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> parameterGroup = generateParameterGroup(); - - if (parameterGroup.size() > taskParameters.getMaxNumOfSubWorkflowInstances()) { - log.warn("the number of sub process instances [{}] exceeds the maximum limit [{}]", parameterGroup.size(), - taskParameters.getMaxNumOfSubWorkflowInstances()); - parameterGroup = parameterGroup.subList(0, taskParameters.getMaxNumOfSubWorkflowInstances()); - } - - // if already exists sub process instance, do not generate again - List existsSubWorkflowInstanceList = - subWorkflowService.getAllDynamicSubWorkflow(workflowInstance.getId(), taskInstance.getTaskCode()); - if (CollectionUtils.isEmpty(existsSubWorkflowInstanceList)) { - generateSubWorkflowInstance(parameterGroup); - } else { - resetProcessInstanceStatus(existsSubWorkflowInstanceList); - } - return new DynamicAsyncTaskExecuteFunction(taskExecutionContext, workflowInstance, taskInstance, this, - commandMapper, - subWorkflowService, taskParameters.getDegreeOfParallelism()); - } + // public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() throws MasterTaskExecuteException { + // List> parameterGroup = generateParameterGroup(); + // + // if (parameterGroup.size() > dynamicParameters.getMaxNumOfSubWorkflowInstances()) { + // log.warn("the number of sub process instances [{}] exceeds the maximum limit [{}]", parameterGroup.size(), + // dynamicParameters.getMaxNumOfSubWorkflowInstances()); + // parameterGroup = parameterGroup.subList(0, dynamicParameters.getMaxNumOfSubWorkflowInstances()); + // } + // + // // if already exists sub process instance, do not generate again + // List existsSubWorkflowInstanceList = + // subWorkflowService.getAllDynamicSubWorkflow(workflowInstance.getId(), taskInstance.getTaskCode()); + // if (CollectionUtils.isEmpty(existsSubWorkflowInstanceList)) { + // generateSubWorkflowInstance(parameterGroup); + // } else { + // resetProcessInstanceStatus(existsSubWorkflowInstanceList); + // } + // return new DynamicAsyncTaskExecuteFunction(taskExecutionContext, workflowInstance, taskInstance, this, + // commandMapper, + // subWorkflowService, dynamicParameters.getDegreeOfParallelism()); + // } public void resetProcessInstanceStatus(List existsSubWorkflowInstanceList) { switch (workflowInstance.getCommandType()) { @@ -250,6 +251,16 @@ private List getDynamicInputParameters() { return dynamicInputParameters; } + @Override + public void start() throws MasterTaskExecuteException { + // todo: + } + + @Override + public TaskExecutionStatus getTaskExecutionState() { + return taskExecutionContext.getCurrentExecutionStatus(); + } + @Override public void pause() throws MasterTaskExecuteException { // todo: support pause @@ -264,6 +275,12 @@ public void kill() { } } + @Override + public ITaskParameterDeserializer getTaskParameterDeserializer() { + return taskParamsJson -> JSONUtils.parseObject(taskParamsJson, new TypeReference() { + }); + } + private void doKillSubWorkflowInstances() throws MasterTaskExecuteException { List existsSubWorkflowInstanceList = subWorkflowService.getAllDynamicSubWorkflow(workflowInstance.getId(), taskInstance.getTaskCode()); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTaskPluginFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicLogicTaskPluginFactory.java similarity index 76% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTaskPluginFactory.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicLogicTaskPluginFactory.java index cf9c599957164..5065e80c7fbc7 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTaskPluginFactory.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/dynamic/DynamicLogicTaskPluginFactory.java @@ -15,16 +15,17 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.task.dynamic; +package org.apache.dolphinscheduler.server.master.engine.executor.plugin.dynamic; import org.apache.dolphinscheduler.dao.mapper.CommandMapper; import org.apache.dolphinscheduler.dao.mapper.WorkflowDefinitionMapper; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory; +import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ILogicTaskPluginFactory; import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.subworkflow.SubWorkflowService; +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; import lombok.extern.slf4j.Slf4j; @@ -54,10 +55,15 @@ public class DynamicLogicTaskPluginFactory implements ILogicTaskPluginFactory { - - private volatile boolean killFlag; +public class LogicFakeTask extends AbstractLogicTask { private Process process; public LogicFakeTask(final IWorkflowExecutionRunnable workflowExecutionRunnable, final TaskExecutionContext taskExecutionContext) { - super(workflowExecutionRunnable, taskExecutionContext, - JSONUtils.parseObject(taskExecutionContext.getTaskParams(), LogicFakeTaskParameters.class)); + super(taskExecutionContext); + onTaskRunning(); } @Override - public void handle() throws MasterTaskExecuteException { + public void start() throws MasterTaskExecuteException { try { final String shellScript = ParameterUtils.convertParameterPlaceholders( taskParameters.getShellScript(), @@ -57,28 +57,39 @@ public void handle() throws MasterTaskExecuteException { final String[] cmd = {"/bin/sh", "-c", shellScript}; process = Runtime.getRuntime().exec(cmd); int exitCode = process.waitFor(); - if (killFlag) { - taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.KILL); + if (taskExecutionStatus != TaskExecutionStatus.RUNNING_EXECUTION) { + // The task has been killed return; } if (exitCode == 0) { - taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.SUCCESS); + onTaskSuccess(); } else { - taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.FAILURE); + onTaskFailed(); } } catch (Exception ex) { throw new MasterTaskExecuteException("FakeTask execute failed", ex); } } + @Override + public void pause() throws MasterTaskExecuteException { + log.info("The LogicFakeTask: {} doesn't support pause", taskExecutionContext.getTaskName()); + } + @Override public void kill() throws MasterTaskExecuteException { - log.info("kill task : {}", taskExecutionContext.getTaskName()); if (process != null && process.isAlive()) { - killFlag = true; - process.destroy(); - log.info("kill task : {} succeed", taskExecutionContext.getTaskName()); + // todo: use shell script to kill the process + process.destroyForcibly(); + onTaskKilled(); + log.info("kill LogicFakeTask : {} succeed", 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/fake/LogicFakeTaskPluginFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/fake/LogicFakeTaskPluginFactory.java similarity index 77% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/fake/LogicFakeTaskPluginFactory.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/fake/LogicFakeTaskPluginFactory.java index 8fc2ea96337c0..9482e2dc35f27 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/fake/LogicFakeTaskPluginFactory.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/fake/LogicFakeTaskPluginFactory.java @@ -15,13 +15,14 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.task.fake; +package org.apache.dolphinscheduler.server.master.engine.executor.plugin.fake; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.task.LogicFakeTaskChannelFactory; import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository; +import org.apache.dolphinscheduler.server.master.engine.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 org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; @@ -33,12 +34,13 @@ public class LogicFakeTaskPluginFactory implements ILogicTaskPluginFactory { @Autowired - private IWorkflowRepository IWorkflowRepository; + private IWorkflowRepository workflowRepository; @Override - public LogicFakeTask createLogicTask(final TaskExecutionContext taskExecutionContext) { + public LogicFakeTask createLogicTask(final ITaskExecutor taskExecutor) { + final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext(); final IWorkflowExecutionRunnable workflowExecutionRunnable = - IWorkflowRepository.get(taskExecutionContext.getWorkflowInstanceId()); + workflowRepository.get(taskExecutionContext.getWorkflowInstanceId()); if (workflowExecutionRunnable == null) { throw new IllegalStateException( "Cannot find the WorkflowExecuteRunnable: " + taskExecutionContext.getWorkflowInstanceId()); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowControlClient.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowControlClient.java similarity index 96% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowControlClient.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowControlClient.java index 69109f09dcdc1..e26fa5775be51 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowControlClient.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowControlClient.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.task.subworkflow; +package org.apache.dolphinscheduler.server.master.engine.executor.plugin.subworkflow; import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; @@ -31,10 +31,10 @@ import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopRequest; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerRequest; +import org.apache.dolphinscheduler.server.master.engine.executor.plugin.subworkflow.trigger.SubWorkflowManualTrigger; import org.apache.dolphinscheduler.server.master.engine.workflow.trigger.WorkflowInstanceRecoverFailureTaskTrigger; import org.apache.dolphinscheduler.server.master.engine.workflow.trigger.WorkflowInstanceRecoverSuspendTaskTrigger; import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; -import org.apache.dolphinscheduler.server.master.runner.task.subworkflow.trigger.SubWorkflowManualTrigger; import lombok.extern.slf4j.Slf4j; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowLogicTask.java similarity index 87% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowLogicTask.java index d5f21bfc53989..b5bc3b4deea80 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowLogicTask.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.task.subworkflow; +package org.apache.dolphinscheduler.server.master.engine.executor.plugin.subworkflow; import org.apache.dolphinscheduler.common.enums.Flag; import org.apache.dolphinscheduler.common.utils.JSONUtils; @@ -34,12 +34,14 @@ import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowInstanceStopResponse; import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowManualTriggerRequest; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; import org.apache.dolphinscheduler.plugin.task.api.parameters.SubWorkflowParameters; +import org.apache.dolphinscheduler.server.master.engine.executor.plugin.AbstractLogicTask; +import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ITaskParameterDeserializer; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; -import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction; -import org.apache.dolphinscheduler.server.master.runner.message.LogicTaskInstanceExecutionEventSenderManager; -import org.apache.dolphinscheduler.server.master.runner.task.BaseAsyncLogicTask; +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorRuntimeContextChangedLifecycleEvent; import lombok.extern.slf4j.Slf4j; @@ -48,7 +50,7 @@ import com.fasterxml.jackson.core.type.TypeReference; @Slf4j -public class SubWorkflowLogicTask extends BaseAsyncLogicTask { +public class SubWorkflowLogicTask extends AbstractLogicTask { private SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext; @@ -56,35 +58,48 @@ public class SubWorkflowLogicTask extends BaseAsyncLogicTask() { - })); + super(taskExecutionContext); + this.taskExecutor = taskExecutor; this.workflowExecutionRunnable = workflowExecutionRunnable; this.applicationContext = applicationContext; this.subWorkflowLogicTaskRuntimeContext = JSONUtils.parseObject( taskExecutionContext.getAppIds(), SubWorkflowLogicTaskRuntimeContext.class); + onTaskRunning(); } @Override - public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() { + public void start() throws MasterTaskExecuteException { subWorkflowLogicTaskRuntimeContext = initializeSubWorkflowInstance(); upsertSubWorkflowRelation(); taskExecutionContext.setAppIds(JSONUtils.toJsonString(subWorkflowLogicTaskRuntimeContext)); - applicationContext - .getBean(LogicTaskInstanceExecutionEventSenderManager.class) - .runningEventSender() - .sendMessage(taskExecutionContext); + taskExecutor.getTaskExecutorEventBus() + .publish(TaskExecutorRuntimeContextChangedLifecycleEvent.of(taskExecutor)); - return new SubWorkflowAsyncTaskExecuteFunction( + subWorkflowTracker = new SubWorkflowTracker( subWorkflowLogicTaskRuntimeContext, applicationContext.getBean(WorkflowInstanceDao.class)); } + @Override + public TaskExecutionStatus getTaskExecutionState() { + if (subWorkflowTracker == null) { + // The sub workflow has not been started + return taskExecutionStatus; + } + taskExecutionStatus = subWorkflowTracker.getSubWorkflowState(); + return taskExecutionStatus; + } + @Override public void pause() throws MasterTaskExecuteException { if (subWorkflowLogicTaskRuntimeContext == null) { @@ -120,6 +135,12 @@ public void kill() throws MasterTaskExecuteException { } } + @Override + public ITaskParameterDeserializer getTaskParameterDeserializer() { + return taskParamsJson -> JSONUtils.parseObject(taskParamsJson, new TypeReference() { + }); + } + private SubWorkflowLogicTaskRuntimeContext initializeSubWorkflowInstance() { // todo: doFailover if the runtime context is not null and task is generated by failover diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskPluginFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowLogicTaskPluginFactory.java similarity index 76% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskPluginFactory.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowLogicTaskPluginFactory.java index a11d3d0a74acf..58de74903b9cb 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskPluginFactory.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowLogicTaskPluginFactory.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.task.subworkflow; +package org.apache.dolphinscheduler.server.master.engine.executor.plugin.subworkflow; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.task.SubWorkflowLogicTaskChannelFactory; import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository; +import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ILogicTaskPluginFactory; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; -import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; -import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory; +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; import lombok.extern.slf4j.Slf4j; @@ -41,14 +41,12 @@ public class SubWorkflowLogicTaskPluginFactory implements ILogicTaskPluginFactor private IWorkflowRepository IWorkflowRepository; @Override - public SubWorkflowLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException { + public SubWorkflowLogicTask createLogicTask(final ITaskExecutor taskExecutor) { + final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext(); final int workflowInstanceId = taskExecutionContext.getWorkflowInstanceId(); final IWorkflowExecutionRunnable workflowExecutionRunnable = IWorkflowRepository.get(workflowInstanceId); - if (workflowExecutionRunnable == null) { - throw new LogicTaskInitializeException( - "Cannot find the WorkflowExecuteRunnable by : " + workflowInstanceId); - } - return new SubWorkflowLogicTask(taskExecutionContext, workflowExecutionRunnable, applicationContext); + return new SubWorkflowLogicTask(taskExecutionContext, workflowExecutionRunnable, taskExecutor, + applicationContext); } @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskRuntimeContext.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowLogicTaskRuntimeContext.java similarity index 94% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskRuntimeContext.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowLogicTaskRuntimeContext.java index 48d08157359f2..20b170cfb133f 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskRuntimeContext.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowLogicTaskRuntimeContext.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.task.subworkflow; +package org.apache.dolphinscheduler.server.master.engine.executor.plugin.subworkflow; import lombok.AllArgsConstructor; import lombok.Builder; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowAsyncTaskExecuteFunction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowTracker.java similarity index 61% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowAsyncTaskExecuteFunction.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowTracker.java index 8938469762ff4..4cc16556081d0 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowAsyncTaskExecuteFunction.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/SubWorkflowTracker.java @@ -15,56 +15,47 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.task.subworkflow; +package org.apache.dolphinscheduler.server.master.engine.executor.plugin.subworkflow; import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao; -import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction; - -import java.time.Duration; +import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; @Slf4j -public class SubWorkflowAsyncTaskExecuteFunction implements AsyncTaskExecuteFunction { - - private static final Duration SUB_WORKFLOW_TASK_EXECUTE_STATE_CHECK_INTERVAL = Duration.ofSeconds(10); +public class SubWorkflowTracker { private final WorkflowInstanceDao workflowInstanceDao; private final SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext; - public SubWorkflowAsyncTaskExecuteFunction(final SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext, - final WorkflowInstanceDao workflowInstanceDao) { + public SubWorkflowTracker(final SubWorkflowLogicTaskRuntimeContext subWorkflowLogicTaskRuntimeContext, + final WorkflowInstanceDao workflowInstanceDao) { this.subWorkflowLogicTaskRuntimeContext = subWorkflowLogicTaskRuntimeContext; this.workflowInstanceDao = workflowInstanceDao; } - @Override - public @NonNull AsyncTaskExecutionStatus getAsyncTaskExecutionStatus() { + public @NonNull TaskExecutionStatus getSubWorkflowState() { final Integer subWorkflowInstanceId = subWorkflowLogicTaskRuntimeContext.getSubWorkflowInstanceId(); final WorkflowInstance subWorkflowInstance = workflowInstanceDao.queryById(subWorkflowInstanceId); if (subWorkflowInstance == null) { log.info("Cannot find the SubWorkflow instance: {}, maybe it has been deleted", subWorkflowInstanceId); - return AsyncTaskExecutionStatus.FAILED; + return TaskExecutionStatus.FAILURE; } switch (subWorkflowInstance.getState()) { case PAUSE: - return AsyncTaskExecutionStatus.PAUSE; + return TaskExecutionStatus.PAUSE; case STOP: - return AsyncTaskExecutionStatus.KILL; + return TaskExecutionStatus.KILL; case SUCCESS: - return AsyncTaskExecutionStatus.SUCCESS; + return TaskExecutionStatus.SUCCESS; case FAILURE: - return AsyncTaskExecutionStatus.FAILED; + return TaskExecutionStatus.FAILURE; default: - return AsyncTaskExecutionStatus.RUNNING; + return TaskExecutionStatus.RUNNING_EXECUTION; } } - @Override - public @NonNull Duration getAsyncTaskStateCheckInterval() { - return SUB_WORKFLOW_TASK_EXECUTE_STATE_CHECK_INTERVAL; - } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/trigger/SubWorkflowManualTrigger.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/trigger/SubWorkflowManualTrigger.java similarity index 94% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/trigger/SubWorkflowManualTrigger.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/trigger/SubWorkflowManualTrigger.java index 5484de2128dc3..cbbc9030083b5 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/trigger/SubWorkflowManualTrigger.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/subworkflow/trigger/SubWorkflowManualTrigger.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.task.subworkflow.trigger; +package org.apache.dolphinscheduler.server.master.engine.executor.plugin.subworkflow.trigger; import org.apache.dolphinscheduler.common.enums.Flag; import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/switchtask/SwitchLogicTask.java similarity index 86% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTask.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/switchtask/SwitchLogicTask.java index 2407a7f0ab4aa..60c78d7d5a1bb 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTask.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/switchtask/SwitchLogicTask.java @@ -15,20 +15,19 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.task.switchtask; +package org.apache.dolphinscheduler.server.master.engine.executor.plugin.switchtask; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.dao.entity.TaskDefinition; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; import org.apache.dolphinscheduler.plugin.task.api.model.Property; import org.apache.dolphinscheduler.plugin.task.api.model.SwitchResultVo; import org.apache.dolphinscheduler.plugin.task.api.parameters.SwitchParameters; +import org.apache.dolphinscheduler.server.master.engine.executor.plugin.AbstractLogicTask; +import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ITaskParameterDeserializer; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; -import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecuteContext; -import org.apache.dolphinscheduler.server.master.runner.task.BaseSyncLogicTask; import org.apache.dolphinscheduler.server.master.utils.SwitchTaskUtils; import org.apache.commons.collections4.CollectionUtils; @@ -43,29 +42,25 @@ import com.fasterxml.jackson.core.type.TypeReference; @Slf4j -public class SwitchLogicTask extends BaseSyncLogicTask { - - public static final String TASK_TYPE = "SWITCH"; +public class SwitchLogicTask extends AbstractLogicTask { private final IWorkflowExecutionRunnable workflowExecutionRunnable; private final TaskInstance taskInstance; public SwitchLogicTask(IWorkflowExecutionRunnable workflowExecutionRunnable, TaskExecutionContext taskExecutionContext) { - super(workflowExecutionRunnable, - taskExecutionContext, - JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference() { - })); + super(taskExecutionContext); this.workflowExecutionRunnable = workflowExecutionRunnable; this.taskInstance = workflowExecutionRunnable .getWorkflowExecuteContext() .getWorkflowExecutionGraph() .getTaskExecutionRunnableById(taskExecutionContext.getTaskInstanceId()) .getTaskInstance(); + onTaskRunning(); } @Override - public void handle() throws MasterTaskExecuteException { + public void start() { if (CollectionUtils.isEmpty(taskParameters.getSwitchResult().getDependTaskList())) { // If the branch is empty then will go into the default branch // This case shouldn't happen, we can directly throw exception and forbid the user to set branch @@ -76,8 +71,9 @@ public void handle() throws MasterTaskExecuteException { } checkIfBranchExist(taskParameters.getNextBranch()); taskInstance.setTaskParams(JSONUtils.toJsonString(taskParameters)); - taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.SUCCESS); - log.info("Switch task execute finished: {}", taskExecutionContext.getCurrentExecutionStatus().name()); + + onTaskSuccess(); + log.info("Switch task execute finished"); } private void moveToDefaultBranch() { @@ -144,4 +140,20 @@ private String getTaskName(Long taskCode) { .orElse(null); } + @Override + public void pause() { + log.info("The SwitchTask does not support pause operation"); + } + + @Override + public void kill() { + log.info("The SwitchTask does not support kill operation"); + } + + @Override + public ITaskParameterDeserializer getTaskParameterDeserializer() { + return taskParamsJson -> JSONUtils.parseObject(taskParamsJson, new TypeReference() { + }); + } + } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTaskPluginFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/switchtask/SwitchLogicTaskPluginFactory.java similarity index 77% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTaskPluginFactory.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/switchtask/SwitchLogicTaskPluginFactory.java index ce5fb6dfe662f..9bb0661635bd2 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTaskPluginFactory.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/executor/plugin/switchtask/SwitchLogicTaskPluginFactory.java @@ -15,13 +15,15 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.task.switchtask; +package org.apache.dolphinscheduler.server.master.engine.executor.plugin.switchtask; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.plugin.task.api.task.SwitchLogicTaskChannelFactory; import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository; +import org.apache.dolphinscheduler.server.master.engine.executor.plugin.ILogicTaskPluginFactory; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; -import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory; +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; import lombok.extern.slf4j.Slf4j; @@ -36,7 +38,8 @@ public class SwitchLogicTaskPluginFactory implements ILogicTaskPluginFactory { - private final LinkedBlockingQueue eventChannel = new LinkedBlockingQueue<>(); - public void publish(final AbstractSystemEvent event) { - try { - eventChannel.put(event); - log.info("Published SystemEvent: {}", event); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new IllegalStateException("The thread has been interrupted", e); - } + super.publish(event); + log.info("Published SystemEvent: {}", event); } public AbstractSystemEvent take() throws InterruptedException { - return eventChannel.take(); + return delayEventQueue.take(); } - - public boolean isEmpty() { - return eventChannel.isEmpty(); - } - } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/system/event/GlobalMasterFailoverEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/system/event/GlobalMasterFailoverEvent.java index a6c790985179a..4f47532a33cee 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/system/event/GlobalMasterFailoverEvent.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/system/event/GlobalMasterFailoverEvent.java @@ -29,7 +29,7 @@ public class GlobalMasterFailoverEvent extends AbstractSystemEvent { private final Date eventTime; public GlobalMasterFailoverEvent(Date eventTime) { - super(eventTime.getTime()); + super(0); this.eventTime = eventTime; } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/system/event/MasterFailoverEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/system/event/MasterFailoverEvent.java index 9eab103f72aec..5abbd0db7fce0 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/system/event/MasterFailoverEvent.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/system/event/MasterFailoverEvent.java @@ -31,7 +31,7 @@ public class MasterFailoverEvent extends AbstractSystemEvent { private MasterFailoverEvent(final String masterAddress, final Date eventTime) { - super(eventTime.getTime()); + super(System.currentTimeMillis() - eventTime.getTime()); this.masterAddress = masterAddress; this.eventTime = eventTime; } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskExecutorClient.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskExecutorClient.java index 55ce24972c25c..3f42e3cf75636 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskExecutorClient.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskExecutorClient.java @@ -19,13 +19,31 @@ import org.apache.dolphinscheduler.server.master.engine.exceptions.TaskKillException; import org.apache.dolphinscheduler.server.master.engine.exceptions.TaskPauseException; +import org.apache.dolphinscheduler.server.master.engine.exceptions.TaskTakeOverException; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; +import org.apache.dolphinscheduler.task.executor.ITaskExecutorLifecycleEventReporter; +import org.apache.dolphinscheduler.task.executor.TaskEngine; /** - * The client of task executor, used to communicate with task executor. + * The client used to communicate with {@link TaskEngine}. */ public interface ITaskExecutorClient { + /** + * Dispatch the task to task executor. + * + * @throws TaskDispatchException If dispatch failed or error occurs. + */ + void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskDispatchException; + + /** + * Reassign the workflow instance host from task executor. + * + * @throws TaskTakeOverException If an error occurs. + */ + boolean reassignWorkflowInstanceHost(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskTakeOverException; + /** * Pause task from task executor. *

This method is not a sync method, it will return immediately after send a kill request to remote executor and receive a response. @@ -34,7 +52,7 @@ public interface ITaskExecutorClient { * * @throws TaskPauseException If an error occurs. */ - void pause(ITaskExecutionRunnable taskExecutionRunnable) throws TaskPauseException; + void pause(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskPauseException; /** * Kill task from task executor. @@ -44,5 +62,14 @@ public interface ITaskExecutorClient { * * @throws TaskKillException If an error occurs. */ - void kill(ITaskExecutionRunnable taskExecutionRunnable) throws TaskKillException; + void kill(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskKillException; + + /** + * Send TaskExecutorLifecycleEventAck to TaskEngine. + *

This method will not throw exception, once send ack failed, the executor engine will retry. + */ + void ackTaskExecutorLifecycleEvent( + final ITaskExecutionRunnable taskExecutionRunnable, + final ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck); + } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskExecutorClientDelegator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskExecutorClientDelegator.java index d157ea8f58645..8ca2765805417 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskExecutorClientDelegator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskExecutorClientDelegator.java @@ -18,6 +18,8 @@ package org.apache.dolphinscheduler.server.master.engine.task.client; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; +import org.apache.dolphinscheduler.task.executor.ITaskExecutorLifecycleEventReporter; /** * The interface of task executor client delegator. It is used to send operation to task executor server. @@ -27,9 +29,32 @@ */ public interface ITaskExecutorClientDelegator { - void dispatch(final ITaskExecutionRunnable taskExecutionRunnable); + /** + * Dispatch the task to task executor. + * + * @throws TaskDispatchException If dispatch failed + */ + void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskDispatchException; + /** + * Take over the task from task executor. + */ + boolean reassignMasterHost(final ITaskExecutionRunnable taskExecutionRunnable); + + /** + * Pause the task, this method doesn't guarantee the task is paused success. + */ void pause(final ITaskExecutionRunnable taskExecutionRunnable); + /** + * Kill the task, this method doesn't guarantee the task is killed success. + */ void kill(final ITaskExecutionRunnable taskExecutionRunnable); + + /** + * Ack the task executor lifecycle event. + */ + void ackTaskExecutorLifecycleEvent( + final ITaskExecutionRunnable taskExecutionRunnable, + final ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskOperator.java deleted file mode 100644 index 9b01b52ccade0..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/ITaskOperator.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.engine.task.client; - -import org.apache.dolphinscheduler.server.master.engine.exceptions.TaskKillException; -import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; - -public interface ITaskOperator { - - void dispatch(ITaskExecutionRunnable taskExecutionRunnable); - - void pause(ITaskExecutionRunnable taskExecutionRunnable); - - void kill(ITaskExecutionRunnable taskExecutionRunnable) throws TaskKillException; - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/LogicTaskExecutorClientDelegator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/LogicTaskExecutorClientDelegator.java index 7a941f9e305ea..5bdcdfcbda59c 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/LogicTaskExecutorClientDelegator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/LogicTaskExecutorClientDelegator.java @@ -21,27 +21,57 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.extract.base.client.Clients; -import org.apache.dolphinscheduler.extract.master.ILogicTaskInstanceOperator; -import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskKillRequest; -import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskKillResponse; -import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskPauseRequest; -import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskPauseResponse; +import org.apache.dolphinscheduler.extract.master.ILogicTaskExecutorOperator; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.engine.exceptions.TaskKillException; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; +import org.apache.dolphinscheduler.task.executor.ITaskExecutorLifecycleEventReporter; +import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchRequest; +import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchResponse; +import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorKillRequest; +import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorKillResponse; +import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorPauseRequest; +import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorPauseResponse; import org.apache.commons.lang3.StringUtils; import lombok.extern.slf4j.Slf4j; +import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; @Slf4j @Component public class LogicTaskExecutorClientDelegator implements ITaskExecutorClientDelegator { + @Autowired + private MasterConfig masterConfig; + @Override - public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) { + public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskDispatchException { + final String logicTaskExecutorAddress = masterConfig.getMasterAddress(); + final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext(); + + taskExecutionContext.setHost(logicTaskExecutorAddress); + taskExecutionRunnable.getTaskInstance().setHost(logicTaskExecutorAddress); + + final TaskExecutorDispatchResponse logicTaskDispatchResponse = Clients + .withService(ILogicTaskExecutorOperator.class) + .withHost(logicTaskExecutorAddress) + .dispatchTask(TaskExecutorDispatchRequest.of(taskExecutionContext)); + if (!logicTaskDispatchResponse.isDispatchSuccess()) { + throw new TaskDispatchException( + String.format("Dispatch LogicTask to %s failed, response is: %s", + taskExecutionContext.getHost(), logicTaskDispatchResponse)); + } + } + @Override + public boolean reassignMasterHost(final ITaskExecutionRunnable taskExecutionRunnable) { + // The Logic Task doesn't support take-over, since the logic task is not executed on the worker. + return false; } @Override @@ -51,10 +81,10 @@ public void pause(final ITaskExecutionRunnable taskExecutionRunnable) { final String taskName = taskInstance.getName(); checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty"); - final LogicTaskPauseResponse pauseResponse = Clients - .withService(ILogicTaskInstanceOperator.class) + final TaskExecutorPauseResponse pauseResponse = Clients + .withService(ILogicTaskExecutorOperator.class) .withHost(taskInstance.getHost()) - .pauseLogicTask(new LogicTaskPauseRequest(taskInstance.getId())); + .pauseTask(TaskExecutorPauseRequest.of(taskInstance.getId())); if (pauseResponse.isSuccess()) { log.info("Pause task {} on executor {} successfully", taskName, executorHost); } else { @@ -69,14 +99,29 @@ public void kill(final ITaskExecutionRunnable taskExecutionRunnable) throws Task final String taskName = taskInstance.getName(); checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty"); - final LogicTaskKillResponse killResponse = Clients - .withService(ILogicTaskInstanceOperator.class) + final TaskExecutorKillResponse killResponse = Clients + .withService(ILogicTaskExecutorOperator.class) .withHost(taskInstance.getHost()) - .killLogicTask(new LogicTaskKillRequest(taskInstance.getId())); + .killTask(TaskExecutorKillRequest.of(taskInstance.getId())); if (killResponse.isSuccess()) { log.info("Kill task {} on executor {} successfully", taskName, executorHost); } else { log.warn("Kill task {} on executor {} failed with response {}", taskName, executorHost, killResponse); } } + + @Override + public void ackTaskExecutorLifecycleEvent( + final ITaskExecutionRunnable taskExecutionRunnable, + final ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck) { + final TaskInstance taskInstance = taskExecutionRunnable.getTaskInstance(); + final String executorHost = taskInstance.getHost(); + checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty"); + + Clients + .withService(ILogicTaskExecutorOperator.class) + .withHost(taskInstance.getHost()) + .ackTaskExecutorLifecycleEvent(taskExecutorLifecycleEventAck); + } + } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java index 2db50f84c9d0e..39315b98f4725 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java @@ -21,26 +21,108 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.extract.base.client.Clients; -import org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator; -import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceKillRequest; -import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstanceKillResponse; -import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstancePauseRequest; -import org.apache.dolphinscheduler.extract.worker.transportor.TaskInstancePauseResponse; +import org.apache.dolphinscheduler.extract.base.utils.Host; +import org.apache.dolphinscheduler.extract.worker.IPhysicalTaskExecutorOperator; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.cluster.loadbalancer.IWorkerLoadBalancer; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; +import org.apache.dolphinscheduler.task.executor.ITaskExecutorLifecycleEventReporter; +import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchRequest; +import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchResponse; +import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorKillRequest; +import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorKillResponse; +import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorPauseRequest; +import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorPauseResponse; +import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorReassignMasterRequest; +import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorReassignMasterResponse; import org.apache.commons.lang3.StringUtils; import lombok.extern.slf4j.Slf4j; +import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; @Slf4j @Component public class PhysicalTaskExecutorClientDelegator implements ITaskExecutorClientDelegator { + @Autowired + private MasterConfig masterConfig; + + @Autowired + private IWorkerLoadBalancer workerLoadBalancer; + @Override - public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) { + public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskDispatchException { + final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext(); + final String taskName = taskExecutionContext.getTaskName(); + final String physicalTaskExecutorAddress = workerLoadBalancer + .select(taskExecutionContext.getWorkerGroup()) + .map(Host::of) + .map(Host::getAddress) + .orElseThrow(() -> new TaskDispatchException( + String.format("Cannot find the host to dispatch Task[id=%s, name=%s]", + taskExecutionContext.getTaskInstanceId(), taskName))); + + taskExecutionContext.setHost(physicalTaskExecutorAddress); + taskExecutionRunnable.getTaskInstance().setHost(physicalTaskExecutorAddress); + + try { + final TaskExecutorDispatchResponse taskExecutorDispatchResponse = Clients + .withService(IPhysicalTaskExecutorOperator.class) + .withHost(physicalTaskExecutorAddress) + .dispatchTask(TaskExecutorDispatchRequest.of(taskExecutionRunnable.getTaskExecutionContext())); + if (!taskExecutorDispatchResponse.isDispatchSuccess()) { + throw new TaskDispatchException( + "Dispatch task: " + taskName + " to " + physicalTaskExecutorAddress + " failed: " + + taskExecutorDispatchResponse); + } + } catch (TaskDispatchException e) { + throw e; + } catch (Exception e) { + throw new TaskDispatchException( + "Dispatch task: " + taskName + " to " + physicalTaskExecutorAddress + " failed", e); + } + } + + @Override + public boolean reassignMasterHost(final ITaskExecutionRunnable taskExecutionRunnable) { + final String taskName = taskExecutionRunnable.getName(); + checkArgument(taskExecutionRunnable.isTaskInstanceInitialized(), + "Task " + taskName + "is not initialized cannot take-over"); + + final TaskInstance taskInstance = taskExecutionRunnable.getTaskInstance(); + final String taskExecutorHost = taskInstance.getHost(); + if (StringUtils.isEmpty(taskExecutorHost)) { + log.debug( + "The task executor: {} host is empty, cannot take-over, this might caused by the task hasn't dispatched", + taskName); + return false; + } + final TaskExecutorReassignMasterRequest taskExecutorReassignMasterRequest = + TaskExecutorReassignMasterRequest.builder() + .taskInstanceId(taskInstance.getId()) + .workflowHost(masterConfig.getMasterAddress()) + .build(); + final TaskExecutorReassignMasterResponse taskExecutorReassignMasterResponse = + Clients + .withService(IPhysicalTaskExecutorOperator.class) + .withHost(taskInstance.getHost()) + .reassignWorkflowInstanceHost(taskExecutorReassignMasterRequest); + boolean success = taskExecutorReassignMasterResponse.isSuccess(); + if (success) { + log.info("Reassign master host {} to {} successfully", taskExecutorHost, taskName); + } else { + log.info("Reassign master host {} on {} failed with response {}", + taskExecutorHost, + taskName, + taskExecutorReassignMasterResponse); + } + return success; } @Override @@ -50,10 +132,10 @@ public void pause(final ITaskExecutionRunnable taskExecutionRunnable) { final String taskName = taskInstance.getName(); checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty"); - final TaskInstancePauseResponse pauseResponse = Clients - .withService(ITaskInstanceOperator.class) + final TaskExecutorPauseResponse pauseResponse = Clients + .withService(IPhysicalTaskExecutorOperator.class) .withHost(taskInstance.getHost()) - .pauseTask(new TaskInstancePauseRequest(taskInstance.getId())); + .pauseTask(TaskExecutorPauseRequest.of(taskInstance.getId())); if (pauseResponse.isSuccess()) { log.info("Pause task {} on executor {} successfully", taskName, executorHost); } else { @@ -68,14 +150,28 @@ public void kill(final ITaskExecutionRunnable taskExecutionRunnable) { final String taskName = taskInstance.getName(); checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty"); - final TaskInstanceKillResponse killResponse = Clients - .withService(ITaskInstanceOperator.class) + final TaskExecutorKillResponse killResponse = Clients + .withService(IPhysicalTaskExecutorOperator.class) .withHost(executorHost) - .killTask(new TaskInstanceKillRequest(taskInstance.getId())); + .killTask(TaskExecutorKillRequest.of(taskInstance.getId())); if (killResponse.isSuccess()) { log.info("Kill task {} on executor {} successfully", taskName, executorHost); } else { log.warn("Kill task {} on executor {} failed with response {}", taskName, executorHost, killResponse); } } + + @Override + public void ackTaskExecutorLifecycleEvent(final ITaskExecutionRunnable taskExecutionRunnable, + final ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck) { + final TaskInstance taskInstance = taskExecutionRunnable.getTaskInstance(); + final String executorHost = taskInstance.getHost(); + checkArgument(StringUtils.isNotEmpty(executorHost), "Executor host is empty"); + + Clients + .withService(IPhysicalTaskExecutorOperator.class) + .withHost(executorHost) + .ackPhysicalTaskExecutorLifecycleEvent(taskExecutorLifecycleEventAck); + } + } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/TaskExecutorClient.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/TaskExecutorClient.java index e83bbd5515c3b..27c70b6b6901a 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/TaskExecutorClient.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/TaskExecutorClient.java @@ -23,7 +23,12 @@ import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils; import org.apache.dolphinscheduler.server.master.engine.exceptions.TaskKillException; import org.apache.dolphinscheduler.server.master.engine.exceptions.TaskPauseException; +import org.apache.dolphinscheduler.server.master.engine.exceptions.TaskTakeOverException; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; +import org.apache.dolphinscheduler.task.executor.ITaskExecutorLifecycleEventReporter; + +import lombok.extern.slf4j.Slf4j; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; @@ -31,6 +36,7 @@ /** * The client of task executor, used to communicate with task executor. */ +@Slf4j @Component public class TaskExecutorClient implements ITaskExecutorClient { @@ -41,7 +47,31 @@ public class TaskExecutorClient implements ITaskExecutorClient { private PhysicalTaskExecutorClientDelegator physicalTaskExecutorClientDelegator; @Override - public void pause(ITaskExecutionRunnable taskExecutionRunnable) throws TaskPauseException { + public void dispatch(ITaskExecutionRunnable taskExecutionRunnable) throws TaskDispatchException { + try { + getTaskExecutorClientDelegator(taskExecutionRunnable).dispatch(taskExecutionRunnable); + } catch (TaskDispatchException taskDispatchException) { + throw taskDispatchException; + } catch (Exception ex) { + throw new TaskDispatchException("Dispatch task: " + taskExecutionRunnable.getName() + " to executor failed", + ex); + } + } + + @Override + public boolean reassignWorkflowInstanceHost(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskTakeOverException { + try { + return getTaskExecutorClientDelegator(taskExecutionRunnable) + .reassignMasterHost(taskExecutionRunnable); + } catch (Exception ex) { + throw new TaskTakeOverException( + "Take over task: " + taskExecutionRunnable.getName() + " from executor failed", + ex); + } + } + + @Override + public void pause(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskPauseException { try { getTaskExecutorClientDelegator(taskExecutionRunnable).pause(taskExecutionRunnable); } catch (Exception ex) { @@ -51,7 +81,7 @@ public void pause(ITaskExecutionRunnable taskExecutionRunnable) throws TaskPause } @Override - public void kill(ITaskExecutionRunnable taskExecutionRunnable) throws TaskKillException { + public void kill(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskKillException { try { getTaskExecutorClientDelegator(taskExecutionRunnable).kill(taskExecutionRunnable); } catch (Exception ex) { @@ -59,6 +89,18 @@ public void kill(ITaskExecutionRunnable taskExecutionRunnable) throws TaskKillEx } } + @Override + public void ackTaskExecutorLifecycleEvent( + final ITaskExecutionRunnable taskExecutionRunnable, + final ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck) { + try { + getTaskExecutorClientDelegator(taskExecutionRunnable) + .ackTaskExecutorLifecycleEvent(taskExecutionRunnable, taskExecutorLifecycleEventAck); + } catch (Exception ex) { + log.error("Send taskExecutorLifecycleEventAck: {} failed", taskExecutorLifecycleEventAck, ex); + } + } + private ITaskExecutorClientDelegator getTaskExecutorClientDelegator(final ITaskExecutionRunnable taskExecutionRunnable) { final TaskInstance taskInstance = taskExecutionRunnable.getTaskInstance(); checkArgument(taskInstance != null, "taskType cannot be empty"); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java index 4e9cb4ad14403..5ddcf13c8917f 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/TaskLifecycleEventType.java @@ -38,6 +38,10 @@ public enum TaskLifecycleEventType implements ILifecycleEventType { * The task instance is running at the target executor server. */ RUNNING, + /** + * The task instance's runtime context changed. + */ + RUNTIME_CONTEXT_CHANGED, /** * Do Timeout strategy of the task instance. */ diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskRunningLifecycleEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskRunningLifecycleEvent.java index 571f23e4e68d9..e95d0c1e3f425 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskRunningLifecycleEvent.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskRunningLifecycleEvent.java @@ -37,8 +37,6 @@ public class TaskRunningLifecycleEvent extends AbstractTaskLifecycleEvent { private final String logPath; - private final String runtimeContext; - private final Date startTime; @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskRuntimeContextChangedEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskRuntimeContextChangedEvent.java new file mode 100644 index 0000000000000..681fb43f63452 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/event/TaskRuntimeContextChangedEvent.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event; + +import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.AbstractTaskLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Getter; + +@Getter +@Builder +@AllArgsConstructor +public class TaskRuntimeContextChangedEvent extends AbstractTaskLifecycleEvent { + + private final ITaskExecutionRunnable taskExecutionRunnable; + + private final String runtimeContext; + + @Override + public ILifecycleEventType getEventType() { + return TaskLifecycleEventType.RUNTIME_CONTEXT_CHANGED; + } + + @Override + public String toString() { + return "TaskRunningLifecycleEvent{" + + "task=" + taskExecutionRunnable.getName() + + ", runtimeContext=" + runtimeContext + + '}'; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskDispatchedLifecycleEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskDispatchedLifecycleEventHandler.java index a0203af46e43f..6cb4043872390 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskDispatchedLifecycleEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskDispatchedLifecycleEventHandler.java @@ -18,14 +18,18 @@ package org.apache.dolphinscheduler.server.master.engine.task.lifecycle.handler; import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.client.TaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskDispatchedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.task.statemachine.ITaskStateAction; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.task.executor.ITaskExecutorLifecycleEventReporter; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorLifecycleEventType; import lombok.extern.slf4j.Slf4j; +import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; @Slf4j @@ -34,12 +38,20 @@ public class TaskDispatchedLifecycleEventHandler extends AbstractTaskLifecycleEventHandler { + @Autowired + private TaskExecutorClient taskExecutorClient; + @Override public void handle(final ITaskStateAction taskStateAction, final IWorkflowExecutionRunnable workflowExecutionRunnable, final ITaskExecutionRunnable taskExecutionRunnable, final TaskDispatchedLifecycleEvent taskDispatchedEvent) { taskStateAction.dispatchedEventAction(workflowExecutionRunnable, taskExecutionRunnable, taskDispatchedEvent); + taskExecutorClient.ackTaskExecutorLifecycleEvent( + taskExecutionRunnable, + new ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck( + taskExecutionRunnable.getId(), + TaskExecutorLifecycleEventType.DISPATCHED)); } @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFailedLifecycleEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFailedLifecycleEventHandler.java index c83ac98946d1f..c636fd937bbf2 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFailedLifecycleEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskFailedLifecycleEventHandler.java @@ -18,23 +18,35 @@ package org.apache.dolphinscheduler.server.master.engine.task.lifecycle.handler; import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.client.TaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.task.statemachine.ITaskStateAction; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.task.executor.ITaskExecutorLifecycleEventReporter; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorLifecycleEventType; +import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; @Component public class TaskFailedLifecycleEventHandler extends AbstractTaskLifecycleEventHandler { + @Autowired + private TaskExecutorClient taskExecutorClient; + @Override public void handle(final ITaskStateAction taskStateAction, final IWorkflowExecutionRunnable workflowExecutionRunnable, final ITaskExecutionRunnable taskExecutionRunnable, final TaskFailedLifecycleEvent event) { taskStateAction.failedEventAction(workflowExecutionRunnable, taskExecutionRunnable, event); + taskExecutorClient.ackTaskExecutorLifecycleEvent( + taskExecutionRunnable, + new ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck( + taskExecutionRunnable.getId(), + TaskExecutorLifecycleEventType.FAILED)); } @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskKilledLifecycleEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskKilledLifecycleEventHandler.java index b84a770dde017..6353b48969951 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskKilledLifecycleEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskKilledLifecycleEventHandler.java @@ -18,23 +18,35 @@ package org.apache.dolphinscheduler.server.master.engine.task.lifecycle.handler; import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.client.TaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKilledLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.task.statemachine.ITaskStateAction; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.task.executor.ITaskExecutorLifecycleEventReporter; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorLifecycleEventType; +import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; @Component public class TaskKilledLifecycleEventHandler extends AbstractTaskLifecycleEventHandler { + @Autowired + private TaskExecutorClient taskExecutorClient; + @Override public void handle(final ITaskStateAction taskStateAction, final IWorkflowExecutionRunnable workflowExecutionRunnable, final ITaskExecutionRunnable taskExecutionRunnable, final TaskKilledLifecycleEvent taskKilledEvent) { taskStateAction.killedEventAction(workflowExecutionRunnable, taskExecutionRunnable, taskKilledEvent); + taskExecutorClient.ackTaskExecutorLifecycleEvent( + taskExecutionRunnable, + new ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck( + taskExecutionRunnable.getId(), + TaskExecutorLifecycleEventType.KILLED)); } @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskPausedLifecycleEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskPausedLifecycleEventHandler.java index 49811b90a1187..009df5af4f3b0 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskPausedLifecycleEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskPausedLifecycleEventHandler.java @@ -18,23 +18,35 @@ package org.apache.dolphinscheduler.server.master.engine.task.lifecycle.handler; import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.client.TaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskPausedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.task.statemachine.ITaskStateAction; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.task.executor.ITaskExecutorLifecycleEventReporter; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorLifecycleEventType; +import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; @Component public class TaskPausedLifecycleEventHandler extends AbstractTaskLifecycleEventHandler { + @Autowired + private TaskExecutorClient taskExecutorClient; + @Override public void handle(final ITaskStateAction taskStateAction, final IWorkflowExecutionRunnable workflowExecutionRunnable, final ITaskExecutionRunnable taskExecutionRunnable, final TaskPausedLifecycleEvent event) { taskStateAction.pausedEventAction(workflowExecutionRunnable, taskExecutionRunnable, event); + taskExecutorClient.ackTaskExecutorLifecycleEvent( + taskExecutionRunnable, + new ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck( + taskExecutionRunnable.getId(), + TaskExecutorLifecycleEventType.PAUSED)); } @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskRunningLifecycleEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskRunningLifecycleEventHandler.java index 4bd5d1ee203b2..fba27c494c7b2 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskRunningLifecycleEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskRunningLifecycleEventHandler.java @@ -18,26 +18,38 @@ package org.apache.dolphinscheduler.server.master.engine.task.lifecycle.handler; import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskRunningLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.task.statemachine.ITaskStateAction; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.task.executor.ITaskExecutorLifecycleEventReporter; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorLifecycleEventType; import lombok.extern.slf4j.Slf4j; +import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; @Slf4j @Component public class TaskRunningLifecycleEventHandler extends AbstractTaskLifecycleEventHandler { + @Autowired + private ITaskExecutorClient taskExecutorClient; + @Override public void handle(final ITaskStateAction taskStateAction, final IWorkflowExecutionRunnable workflowExecutionRunnable, final ITaskExecutionRunnable taskExecutionRunnable, final TaskRunningLifecycleEvent taskRunningEvent) { taskStateAction.startedEventAction(workflowExecutionRunnable, taskExecutionRunnable, taskRunningEvent); + taskExecutorClient.ackTaskExecutorLifecycleEvent( + taskExecutionRunnable, + new ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck( + taskExecutionRunnable.getId(), + TaskExecutorLifecycleEventType.RUNNING)); } @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskRuntimeContextChangedLifecycleEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskRuntimeContextChangedLifecycleEventHandler.java new file mode 100644 index 0000000000000..adcb4a2ec1a83 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskRuntimeContextChangedLifecycleEventHandler.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.engine.task.lifecycle.handler; + +import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskRuntimeContextChangedEvent; +import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.engine.task.statemachine.ITaskStateAction; +import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.task.executor.ITaskExecutorLifecycleEventReporter; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorLifecycleEventType; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class TaskRuntimeContextChangedLifecycleEventHandler + extends + AbstractTaskLifecycleEventHandler { + + @Autowired + private ITaskExecutorClient taskExecutorClient; + + @Override + public void handle(final ITaskStateAction taskStateAction, + final IWorkflowExecutionRunnable workflowExecutionRunnable, + final ITaskExecutionRunnable taskExecutionRunnable, + final TaskRuntimeContextChangedEvent event) { + taskStateAction.runtimeContextChangedEventAction(workflowExecutionRunnable, taskExecutionRunnable, event); + + taskExecutorClient.ackTaskExecutorLifecycleEvent( + taskExecutionRunnable, + new ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck( + taskExecutionRunnable.getId(), + TaskExecutorLifecycleEventType.RUNTIME_CONTEXT_CHANGE)); + } + + @Override + public ILifecycleEventType matchEventType() { + return TaskLifecycleEventType.RUNTIME_CONTEXT_CHANGED; + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskSuccessLifecycleEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskSuccessLifecycleEventHandler.java index b79ec46bf8443..3e009100e6ab8 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskSuccessLifecycleEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/lifecycle/handler/TaskSuccessLifecycleEventHandler.java @@ -18,23 +18,35 @@ package org.apache.dolphinscheduler.server.master.engine.task.lifecycle.handler; import org.apache.dolphinscheduler.server.master.engine.ILifecycleEventType; +import org.apache.dolphinscheduler.server.master.engine.task.client.TaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.TaskLifecycleEventType; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskSuccessLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.task.statemachine.ITaskStateAction; import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; +import org.apache.dolphinscheduler.task.executor.ITaskExecutorLifecycleEventReporter; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorLifecycleEventType; +import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; @Component public class TaskSuccessLifecycleEventHandler extends AbstractTaskLifecycleEventHandler { + @Autowired + private TaskExecutorClient taskExecutorClient; + @Override public void handle(final ITaskStateAction taskStateAction, final IWorkflowExecutionRunnable workflowExecutionRunnable, final ITaskExecutionRunnable taskExecutionRunnable, final TaskSuccessLifecycleEvent taskSuccessEvent) { taskStateAction.succeedEventAction(workflowExecutionRunnable, taskExecutionRunnable, taskSuccessEvent); + taskExecutorClient.ackTaskExecutorLifecycleEvent( + taskExecutionRunnable, + new ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck( + taskExecutionRunnable.getId(), + TaskExecutorLifecycleEventType.SUCCESS)); } @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/ITaskExecutionRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/ITaskExecutionRunnable.java index ac01b60873645..d0e0ad8d9d15f 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/ITaskExecutionRunnable.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/ITaskExecutionRunnable.java @@ -32,6 +32,14 @@ public interface ITaskExecutionRunnable extends Comparable { + /** + * Get the task instance id. + *

Need to know the id might change since the task instance might be regenerated. + */ + default int getId() { + return getTaskInstance().getId(); + } + default String getName() { return getTaskDefinition().getName(); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskExecutionRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskExecutionRunnable.java index 1c364a7e224f9..9f45490a72e80 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskExecutionRunnable.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskExecutionRunnable.java @@ -25,21 +25,16 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition; import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; -import org.apache.dolphinscheduler.extract.base.client.Clients; -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.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils; -import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.engine.WorkflowEventBus; import org.apache.dolphinscheduler.server.master.engine.graph.IWorkflowExecutionGraph; +import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKillLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskPauseLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskStartLifecycleEvent; import org.apache.dolphinscheduler.server.master.runner.TaskExecutionContextFactory; -import org.apache.commons.lang3.StringUtils; +import javax.annotation.Nullable; import lombok.Getter; import lombok.extern.slf4j.Slf4j; @@ -62,7 +57,7 @@ public class TaskExecutionRunnable implements ITaskExecutionRunnable { @Getter private final WorkflowInstance workflowInstance; @Getter - private TaskInstance taskInstance; + private @Nullable TaskInstance taskInstance; @Getter private final TaskDefinition taskDefinition; @Getter @@ -159,24 +154,8 @@ private void initializeTaskExecutionContext() { private boolean takeOverTaskFromExecutor() { checkState(isTaskInstanceInitialized(), "The task instance is null, can't take over from executor."); - if (TaskTypeUtils.isLogicTask(taskInstance.getTaskType())) { - return false; - } - if (StringUtils.isEmpty(taskInstance.getHost())) { - log.debug("Task: {} host is empty, cannot take over the task from executor(This is normal case).", - taskInstance.getName()); - return false; - } try { - final TakeOverTaskRequest takeOverTaskRequest = TakeOverTaskRequest.builder() - .taskInstanceId(taskInstance.getId()) - .workflowHost(applicationContext.getBean(MasterConfig.class).getMasterAddress()) - .build(); - final TakeOverTaskResponse takeOverTaskResponse = Clients - .withService(ITaskInstanceOperator.class) - .withHost(taskInstance.getHost()) - .takeOverTask(takeOverTaskRequest); - return takeOverTaskResponse.isSuccess(); + return applicationContext.getBean(ITaskExecutorClient.class).reassignWorkflowInstanceHost(this); } catch (Exception ex) { log.warn("Take over task: {} failed", taskInstance.getName(), ex); return false; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskExecutionRunnableBuilder.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskExecutionRunnableBuilder.java index 24d1dbd04e762..ad213de495cfe 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskExecutionRunnableBuilder.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskExecutionRunnableBuilder.java @@ -25,6 +25,8 @@ import org.apache.dolphinscheduler.server.master.engine.WorkflowEventBus; import org.apache.dolphinscheduler.server.master.engine.graph.IWorkflowExecutionGraph; +import javax.annotation.Nullable; + import lombok.AllArgsConstructor; import lombok.Builder; import lombok.Getter; @@ -41,7 +43,7 @@ public class TaskExecutionRunnableBuilder { private final Project project; private final WorkflowInstance workflowInstance; private final TaskDefinition taskDefinition; - private final TaskInstance taskInstance; + private final @Nullable TaskInstance taskInstance; private final WorkflowEventBus workflowEventBus; private final ApplicationContext applicationContext; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java index 1de71817460c1..867a6edeece3e 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/AbstractTaskStateAction.java @@ -38,6 +38,7 @@ import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskPausedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskRetryLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskRunningLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskRuntimeContextChangedEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskSuccessLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskInstanceFactories; @@ -110,15 +111,23 @@ public void dispatchedEventAction(final IWorkflowExecutionRunnable workflowExecu taskInstanceDao.updateById(taskInstance); } + @Override + public void runtimeContextChangedEventAction(final IWorkflowExecutionRunnable workflowExecutionRunnable, + final ITaskExecutionRunnable taskExecutionRunnable, + final TaskRuntimeContextChangedEvent taskRuntimeContextChangedEvent) { + final TaskInstance taskInstance = taskExecutionRunnable.getTaskInstance(); + if (StringUtils.isNotEmpty(taskRuntimeContextChangedEvent.getRuntimeContext())) { + taskInstance.setAppLink(taskRuntimeContextChangedEvent.getRuntimeContext()); + } + taskInstanceDao.updateById(taskInstance); + } + protected void persistentTaskInstanceStartedEventToDB(final ITaskExecutionRunnable taskExecutionRunnable, final TaskRunningLifecycleEvent taskRunningEvent) { final TaskInstance taskInstance = taskExecutionRunnable.getTaskInstance(); taskInstance.setState(TaskExecutionStatus.RUNNING_EXECUTION); taskInstance.setStartTime(taskRunningEvent.getStartTime()); taskInstance.setLogPath(taskRunningEvent.getLogPath()); - if (StringUtils.isNotEmpty(taskRunningEvent.getRuntimeContext())) { - taskInstance.setAppLink(taskRunningEvent.getRuntimeContext()); - } taskInstanceDao.updateById(taskInstance); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java index 432c7d6d7aa13..ac4debfdbeb13 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/ITaskStateAction.java @@ -28,6 +28,7 @@ import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskPausedLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskRetryLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskRunningLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskRuntimeContextChangedEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskStartLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskSuccessLifecycleEvent; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; @@ -66,6 +67,14 @@ void startedEventAction(final IWorkflowExecutionRunnable workflowExecutionRunnab final ITaskExecutionRunnable taskExecutionRunnable, final TaskRunningLifecycleEvent taskRunningEvent); + /** + * Perform the necessary actions when the task in a certain state receive a {@link TaskRuntimeContextChangedEvent}. + *

This method is called when the master receive task runtime context changed event from executor. + */ + void runtimeContextChangedEventAction(final IWorkflowExecutionRunnable workflowExecutionRunnable, + final ITaskExecutionRunnable taskExecutionRunnable, + final TaskRuntimeContextChangedEvent taskRuntimeContextChangedEvent); + /** * Perform the necessary actions when the task in a certain state receive a {@link TaskRetryLifecycleEvent}. *

This method is called when the task need to retry. diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/TaskSubmittedStateAction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/TaskSubmittedStateAction.java index aca7b77ffd050..184e5d4c67132 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/TaskSubmittedStateAction.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/statemachine/TaskSubmittedStateAction.java @@ -169,7 +169,7 @@ public void failedEventAction(final IWorkflowExecutionRunnable workflowExecution final ITaskExecutionRunnable taskExecutionRunnable, final TaskFailedLifecycleEvent taskFailedEvent) { throwExceptionIfStateIsNotMatch(taskExecutionRunnable); - logWarningIfCannotDoAction(taskExecutionRunnable, taskFailedEvent); + super.failedEventAction(workflowExecutionRunnable, taskExecutionRunnable, taskFailedEvent); } @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicITaskInstanceDispatchOperationFunction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicITaskInstanceDispatchOperationFunction.java deleted file mode 100644 index 57b7ae3976450..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicITaskInstanceDispatchOperationFunction.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.rpc; - -import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskDispatchRequest; -import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskDispatchResponse; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; -import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutionContextHolder; -import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutor; -import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutorFactoryBuilder; -import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutorThreadPoolManager; -import org.apache.dolphinscheduler.server.master.runner.message.LogicTaskInstanceExecutionEventSenderManager; - -import lombok.extern.slf4j.Slf4j; - -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Component; - -@Slf4j -@Component -public class LogicITaskInstanceDispatchOperationFunction - implements - ITaskInstanceOperationFunction { - - @Autowired - private MasterTaskExecutorFactoryBuilder masterTaskExecutorFactoryBuilder; - - @Autowired - private MasterTaskExecutorThreadPoolManager masterTaskExecutorThreadPool; - - @Autowired - private LogicTaskInstanceExecutionEventSenderManager logicTaskInstanceExecutionEventSenderManager; - - @Override - public LogicTaskDispatchResponse operate(LogicTaskDispatchRequest taskDispatchRequest) { - log.info("Received dispatchLogicTask request: {}", taskDispatchRequest); - TaskExecutionContext taskExecutionContext = taskDispatchRequest.getTaskExecutionContext(); - try { - final int taskInstanceId = taskExecutionContext.getTaskInstanceId(); - final int workflowInstanceId = taskExecutionContext.getWorkflowInstanceId(); - final String taskInstanceName = taskExecutionContext.getTaskName(); - - taskExecutionContext.setLogPath(LogUtils.getTaskInstanceLogFullPath(taskExecutionContext)); - - LogUtils.setWorkflowAndTaskInstanceIDMDC(workflowInstanceId, taskInstanceId); - LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath()); - - MasterTaskExecutionContextHolder.putTaskExecutionContext(taskExecutionContext); - - final MasterTaskExecutor masterTaskExecutor = masterTaskExecutorFactoryBuilder - .createMasterTaskExecutorFactory(taskExecutionContext.getTaskType()) - .createMasterTaskExecutor(taskExecutionContext); - if (masterTaskExecutorThreadPool.submitMasterTaskExecutor(masterTaskExecutor)) { - log.info("Submit LogicTask: {} to MasterTaskExecutorThreadPool success", taskInstanceName); - return LogicTaskDispatchResponse.success(taskInstanceId); - } else { - log.error("Submit LogicTask: {} to MasterTaskExecutorThreadPool failed", taskInstanceName); - return LogicTaskDispatchResponse.failed(taskInstanceId, "MasterTaskExecutorThreadPool is full"); - } - } finally { - LogUtils.removeWorkflowAndTaskInstanceIdMDC(); - LogUtils.removeTaskInstanceLogFullPathMDC(); - } - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicITaskInstanceKillOperationFunction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicITaskInstanceKillOperationFunction.java deleted file mode 100644 index 59e18993da6a5..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicITaskInstanceKillOperationFunction.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.master.rpc; - -import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskKillRequest; -import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskKillResponse; -import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; -import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; -import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutor; -import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutorHolder; -import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutorThreadPoolManager; - -import lombok.extern.slf4j.Slf4j; - -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Component; - -@Slf4j -@Component -public class LogicITaskInstanceKillOperationFunction - implements - ITaskInstanceOperationFunction { - - @Autowired - private MasterTaskExecutorThreadPoolManager masterTaskExecutorThreadPool; - - @Override - public LogicTaskKillResponse operate(LogicTaskKillRequest taskKillRequest) { - final int taskInstanceId = taskKillRequest.getTaskInstanceId(); - try { - LogUtils.setTaskInstanceIdMDC(taskKillRequest.getTaskInstanceId()); - log.info("Received killLogicTask request: {}", taskKillRequest); - final MasterTaskExecutor masterTaskExecutor = - MasterTaskExecutorHolder.getMasterTaskExecutor(taskInstanceId); - if (masterTaskExecutor == null) { - log.error("Cannot find the MasterTaskExecuteRunnable, this task may already been killed"); - return LogicTaskKillResponse.fail("Cannot find the MasterTaskExecuteRunnable"); - } - try { - masterTaskExecutor.cancelTask(); - return LogicTaskKillResponse.success(); - } catch (MasterTaskExecuteException e) { - log.error("Cancel MasterTaskExecuteRunnable failed ", e); - return LogicTaskKillResponse.fail("Cancel MasterTaskExecuteRunnable failed: " + e.getMessage()); - } - } finally { - LogUtils.removeTaskInstanceIdMDC(); - } - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicITaskInstancePauseOperationFunction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicITaskInstancePauseOperationFunction.java deleted file mode 100644 index 12b45e6fe39af..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicITaskInstancePauseOperationFunction.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.rpc; - -import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskPauseRequest; -import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskPauseResponse; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; -import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; -import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutor; -import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutorHolder; - -import lombok.extern.slf4j.Slf4j; - -import org.springframework.stereotype.Component; - -@Slf4j -@Component -public class LogicITaskInstancePauseOperationFunction - implements - ITaskInstanceOperationFunction { - - @Override - public LogicTaskPauseResponse operate(LogicTaskPauseRequest taskPauseRequest) { - try { - LogUtils.setTaskInstanceIdMDC(taskPauseRequest.getTaskInstanceId()); - final MasterTaskExecutor masterTaskExecutor = - MasterTaskExecutorHolder.getMasterTaskExecutor(taskPauseRequest.getTaskInstanceId()); - if (masterTaskExecutor == null) { - log.info("Cannot find the MasterTaskExecuteRunnable"); - return LogicTaskPauseResponse.fail("Cannot find the MasterTaskExecuteRunnable"); - } - final TaskExecutionContext taskExecutionContext = masterTaskExecutor.getTaskExecutionContext(); - LogUtils.setWorkflowAndTaskInstanceIDMDC(taskExecutionContext.getWorkflowInstanceId(), - taskExecutionContext.getTaskInstanceId()); - masterTaskExecutor.pauseTask(); - return LogicTaskPauseResponse.success(); - } catch (MasterTaskExecuteException e) { - log.error("Pause MasterTaskExecuteRunnable failed", e); - return LogicTaskPauseResponse.fail("Pause MasterTaskExecuteRunnable failed: " + e.getMessage()); - } finally { - LogUtils.removeTaskInstanceIdMDC(); - LogUtils.removeWorkflowAndTaskInstanceIdMDC(); - } - } - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicTaskExecutorOperatorImpl.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicTaskExecutorOperatorImpl.java new file mode 100644 index 0000000000000..ee4194574813b --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicTaskExecutorOperatorImpl.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.rpc; + +import org.apache.dolphinscheduler.extract.master.ILogicTaskExecutorOperator; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.engine.executor.LogicTaskEngine; +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.commons.lang3.exception.ExceptionUtils; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Service; + +@Slf4j +@Service +public class LogicTaskExecutorOperatorImpl implements ILogicTaskExecutorOperator { + + @Autowired + private LogicTaskEngine logicTaskEngine; + + @Override + public TaskExecutorDispatchResponse dispatchTask(final TaskExecutorDispatchRequest taskExecutorDispatchRequest) { + log.info("Receive TaskExecutorDispatchRequest: {}", taskExecutorDispatchRequest); + final TaskExecutionContext taskExecutionContext = taskExecutorDispatchRequest.getTaskExecutionContext(); + try { + logicTaskEngine.dispatchLogicTask(taskExecutionContext); + log.info("Handle TaskExecutorDispatchRequest: {} success", taskExecutorDispatchRequest); + return TaskExecutorDispatchResponse.success(); + } catch (Throwable throwable) { + log.error("Handle TaskExecutorDispatchRequest: {} failed", taskExecutorDispatchRequest, throwable); + return TaskExecutorDispatchResponse.failed(ExceptionUtils.getMessage(throwable)); + } + } + + @Override + public TaskExecutorPauseResponse pauseTask(final TaskExecutorPauseRequest taskPauseRequest) { + log.info("Receive TaskExecutorPauseRequest: {}", taskPauseRequest); + final int taskInstanceId = taskPauseRequest.getTaskInstanceId(); + try { + logicTaskEngine.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 void ackTaskExecutorLifecycleEvent(final ITaskExecutorLifecycleEventReporter.TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck) { + log.info("Receive TaskExecutorLifecycleEventAck: {}", taskExecutorLifecycleEventAck); + logicTaskEngine.ackLogicTaskExecutionEvent(taskExecutorLifecycleEventAck); + } + + @Override + public TaskExecutorKillResponse killTask(final TaskExecutorKillRequest taskKillRequest) { + log.info("Receive TaskExecutorKillRequest: {}", taskKillRequest); + final int taskInstanceId = taskKillRequest.getTaskInstanceId(); + try { + logicTaskEngine.killLogicTask(taskInstanceId); + log.info("Handle TaskExecutorKillRequest: {} success", taskKillRequest); + return TaskExecutorKillResponse.success(); + } catch (Throwable throwable) { + log.error("Handle TaskExecutorKillRequest: {} failed", taskKillRequest, throwable); + return TaskExecutorKillResponse.fail(ExceptionUtils.getMessage(throwable)); + } + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicTaskInstanceOperatorImpl.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicTaskInstanceOperatorImpl.java deleted file mode 100644 index 39446bd89d267..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/LogicTaskInstanceOperatorImpl.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.master.rpc; - -import org.apache.dolphinscheduler.extract.master.ILogicTaskInstanceOperator; -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; - -import lombok.extern.slf4j.Slf4j; - -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Service; - -@Slf4j -@Service -public class LogicTaskInstanceOperatorImpl implements ILogicTaskInstanceOperator { - - @Autowired - private LogicTaskInstanceOperationFunctionManager logicTaskInstanceOperationFunctionManager; - - @Override - public LogicTaskDispatchResponse dispatchLogicTask(LogicTaskDispatchRequest taskDispatchRequest) { - return logicTaskInstanceOperationFunctionManager.getLogicTaskInstanceDispatchOperationFunction() - .operate(taskDispatchRequest); - } - - @Override - public LogicTaskKillResponse killLogicTask(LogicTaskKillRequest taskKillRequest) { - return logicTaskInstanceOperationFunctionManager.getLogicTaskInstanceKillOperationFunction() - .operate(taskKillRequest); - } - - @Override - public LogicTaskPauseResponse pauseLogicTask(LogicTaskPauseRequest taskPauseRequest) { - return logicTaskInstanceOperationFunctionManager.getLogicTaskInstancePauseOperationFunction() - .operate(taskPauseRequest); - } - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterLogServiceImpl.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterLogServiceImpl.java index 0fa7cbffd905b..98e7461f50d2d 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterLogServiceImpl.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterLogServiceImpl.java @@ -20,14 +20,11 @@ import org.apache.dolphinscheduler.common.utils.FileUtils; import org.apache.dolphinscheduler.common.utils.LogUtils; 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 java.util.Collections; import java.util.List; import lombok.extern.slf4j.Slf4j; @@ -58,11 +55,6 @@ public TaskInstanceLogPageQueryResponse pageQueryTaskInstanceLog(TaskInstanceLog return new TaskInstanceLogPageQueryResponse(logContent); } - @Override - public GetAppIdResponse getAppId(GetAppIdRequest getAppIdRequest) { - return new GetAppIdResponse(Collections.emptyList()); - } - @Override public void removeTaskInstanceLog(String taskInstanceLogAbsolutePath) { FileUtils.deleteFile(taskInstanceLogAbsolutePath); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/TaskExecutionEventListenerImpl.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/TaskExecutionEventListenerImpl.java deleted file mode 100644 index b788337033d27..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/TaskExecutionEventListenerImpl.java +++ /dev/null @@ -1,210 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.rpc; - -import org.apache.dolphinscheduler.extract.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.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; -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.TaskTypeUtils; -import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository; -import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskDispatchedLifecycleEvent; -import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; -import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKilledLifecycleEvent; -import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskPausedLifecycleEvent; -import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskRunningLifecycleEvent; -import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskSuccessLifecycleEvent; -import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; -import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; - -import java.util.Date; - -import lombok.extern.slf4j.Slf4j; - -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Service; - -@Slf4j -@Service -public class TaskExecutionEventListenerImpl implements ITaskExecutionEventListener { - - @Autowired - private IWorkflowRepository IWorkflowRepository; - - @Override - public void onTaskInstanceDispatched(TaskExecutionDispatchEvent taskExecutionDispatchEvent) { - final ITaskExecutionRunnable taskExecutionRunnable = getTaskExecutionRunnable(taskExecutionDispatchEvent); - final TaskDispatchedLifecycleEvent taskDispatchedLifecycleEvent = TaskDispatchedLifecycleEvent.builder() - .taskExecutionRunnable(taskExecutionRunnable) - .executorHost(taskExecutionDispatchEvent.getTaskInstanceHost()) - .build(); - - taskExecutionRunnable.getWorkflowEventBus().publish(taskDispatchedLifecycleEvent); - // Once the master receive the event, then will send ack to the worker - // This means the worker will not retry to send the event to the master - // So once the master failover and we take over the task instance success, then we should fetch the latest task - // instance state. - // The logic task doesn't need to send ack - if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) { - Clients - .withService(ITaskInstanceExecutionEventAckListener.class) - .withHost(taskExecutionDispatchEvent.getTaskInstanceHost()) - .handleTaskInstanceDispatchedEventAck( - TaskInstanceExecutionDispatchedEventAck - .success(taskExecutionDispatchEvent.getTaskInstanceId())); - } - } - - @Override - public void onTaskInstanceExecutionRunning(TaskExecutionRunningEvent taskExecutionRunningEvent) { - final ITaskExecutionRunnable taskExecutionRunnable = getTaskExecutionRunnable(taskExecutionRunningEvent); - final TaskRunningLifecycleEvent taskRunningEvent = TaskRunningLifecycleEvent.builder() - .taskExecutionRunnable(taskExecutionRunnable) - .startTime(new Date(taskExecutionRunningEvent.getStartTime())) - .logPath(taskExecutionRunningEvent.getLogPath()) - .runtimeContext(taskExecutionRunningEvent.getAppIds()) - .build(); - - taskExecutionRunnable.getWorkflowEventBus().publish(taskRunningEvent); - // Once the master receive the event, then will send ack to the worker - // This means the worker will not retry to send the event to the master - // So once the master failover and we take over the task instance success, then we should fetch the latest task - // instance state. - // The logic task doesn't need to send ack - if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) { - Clients - .withService(ITaskInstanceExecutionEventAckListener.class) - .withHost(taskExecutionRunningEvent.getTaskInstanceHost()) - .handleTaskInstanceExecutionRunningEventAck( - TaskInstanceExecutionRunningEventAck - .success(taskExecutionRunningEvent.getTaskInstanceId())); - } - } - - @Override - public void onTaskInstanceExecutionSuccess(final TaskExecutionSuccessEvent taskExecutionSuccessEvent) { - final ITaskExecutionRunnable taskExecutionRunnable = getTaskExecutionRunnable(taskExecutionSuccessEvent); - final TaskSuccessLifecycleEvent taskSuccessEvent = TaskSuccessLifecycleEvent.builder() - .taskExecutionRunnable(taskExecutionRunnable) - .endTime(new Date(taskExecutionSuccessEvent.getEndTime())) - .varPool(taskExecutionSuccessEvent.getVarPool()) - .build(); - taskExecutionRunnable.getWorkflowEventBus().publish(taskSuccessEvent); - // Once the master receive the event, then will send ack to the worker - // This means the worker will not retry to send the event to the master - // So once the master failover and we take over the task instance success, then we should fetch the latest task - // instance state. - if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) { - Clients - .withService(ITaskInstanceExecutionEventAckListener.class) - .withHost(taskExecutionSuccessEvent.getTaskInstanceHost()) - .handleTaskExecutionSuccessEventAck( - TaskExecutionSuccessEventAck.success(taskExecutionSuccessEvent.getTaskInstanceId())); - } - } - - @Override - public void onTaskInstanceExecutionFailed(TaskExecutionFailedEvent taskExecutionFailedEvent) { - final ITaskExecutionRunnable taskExecutionRunnable = getTaskExecutionRunnable(taskExecutionFailedEvent); - final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder() - .taskExecutionRunnable(taskExecutionRunnable) - .endTime(new Date(taskExecutionFailedEvent.getEndTime())) - .build(); - taskExecutionRunnable.getWorkflowEventBus().publish(taskFailedEvent); - // Once the master receive the event, then will send ack to the worker - // This means the worker will not retry to send the event to the master - // So once the master failover and we take over the task instance success, then we should fetch the latest task - // instance state. - if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) { - Clients - .withService(ITaskInstanceExecutionEventAckListener.class) - .withHost(taskExecutionFailedEvent.getTaskInstanceHost()) - .handleTaskExecutionFailedEventAck( - TaskExecutionFailedEventAck.success(taskExecutionFailedEvent.getTaskInstanceId())); - } - } - - @Override - public void onTaskInstanceExecutionKilled(TaskExecutionKilledEvent taskExecutionKilledEvent) { - final ITaskExecutionRunnable taskExecutionRunnable = getTaskExecutionRunnable(taskExecutionKilledEvent); - final TaskKilledLifecycleEvent taskKilledEvent = TaskKilledLifecycleEvent.builder() - .taskExecutionRunnable(taskExecutionRunnable) - .endTime(new Date(taskExecutionKilledEvent.getEndTime())) - .build(); - taskExecutionRunnable.getWorkflowEventBus().publish(taskKilledEvent); - // Once the master receive the event, then will send ack to the worker - // This means the worker will not retry to send the event to the master - // So once the master failover and we take over the task instance success, then we should fetch the latest task - // instance state. - if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) { - Clients - .withService(ITaskInstanceExecutionEventAckListener.class) - .withHost(taskExecutionKilledEvent.getTaskInstanceHost()) - .handleTaskExecutionKilledEventAck( - TaskExecutionKilledEventAck.success(taskExecutionKilledEvent.getTaskInstanceId())); - } - } - - @Override - public void onTaskInstanceExecutionPaused(TaskExecutionPausedEvent taskExecutionPausedEvent) { - final ITaskExecutionRunnable taskExecutionRunnable = getTaskExecutionRunnable(taskExecutionPausedEvent); - final TaskPausedLifecycleEvent taskPausedEvent = TaskPausedLifecycleEvent.of(taskExecutionRunnable); - taskExecutionRunnable.getWorkflowEventBus().publish(taskPausedEvent); - // Once the master receive the event, then will send ack to the worker - // This means the worker will not retry to send the event to the master - // So once the master failover and we take over the task instance success, then we should fetch the latest task - // instance state. - if (!TaskTypeUtils.isLogicTask(taskExecutionRunnable.getTaskDefinition().getTaskType())) { - Clients - .withService(ITaskInstanceExecutionEventAckListener.class) - .withHost(taskExecutionPausedEvent.getTaskInstanceHost()) - .handleTaskExecutionPausedEventAck( - TaskExecutionPausedEventAck.success(taskExecutionPausedEvent.getTaskInstanceId())); - } - } - - private ITaskExecutionRunnable getTaskExecutionRunnable(final ITaskExecutionEvent taskExecutionEvent) { - final int workflowInstanceId = taskExecutionEvent.getWorkflowInstanceId(); - final int taskInstanceId = taskExecutionEvent.getTaskInstanceId(); - - final IWorkflowExecutionRunnable workflowExecutionRunnable = IWorkflowRepository.get(workflowInstanceId); - if (workflowExecutionRunnable == null) { - throw new IllegalArgumentException("Cannot find the WorkflowExecuteRunnable: " + workflowInstanceId); - } - final ITaskExecutionRunnable taskExecutionRunnable = workflowExecutionRunnable.getWorkflowExecuteContext() - .getWorkflowExecutionGraph() - .getTaskExecutionRunnableById(taskInstanceId); - if (taskExecutionRunnable == null) { - throw new IllegalArgumentException("Cannot find the TaskExecuteRunnable: " + taskInstanceId); - } - return taskExecutionRunnable; - } - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/TaskExecutorEventListenerImpl.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/TaskExecutorEventListenerImpl.java new file mode 100644 index 0000000000000..202e45834352d --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/TaskExecutorEventListenerImpl.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.rpc; + +import org.apache.dolphinscheduler.extract.master.ITaskExecutorEventListener; +import org.apache.dolphinscheduler.server.master.engine.IWorkflowRepository; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskDispatchedLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskKilledLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskPausedLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskRunningLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskRuntimeContextChangedEvent; +import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskSuccessLifecycleEvent; +import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; +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 java.util.Date; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Service; + +@Slf4j +@Service +public class TaskExecutorEventListenerImpl implements ITaskExecutorEventListener { + + @Autowired + private IWorkflowRepository workflowRepository; + + @Override + public void onTaskExecutorDispatched(final TaskExecutorDispatchedLifecycleEvent taskExecutorDispatchedLifecycleEvent) { + final ITaskExecutionRunnable taskExecutionRunnable = + getTaskExecutionRunnable(taskExecutorDispatchedLifecycleEvent); + final TaskDispatchedLifecycleEvent taskDispatchedLifecycleEvent = TaskDispatchedLifecycleEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .executorHost(taskExecutorDispatchedLifecycleEvent.getTaskInstanceHost()) + .build(); + + taskExecutionRunnable.getWorkflowEventBus().publish(taskDispatchedLifecycleEvent); + } + + @Override + public void onTaskExecutorRunning(final TaskExecutorStartedLifecycleEvent taskExecutorStartedLifecycleEvent) { + final ITaskExecutionRunnable taskExecutionRunnable = + getTaskExecutionRunnable(taskExecutorStartedLifecycleEvent); + final TaskRunningLifecycleEvent taskRunningEvent = TaskRunningLifecycleEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .startTime(new Date(taskExecutorStartedLifecycleEvent.getStartTime())) + .logPath(taskExecutorStartedLifecycleEvent.getLogPath()) + .build(); + + taskExecutionRunnable.getWorkflowEventBus().publish(taskRunningEvent); + } + + @Override + public void onTaskExecutorRuntimeContextChanged(final TaskExecutorRuntimeContextChangedLifecycleEvent taskExecutorRuntimeContextChangedLifecycleEventr) { + final ITaskExecutionRunnable taskExecutionRunnable = + getTaskExecutionRunnable(taskExecutorRuntimeContextChangedLifecycleEventr); + + final TaskRuntimeContextChangedEvent taskRuntimeContextChangedEvent = TaskRuntimeContextChangedEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .runtimeContext(taskExecutorRuntimeContextChangedLifecycleEventr.getAppIds()) + .build(); + + taskExecutionRunnable.getWorkflowEventBus().publish(taskRuntimeContextChangedEvent); + } + + @Override + public void onTaskExecutorSuccess(final TaskExecutorSuccessLifecycleEvent taskExecutorSuccessLifecycleEvent) { + final ITaskExecutionRunnable taskExecutionRunnable = + getTaskExecutionRunnable(taskExecutorSuccessLifecycleEvent); + final TaskSuccessLifecycleEvent taskSuccessEvent = TaskSuccessLifecycleEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .endTime(new Date(taskExecutorSuccessLifecycleEvent.getEndTime())) + .varPool(taskExecutorSuccessLifecycleEvent.getVarPool()) + .build(); + taskExecutionRunnable.getWorkflowEventBus().publish(taskSuccessEvent); + } + + @Override + public void onTaskExecutorFailed(final TaskExecutorFailedLifecycleEvent taskExecutorFailedLifecycleEvent) { + final ITaskExecutionRunnable taskExecutionRunnable = getTaskExecutionRunnable(taskExecutorFailedLifecycleEvent); + final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .endTime(new Date(taskExecutorFailedLifecycleEvent.getEndTime())) + .build(); + taskExecutionRunnable.getWorkflowEventBus().publish(taskFailedEvent); + } + + @Override + public void onTaskExecutorKilled(final TaskExecutorKilledLifecycleEvent taskExecutorKilledLifecycleEvent) { + final ITaskExecutionRunnable taskExecutionRunnable = getTaskExecutionRunnable(taskExecutorKilledLifecycleEvent); + final TaskKilledLifecycleEvent taskKilledEvent = TaskKilledLifecycleEvent.builder() + .taskExecutionRunnable(taskExecutionRunnable) + .endTime(new Date(taskExecutorKilledLifecycleEvent.getEndTime())) + .build(); + taskExecutionRunnable.getWorkflowEventBus().publish(taskKilledEvent); + } + + @Override + public void onTaskExecutorPaused(final TaskExecutorPausedLifecycleEvent taskExecutorPausedLifecycleEvent) { + final ITaskExecutionRunnable taskExecutionRunnable = getTaskExecutionRunnable(taskExecutorPausedLifecycleEvent); + final TaskPausedLifecycleEvent taskPausedEvent = TaskPausedLifecycleEvent.of(taskExecutionRunnable); + taskExecutionRunnable.getWorkflowEventBus().publish(taskPausedEvent); + } + + private ITaskExecutionRunnable getTaskExecutionRunnable(final IReportableTaskExecutorLifecycleEvent reportableTaskExecutorLifecycleEvent) { + final int workflowInstanceId = reportableTaskExecutorLifecycleEvent.getWorkflowInstanceId(); + final int taskInstanceId = reportableTaskExecutorLifecycleEvent.getTaskInstanceId(); + + final IWorkflowExecutionRunnable workflowExecutionRunnable = workflowRepository.get(workflowInstanceId); + if (workflowExecutionRunnable == null) { + throw new IllegalArgumentException("Cannot find the WorkflowExecuteRunnable: " + workflowInstanceId); + } + final ITaskExecutionRunnable taskExecutionRunnable = workflowExecutionRunnable.getWorkflowExecuteContext() + .getWorkflowExecutionGraph() + .getTaskExecutionRunnableById(taskInstanceId); + if (taskExecutionRunnable == null) { + throw new IllegalArgumentException("Cannot find the TaskExecuteRunnable: " + taskInstanceId); + } + return taskExecutionRunnable; + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooper.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooper.java index e2657b697d2d5..23883da8b64ee 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooper.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooper.java @@ -20,9 +20,8 @@ import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; +import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; -import org.apache.dolphinscheduler.server.master.runner.dispatcher.TaskDispatchFactory; -import org.apache.dolphinscheduler.server.master.runner.dispatcher.TaskDispatcher; import java.util.concurrent.atomic.AtomicBoolean; @@ -39,7 +38,7 @@ public class GlobalTaskDispatchWaitingQueueLooper extends BaseDaemonThread imple private GlobalTaskDispatchWaitingQueue globalTaskDispatchWaitingQueue; @Autowired - private TaskDispatchFactory taskDispatchFactory; + private ITaskExecutorClient taskExecutorClient; private final AtomicBoolean RUNNING_FLAG = new AtomicBoolean(false); @@ -74,8 +73,7 @@ void doDispatch() { log.warn("The TaskInstance {} state is : {}, will not dispatch", taskInstance.getName(), status); return; } - final TaskDispatcher taskDispatcher = taskDispatchFactory.getTaskDispatcher(taskInstance); - taskDispatcher.dispatchTask(taskExecutionRunnable); + taskExecutorClient.dispatch(taskExecutionRunnable); } catch (Exception e) { // If dispatch failed, will put the task back to the queue // The task will be dispatched after waiting time. diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecutorBootstrap.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecutorBootstrap.java deleted file mode 100644 index 7d3bf6940b4b9..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecutorBootstrap.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.master.runner; - -import org.apache.dolphinscheduler.server.master.runner.execute.AsyncMasterTaskDelayQueueLooper; - -import lombok.extern.slf4j.Slf4j; - -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Component; - -@Slf4j -@Component -public class MasterTaskExecutorBootstrap implements AutoCloseable { - - @Autowired - private GlobalTaskDispatchWaitingQueueLooper globalTaskDispatchWaitingQueueLooper; - - @Autowired - private AsyncMasterTaskDelayQueueLooper asyncMasterTaskDelayQueueLooper; - - public synchronized void start() { - log.info("MasterTaskExecutorBootstrap starting..."); - globalTaskDispatchWaitingQueueLooper.start(); - asyncMasterTaskDelayQueueLooper.start(); - log.info("MasterTaskExecutorBootstrap started..."); - } - - @Override - public void close() throws Exception { - log.info("MasterTaskExecutorBootstrap closing..."); - try ( - final GlobalTaskDispatchWaitingQueueLooper globalTaskDispatchWaitingQueueLooper1 = - globalTaskDispatchWaitingQueueLooper; - final AsyncMasterTaskDelayQueueLooper asyncMasterTaskDelayQueueLooper1 = - asyncMasterTaskDelayQueueLooper) { - // closed the resource - } - log.info("MasterTaskExecutorBootstrap closed..."); - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecuteRunnableFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecuteRunnableFactory.java deleted file mode 100644 index 4a97213c62704..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecuteRunnableFactory.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner; - -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; -import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; -import org.apache.dolphinscheduler.server.master.exception.TaskExecuteRunnableCreateException; - -/** - * Use to create TaskExecuteRunnable - * - * @param TaskExecuteRunnable - */ -public interface TaskExecuteRunnableFactory { - - /** - * Create a task execution runnable which related to the task code - * - * @param workflowExecuteRunnable - * @param taskCode - * @throws TaskExecuteRunnableCreateException - */ - ITaskExecutionRunnable createTaskExecutionRunnable(IWorkflowExecutionRunnable workflowExecuteRunnable, - Long taskCode); - - T createTaskExecuteRunnable(TaskInstance taskInstance) throws TaskExecuteRunnableCreateException; - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/BaseTaskDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/BaseTaskDispatcher.java deleted file mode 100644 index 262b302d9b6f5..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/BaseTaskDispatcher.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.master.runner.dispatcher; - -import org.apache.dolphinscheduler.extract.base.utils.Host; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; -import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; -import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException; - -import java.util.Optional; - -import lombok.extern.slf4j.Slf4j; - -@Slf4j -public abstract class BaseTaskDispatcher implements TaskDispatcher { - - @Override - public void dispatchTask(ITaskExecutionRunnable taskExecutionRunnable) throws TaskDispatchException { - final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext(); - final String taskName = taskExecutionRunnable.getTaskExecutionContext().getTaskName(); - final String taskInstanceDispatchAddress; - try { - taskInstanceDispatchAddress = getTaskInstanceDispatchHost(taskExecutionRunnable) - .map(Host::getAddress) - .orElseThrow(() -> new TaskDispatchException("Cannot find the host to execute task: " + taskName)); - } catch (WorkerGroupNotFoundException workerGroupNotFoundException) { - // todo: this is a temporary solution, we should refactor the ServerNodeManager to make sure there won't - // throw WorkerGroupNotFoundException unless the worker group is not exist in database - throw new TaskDispatchException("Dispatch task: " + taskName + " failed", workerGroupNotFoundException); - } - // We inject the host here to avoid when we dispatched the task to worker, but the worker is crash. - // Then we can use the host to do worker failover. - taskExecutionContext.setHost(taskInstanceDispatchAddress); - taskExecutionRunnable.getTaskInstance().setHost(taskInstanceDispatchAddress); - doDispatch(taskExecutionRunnable); - // todo: update the task state and host here, otherwise when the master failover the task host is null - // but it already dispatched to worker - // Or when the worker receive the task, it should wait the master send a start event to it. - // the second solution is better - log.info("Success dispatch task {} to {}.", taskName, taskInstanceDispatchAddress); - } - - protected abstract void doDispatch(ITaskExecutionRunnable ITaskExecutionRunnable) throws TaskDispatchException; - - protected abstract Optional getTaskInstanceDispatchHost(ITaskExecutionRunnable taskExecutionContext) throws TaskDispatchException, WorkerGroupNotFoundException; - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcher.java deleted file mode 100644 index ae87b6e33eafb..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcher.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.server.master.runner.dispatcher; - -import org.apache.dolphinscheduler.extract.base.utils.Host; -import org.apache.dolphinscheduler.extract.master.ILogicTaskInstanceOperator; -import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskDispatchRequest; -import org.apache.dolphinscheduler.extract.master.transportor.LogicTaskDispatchResponse; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.server.master.config.MasterConfig; -import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; -import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; - -import java.util.Optional; - -import lombok.extern.slf4j.Slf4j; - -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Component; - -@Slf4j -@Component -public class MasterTaskDispatcher extends BaseTaskDispatcher { - - private final Optional masterTaskExecuteHost; - - @Autowired - private ILogicTaskInstanceOperator logicTaskInstanceOperator; - - public MasterTaskDispatcher(MasterConfig masterConfig) { - this.masterTaskExecuteHost = Optional.of(Host.of(masterConfig.getMasterAddress())); - } - - @Override - protected void doDispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskDispatchException { - final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext(); - try { - final LogicTaskDispatchRequest logicTaskDispatchRequest = - new LogicTaskDispatchRequest(taskExecutionContext); - final LogicTaskDispatchResponse logicTaskDispatchResponse = logicTaskInstanceOperator.dispatchLogicTask( - logicTaskDispatchRequest); - if (!logicTaskDispatchResponse.isDispatchSuccess()) { - throw new TaskDispatchException( - String.format("Dispatch LogicTask to %s failed, response is: %s", - taskExecutionContext.getHost(), logicTaskDispatchResponse)); - } - } catch (TaskDispatchException e) { - throw e; - } catch (Exception e) { - throw new TaskDispatchException(String.format("Dispatch task to %s failed", - taskExecutionContext.getHost()), e); - } - } - - @Override - protected Optional getTaskInstanceDispatchHost(ITaskExecutionRunnable taskExecutionContext) { - return masterTaskExecuteHost; - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatcher.java deleted file mode 100644 index 4f612a61abadd..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatcher.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.dispatcher; - -import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; -import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; - -/** - * Used to do task dispatcher. - */ -public interface TaskDispatcher { - - void dispatchTask(ITaskExecutionRunnable ITaskExecutionRunnable) throws TaskDispatchException; - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.java deleted file mode 100644 index 9ff7f75772138..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.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.server.master.runner.dispatcher; - -import static com.google.common.base.Preconditions.checkNotNull; - -import org.apache.dolphinscheduler.extract.base.client.Clients; -import org.apache.dolphinscheduler.extract.base.utils.Host; -import org.apache.dolphinscheduler.extract.worker.ITaskInstanceOperator; -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.server.master.cluster.loadbalancer.IWorkerLoadBalancer; -import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; -import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; - -import java.util.Optional; - -import lombok.extern.slf4j.Slf4j; - -import org.springframework.stereotype.Component; - -@Slf4j -@Component -public class WorkerTaskDispatcher extends BaseTaskDispatcher { - - private final IWorkerLoadBalancer workerLoadBalancer; - - public WorkerTaskDispatcher(IWorkerLoadBalancer workerLoadBalancer) { - this.workerLoadBalancer = checkNotNull(workerLoadBalancer); - } - - @Override - protected void doDispatch(ITaskExecutionRunnable ITaskExecutionRunnable) throws TaskDispatchException { - final TaskExecutionContext taskExecutionContext = ITaskExecutionRunnable.getTaskExecutionContext(); - final String taskName = taskExecutionContext.getTaskName(); - final String workerAddress = taskExecutionContext.getHost(); - try { - final TaskInstanceDispatchResponse taskInstanceDispatchResponse = Clients - .withService(ITaskInstanceOperator.class) - .withHost(workerAddress) - .dispatchTask(new TaskInstanceDispatchRequest(ITaskExecutionRunnable.getTaskExecutionContext())); - if (!taskInstanceDispatchResponse.isDispatchSuccess()) { - throw new TaskDispatchException("Dispatch task: " + taskName + " to " + workerAddress + " failed: " - + taskInstanceDispatchResponse); - } - } catch (TaskDispatchException e) { - throw e; - } catch (Exception e) { - throw new TaskDispatchException("Dispatch task: " + taskName + " to " + workerAddress + " failed", e); - } - } - - @Override - protected Optional getTaskInstanceDispatchHost(ITaskExecutionRunnable ITaskExecutionRunnable) { - String workerGroup = ITaskExecutionRunnable.getTaskExecutionContext().getWorkerGroup(); - return workerLoadBalancer.select(workerGroup).map(Host::of); - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueueLooper.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueueLooper.java deleted file mode 100644 index f797a684fbcb7..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueueLooper.java +++ /dev/null @@ -1,133 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.execute; - -import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; - -import java.util.concurrent.atomic.AtomicBoolean; - -import lombok.extern.slf4j.Slf4j; - -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Component; - -@Slf4j -@Component -public class AsyncMasterTaskDelayQueueLooper extends BaseDaemonThread implements AutoCloseable { - - @Autowired - private AsyncMasterTaskDelayQueue asyncMasterTaskDelayQueue; - - @Autowired - private MasterAsyncTaskExecutorThreadPool masterAsyncTaskExecutorThreadPool; - private static final AtomicBoolean RUNNING_FLAG = new AtomicBoolean(false); - - public AsyncMasterTaskDelayQueueLooper() { - super("AsyncMasterTaskDelayQueueLooper"); - } - - @Override - public synchronized void start() { - if (!RUNNING_FLAG.compareAndSet(false, true)) { - log.info("The AsyncMasterTaskDelayQueueLooper has already been started, will not start again"); - return; - } - - log.info("AsyncMasterTaskDelayQueueLooper starting..."); - super.start(); - log.info("AsyncMasterTaskDelayQueueLooper started..."); - } - - @Override - public void run() { - while (RUNNING_FLAG.get()) { - AsyncTaskExecutionContext asyncTaskExecutionContext; - try { - asyncTaskExecutionContext = asyncMasterTaskDelayQueue.pollAsyncTask(); - } catch (InterruptedException e) { - log.error("AsyncConditionTaskLooper has been interrupted, will break this loop", e); - Thread.currentThread().interrupt(); - break; - } - final TaskExecutionContext taskExecutionContext = asyncTaskExecutionContext.getTaskExecutionContext(); - try { - LogUtils.setWorkflowAndTaskInstanceIDMDC(taskExecutionContext.getWorkflowInstanceId(), - taskExecutionContext.getTaskInstanceId()); - LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath()); - - if (MasterTaskExecutionContextHolder - .getTaskExecutionContext(taskExecutionContext.getTaskInstanceId()) == null) { - log.warn( - "Cannot find the taskInstance from TaskExecutionContextCacheManager, the task may already been killed, will stop the async master task"); - continue; - } - masterAsyncTaskExecutorThreadPool.getThreadPool().execute(() -> { - final AsyncTaskExecuteFunction asyncTaskExecuteFunction = - asyncTaskExecutionContext.getAsyncTaskExecuteFunction(); - final AsyncTaskCallbackFunction asyncTaskCallbackFunction = - asyncTaskExecutionContext.getAsyncTaskCallbackFunction(); - try { - LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath()); - LogUtils.setTaskInstanceIdMDC(taskExecutionContext.getTaskInstanceId()); - AsyncTaskExecuteFunction.AsyncTaskExecutionStatus asyncTaskExecutionStatus = - asyncTaskExecuteFunction.getAsyncTaskExecutionStatus(); - switch (asyncTaskExecutionStatus) { - case RUNNING: - // If the task status is running, means the task real status is not finished. We - // will - // put it back to the queue to get the status again. - asyncMasterTaskDelayQueue.addAsyncTask(asyncTaskExecutionContext); - break; - case PAUSE: - asyncTaskCallbackFunction.executePause(); - break; - case KILL: - asyncTaskCallbackFunction.executeKilled(); - break; - case SUCCESS: - asyncTaskCallbackFunction.executeSuccess(); - break; - case FAILED: - asyncTaskCallbackFunction.executeFailed(); - break; - } - } catch (Exception ex) { - asyncTaskCallbackFunction.executeThrowing(ex); - } finally { - LogUtils.removeTaskInstanceLogFullPathMDC(); - LogUtils.removeTaskInstanceIdMDC(); - } - }); - } finally { - LogUtils.removeTaskInstanceLogFullPathMDC(); - LogUtils.removeWorkflowAndTaskInstanceIdMDC(); - } - } - log.info("AsyncMasterTaskDelayQueueLooper closed..."); - } - - @Override - public void close() throws Exception { - if (!RUNNING_FLAG.compareAndSet(true, false)) { - log.warn("The AsyncMasterTaskDelayQueueLooper is not started, will not close"); - return; - } - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskExecutor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskExecutor.java deleted file mode 100644 index d289a318daa3b..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskExecutor.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.execute; - -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; -import org.apache.dolphinscheduler.server.master.runner.message.LogicTaskInstanceExecutionEventSenderManager; -import org.apache.dolphinscheduler.server.master.runner.task.IAsyncLogicTask; -import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder; - -import lombok.extern.slf4j.Slf4j; - -@Slf4j -public class AsyncMasterTaskExecutor extends MasterTaskExecutor { - - private final AsyncMasterTaskDelayQueue asyncMasterTaskDelayQueue; - - public AsyncMasterTaskExecutor(TaskExecutionContext taskExecutionContext, - LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder, - LogicTaskInstanceExecutionEventSenderManager logicTaskInstanceExecutionEventSenderManager, - AsyncMasterTaskDelayQueue asyncTaskDelayQueue) { - super(taskExecutionContext, logicTaskPluginFactoryBuilder, logicTaskInstanceExecutionEventSenderManager); - this.asyncMasterTaskDelayQueue = asyncTaskDelayQueue; - } - - @Override - protected void executeTask() throws MasterTaskExecuteException { - if (logicTask == null) { - throw new MasterTaskExecuteException("The task plugin instance is null"); - } - final IAsyncLogicTask iAsyncLogicTask = (IAsyncLogicTask) logicTask; - // we execute the handle method here, but for async task, this method will not block - // submit the task to async task queue - final AsyncTaskExecutionContext asyncTaskExecutionContext = new AsyncTaskExecutionContext( - taskExecutionContext, - iAsyncLogicTask.getAsyncTaskExecuteFunction(), - new AsyncTaskCallbackFunctionImpl(this)); - asyncMasterTaskDelayQueue.addAsyncTask(asyncTaskExecutionContext); - } - - @Override - protected void afterExecute() { - // Do nothing, async task is not finished yet - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskExecutorFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskExecutorFactory.java deleted file mode 100644 index 6c81cd78626dc..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskExecutorFactory.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.server.master.runner.execute; - -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 org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Component; - -@Component -public class AsyncMasterTaskExecutorFactory - implements - MasterTaskExecutorFactory { - - @Autowired - private LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder; - - @Autowired - private LogicTaskInstanceExecutionEventSenderManager logicTaskInstanceExecutionEventSenderManager; - - @Autowired - private AsyncMasterTaskDelayQueue asyncTaskDelayQueue; - - @Override - public AsyncMasterTaskExecutor createMasterTaskExecutor(TaskExecutionContext taskExecutionContext) { - return new AsyncMasterTaskExecutor(taskExecutionContext, - logicTaskPluginFactoryBuilder, - logicTaskInstanceExecutionEventSenderManager, - asyncTaskDelayQueue); - } - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunction.java deleted file mode 100644 index df9ed8f5b5153..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunction.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.execute; - -import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; - -public interface AsyncTaskCallbackFunction { - - void executeSuccess() throws MasterTaskExecuteException; - - void executeFailed() throws MasterTaskExecuteException; - - void executeThrowing(Throwable throwable); - - void executePause(); - - void executeKilled(); -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunctionImpl.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunctionImpl.java deleted file mode 100644 index ae7a480cd073f..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunctionImpl.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.master.runner.execute; - -import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; -import org.apache.dolphinscheduler.plugin.task.api.log.TaskInstanceLogHeader; - -import lombok.NonNull; -import lombok.extern.slf4j.Slf4j; - -@Slf4j -public class AsyncTaskCallbackFunctionImpl implements AsyncTaskCallbackFunction { - - private final AsyncMasterTaskExecutor asyncMasterTaskExecuteRunnable; - - public AsyncTaskCallbackFunctionImpl(@NonNull AsyncMasterTaskExecutor asyncMasterTaskExecuteRunnable) { - this.asyncMasterTaskExecuteRunnable = asyncMasterTaskExecuteRunnable; - } - - @Override - public void executeSuccess() { - asyncMasterTaskExecuteRunnable.getTaskExecutionContext() - .setCurrentExecutionStatus(TaskExecutionStatus.SUCCESS); - executeFinished(); - } - - @Override - public void executeFailed() { - asyncMasterTaskExecuteRunnable.getTaskExecutionContext() - .setCurrentExecutionStatus(TaskExecutionStatus.FAILURE); - executeFinished(); - } - - @Override - public void executeThrowing(Throwable throwable) { - asyncMasterTaskExecuteRunnable.afterThrowing(throwable); - } - - @Override - public void executePause() { - asyncMasterTaskExecuteRunnable.getTaskExecutionContext() - .setCurrentExecutionStatus(TaskExecutionStatus.PAUSE); - executeFinished(); - } - - @Override - public void executeKilled() { - asyncMasterTaskExecuteRunnable.getTaskExecutionContext() - .setCurrentExecutionStatus(TaskExecutionStatus.KILL); - executeFinished(); - } - - private void executeFinished() { - TaskInstanceLogHeader.printFinalizeTaskHeader(); - int taskInstanceId = asyncMasterTaskExecuteRunnable.getTaskExecutionContext().getTaskInstanceId(); - MasterTaskExecutionContextHolder.removeTaskExecutionContext(taskInstanceId); - MasterTaskExecutorHolder.removeMasterTaskExecutor(taskInstanceId); - log.info("Task execute finished, removed the TaskExecutionContext"); - asyncMasterTaskExecuteRunnable.sendTaskResult(); - log.info( - "Execute task finished, will send the task execute result to master, the current task execute result is {}", - asyncMasterTaskExecuteRunnable.getTaskExecutionContext().getCurrentExecutionStatus().name()); - asyncMasterTaskExecuteRunnable.closeLogAppender(); - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecutionContext.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecutionContext.java deleted file mode 100644 index 14ec0918865fc..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecutionContext.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.master.runner.execute; - -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; - -import java.util.concurrent.Delayed; -import java.util.concurrent.TimeUnit; - -import lombok.Data; -import lombok.NonNull; - -@Data -public class AsyncTaskExecutionContext implements Delayed { - - private final TaskExecutionContext taskExecutionContext; - - private final AsyncTaskExecuteFunction asyncTaskExecuteFunction; - - private final AsyncTaskCallbackFunction asyncTaskCallbackFunction; - - private long currentStartTime; - private int executeTimes; - private final long executeInterval; - - public AsyncTaskExecutionContext(@NonNull TaskExecutionContext taskExecutionContext, - @NonNull AsyncTaskExecuteFunction asyncTaskExecuteFunction, - @NonNull AsyncTaskCallbackFunction asyncTaskCallbackFunction) { - this.taskExecutionContext = taskExecutionContext; - this.asyncTaskExecuteFunction = asyncTaskExecuteFunction; - this.asyncTaskCallbackFunction = asyncTaskCallbackFunction; - this.currentStartTime = 0; - this.executeTimes = 0; - this.executeInterval = Math.max(asyncTaskExecuteFunction.getAsyncTaskStateCheckInterval().toMillis(), 1000L); - } - - public synchronized void refreshStartTime() { - if (executeTimes != 0) { - // The first time doesn't have delay - currentStartTime = System.currentTimeMillis(); - } - executeTimes++; - } - - @Override - public long getDelay(TimeUnit unit) { - long nextExecuteTimeDelay = currentStartTime + executeInterval - System.currentTimeMillis(); - return unit.convert(nextExecuteTimeDelay, TimeUnit.MILLISECONDS); - } - - @Override - public int compareTo(Delayed o) { - if (o == null) { - return 1; - } - return Long.compare(this.getDelay(TimeUnit.SECONDS), o.getDelay(TimeUnit.SECONDS)); - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/IMasterTaskExecutorThreadPool.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/IMasterTaskExecutorThreadPool.java deleted file mode 100644 index 6c8e2caa73907..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/IMasterTaskExecutorThreadPool.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.execute; - -public interface IMasterTaskExecutorThreadPool { - - boolean submitMasterTaskExecutor(T masterTaskExecutor); - - boolean removeMasterTaskExecutor(T masterTaskExecutor); -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterAsyncTaskExecutorThreadPool.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterAsyncTaskExecutorThreadPool.java deleted file mode 100644 index 2761958cd6412..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterAsyncTaskExecutorThreadPool.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.master.runner.execute; - -import org.apache.dolphinscheduler.common.thread.ThreadUtils; -import org.apache.dolphinscheduler.server.master.config.MasterConfig; - -import java.util.concurrent.ThreadPoolExecutor; - -import lombok.extern.slf4j.Slf4j; - -import org.springframework.stereotype.Component; - -@Slf4j -@Component -public class MasterAsyncTaskExecutorThreadPool implements IMasterTaskExecutorThreadPool { - - private final ThreadPoolExecutor threadPoolExecutor; - - public MasterAsyncTaskExecutorThreadPool(MasterConfig masterConfig) { - this.threadPoolExecutor = ThreadUtils.newDaemonFixedThreadExecutor("MasterAsyncTaskExecutorThreadPool", - masterConfig.getMasterSyncTaskExecutorThreadPoolSize()); - } - - @Override - public boolean submitMasterTaskExecutor(AsyncMasterTaskExecutor asyncMasterTaskExecutor) { - synchronized (MasterAsyncTaskExecutorThreadPool.class) { - // todo: check if the thread pool is overload - threadPoolExecutor.execute(asyncMasterTaskExecutor); - return true; - } - } - - @Override - public boolean removeMasterTaskExecutor(AsyncMasterTaskExecutor asyncMasterTaskExecutor) { - return threadPoolExecutor.remove(asyncMasterTaskExecutor); - } - - // todo: remove this method, it's not a good idea to expose the ThreadPoolExecutor to out side. - ThreadPoolExecutor getThreadPool() { - return threadPoolExecutor; - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterSyncTaskExecutorThreadPool.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterSyncTaskExecutorThreadPool.java deleted file mode 100644 index 4ce59f1f49284..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterSyncTaskExecutorThreadPool.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.execute; - -import org.apache.dolphinscheduler.common.thread.ThreadUtils; -import org.apache.dolphinscheduler.server.master.config.MasterConfig; - -import java.util.concurrent.ThreadPoolExecutor; - -import lombok.extern.slf4j.Slf4j; - -import org.springframework.stereotype.Component; - -@Slf4j -@Component -public class MasterSyncTaskExecutorThreadPool implements IMasterTaskExecutorThreadPool { - - private final ThreadPoolExecutor threadPoolExecutor; - - public MasterSyncTaskExecutorThreadPool(MasterConfig masterConfig) { - this.threadPoolExecutor = ThreadUtils.newDaemonFixedThreadExecutor("MasterSyncTaskExecutorThreadPool", - masterConfig.getMasterSyncTaskExecutorThreadPoolSize()); - } - - @Override - public boolean submitMasterTaskExecutor(SyncMasterTaskExecutor syncMasterTaskExecutor) { - synchronized (MasterSyncTaskExecutorThreadPool.class) { - // todo: check if the thread pool is overload - threadPoolExecutor.execute(syncMasterTaskExecutor); - return true; - } - } - - @Override - public boolean removeMasterTaskExecutor(SyncMasterTaskExecutor syncMasterTaskExecutor) { - return threadPoolExecutor.remove(syncMasterTaskExecutor); - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutionContextHolder.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutionContextHolder.java deleted file mode 100644 index edcbbe67089de..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutionContextHolder.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.execute; - -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; - -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -import lombok.extern.slf4j.Slf4j; - -@Slf4j -public class MasterTaskExecutionContextHolder { - - private static final Map TASK_EXECUTION_CONTEXT_MAP = new ConcurrentHashMap<>(); - - public static void putTaskExecutionContext(TaskExecutionContext taskExecutionContext) { - if (TASK_EXECUTION_CONTEXT_MAP.containsKey(taskExecutionContext.getTaskInstanceId())) { - log.error("The TaskExecutionContext {} already exists in the MasterTaskExecutionContextHolder", - taskExecutionContext); - } - TASK_EXECUTION_CONTEXT_MAP.put(taskExecutionContext.getTaskInstanceId(), taskExecutionContext); - } - - public static TaskExecutionContext getTaskExecutionContext(int taskInstanceId) { - return TASK_EXECUTION_CONTEXT_MAP.get(taskInstanceId); - } - - public static void removeTaskExecutionContext(int taskInstanceId) { - TASK_EXECUTION_CONTEXT_MAP.remove(taskInstanceId); - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutor.java deleted file mode 100644 index d21e81fe97df0..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutor.java +++ /dev/null @@ -1,203 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.execute; - -import static ch.qos.logback.classic.ClassicConstants.FINALIZE_SESSION_MARKER; -import static org.apache.dolphinscheduler.common.constants.Constants.DRY_RUN_FLAG_YES; - -import org.apache.dolphinscheduler.common.log.remote.RemoteLogUtils; -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.plugin.task.api.utils.LogUtils; -import org.apache.dolphinscheduler.server.master.exception.LogicTaskFactoryNotFoundException; -import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; -import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; -import org.apache.dolphinscheduler.server.master.runner.message.LogicTaskInstanceExecutionEventSenderManager; -import org.apache.dolphinscheduler.server.master.runner.task.ILogicTask; -import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder; - -import lombok.Getter; -import lombok.extern.slf4j.Slf4j; - -@Slf4j -public abstract class MasterTaskExecutor implements Runnable { - - @Getter - protected final TaskExecutionContext taskExecutionContext; - @Getter - protected ILogicTask logicTask; - - protected final LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder; - protected final LogicTaskInstanceExecutionEventSenderManager logicTaskInstanceExecutionEventSenderManager; - - public MasterTaskExecutor(TaskExecutionContext taskExecutionContext, - LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder, - LogicTaskInstanceExecutionEventSenderManager logicTaskInstanceExecutionEventSenderManager) { - this.taskExecutionContext = taskExecutionContext; - this.logicTaskPluginFactoryBuilder = logicTaskPluginFactoryBuilder; - this.logicTaskInstanceExecutionEventSenderManager = logicTaskInstanceExecutionEventSenderManager; - } - - protected abstract void executeTask() throws MasterTaskExecuteException; - - protected abstract void afterExecute() throws MasterTaskExecuteException; - - protected void afterThrowing(Throwable throwable) { - TaskInstanceLogHeader.printFinalizeTaskHeader(); - try { - log.error("Get a exception when execute the task, will try to cancel the task", throwable); - cancelTask(); - } catch (Exception e) { - log.error("Cancel task failed,", e); - } - taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.FAILURE); - sendTaskResult(); - log.info( - "Get a exception when execute the task, sent the task execute result to master, the current task execute result is {}", - taskExecutionContext.getCurrentExecutionStatus()); - MasterTaskExecutionContextHolder.removeTaskExecutionContext(taskExecutionContext.getTaskInstanceId()); - MasterTaskExecutorHolder.removeMasterTaskExecutor(taskExecutionContext.getTaskInstanceId()); - log.info("Get a exception when execute the task, removed the TaskExecutionContext"); - closeLogAppender(); - } - - public void cancelTask() throws MasterTaskExecuteException { - if (logicTask != null) { - logicTask.kill(); - } - } - - public void pauseTask() throws MasterTaskExecuteException { - if (logicTask != null) { - logicTask.pause(); - } - } - - @Override - public void run() { - try { - LogUtils.setWorkflowAndTaskInstanceIDMDC( - taskExecutionContext.getWorkflowInstanceId(), - taskExecutionContext.getTaskInstanceId()); - LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath()); - - TaskInstanceLogHeader.printInitializeTaskContextHeader(); - initializeTask(); - - TaskInstanceLogHeader.printLoadTaskInstancePluginHeader(); - beforeExecute(); - - if (DRY_RUN_FLAG_YES == taskExecutionContext.getDryRun()) { - taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.SUCCESS); - taskExecutionContext.setEndTime(System.currentTimeMillis()); - MasterTaskExecutorHolder.removeMasterTaskExecutor(taskExecutionContext.getTaskInstanceId()); - logicTaskInstanceExecutionEventSenderManager.successEventSender().sendMessage(taskExecutionContext); - log.info( - "The current execute mode is dry run, will stop the logic task and set the taskInstance status to success"); - return; - } - TaskInstanceLogHeader.printExecuteTaskHeader(); - executeTask(); - - afterExecute(); - } catch (Throwable ex) { - log.error("Task execute failed, due to meet an exception", ex); - afterThrowing(ex); - } 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() throws LogicTaskFactoryNotFoundException, LogicTaskInitializeException { - taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.RUNNING_EXECUTION); - logicTaskInstanceExecutionEventSenderManager.runningEventSender().sendMessage(taskExecutionContext); - log.info("Send task status {} to master {}", taskExecutionContext.getCurrentExecutionStatus().name(), - taskExecutionContext.getWorkflowInstanceHost()); - - logicTask = logicTaskPluginFactoryBuilder.createILogicTaskPluginFactory(taskExecutionContext.getTaskType()) - .createLogicTask(taskExecutionContext); - log.info("Initialized task plugin instance: {} successfully", taskExecutionContext.getTaskType()); - - logicTask.getTaskParameters().setVarPool(taskExecutionContext.getVarPool()); - log.info("Initialize taskVarPool: {} successfully", taskExecutionContext.getVarPool()); - - } - - protected void closeLogAppender() { - try { - if (RemoteLogUtils.isRemoteLoggingEnable()) { - RemoteLogUtils.sendRemoteLog(taskExecutionContext.getLogPath()); - log.info("Send task log {} to remote storage successfully", taskExecutionContext.getLogPath()); - } - } catch (Exception ex) { - log.error("Send task log {} to remote storage failed", taskExecutionContext.getLogPath(), ex); - } finally { - log.info(FINALIZE_SESSION_MARKER, FINALIZE_SESSION_MARKER.toString()); - } - } - - protected void sendTaskResult() { - try { - taskExecutionContext.setEndTime(System.currentTimeMillis()); - taskExecutionContext.setVarPool(JSONUtils.toJsonString(logicTask.getTaskParameters().getVarPool())); - switch (taskExecutionContext.getCurrentExecutionStatus()) { - case KILL: - logicTaskInstanceExecutionEventSenderManager.killedEventSender().sendMessage(taskExecutionContext); - break; - case PAUSE: - logicTaskInstanceExecutionEventSenderManager.pausedEventSender().sendMessage(taskExecutionContext); - break; - case FAILURE: - logicTaskInstanceExecutionEventSenderManager.failedEventSender().sendMessage(taskExecutionContext); - break; - case SUCCESS: - logicTaskInstanceExecutionEventSenderManager.successEventSender().sendMessage(taskExecutionContext); - break; - default: - logicTaskInstanceExecutionEventSenderManager.failedEventSender().sendMessage(taskExecutionContext); - break; - } - log.info("Send task status: {} to master: {} successfully", - taskExecutionContext.getCurrentExecutionStatus().name(), - taskExecutionContext.getWorkflowInstanceHost()); - } catch (Exception ex) { - log.error("Send task status: {} to master: {} failed", - taskExecutionContext.getCurrentExecutionStatus().name(), - taskExecutionContext.getWorkflowInstanceHost(), ex); - } - } - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorFactoryBuilder.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorFactoryBuilder.java deleted file mode 100644 index 8fa6002463884..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorFactoryBuilder.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.execute; - -import org.apache.dolphinscheduler.plugin.task.api.task.SubWorkflowLogicTaskChannelFactory; -import org.apache.dolphinscheduler.server.master.runner.task.dependent.DependentLogicTask; -import org.apache.dolphinscheduler.server.master.runner.task.dynamic.DynamicLogicTask; - -import java.util.Set; - -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Component; - -import com.google.common.collect.Sets; - -/** - * Use to create MasterTaskExecutorFactory - */ -@Component -public class MasterTaskExecutorFactoryBuilder { - - @Autowired - private AsyncMasterTaskExecutorFactory asyncMasterTaskExecutorFactory; - - @Autowired - private SyncMasterTaskExecutorFactory syncMasterTaskExecutorFactory; - - private static final Set ASYNC_TASK_TYPE = Sets.newHashSet( - DependentLogicTask.TASK_TYPE, - SubWorkflowLogicTaskChannelFactory.NAME, - DynamicLogicTask.TASK_TYPE); - - public MasterTaskExecutorFactory createMasterTaskExecutorFactory(String taskType) { - if (ASYNC_TASK_TYPE.contains(taskType)) { - return asyncMasterTaskExecutorFactory; - } - return syncMasterTaskExecutorFactory; - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorHolder.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorHolder.java deleted file mode 100644 index 962bfc6ed34be..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorHolder.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.server.master.runner.execute; - -import java.util.Collection; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -import lombok.experimental.UtilityClass; -import lombok.extern.slf4j.Slf4j; - -@Slf4j -@UtilityClass -public class MasterTaskExecutorHolder { - - private static final Map SUBMITTED_MASTER_TASK_MAP = new ConcurrentHashMap<>(); - - public void putMasterTaskExecuteRunnable(MasterTaskExecutor masterTaskExecutor) { - SUBMITTED_MASTER_TASK_MAP.put(masterTaskExecutor.getTaskExecutionContext().getTaskInstanceId(), - masterTaskExecutor); - } - - public MasterTaskExecutor getMasterTaskExecutor(Integer taskInstanceId) { - return SUBMITTED_MASTER_TASK_MAP.get(taskInstanceId); - } - - public void removeMasterTaskExecutor(Integer taskInstanceId) { - SUBMITTED_MASTER_TASK_MAP.remove(taskInstanceId); - } - - public Collection getAllMasterTaskExecutor() { - return SUBMITTED_MASTER_TASK_MAP.values(); - } - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorThreadPoolManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorThreadPoolManager.java deleted file mode 100644 index 9c30d49f8fb21..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorThreadPoolManager.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.execute; - -import org.apache.dolphinscheduler.server.master.runner.message.LogicTaskInstanceExecutionEventSenderManager; - -import lombok.extern.slf4j.Slf4j; - -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Component; - -@Slf4j -@Component -public class MasterTaskExecutorThreadPoolManager { - - @Autowired - private MasterSyncTaskExecutorThreadPool masterSyncTaskExecutorThreadPool; - - @Autowired - private MasterAsyncTaskExecutorThreadPool masterAsyncTaskExecutorThreadPool; - - @Autowired - private LogicTaskInstanceExecutionEventSenderManager logicTaskInstanceExecutionEventSenderManager; - - public boolean submitMasterTaskExecutor(final MasterTaskExecutor masterTaskExecutor) { - MasterTaskExecutorHolder.putMasterTaskExecuteRunnable(masterTaskExecutor); - sendDispatchedEvent(masterTaskExecutor); - if (masterTaskExecutor instanceof SyncMasterTaskExecutor) { - return masterSyncTaskExecutorThreadPool - .submitMasterTaskExecutor((SyncMasterTaskExecutor) masterTaskExecutor); - } - if (masterTaskExecutor instanceof AsyncMasterTaskExecutor) { - return masterAsyncTaskExecutorThreadPool - .submitMasterTaskExecutor((AsyncMasterTaskExecutor) masterTaskExecutor); - } - throw new IllegalArgumentException("Unknown type of MasterTaskExecutor: " + masterTaskExecutor); - } - - public boolean removeMasterTaskExecutor(final MasterTaskExecutor masterTaskExecutor) { - if (masterTaskExecutor instanceof SyncMasterTaskExecutor) { - return masterSyncTaskExecutorThreadPool - .removeMasterTaskExecutor((SyncMasterTaskExecutor) masterTaskExecutor); - } - if (masterTaskExecutor instanceof AsyncMasterTaskExecutor) { - return masterAsyncTaskExecutorThreadPool - .removeMasterTaskExecutor((AsyncMasterTaskExecutor) masterTaskExecutor); - } - throw new IllegalArgumentException("Unknown type of MasterTaskExecutor: " + masterTaskExecutor); - } - - private void sendDispatchedEvent(final MasterTaskExecutor masterTaskExecutor) { - logicTaskInstanceExecutionEventSenderManager.dispatchEventSender() - .sendMessage(masterTaskExecutor.getTaskExecutionContext()); - } - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/SyncMasterTaskExecutor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/SyncMasterTaskExecutor.java deleted file mode 100644 index 7f303487efed3..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/SyncMasterTaskExecutor.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.execute; - -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.plugin.task.api.log.TaskInstanceLogHeader; -import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; -import org.apache.dolphinscheduler.server.master.runner.message.LogicTaskInstanceExecutionEventSenderManager; -import org.apache.dolphinscheduler.server.master.runner.task.ISyncLogicTask; -import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder; - -import lombok.extern.slf4j.Slf4j; - -@Slf4j -public class SyncMasterTaskExecutor extends MasterTaskExecutor { - - public SyncMasterTaskExecutor(TaskExecutionContext taskExecutionContext, - LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder, - LogicTaskInstanceExecutionEventSenderManager logicTaskInstanceExecutionEventSenderManager) { - super(taskExecutionContext, logicTaskPluginFactoryBuilder, logicTaskInstanceExecutionEventSenderManager); - } - - @Override - protected void executeTask() throws MasterTaskExecuteException { - if (logicTask == null) { - throw new MasterTaskExecuteException("The task plugin instance is null"); - } - ISyncLogicTask iSyncLogicTask = (ISyncLogicTask) logicTask; - iSyncLogicTask.handle(); - } - - protected void afterExecute() throws MasterTaskExecuteException { - TaskInstanceLogHeader.printFinalizeTaskHeader(); - // todo: move this to constructor to avoid check every time - if (logicTask == null) { - throw new MasterTaskExecuteException("The current task instance is null"); - } - sendTaskResult(); - log.info( - "Execute task finished, will send the task execute result to master, the current task execute result is {}", - taskExecutionContext.getCurrentExecutionStatus().name()); - closeLogAppender(); - MasterTaskExecutionContextHolder.removeTaskExecutionContext(taskExecutionContext.getTaskInstanceId()); - MasterTaskExecutorHolder.removeMasterTaskExecutor(taskExecutionContext.getTaskInstanceId()); - } - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecuteDispatchEventSender.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecuteDispatchEventSender.java deleted file mode 100644 index 2f45baa51bf19..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecuteDispatchEventSender.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.message; - -import org.apache.dolphinscheduler.extract.base.client.Clients; -import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; -import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionDispatchEvent; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; - -import org.springframework.stereotype.Component; - -@Component -public class LogicTaskInstanceExecuteDispatchEventSender - implements - LogicTaskInstanceExecutionEventSender { - - @Override - public void sendMessage(TaskExecutionDispatchEvent taskExecutionDispatchEvent) { - Clients - .withService(ITaskExecutionEventListener.class) - .withHost(taskExecutionDispatchEvent.getWorkflowInstanceHost()) - .onTaskInstanceDispatched(taskExecutionDispatchEvent); - } - - @Override - public TaskExecutionDispatchEvent buildMessage(TaskExecutionContext taskExecutionContext) { - final TaskExecutionDispatchEvent taskExecutionDispatchEvent = - TaskExecutionDispatchEvent.builder() - .workflowInstanceId(taskExecutionContext.getWorkflowInstanceId()) - .workflowInstanceHost(taskExecutionContext.getWorkflowInstanceHost()) - .taskInstanceId(taskExecutionContext.getTaskInstanceId()) - .taskInstanceHost(taskExecutionContext.getHost()) - .eventCreateTime(System.currentTimeMillis()) - .eventSendTime(System.currentTimeMillis()) - .build(); - return taskExecutionDispatchEvent; - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecuteRunningEventSender.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecuteRunningEventSender.java deleted file mode 100644 index 8e27246a91cc3..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecuteRunningEventSender.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.message; - -import org.apache.dolphinscheduler.extract.base.client.Clients; -import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; -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 LogicTaskInstanceExecuteRunningEventSender - implements - LogicTaskInstanceExecutionEventSender { - - @Override - public void sendMessage(TaskExecutionRunningEvent taskInstanceExecutionRunningEvent) { - Clients - .withService(ITaskExecutionEventListener.class) - .withHost(taskInstanceExecutionRunningEvent.getWorkflowInstanceHost()) - .onTaskInstanceExecutionRunning(taskInstanceExecutionRunningEvent); - } - - @Override - public TaskExecutionRunningEvent buildMessage(@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; - } - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionEventSenderManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionEventSenderManager.java deleted file mode 100644 index eb3ac11e1312b..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionEventSenderManager.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.message; - -import lombok.extern.slf4j.Slf4j; - -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Component; - -@Slf4j -@Component -public class LogicTaskInstanceExecutionEventSenderManager { - - @Autowired - private LogicTaskInstanceExecuteDispatchEventSender logicTaskInstanceExecuteDispatchEventSender; - - @Autowired - private LogicTaskInstanceExecuteRunningEventSender logicTaskInstanceExecuteRunningEventSender; - - @Autowired - private LogicTaskInstanceExecutionFailedEventSender logicTaskInstanceExecutionFailedEventSender; - - @Autowired - private LogicTaskInstanceExecutionPausedEventSender logicTaskInstanceExecutionPausedEventSender; - - @Autowired - private LogicTaskInstanceExecutionSuccessEventSender logicTaskInstanceExecutionSuccessEventSender; - - @Autowired - private LogicTaskInstanceExecutionKilledEventSender logicTaskInstanceExecutionKilledEventSender; - - public LogicTaskInstanceExecuteDispatchEventSender dispatchEventSender() { - return logicTaskInstanceExecuteDispatchEventSender; - } - - public LogicTaskInstanceExecuteRunningEventSender runningEventSender() { - return logicTaskInstanceExecuteRunningEventSender; - } - - public LogicTaskInstanceExecutionFailedEventSender failedEventSender() { - return logicTaskInstanceExecutionFailedEventSender; - } - - public LogicTaskInstanceExecutionPausedEventSender pausedEventSender() { - return logicTaskInstanceExecutionPausedEventSender; - } - - public LogicTaskInstanceExecutionSuccessEventSender successEventSender() { - return logicTaskInstanceExecutionSuccessEventSender; - } - - public LogicTaskInstanceExecutionKilledEventSender killedEventSender() { - return logicTaskInstanceExecutionKilledEventSender; - } - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseAsyncLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseAsyncLogicTask.java deleted file mode 100644 index 2b9d5728a1665..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseAsyncLogicTask.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.task; - -import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters; - -import lombok.extern.slf4j.Slf4j; - -@Slf4j -public abstract class BaseAsyncLogicTask implements IAsyncLogicTask { - - protected final TaskExecutionContext taskExecutionContext; - protected final T taskParameters; - - protected BaseAsyncLogicTask(TaskExecutionContext taskExecutionContext, T taskParameters) { - this.taskExecutionContext = taskExecutionContext; - this.taskParameters = taskParameters; - log.info("Success initialize task parameters: \n{}", JSONUtils.toPrettyJsonString(taskParameters)); - } - - @Override - public TaskExecutionContext getTaskExecutionContext() { - return taskExecutionContext; - } - - @Override - public AbstractParameters getTaskParameters() { - return taskParameters; - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseSyncLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseSyncLogicTask.java deleted file mode 100644 index c971245b8cc61..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseSyncLogicTask.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.task; - -import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters; -import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable; -import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; - -import lombok.extern.slf4j.Slf4j; - -@Slf4j -public abstract class BaseSyncLogicTask implements ISyncLogicTask { - - protected final TaskExecutionContext taskExecutionContext; - - protected final IWorkflowExecutionRunnable workflowExecutionRunnable; - protected final TaskInstance taskInstance; - protected final T taskParameters; - - protected BaseSyncLogicTask(IWorkflowExecutionRunnable workflowExecutionRunnable, - TaskExecutionContext taskExecutionContext, - T taskParameters) { - this.taskExecutionContext = taskExecutionContext; - this.workflowExecutionRunnable = workflowExecutionRunnable; - this.taskInstance = workflowExecutionRunnable - .getWorkflowExecuteContext() - .getWorkflowExecutionGraph() - .getTaskExecutionRunnableById(taskExecutionContext.getTaskInstanceId()) - .getTaskInstance(); - this.taskParameters = taskParameters; - log.info("Success initialize task parameters: \n{}", JSONUtils.toPrettyJsonString(taskParameters)); - } - - @Override - public void pause() throws MasterTaskExecuteException { - log.warn("The Sync task does not support pause operation"); - } - - @Override - public void kill() throws MasterTaskExecuteException { - log.warn("The Sync task does not support kill operation"); - } - - @Override - public TaskExecutionContext getTaskExecutionContext() { - return taskExecutionContext; - } - - @Override - public AbstractParameters getTaskParameters() { - return taskParameters; - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/IAsyncLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/IAsyncLogicTask.java deleted file mode 100644 index 88d06ed1e46d5..0000000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/IAsyncLogicTask.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.task; - -import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; -import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction; - -public interface IAsyncLogicTask extends ILogicTask { - - AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() throws MasterTaskExecuteException; -} diff --git a/dolphinscheduler-master/src/main/resources/application.yaml b/dolphinscheduler-master/src/main/resources/application.yaml index 964fca836ce6b..76cc257bd2737 100644 --- a/dolphinscheduler-master/src/main/resources/application.yaml +++ b/dolphinscheduler-master/src/main/resources/application.yaml @@ -86,10 +86,11 @@ master: listen-port: 5678 # The number of workflow event bus fire worker used to fire the workflow event, default is 2*CPU core + 1. # workflow-event-bus-fire-thread-count: 10 - # The number of threads used to execute sync logic task e.g. Switch/Condition, default is CPU core. - # master-sync-task-executor-thread-pool-size: 10 - # The number of threads used to execute async logic task e.g. Dependent/SubWorkflow default is CPU core. - # master-async-task-executor-thread-pool-size: 10 + # logic-task-config: + # The number of threads used to execute logic task. + # task-executor-thread-count: 10 + # The number of threads used to fire logic task events. + # task-executor-event-bus-fire-thread-count: 10 max-heartbeat-interval: 10s server-load-protection: # If set true, will open master overload protection diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooperTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooperTest.java index 9c31ae48cf1c0..f36869cb03ffa 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooperTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooperTest.java @@ -36,11 +36,10 @@ import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; import org.apache.dolphinscheduler.server.master.engine.WorkflowEventBus; import org.apache.dolphinscheduler.server.master.engine.graph.WorkflowExecutionGraph; +import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnable; import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnableBuilder; -import org.apache.dolphinscheduler.server.master.runner.dispatcher.TaskDispatchFactory; -import org.apache.dolphinscheduler.server.master.runner.dispatcher.TaskDispatcher; import java.util.HashMap; @@ -64,7 +63,7 @@ class GlobalTaskDispatchWaitingQueueLooperTest { private GlobalTaskDispatchWaitingQueue globalTaskDispatchWaitingQueue; @Mock - private TaskDispatchFactory taskDispatchFactory; + private ITaskExecutorClient taskExecutorClient; @Test void testTaskExecutionRunnableStatusIsNotSubmitted() throws Exception { @@ -75,14 +74,13 @@ void testTaskExecutionRunnableStatusIsNotSubmitted() throws Exception { final ITaskExecutionRunnable defaultTaskExecuteRunnable = createTaskExecuteRunnable(taskInstance, workflowInstance); - TaskDispatcher taskDispatcher = mock(TaskDispatcher.class); - when(taskDispatchFactory.getTaskDispatcher(taskInstance)).thenReturn(taskDispatcher); - doNothing().when(taskDispatcher).dispatchTask(any()); + doNothing().when(taskExecutorClient).dispatch(any()); when(globalTaskDispatchWaitingQueue.takeTaskExecuteRunnable()).thenReturn(defaultTaskExecuteRunnable); globalTaskDispatchWaitingQueueLooper.doDispatch(); await().during(ofSeconds(1)) - .untilAsserted(() -> verify(taskDispatchFactory, never()).getTaskDispatcher(taskInstance)); + .untilAsserted(() -> verify(taskExecutorClient, never()).dispatch(any())); + globalTaskDispatchWaitingQueueLooper.close(); } @Test @@ -94,15 +92,12 @@ void testTaskExecutionRunnableStatusIsSubmitted() throws Exception { final ITaskExecutionRunnable defaultTaskExecuteRunnable = createTaskExecuteRunnable(taskInstance, workflowInstance); - TaskDispatcher taskDispatcher = mock(TaskDispatcher.class); - when(taskDispatchFactory.getTaskDispatcher(taskInstance)).thenReturn(taskDispatcher); - doNothing().when(taskDispatcher).dispatchTask(any()); + doNothing().when(taskExecutorClient).dispatch(any()); when(globalTaskDispatchWaitingQueue.takeTaskExecuteRunnable()).thenReturn(defaultTaskExecuteRunnable); globalTaskDispatchWaitingQueueLooper.doDispatch(); await().atMost(ofSeconds(1)).untilAsserted(() -> { - verify(taskDispatchFactory, atLeastOnce()).getTaskDispatcher(any(TaskInstance.class)); - verify(taskDispatcher, atLeastOnce()).dispatchTask(any(ITaskExecutionRunnable.class)); + verify(taskExecutorClient, atLeastOnce()).dispatch(any(ITaskExecutionRunnable.class)); }); } diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcherTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcherTest.java deleted file mode 100644 index 80a2c9427d9d1..0000000000000 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcherTest.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.dispatcher; - -import org.apache.dolphinscheduler.extract.base.utils.Host; -import org.apache.dolphinscheduler.server.master.config.MasterConfig; -import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Mockito; -import org.mockito.junit.jupiter.MockitoExtension; - -@ExtendWith(MockitoExtension.class) -public class MasterTaskDispatcherTest { - - @Test - public void getTaskExecuteHost() { - MasterConfig masterConfig = Mockito.mock(MasterConfig.class); - Mockito.when(masterConfig.getMasterAddress()).thenReturn("localhost:5678"); - ITaskExecutionRunnable ITaskExecutionRunnable = Mockito.mock(ITaskExecutionRunnable.class); - - MasterTaskDispatcher masterTaskDispatcher = new MasterTaskDispatcher(masterConfig); - Host taskInstanceDispatchHost = masterTaskDispatcher.getTaskInstanceDispatchHost(ITaskExecutionRunnable) - .orElseThrow(() -> new IllegalArgumentException("Cannot get the ")); - Assertions.assertEquals(masterConfig.getMasterAddress(), taskInstanceDispatchHost.getAddress()); - } -} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatchFactoryTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatchFactoryTest.java deleted file mode 100644 index dd63abcfa9d62..0000000000000 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatchFactoryTest.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.master.runner.dispatcher; - -import static com.google.common.truth.Truth.assertThat; - -import org.apache.dolphinscheduler.plugin.task.api.task.ConditionsLogicTaskChannelFactory; -import org.apache.dolphinscheduler.plugin.task.api.task.DependentLogicTaskChannelFactory; -import org.apache.dolphinscheduler.plugin.task.api.task.DynamicLogicTaskChannelFactory; -import org.apache.dolphinscheduler.plugin.task.api.task.SubWorkflowLogicTaskChannelFactory; -import org.apache.dolphinscheduler.plugin.task.api.task.SwitchLogicTaskChannelFactory; -import org.apache.dolphinscheduler.plugin.task.shell.ShellTaskChannelFactory; - -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; -import org.mockito.InjectMocks; -import org.mockito.Mock; -import org.mockito.junit.jupiter.MockitoExtension; - -@ExtendWith(MockitoExtension.class) -public class TaskDispatchFactoryTest { - - @InjectMocks - private TaskDispatchFactory taskDispatchFactory; - - @Mock - private MasterTaskDispatcher masterTaskDispatcher; - - @Mock - private WorkerTaskDispatcher workerTaskDispatcher; - - @ParameterizedTest - @ValueSource(strings = { - ConditionsLogicTaskChannelFactory.NAME, - DependentLogicTaskChannelFactory.NAME, - DynamicLogicTaskChannelFactory.NAME, - SubWorkflowLogicTaskChannelFactory.NAME, - SwitchLogicTaskChannelFactory.NAME}) - public void getTaskDispatcher_withLogicTask(String taskType) { - assertThat(taskDispatchFactory.getTaskDispatcher(taskType)).isSameInstanceAs(masterTaskDispatcher); - } - - @ParameterizedTest - @ValueSource(strings = {ShellTaskChannelFactory.NAME}) - public void getTaskDispatcher_withWorkerTask(String taskType) { - assertThat(taskDispatchFactory.getTaskDispatcher(taskType)).isSameInstanceAs(workerTaskDispatcher); - } -} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcherTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcherTest.java deleted file mode 100644 index 993962544d550..0000000000000 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcherTest.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.dispatcher; - -import org.apache.dolphinscheduler.extract.base.utils.Host; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.server.master.cluster.loadbalancer.IWorkerLoadBalancer; -import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; - -import java.util.Optional; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Mockito; -import org.mockito.junit.jupiter.MockitoExtension; - -@ExtendWith(MockitoExtension.class) -public class WorkerTaskDispatcherTest { - - @Test - public void getTaskInstanceDispatchHost() { - IWorkerLoadBalancer workerLoadBalancer = Mockito.mock(IWorkerLoadBalancer.class); - Mockito.when(workerLoadBalancer.select(Mockito.any())).thenReturn(Optional.of("localhost:1234")); - WorkerTaskDispatcher workerTaskDispatcher = new WorkerTaskDispatcher(workerLoadBalancer); - - ITaskExecutionRunnable ITaskExecutionRunnable = Mockito.mock(ITaskExecutionRunnable.class); - Mockito.when(ITaskExecutionRunnable.getTaskExecutionContext()).thenReturn(new TaskExecutionContext()); - Optional taskInstanceDispatchHost = - workerTaskDispatcher.getTaskInstanceDispatchHost(ITaskExecutionRunnable); - Assertions.assertEquals("localhost:1234", taskInstanceDispatchHost.get().getAddress()); - } -} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicAsyncTaskExecuteFunctionTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicAsyncTaskExecuteFunctionTest.java deleted file mode 100644 index eda6d92088cdc..0000000000000 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicAsyncTaskExecuteFunctionTest.java +++ /dev/null @@ -1,141 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.task.dynamic; - -import static org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction.AsyncTaskExecutionStatus; - -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; -import org.apache.dolphinscheduler.dao.mapper.CommandMapper; -import org.apache.dolphinscheduler.plugin.task.api.parameters.DynamicParameters; -import org.apache.dolphinscheduler.service.subworkflow.SubWorkflowService; - -import java.util.Arrays; -import java.util.List; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Mock; -import org.mockito.Mockito; -import org.mockito.junit.jupiter.MockitoExtension; - -@ExtendWith(MockitoExtension.class) -class DynamicAsyncTaskExecuteFunctionTest { - - @Mock - private WorkflowInstance workflowInstance; - - @Mock - private TaskInstance taskInstance; - - @Mock - private SubWorkflowService subWorkflowService; - - @Mock - private CommandMapper commandMapper; - - @Mock - private DynamicLogicTask dynamicLogicTask; - - private DynamicAsyncTaskExecuteFunction function; - - @BeforeEach - void setUp() { - workflowInstance = new WorkflowInstance(); - taskInstance = new TaskInstance(); - - workflowInstance.setId(1); - taskInstance.setTaskCode(2L); - - function = new DynamicAsyncTaskExecuteFunction( - null, - workflowInstance, - taskInstance, - dynamicLogicTask, - commandMapper, - subWorkflowService, - 0); - } - - @Test - void shouldReturnSuccessWhenAllSubProcessInstancesFinishedSuccessfully() { - // Given - List workflowInstances = Arrays.asList(Mockito.mock(WorkflowInstance.class)); - - Mockito.when(workflowInstances.get(0).getCommandParam()).thenReturn("{}"); - Mockito.when(subWorkflowService.getAllDynamicSubWorkflow(1, 2L)).thenReturn(workflowInstances); - Mockito.when(subWorkflowService.filterFinishProcessInstances(Mockito.any())).thenReturn(workflowInstances); - Mockito.when(subWorkflowService.filterSuccessProcessInstances(Mockito.any())).thenReturn(workflowInstances); - - // When - DynamicParameters dynamicParameters = new DynamicParameters(); - Mockito.when(dynamicLogicTask.getTaskParameters()).thenReturn(dynamicParameters); - AsyncTaskExecutionStatus status = function.getAsyncTaskExecutionStatus(); - - // Then - Assertions.assertEquals(AsyncTaskExecutionStatus.SUCCESS, status); - } - - @Test - void shouldReturnFailedWhenSomeSubProcessInstancesFinishedUnsuccessfully() { - // Given - List workflowInstances = - Arrays.asList(Mockito.mock(WorkflowInstance.class), Mockito.mock(WorkflowInstance.class)); - Mockito.when(subWorkflowService.getAllDynamicSubWorkflow(1, 2L)).thenReturn(workflowInstances); - Mockito.when(subWorkflowService.filterFinishProcessInstances(Mockito.anyList())).thenReturn(workflowInstances); - Mockito.when(subWorkflowService.filterSuccessProcessInstances(Mockito.anyList())) - .thenReturn(Arrays.asList(workflowInstances.get(0))); - - // When - AsyncTaskExecutionStatus status = function.getAsyncTaskExecutionStatus(); - - // Then - Assertions.assertEquals(AsyncTaskExecutionStatus.FAILED, status); - } - - @Test - void shouldReturnRunningWhenSomeSubProcessInstancesAreRunning() { - // Given - List workflowInstances = Arrays.asList(Mockito.mock(WorkflowInstance.class)); - Mockito.when(subWorkflowService.getAllDynamicSubWorkflow(1, 2L)).thenReturn(workflowInstances); - Mockito.when(subWorkflowService.filterFinishProcessInstances(Mockito.anyList())).thenReturn(Arrays.asList()); - - // When - AsyncTaskExecutionStatus status = function.getAsyncTaskExecutionStatus(); - - // Then - Assertions.assertEquals(AsyncTaskExecutionStatus.RUNNING, status); - } - - @Test - void shouldReturnFailedWhenLogicTaskIsCancelled() { - // Given - List workflowInstances = Arrays.asList(Mockito.mock(WorkflowInstance.class)); - Mockito.when(subWorkflowService.getAllDynamicSubWorkflow(1, 2L)).thenReturn(workflowInstances); - Mockito.when(dynamicLogicTask.isCancel()).thenReturn(true); - - // When - AsyncTaskExecutionStatus status = function.getAsyncTaskExecutionStatus(); - - // Then - Assertions.assertEquals(AsyncTaskExecutionStatus.FAILED, status); - } - -} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicCommandUtilsTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicCommandUtilsTest.java deleted file mode 100644 index 9074679d77218..0000000000000 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicCommandUtilsTest.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.task.dynamic; - -import org.apache.dolphinscheduler.common.constants.CommandKeyConstants; -import org.apache.dolphinscheduler.common.enums.CommandType; -import org.apache.dolphinscheduler.common.enums.TaskDependType; -import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.dao.entity.Command; -import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; - -import java.util.HashMap; -import java.util.Map; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -class DynamicCommandUtilsTest { - - private WorkflowInstance workflowInstance; - private Long subProcessDefinitionCode; - private Integer subProcessDefinitionVersion; - private Map parameters; - - @BeforeEach - void setUp() { - workflowInstance = new WorkflowInstance(); - subProcessDefinitionCode = 1L; - subProcessDefinitionVersion = 1; - parameters = new HashMap<>(); - - // Populate processInstance with some dummy data - workflowInstance.setCommandType(CommandType.START_PROCESS); - workflowInstance.setFailureStrategy(null); // update this - workflowInstance.setWarningType(null); // update this - workflowInstance.setGlobalParams("{\"prop\":\"value\"}"); - workflowInstance.setExecutorId(1); - workflowInstance.setWarningGroupId(1); - workflowInstance.setWorkflowInstancePriority(null); // update this - workflowInstance.setWorkerGroup("worker"); - workflowInstance.setTenantCode("unit-root"); - workflowInstance.setDryRun(0); - } - - @Test - void testCreateCommand() { - Command command = DynamicCommandUtils.createCommand(workflowInstance, subProcessDefinitionCode, - subProcessDefinitionVersion, parameters); - - Assertions.assertEquals(CommandType.DYNAMIC_GENERATION, command.getCommandType()); - Assertions.assertEquals(subProcessDefinitionCode, command.getWorkflowDefinitionCode()); - Assertions.assertEquals(subProcessDefinitionVersion, command.getWorkflowDefinitionVersion()); - Assertions.assertEquals(TaskDependType.TASK_POST, command.getTaskDependType()); - Assertions.assertEquals(workflowInstance.getFailureStrategy(), command.getFailureStrategy()); - Assertions.assertEquals(workflowInstance.getWarningType(), command.getWarningType()); - Assertions.assertEquals(workflowInstance.getExecutorId(), command.getExecutorId()); - Assertions.assertEquals(workflowInstance.getWarningGroupId(), command.getWarningGroupId()); - Assertions.assertEquals(workflowInstance.getWorkflowInstancePriority(), command.getWorkflowInstancePriority()); - Assertions.assertEquals(workflowInstance.getWorkerGroup(), command.getWorkerGroup()); - Assertions.assertEquals(workflowInstance.getDryRun(), command.getDryRun()); - Assertions.assertEquals(workflowInstance.getTenantCode(), command.getTenantCode()); - } - - @Test - void testGetDataFromCommandParam() { - Command command = new Command(); - DynamicCommandUtils.addDataToCommandParam(command, "testKey", "testData"); - String data = DynamicCommandUtils.getDataFromCommandParam(command.getCommandParam(), "testKey"); - - Assertions.assertEquals("testData", data); - } - - @Test - void testCreateCommandCommandType() { - // Scenario 1: CommandType is START_PROCESS - workflowInstance.setCommandType(CommandType.START_PROCESS); - Command command1 = DynamicCommandUtils.createCommand(workflowInstance, subProcessDefinitionCode, - subProcessDefinitionVersion, parameters); - Assertions.assertEquals(CommandType.DYNAMIC_GENERATION, command1.getCommandType()); - - // Scenario 2: CommandType is not START_PROCESS - workflowInstance.setCommandType(CommandType.START_FAILURE_TASK_PROCESS); - Command command2 = DynamicCommandUtils.createCommand(workflowInstance, subProcessDefinitionCode, - subProcessDefinitionVersion, parameters); - Assertions.assertEquals(CommandType.START_FAILURE_TASK_PROCESS, command2.getCommandType()); - } - - @Test - void testCreateCommandStartParams() { - // Scenario: Add some data to parameters - parameters.put("testKey", "testValue"); - Command command = DynamicCommandUtils.createCommand(workflowInstance, subProcessDefinitionCode, - subProcessDefinitionVersion, parameters); - - String startParamsJson = DynamicCommandUtils.getDataFromCommandParam(command.getCommandParam(), - CommandKeyConstants.CMD_PARAM_START_PARAMS); - Map startParams = JSONUtils.toMap(startParamsJson); - - Assertions.assertEquals("testValue", startParams.get("testKey")); - } - - @Test - void testCreateCommandGlobalParams() { - // Scenario: processInstance has globalParams - parameters.put("testKey", "testValue"); - workflowInstance.setGlobalParams("[{\"prop\":\"globalKey\",\"value\":\"globalValue\"}]"); - - Command command = DynamicCommandUtils.createCommand(workflowInstance, subProcessDefinitionCode, - subProcessDefinitionVersion, parameters); - - String startParamsJson = DynamicCommandUtils.getDataFromCommandParam(command.getCommandParam(), - CommandKeyConstants.CMD_PARAM_START_PARAMS); - Map startParams = JSONUtils.toMap(startParamsJson); - - Assertions.assertEquals("testValue", startParams.get("testKey")); - Assertions.assertEquals("globalValue", startParams.get("globalKey")); - } - -} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTaskTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTaskTest.java deleted file mode 100644 index bcd6929f1fe58..0000000000000 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/task/dynamic/DynamicLogicTaskTest.java +++ /dev/null @@ -1,175 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.dolphinscheduler.server.master.runner.task.dynamic; - -import org.apache.dolphinscheduler.common.enums.CommandType; -import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; -import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; -import org.apache.dolphinscheduler.dao.mapper.CommandMapper; -import org.apache.dolphinscheduler.dao.mapper.WorkflowDefinitionMapper; -import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; -import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.plugin.task.api.model.DynamicInputParameter; -import org.apache.dolphinscheduler.plugin.task.api.parameters.DynamicParameters; -import org.apache.dolphinscheduler.service.process.ProcessService; -import org.apache.dolphinscheduler.service.subworkflow.SubWorkflowService; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Mock; -import org.mockito.Mockito; -import org.mockito.junit.jupiter.MockitoExtension; - -@ExtendWith(MockitoExtension.class) -class DynamicLogicTaskTest { - - @Mock - private WorkflowInstanceDao workflowInstanceDao; - - @Mock - private TaskInstanceDao taskInstanceDao; - - @Mock - private SubWorkflowService subWorkflowService; - - @Mock - private ProcessService processService; - - @Mock - private WorkflowDefinitionMapper processDefineMapper; - - @Mock - private CommandMapper commandMapper; - - private DynamicParameters dynamicParameters; - - private WorkflowInstance workflowInstance; - - private TaskExecutionContext taskExecutionContext; - - private DynamicLogicTask dynamicLogicTask; - - @BeforeEach - public void setUp() { - // Set up your test environment before each test. - dynamicParameters = new DynamicParameters(); - taskExecutionContext = Mockito.mock(TaskExecutionContext.class); - workflowInstance = new WorkflowInstance(); - Mockito.when(workflowInstanceDao.queryById(Mockito.any())).thenReturn(workflowInstance); - dynamicLogicTask = new DynamicLogicTask( - taskExecutionContext, - workflowInstanceDao, - taskInstanceDao, - subWorkflowService, - processService, - processDefineMapper, - commandMapper); - } - - @Test - void testGenerateParameterGroup() { - DynamicInputParameter dynamicInputParameter1 = new DynamicInputParameter(); - dynamicInputParameter1.setName("param1"); - dynamicInputParameter1.setValue("a,b,c"); - dynamicInputParameter1.setSeparator(","); - - DynamicInputParameter dynamicInputParameter2 = new DynamicInputParameter(); - dynamicInputParameter2.setName("param2"); - dynamicInputParameter2.setValue("1. 2 . 3"); - dynamicInputParameter2.setSeparator("."); - - List dynamicInputParameters = - Arrays.asList(dynamicInputParameter1, dynamicInputParameter2); - dynamicParameters.setListParameters(dynamicInputParameters); - dynamicParameters.setFilterCondition("b,2"); - - Mockito.when(taskExecutionContext.getPrepareParamsMap()).thenReturn(new HashMap<>()); - Mockito.when(taskExecutionContext.getTaskParams()) - .thenReturn(JSONUtils.toJsonString(dynamicParameters)); - - dynamicLogicTask = new DynamicLogicTask( - taskExecutionContext, - workflowInstanceDao, - taskInstanceDao, - subWorkflowService, - processService, - processDefineMapper, - commandMapper); - - List> parameterGroup = dynamicLogicTask.generateParameterGroup(); - - Assertions.assertEquals(4, parameterGroup.size()); // expected cartesian product without filtered values is 6 - - // Assert the value of parameter groups. Adjust these according to your expectations. - // Here we only check for a few representative cases to keep the test concise. - Map expectedMap1 = new HashMap<>(); - expectedMap1.put("param1", "a"); - expectedMap1.put("param2", "1"); - - Map expectedMap2 = new HashMap<>(); - expectedMap2.put("param1", "a"); - expectedMap2.put("param2", "3"); - - Map expectedMap3 = new HashMap<>(); - expectedMap3.put("param1", "c"); - expectedMap3.put("param2", "1"); - - Map expectedMap4 = new HashMap<>(); - expectedMap4.put("param1", "c"); - expectedMap4.put("param2", "3"); - - assert (parameterGroup.containsAll(Arrays.asList(expectedMap1, expectedMap2, expectedMap3, expectedMap4))); - } - - @Test - void testResetProcessInstanceStatus_RepeatRunning() { - workflowInstance.setCommandType(CommandType.REPEAT_RUNNING); - WorkflowInstance subWorkflowInstance = new WorkflowInstance(); - List subWorkflowInstances = Arrays.asList(subWorkflowInstance); - - dynamicLogicTask.resetProcessInstanceStatus(subWorkflowInstances); - - Mockito.verify(workflowInstanceDao).updateById(subWorkflowInstance); - Assertions.assertEquals(WorkflowExecutionStatus.WAIT_TO_RUN, subWorkflowInstance.getState()); - } - - @Test - void testResetProcessInstanceStatus_StartFailureTaskProcess() { - workflowInstance.setCommandType(CommandType.START_FAILURE_TASK_PROCESS); - WorkflowInstance failedSubWorkflowInstance = new WorkflowInstance(); - failedSubWorkflowInstance.setState(WorkflowExecutionStatus.FAILURE); - List subWorkflowInstances = Arrays.asList(failedSubWorkflowInstance); - Mockito.when(subWorkflowService.filterFailedProcessInstances(subWorkflowInstances)) - .thenReturn(Arrays.asList(failedSubWorkflowInstance)); - - dynamicLogicTask.resetProcessInstanceStatus(subWorkflowInstances); - - Mockito.verify(workflowInstanceDao).updateById(failedSubWorkflowInstance); - Assertions.assertEquals(WorkflowExecutionStatus.WAIT_TO_RUN, failedSubWorkflowInstance.getState()); - } - -} diff --git a/dolphinscheduler-standalone-server/src/main/resources/application.yaml b/dolphinscheduler-standalone-server/src/main/resources/application.yaml index 97b9b6d22d0c4..b8bfc6d174e7b 100644 --- a/dolphinscheduler-standalone-server/src/main/resources/application.yaml +++ b/dolphinscheduler-standalone-server/src/main/resources/application.yaml @@ -317,4 +317,4 @@ spring: driver-class-name: com.mysql.cj.jdbc.Driver url: jdbc:mysql://127.0.0.1:3306/dolphinscheduler?useUnicode=true&characterEncoding=UTF-8 username: root - password: root + password: root@123 diff --git a/dolphinscheduler-task-executor/pom.xml b/dolphinscheduler-task-executor/pom.xml new file mode 100644 index 0000000000000..cd55c08d22faa --- /dev/null +++ b/dolphinscheduler-task-executor/pom.xml @@ -0,0 +1,34 @@ + + + 4.0.0 + + org.apache.dolphinscheduler + dolphinscheduler + dev-SNAPSHOT + + + dolphinscheduler-task-executor + + + + + org.apache.dolphinscheduler + dolphinscheduler-eventbus + ${project.version} + + + + org.apache.dolphinscheduler + dolphinscheduler-task-api + ${project.version} + + + + org.apache.dolphinscheduler + dolphinscheduler-common + ${project.version} + + + + diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/AbstractTaskExecutor.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/AbstractTaskExecutor.java new file mode 100644 index 0000000000000..79ce2bc963971 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/AbstractTaskExecutor.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.task.executor; + +import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; + +import lombok.Getter; + +public abstract class AbstractTaskExecutor implements ITaskExecutor { + + @Getter + protected long latestStateRefreshTime; + + @Getter + protected TaskExecutorState taskExecutorState; + + public AbstractTaskExecutor() { + transitTaskExecutorState(TaskExecutorState.INITIALIZED); + } + + @Override + public TaskExecutorState refreshTaskExecutorState() { + latestStateRefreshTime = System.currentTimeMillis(); + taskExecutorState = TaskExecutorStateMappings.mapState(doTrackTaskPluginStatus()); + return taskExecutorState; + } + + @Override + public void transitTaskExecutorState(final TaskExecutorState taskExecutorState) { + this.taskExecutorState = taskExecutorState; + } + + protected abstract TaskExecutionStatus doTrackTaskPluginStatus(); + +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskEngine.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskEngine.java new file mode 100644 index 0000000000000..ce0e85bbc7e94 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskEngine.java @@ -0,0 +1,64 @@ +/* + * 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.task.executor; + +import org.apache.dolphinscheduler.task.executor.exceptions.TaskExecutorRuntimeException; + +/** + * The TaskEngine interface used to responsible for task runtime. + */ +public interface ITaskEngine extends AutoCloseable { + + /** + * Start the task engine. + */ + void start(); + + /** + * Submit the task to the task engine. + * + * @param taskExecutor the task executor to be submitted + * @throws TaskExecutorRuntimeException if submit failed. + */ + void submitTask(final ITaskExecutor taskExecutor) throws TaskExecutorRuntimeException; + + /** + * Pause the task by the task id. + *

This method will not block, only send the pause signal to the task executor + * + * @param taskExecutorId the id of the task executor will be paused + * @throws TaskExecutorRuntimeException if pause failed. + */ + void pauseTask(final int taskExecutorId) throws TaskExecutorRuntimeException; + + /** + * Kill the task by the task id. + *

This method will not block, only send the kill signal to the task executor. + * + * @param taskExecutorId the id of the task executor will be killed. + * @throws TaskExecutorRuntimeException if kill failed. + */ + void killTask(final int taskExecutorId) throws TaskExecutorRuntimeException; + + /** + * Shutdown the task engine. + */ + @Override + void close(); + +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutor.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutor.java new file mode 100644 index 0000000000000..489487d7ccdcd --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutor.java @@ -0,0 +1,71 @@ +/* + * 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.task.executor; + +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; + +public interface ITaskExecutor extends ITaskExecutorStateTransition { + + /** + * Get the id of the task executor. + *

Each task executor has a unique id. + */ + default Integer getId() { + return getTaskExecutionContext().getTaskInstanceId(); + } + + /** + * Get the name of the task executor. + */ + default String getName() { + return getTaskExecutionContext().getTaskName(); + } + + /** + * Get the type of the task executor. + */ + default String getTaskType() { + return getTaskExecutionContext().getTaskType(); + } + + /** + * Get the TaskExecutionContext of the task executor. + */ + TaskExecutionContext getTaskExecutionContext(); + + /** + * Start the task executor. + */ + void start(); + + /** + * Pause the task executor. + */ + void pause(); + + /** + * Kill the task executor. + */ + void kill(); + + /** + * Get the EventBus belongs to the task executor. + */ + TaskExecutorEventBus getTaskExecutorEventBus(); + +} diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskInstanceExecutionEventSender.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorEventBusCoordinator.java similarity index 58% rename from dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskInstanceExecutionEventSender.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorEventBusCoordinator.java index 228f7c76fa231..5b0dcbd63866c 100644 --- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskInstanceExecutionEventSender.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorEventBusCoordinator.java @@ -15,26 +15,29 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.worker.message; +package org.apache.dolphinscheduler.task.executor; -import org.apache.dolphinscheduler.extract.master.transportor.ITaskExecutionEvent; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.task.executor.listener.ITaskExecutorLifecycleEventListener; -public interface TaskInstanceExecutionEventSender { +/** + * The coordinator responsible to fire the {@link TaskExecutorEventBus}. + */ +public interface ITaskExecutorEventBusCoordinator extends AutoCloseable { /** - * Send the message - * + * Start the coordinator. */ - void sendEvent(T message); + void start(); /** - * Build the message from task context and message received address. + * Register the task executor event listener */ - T buildEvent(TaskExecutionContext taskExecutionContext); + void registerTaskExecutorLifecycleEventListener(final ITaskExecutorLifecycleEventListener taskExecutorLifecycleEventListener); /** - * The message type can be sent by this sender. + * Shutdown the coordinator */ - ITaskExecutionEvent.TaskInstanceExecutionEventType getMessageType(); + @Override + void close(); + } diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorEventRemoteReporterClient.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorEventRemoteReporterClient.java new file mode 100644 index 0000000000000..2a3d0649b254f --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorEventRemoteReporterClient.java @@ -0,0 +1,25 @@ +/* + * 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.task.executor; + +import org.apache.dolphinscheduler.task.executor.event.IReportableTaskExecutorLifecycleEvent; + +public interface ITaskExecutorEventRemoteReporterClient { + + void reportTaskExecutionEventToMaster(final IReportableTaskExecutorLifecycleEvent reportableTaskExecutorLifecycleEvent); +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorFactory.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorFactory.java similarity index 80% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorFactory.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorFactory.java index d1f76aedbc707..a0a582259d004 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutorFactory.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorFactory.java @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.execute; +package org.apache.dolphinscheduler.task.executor; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -public interface MasterTaskExecutorFactory { - - T createMasterTaskExecutor(TaskExecutionContext taskExecutionContext); +public interface ITaskExecutorFactory { + ITaskExecutor createTaskExecutor(final TaskExecutionContext taskExecutionContext); } diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorLifecycleEventReporter.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorLifecycleEventReporter.java new file mode 100644 index 0000000000000..f0e4e5f56dd33 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorLifecycleEventReporter.java @@ -0,0 +1,72 @@ +/* + * 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.task.executor; + +import org.apache.dolphinscheduler.task.executor.event.IReportableTaskExecutorLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorLifecycleEventType; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +/** + * The interface of task executor event reporter, used to report task execution events to the master. + *

Once the task executor lifecycle changed, need to report the event to the master. + */ +public interface ITaskExecutorLifecycleEventReporter extends AutoCloseable { + + /** + * Start the reporter. + */ + void start(); + + /** + * Receive the task executor event, which need to be reported to the master. + *

This method is not sync, will only put the event into a pending queue and send the event in another thread. + */ + void reportTaskExecutorLifecycleEvent(final IReportableTaskExecutorLifecycleEvent reportableTaskExecutorLifecycleEvent); + + /** + * Receive the task execution event ACK, which means the master has received the event. + *

Once the reporter received the ACK, will remove the event from the pending queue, otherwise will retry to send the event to master at interval. + */ + void receiveTaskExecutorLifecycleEventACK(final TaskExecutorLifecycleEventAck taskExecutorLifecycleEventAck); + + /** + * Reassign the workflow instance host of the IReportableTaskExecutorLifecycleEvent. + *

This method is used to reassign the workflow instance host of the IReportableTaskExecutorLifecycleEvent, once the workflow's host changed. + */ + boolean reassignWorkflowInstanceHost(int taskInstanceId, String workflowHost); + + /** + * Shutdown the reporter. + */ + @Override + void close(); + + @Data + @NoArgsConstructor + @AllArgsConstructor + class TaskExecutorLifecycleEventAck { + + private int taskExecutorId; + + private TaskExecutorLifecycleEventType taskExecutorLifecycleEventType; + } + +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorRepository.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorRepository.java new file mode 100644 index 0000000000000..69828cfb7a624 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorRepository.java @@ -0,0 +1,34 @@ +/* + * 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.task.executor; + +import java.util.Collection; + +public interface ITaskExecutorRepository { + + void put(final ITaskExecutor taskExecutor); + + ITaskExecutor get(final Integer taskExecutorId); + + Collection getAll(); + + boolean contains(final Integer taskExecutorId); + + void remove(final Integer taskExecutorId); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionEventSender.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorStateTransition.java similarity index 57% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionEventSender.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorStateTransition.java index 3b3ff7f6454f6..38082e98d3eea 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionEventSender.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/ITaskExecutorStateTransition.java @@ -15,26 +15,35 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.message; +package org.apache.dolphinscheduler.task.executor; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +public interface ITaskExecutorStateTransition { -// todo: unify with WorkerMessageSender -public interface LogicTaskInstanceExecutionEventSender { + /** + * Track the state of the task executor. + */ + TaskExecutorState refreshTaskExecutorState(); /** - * Send the message + * Get the latest state track time. */ - void sendMessage(T message); + long getLatestStateRefreshTime(); - default void sendMessage(TaskExecutionContext taskExecutionContext) { - T message = buildMessage(taskExecutionContext); - sendMessage(message); + /** + * The interval of state refresh in ms. + */ + default long getStateRefreshInterval() { + return 10_000; } /** - * Build the message from task context and message received address. + * Set the task executor status. + */ + void transitTaskExecutorState(final TaskExecutorState taskExecutorState); + + /** + * Get the task executor status. */ - T buildMessage(TaskExecutionContext taskExecutionContext); + TaskExecutorState getTaskExecutorState(); } diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskEngine.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskEngine.java new file mode 100644 index 0000000000000..1c5f80859830b --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskEngine.java @@ -0,0 +1,99 @@ +/* + * 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.task.executor; + +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.task.executor.container.ITaskExecutorContainer; +import org.apache.dolphinscheduler.task.executor.container.ITaskExecutorContainerDelegator; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorDispatchedLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorKillLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorPauseLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.exceptions.TaskExecutorRuntimeException; +import org.apache.dolphinscheduler.task.executor.log.TaskExecutorMDCUtils; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class TaskEngine implements ITaskEngine { + + private final ITaskExecutorRepository taskExecutorRepository; + + private final ITaskExecutorContainerDelegator taskExecutorContainerDelegator; + + private final TaskExecutorEventBusCoordinator taskExecutorEventBusCoordinator; + + public TaskEngine(final TaskEngineBuilder taskEngineBuilder) { + this.taskExecutorContainerDelegator = taskEngineBuilder.getTaskExecutorContainerDelegator(); + this.taskExecutorRepository = taskEngineBuilder.getTaskExecutorRepository(); + this.taskExecutorEventBusCoordinator = taskEngineBuilder.getTaskExecutorEventBusCoordinator(); + } + + public void start() { + taskExecutorEventBusCoordinator.start(); + log.info("TaskEngine started"); + } + + @Override + public void submitTask(final ITaskExecutor taskExecutor) throws TaskExecutorRuntimeException { + try (final TaskExecutorMDCUtils.MDCAutoClosable ignore = TaskExecutorMDCUtils.logWithMDC(taskExecutor)) { + getExecutorContainer(taskExecutor).dispatch(taskExecutor); + taskExecutorRepository.put(taskExecutor); + taskExecutor.getTaskExecutorEventBus().publish(TaskExecutorDispatchedLifecycleEvent.of(taskExecutor)); + getExecutorContainer(taskExecutor).start(taskExecutor); + } + } + + @Override + public void pauseTask(final int taskExecutorId) throws TaskExecutorRuntimeException { + final ITaskExecutor taskExecutor = getTaskExecutor(taskExecutorId); + try (final TaskExecutorMDCUtils.MDCAutoClosable ignore = TaskExecutorMDCUtils.logWithMDC(taskExecutor)) { + taskExecutor.getTaskExecutorEventBus().publish(TaskExecutorPauseLifecycleEvent.of(taskExecutor)); + } + } + + @Override + public void killTask(final int taskExecutorId) throws TaskExecutorRuntimeException { + final ITaskExecutor taskExecutor = getTaskExecutor(taskExecutorId); + try (final TaskExecutorMDCUtils.MDCAutoClosable ignore = TaskExecutorMDCUtils.logWithMDC(taskExecutor)) { + taskExecutor.getTaskExecutorEventBus().publish(TaskExecutorKillLifecycleEvent.of(taskExecutor)); + } + } + + @Override + public void close() { + try (final TaskExecutorEventBusCoordinator ignore1 = taskExecutorEventBusCoordinator) { + // Ignore + } + } + + private ITaskExecutor getTaskExecutor(final int taskId) { + final ITaskExecutor taskExecutor = taskExecutorRepository.get(taskId); + if (taskExecutor == null) { + // todo: throw TaskExecutorNotFoundException + throw new IllegalStateException("Cannot find the TaskExecutor(id=" + taskId + ")"); + } + return taskExecutor; + } + + private ITaskExecutorContainer getExecutorContainer(final ITaskExecutor taskExecutor) { + final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext(); + final String taskType = taskExecutionContext.getTaskType(); + return taskExecutorContainerDelegator.getExecutorContainer(taskType); + } + +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskEngineBuilder.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskEngineBuilder.java new file mode 100644 index 0000000000000..80ca4e6c96524 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskEngineBuilder.java @@ -0,0 +1,39 @@ +/* + * 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.task.executor; + +import org.apache.dolphinscheduler.task.executor.container.ITaskExecutorContainerDelegator; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +public class TaskEngineBuilder { + + private ITaskExecutorRepository taskExecutorRepository; + + private ITaskExecutorContainerDelegator taskExecutorContainerDelegator; + + private TaskExecutorEventBusCoordinator TaskExecutorEventBusCoordinator; + +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskExecutorEventBus.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskExecutorEventBus.java new file mode 100644 index 0000000000000..0bff84871b506 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskExecutorEventBus.java @@ -0,0 +1,33 @@ +/* + * 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.task.executor; + +import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.eventbus.AbstractDelayEventBus; +import org.apache.dolphinscheduler.task.executor.event.AbstractTaskExecutorLifecycleEvent; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class TaskExecutorEventBus extends AbstractDelayEventBus { + + public void publish(final AbstractTaskExecutorLifecycleEvent event) { + super.publish(event); + log.info("Publish {}: {}", event.getClass().getSimpleName(), JSONUtils.toPrettyJsonString(event)); + } +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskExecutorEventBusCoordinator.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskExecutorEventBusCoordinator.java new file mode 100644 index 0000000000000..881c6ea8207e7 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskExecutorEventBusCoordinator.java @@ -0,0 +1,197 @@ +/* + * 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.task.executor; + +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.dolphinscheduler.common.thread.ThreadUtils; +import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.task.executor.event.AbstractTaskExecutorLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.ITaskExecutorLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorDispatchedLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorFailedLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorFinalizeLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorKillLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorKilledLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorPauseLifecycleEvent; +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 org.apache.dolphinscheduler.task.executor.listener.ITaskExecutorLifecycleEventListener; +import org.apache.dolphinscheduler.task.executor.log.TaskExecutorMDCUtils; + +import org.apache.commons.collections4.CollectionUtils; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class TaskExecutorEventBusCoordinator implements ITaskExecutorEventBusCoordinator { + + private final ITaskExecutorRepository taskExecutorRepository; + + private final List taskExecutorLifecycleEventListeners; + + private static final int DEFAULT_WORKER_SIZE = 1; + + private static final long DEFAULT_FIRE_INTERVAL = 100; + + private static final Set firingTaskExecutorIds = ConcurrentHashMap.newKeySet(); + + private ScheduledExecutorService mainExecutorThreadPool; + + private ThreadPoolExecutor workerExecutorThreadPool; + + public TaskExecutorEventBusCoordinator(final ITaskExecutorRepository taskExecutorRepository) { + this.taskExecutorRepository = taskExecutorRepository; + this.taskExecutorLifecycleEventListeners = new ArrayList<>(); + } + + public void start() { + mainExecutorThreadPool = ThreadUtils.newDaemonScheduledExecutorService( + "TaskExecutorEventBusCoordinator-main-%d", 1); + // todo: use a event dispatcher to control a condition + mainExecutorThreadPool.scheduleWithFixedDelay( + this::fireTaskExecutorEventBus, + 0, + DEFAULT_FIRE_INTERVAL, + TimeUnit.MILLISECONDS); + + workerExecutorThreadPool = ThreadUtils.newDaemonFixedThreadExecutor( + "TaskExecutorEventBusCoordinator-worker-%d", DEFAULT_WORKER_SIZE); + log.info("TaskExecutorEventBusCoordinator started, worker size: {}", DEFAULT_WORKER_SIZE); + } + + @Override + public void registerTaskExecutorLifecycleEventListener(final ITaskExecutorLifecycleEventListener taskExecutorLifecycleEventListener) { + checkNotNull(taskExecutorLifecycleEventListener); + taskExecutorLifecycleEventListeners.add(taskExecutorLifecycleEventListener); + } + + @Override + public void close() { + mainExecutorThreadPool.shutdownNow(); + log.info("TaskExecutorEventBusCoordinator closed"); + } + + private void fireTaskExecutorEventBus() { + try { + + final Collection taskExecutors = taskExecutorRepository.getAll(); + if (CollectionUtils.isEmpty(taskExecutors)) { + return; + } + for (final ITaskExecutor taskExecutor : taskExecutors) { + if (isFiring(taskExecutor)) { + continue; + } + final Integer taskExecutorId = taskExecutor.getId(); + CompletableFuture + .runAsync(() -> firingTaskExecutorIds.add(taskExecutorId), workerExecutorThreadPool) + .thenAccept((v) -> doFireTaskExecutorEventBus(taskExecutor)) + .whenComplete((v, e) -> firingTaskExecutorIds.remove(taskExecutorId)); + } + } catch (Throwable throwable) { + log.error("Fire TaskExecutorEventBus error", throwable); + } + } + + private void doFireTaskExecutorEventBus(final ITaskExecutor taskExecutor) { + try (final TaskExecutorMDCUtils.MDCAutoClosable ignored = TaskExecutorMDCUtils.logWithMDC(taskExecutor)) { + final TaskExecutorEventBus taskExecutorEventBus = taskExecutor.getTaskExecutorEventBus(); + if (taskExecutorEventBus.isEmpty()) { + return; + } + Optional headEventOptional = taskExecutorEventBus.poll(); + if (!headEventOptional.isPresent()) { + return; + } + final ITaskExecutorLifecycleEvent taskExecutorLifecycleEvent = headEventOptional.get(); + try { + for (final ITaskExecutorLifecycleEventListener taskExecutorLifecycleEventListener : taskExecutorLifecycleEventListeners) { + switch (taskExecutorLifecycleEvent.getType()) { + case DISPATCHED: + taskExecutorLifecycleEventListener.onTaskExecutorDispatchedLifecycleEvent( + ((TaskExecutorDispatchedLifecycleEvent) taskExecutorLifecycleEvent)); + break; + case RUNNING: + taskExecutorLifecycleEventListener.onTaskExecutorStartedLifecycleEvent( + ((TaskExecutorStartedLifecycleEvent) taskExecutorLifecycleEvent)); + break; + case RUNTIME_CONTEXT_CHANGE: + taskExecutorLifecycleEventListener.onTaskExecutorRuntimeContextChangedEvent( + ((TaskExecutorRuntimeContextChangedLifecycleEvent) taskExecutorLifecycleEvent)); + break; + case PAUSE: + taskExecutorLifecycleEventListener.onTaskExecutorPauseLifecycleEvent( + ((TaskExecutorPauseLifecycleEvent) taskExecutorLifecycleEvent)); + break; + case PAUSED: + taskExecutorLifecycleEventListener.onTaskExecutorPausedLifecycleEvent( + ((TaskExecutorPausedLifecycleEvent) taskExecutorLifecycleEvent)); + break; + case KILL: + taskExecutorLifecycleEventListener.onTaskExecutorKillLifecycleEvent( + ((TaskExecutorKillLifecycleEvent) taskExecutorLifecycleEvent)); + break; + case KILLED: + taskExecutorLifecycleEventListener.onTaskExecutorKilledLifecycleEvent( + ((TaskExecutorKilledLifecycleEvent) taskExecutorLifecycleEvent)); + break; + case SUCCESS: + taskExecutorLifecycleEventListener.onTaskExecutorSuccessLifecycleEvent( + ((TaskExecutorSuccessLifecycleEvent) taskExecutorLifecycleEvent)); + break; + case FAILED: + taskExecutorLifecycleEventListener.onTaskExecutorFailLifecycleEvent( + ((TaskExecutorFailedLifecycleEvent) taskExecutorLifecycleEvent)); + break; + case FINALIZE: + taskExecutorLifecycleEventListener.onTaskExecutorFinalizeLifecycleEvent( + ((TaskExecutorFinalizeLifecycleEvent) taskExecutorLifecycleEvent)); + break; + default: + throw new IllegalArgumentException( + "Unsupported ITaskExecutorLifecycleEvent: " + taskExecutorLifecycleEvent); + } + } + log.info("Success fire {}: {} ", + taskExecutorLifecycleEvent.getClass().getSimpleName(), + JSONUtils.toPrettyJsonString(taskExecutorLifecycleEvent)); + } catch (Exception e) { + log.error("Fire ITaskExecutorLifecycleEvent: {} error", taskExecutorLifecycleEvent, e); + } + } + } + + private boolean isFiring(final ITaskExecutor taskExecutor) { + return firingTaskExecutorIds.contains(taskExecutor.getId()); + } + +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskExecutorLifecycleEventRemoteReporter.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskExecutorLifecycleEventRemoteReporter.java new file mode 100644 index 0000000000000..1cd730bdf54ab --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskExecutorLifecycleEventRemoteReporter.java @@ -0,0 +1,272 @@ +/* + * 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.task.executor; + +import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; +import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.task.executor.event.IReportableTaskExecutorLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorLifecycleEventType; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class TaskExecutorLifecycleEventRemoteReporter extends BaseDaemonThread + implements + ITaskExecutorLifecycleEventReporter { + + private static final Long DEFAULT_TASK_EXECUTOR_EVENT_RETRY_INTERVAL = TimeUnit.MINUTES.toMillis(3); + + private final String reporterName; + + private final Map eventChannels = new ConcurrentHashMap<>(); + + private final ITaskExecutorEventRemoteReporterClient taskExecutorEventRemoteReporterClient; + + private volatile boolean runningFlag; + + private final Lock eventChannelsLock = new ReentrantLock(); + + private final Condition taskExecutionEventEmptyCondition = eventChannelsLock.newCondition(); + + public TaskExecutorLifecycleEventRemoteReporter(final String reporterName, + final ITaskExecutorEventRemoteReporterClient taskExecutorEventRemoteReporterClient) { + super(reporterName); + this.reporterName = reporterName; + this.taskExecutorEventRemoteReporterClient = taskExecutorEventRemoteReporterClient; + } + + @Override + public void start() { + // start a thread to send the events + this.runningFlag = true; + super.start(); + log.info("{} started", reporterName); + } + + @Override + public void run() { + while (runningFlag) { + try { + for (final ReportableTaskExecutorLifecycleEventChannel eventChannel : eventChannels.values()) { + if (eventChannel.isEmpty()) { + continue; + } + handleTaskExecutionEventChannel(eventChannel); + } + tryToWaitIfAllTaskExecutionEventChannelEmpty(); + waitIfAnyTaskExecutionEventChannelRetryIntervalPassed(); + } catch (InterruptedException e) { + log.info("{} interrupted", reporterName); + Thread.currentThread().interrupt(); + break; + } catch (Exception ex) { + log.error("Fire ReportableTaskExecutorLifecycleEventChannel error", ex); + } + } + log.info("{} break loop", reporterName); + } + + @Override + public void reportTaskExecutorLifecycleEvent(final IReportableTaskExecutorLifecycleEvent reportableTaskExecutorLifecycleEvent) { + eventChannelsLock.lock(); + try { + log.debug("Report : {}", JSONUtils.toPrettyJsonString(reportableTaskExecutorLifecycleEvent)); + int taskInstanceId = reportableTaskExecutorLifecycleEvent.getTaskInstanceId(); + eventChannels.computeIfAbsent( + taskInstanceId, + k -> new ReportableTaskExecutorLifecycleEventChannel(taskInstanceId)) + .addTaskExecutionEvent(reportableTaskExecutorLifecycleEvent); + taskExecutionEventEmptyCondition.signalAll(); + } finally { + eventChannelsLock.unlock(); + } + + } + + @Override + public void receiveTaskExecutorLifecycleEventACK(final TaskExecutorLifecycleEventAck eventAck) { + final int taskExecutorId = eventAck.getTaskExecutorId(); + final TaskExecutorLifecycleEventType eventType = eventAck.getTaskExecutorLifecycleEventType(); + log.info("Receive TaskExecutionEventACK: {} from TaskExecutor: {}", + eventType, + taskExecutorId); + eventChannelsLock.lock(); + try { + final ReportableTaskExecutorLifecycleEventChannel eventChannel = eventChannels.get(taskExecutorId); + if (eventChannel == null) { + return; + } + if (eventChannel.remove(eventAck.getTaskExecutorLifecycleEventType())) { + log.info("Success removed ReportableTaskExecutorLifecycleEvent by ack: {}", eventAck); + } else { + log.info("Failed removed ReportableTaskExecutorLifecycleEvent by ack: {}", eventAck); + } + if (eventChannel.isEmpty()) { + eventChannels.remove(taskExecutorId); + log.debug("Removed ReportableTaskExecutorLifecycleEventChannel: {}", taskExecutorId); + } + taskExecutionEventEmptyCondition.signalAll(); + } finally { + eventChannelsLock.unlock(); + } + } + + @Override + public boolean reassignWorkflowInstanceHost(int taskInstanceId, String workflowHost) { + eventChannelsLock.lock(); + try { + final ReportableTaskExecutorLifecycleEventChannel eventChannel = eventChannels.get(taskInstanceId); + if (eventChannel == null) { + return false; + } + eventChannel.taskExecutionEventsQueue.forEach(event -> event.setWorkflowInstanceHost(workflowHost)); + return true; + } finally { + eventChannelsLock.unlock(); + } + } + + @Override + public void close() { + // shutdown the thread + runningFlag = false; + log.info("{} closed", reporterName); + } + + private void handleTaskExecutionEventChannel(final ReportableTaskExecutorLifecycleEventChannel reportableTaskExecutorLifecycleEventChannel) { + if (reportableTaskExecutorLifecycleEventChannel.isEmpty()) { + return; + } + while (!reportableTaskExecutorLifecycleEventChannel.isEmpty()) { + final IReportableTaskExecutorLifecycleEvent headEvent = reportableTaskExecutorLifecycleEventChannel.peek(); + try { + if (isTaskExecutorEventNeverSent(headEvent) || isRetryIntervalExceeded(headEvent)) { + taskExecutorEventRemoteReporterClient.reportTaskExecutionEventToMaster(headEvent); + continue; + } + if (log.isDebugEnabled()) { + log.debug( + "The ReportableTaskExecutorLifecycleEvent: {} latest send time: {} doesn't exceeded retry interval", + headEvent, + headEvent.getLatestReportTime()); + } + break; + } catch (Exception ex) { + log.error("Send TaskExecutionEvent: {} to master error will retry after 1 minutes", headEvent, ex); + break; + } + } + } + + private boolean isAllTaskExecutorEventChannelEmpty() { + return eventChannels + .values() + .stream() + .allMatch(ReportableTaskExecutorLifecycleEventChannel::isEmpty); + } + + private long getOldestReportTime() { + return eventChannels.values() + .stream() + .filter(ReportableTaskExecutorLifecycleEventChannel::isNotEmpty) + .map(ReportableTaskExecutorLifecycleEventChannel::peek) + .filter(event -> !isTaskExecutorEventNeverSent(event)) + .map(IReportableTaskExecutorLifecycleEvent::getLatestReportTime) + .min(Long::compareTo) + .orElse(0L); + } + + private boolean isTaskExecutorEventNeverSent(final IReportableTaskExecutorLifecycleEvent headEvent) { + return headEvent.getLatestReportTime() == null; + } + + private boolean isRetryIntervalExceeded(final IReportableTaskExecutorLifecycleEvent reportableTaskExecutorLifecycleEvent) { + if (isTaskExecutorEventNeverSent(reportableTaskExecutorLifecycleEvent)) { + return true; + } + long currentTime = System.currentTimeMillis(); + return currentTime - reportableTaskExecutorLifecycleEvent + .getLatestReportTime() > DEFAULT_TASK_EXECUTOR_EVENT_RETRY_INTERVAL; + } + + private void tryToWaitIfAllTaskExecutionEventChannelEmpty() throws InterruptedException { + eventChannelsLock.lock(); + while (isAllTaskExecutorEventChannelEmpty()) { + taskExecutionEventEmptyCondition.await(); + } + eventChannelsLock.unlock(); + } + + private void waitIfAnyTaskExecutionEventChannelRetryIntervalPassed() throws InterruptedException { + eventChannelsLock.lock(); + try { + final long waitInterval = + (getOldestReportTime() + DEFAULT_TASK_EXECUTOR_EVENT_RETRY_INTERVAL) - System.currentTimeMillis(); + if (waitInterval <= 0) { + return; + } + taskExecutionEventEmptyCondition.await(waitInterval, TimeUnit.MILLISECONDS); + } finally { + eventChannelsLock.unlock(); + } + } + + public static class ReportableTaskExecutorLifecycleEventChannel { + + @Getter + private final int taskExecutorId; + + private final LinkedBlockingQueue taskExecutionEventsQueue; + + // todo: remove the master address from the channel, we need to get the master address from the TaskExecutor + public ReportableTaskExecutorLifecycleEventChannel(int taskExecutorId) { + this.taskExecutorId = taskExecutorId; + this.taskExecutionEventsQueue = new LinkedBlockingQueue<>(); + } + + public void addTaskExecutionEvent(final IReportableTaskExecutorLifecycleEvent reportableTaskExecutorLifecycleEvent) { + taskExecutionEventsQueue.add(reportableTaskExecutorLifecycleEvent); + } + + public IReportableTaskExecutorLifecycleEvent peek() { + return taskExecutionEventsQueue.peek(); + } + + public boolean remove(TaskExecutorLifecycleEventType type) { + return taskExecutionEventsQueue.removeIf(event -> event.getType() == type); + } + + public boolean isEmpty() { + return taskExecutionEventsQueue.isEmpty(); + } + + public boolean isNotEmpty() { + return !isEmpty(); + } + + } +} diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/ITaskInstanceOperationFunction.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskExecutorState.java similarity index 82% rename from dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/ITaskInstanceOperationFunction.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskExecutorState.java index e8865c985cb14..ddb128892e013 100644 --- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/operator/ITaskInstanceOperationFunction.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskExecutorState.java @@ -15,10 +15,20 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.worker.runner.operator; +package org.apache.dolphinscheduler.task.executor; -public interface ITaskInstanceOperationFunction { +public enum TaskExecutorState { - Y operate(X x); + INITIALIZED, + + RUNNING, + + PAUSED, + + KILLED, + + FAILED, + + SUCCEEDED, } diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionRunningEvent.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskExecutorStateMappings.java similarity index 52% rename from dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionRunningEvent.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskExecutorStateMappings.java index b99d8ea28a3fc..5b25dba5af209 100644 --- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionRunningEvent.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/TaskExecutorStateMappings.java @@ -15,45 +15,27 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.extract.master.transportor; +package org.apache.dolphinscheduler.task.executor; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; -import lombok.AllArgsConstructor; -import lombok.Data; -import lombok.NoArgsConstructor; - -@Data -@NoArgsConstructor -@AllArgsConstructor -public class TaskExecutionRunningEvent implements ITaskExecutionEvent { - - private int taskInstanceId; - - private int workflowInstanceId; - - private long startTime; - - private String taskInstanceHost; - - private String workflowInstanceHost; - - private TaskExecutionStatus status; - - private String logPath; - - private String executePath; - - private int processId; - - private String appIds; - - private long eventCreateTime; - - private long eventSendTime; - - @Override - public TaskInstanceExecutionEventType getEventType() { - return TaskInstanceExecutionEventType.RUNNING; +public class TaskExecutorStateMappings { + + public static TaskExecutorState mapState(final TaskExecutionStatus taskExecutionStatus) { + switch (taskExecutionStatus) { + case RUNNING_EXECUTION: + return TaskExecutorState.RUNNING; + case SUCCESS: + return TaskExecutorState.SUCCEEDED; + case FAILURE: + return TaskExecutorState.FAILED; + case KILL: + return TaskExecutorState.KILLED; + case PAUSE: + return TaskExecutorState.PAUSED; + default: + return TaskExecutorState.INITIALIZED; + } } + } diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/AbstractTaskExecutorContainer.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/AbstractTaskExecutorContainer.java new file mode 100644 index 0000000000000..3ddbdc004d980 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/AbstractTaskExecutorContainer.java @@ -0,0 +1,122 @@ +/* + * 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.task.executor.container; + +import static ch.qos.logback.classic.ClassicConstants.FINALIZE_SESSION_MARKER; + +import org.apache.dolphinscheduler.common.log.remote.RemoteLogUtils; +import org.apache.dolphinscheduler.common.thread.ThreadUtils; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; +import org.apache.dolphinscheduler.task.executor.exceptions.TaskExecutorRuntimeException; +import org.apache.dolphinscheduler.task.executor.worker.TaskExecutorWorker; +import org.apache.dolphinscheduler.task.executor.worker.TaskExecutorWorkers; + +import java.util.Optional; +import java.util.concurrent.ThreadPoolExecutor; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public abstract class AbstractTaskExecutorContainer implements ITaskExecutorContainer { + + private final TaskExecutorRouterTable taskExecutorRouterTable; + + protected final ThreadPoolExecutor taskExecutorThreadPool; + + protected final TaskExecutorWorkers taskExecutorWorkers; + + public AbstractTaskExecutorContainer(final TaskExecutorContainerBuilder taskExecutorContainerBuilder, + final TaskExecutorContainerConfig config) { + this.taskExecutorThreadPool = ThreadUtils.newDaemonFixedThreadExecutor( + taskExecutorContainerBuilder.getTaskExecutorContainerName() + "-taskExecutorWorker-%d", + config.getTaskExecutorThreadPoolSize()); + this.taskExecutorWorkers = new TaskExecutorWorkers(config.getTaskExecutorThreadPoolSize()); + this.taskExecutorRouterTable = new TaskExecutorRouterTable(); + startAllThreadTaskExecutorWorker(); + } + + @Override + public void dispatch(final ITaskExecutor taskExecutor) { + synchronized (this) { + Optional taskExecutorWorkerCandidate = getTaskExecutorWorkerCandidate(taskExecutor); + if (!taskExecutorWorkerCandidate.isPresent()) { + log.info("All ExclusiveThreadTaskExecutorWorker are busy, cannot submit taskExecutor(id={})", + taskExecutor.getId()); + throw new TaskExecutorRuntimeException("All ExclusiveThreadTaskExecutorWorker are busy"); + } + final TaskExecutorWorker taskExecutorWorker = taskExecutorWorkerCandidate.get(); + taskExecutorWorker.registerTaskExecutor(taskExecutor); + taskExecutorRouterTable.registerTaskExecutor(taskExecutor, taskExecutorWorker); + } + } + + @Override + public void start(final ITaskExecutor taskExecutor) { + final Integer workerId = taskExecutorRouterTable.getTaskExecutorWorkerId(taskExecutor); + if (workerId == null) { + throw new IllegalStateException( + "The taskExecutor: " + taskExecutor.getId() + " is not registered to any worker"); + } + final TaskExecutorWorker taskExecutorWorker = taskExecutorWorkers.getTaskExecutorWorkerById(workerId); + taskExecutorWorker.fireTaskExecutor(taskExecutor); + } + + @Override + public void pause(final ITaskExecutor taskExecutor) { + taskExecutor.pause(); + } + + @Override + public void kill(final ITaskExecutor taskExecutor) { + taskExecutor.kill(); + } + + @Override + public void finalize(final ITaskExecutor taskExecutor) { + if (taskExecutorRouterTable.isTaskExecutorRegistered(taskExecutor)) { + final Integer taskExecutorWorkerId = taskExecutorRouterTable.getTaskExecutorWorkerId(taskExecutor); + taskExecutorWorkers.getTaskExecutorWorkerById(taskExecutorWorkerId).unRegisterTaskExecutor(taskExecutor); + taskExecutorRouterTable.unregisterTaskExecutor(taskExecutor); + } + log.info(FINALIZE_SESSION_MARKER, FINALIZE_SESSION_MARKER.toString()); + pushTaskExecutorLogToRemote(taskExecutor); + } + + private void startAllThreadTaskExecutorWorker() { + for (final TaskExecutorWorker taskExecutorWorker : taskExecutorWorkers.getTaskExecutorWorkers()) { + taskExecutorThreadPool.submit(taskExecutorWorker::start); + } + } + + protected abstract Optional getTaskExecutorWorkerCandidate(final ITaskExecutor taskExecutor); + + private void pushTaskExecutorLogToRemote(final ITaskExecutor taskExecutor) { + // todo: move this to customer listener, e.g. RemoteLogApaptor + final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext(); + try { + if (RemoteLogUtils.isRemoteLoggingEnable()) { + RemoteLogUtils.sendRemoteLog(taskExecutionContext.getLogPath()); + log.info("Send task log {} to remote storage successfully", taskExecutionContext.getLogPath()); + } + } catch (Exception ex) { + log.error("Send task log {} to remote storage failed", taskExecutionContext.getLogPath(), ex); + } + } + +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/ExclusiveThreadTaskExecutorContainer.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/ExclusiveThreadTaskExecutorContainer.java new file mode 100644 index 0000000000000..952a93d24748c --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/ExclusiveThreadTaskExecutorContainer.java @@ -0,0 +1,43 @@ +/* + * 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.task.executor.container; + +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; +import org.apache.dolphinscheduler.task.executor.worker.TaskExecutorWorker; + +import java.util.Optional; + +import lombok.extern.slf4j.Slf4j; + +/** + * The container to execute a {@link ITaskExecutor} with exclusive thread mode. + *

Each task executor will be executed by a single thread. + */ +@Slf4j +public class ExclusiveThreadTaskExecutorContainer extends AbstractTaskExecutorContainer { + + public ExclusiveThreadTaskExecutorContainer(final TaskExecutorContainerBuilder taskExecutorContainerBuilder) { + super(taskExecutorContainerBuilder, taskExecutorContainerBuilder.getTaskExecutorContainerConfig()); + } + + @Override + protected Optional getTaskExecutorWorkerCandidate(ITaskExecutor taskExecutor) { + return taskExecutorWorkers.getIdleTaskExecutorWorker(); + } + +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/ITaskExecutorContainer.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/ITaskExecutorContainer.java new file mode 100644 index 0000000000000..fd05eb03f9b80 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/ITaskExecutorContainer.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.task.executor.container; + +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; +import org.apache.dolphinscheduler.task.executor.exceptions.TaskExecutorRuntimeException; + +/** + * The task executor container used to execute the submitted ILogicTaskExecutor. + */ +public interface ITaskExecutorContainer { + + /** + * Dispatch the task executor to the container. + *

If dispatch failed, throw TaskExecutorRuntimeException. + */ + void dispatch(final ITaskExecutor taskExecutor) throws TaskExecutorRuntimeException; + + /** + * Start the task executor in the container. + */ + void start(final ITaskExecutor taskExecutor); + + /** + * Pause the task executor by the task instance id. + */ + void pause(final ITaskExecutor taskExecutor); + + /** + * Kill the task executor by the task instance id. + */ + void kill(final ITaskExecutor taskExecutor); + + /** + * Finalize the task executor. + */ + void finalize(final ITaskExecutor taskExecutor); + +} diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorFactory.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/ITaskExecutorContainerDelegator.java similarity index 81% rename from dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorFactory.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/ITaskExecutorContainerDelegator.java index 5a185dbba0c3a..ddb4864c666b3 100644 --- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutorFactory.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/ITaskExecutorContainerDelegator.java @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.worker.runner; +package org.apache.dolphinscheduler.task.executor.container; -public interface WorkerTaskExecutorFactory { +public interface ITaskExecutorContainerDelegator { + + ITaskExecutorContainer getExecutorContainer(final String taskType); - T createWorkerTaskExecutor(); } diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/SharedThreadTaskExecutorContainer.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/SharedThreadTaskExecutorContainer.java new file mode 100644 index 0000000000000..1839d80841dae --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/SharedThreadTaskExecutorContainer.java @@ -0,0 +1,41 @@ +/* + * 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.task.executor.container; + +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; +import org.apache.dolphinscheduler.task.executor.worker.TaskExecutorWorker; + +import java.util.Optional; + +import lombok.extern.slf4j.Slf4j; + +/** + * The container to execute a {@link ITaskExecutor} with shared thread mode. + */ +@Slf4j +public class SharedThreadTaskExecutorContainer extends AbstractTaskExecutorContainer { + + public SharedThreadTaskExecutorContainer(final TaskExecutorContainerBuilder taskExecutorContainerBuilder) { + super(taskExecutorContainerBuilder, taskExecutorContainerBuilder.getTaskExecutorContainerConfig()); + } + + @Override + protected Optional getTaskExecutorWorkerCandidate(final ITaskExecutor taskExecutor) { + return Optional.of(taskExecutorWorkers.getRandomTaskExecutorWorker()); + } +} diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionDispatchEvent.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/TaskExecutorContainerBuilder.java similarity index 65% rename from dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionDispatchEvent.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/TaskExecutorContainerBuilder.java index 91838a58080aa..8640bb0fa7896 100644 --- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionDispatchEvent.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/TaskExecutorContainerBuilder.java @@ -15,7 +15,10 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.extract.master.transportor; +package org.apache.dolphinscheduler.task.executor.container; + +import org.apache.dolphinscheduler.task.executor.ITaskExecutorRepository; +import org.apache.dolphinscheduler.task.executor.TaskExecutorEventBusCoordinator; import lombok.AllArgsConstructor; import lombok.Builder; @@ -24,24 +27,15 @@ @Data @Builder -@NoArgsConstructor @AllArgsConstructor -public class TaskExecutionDispatchEvent implements ITaskExecutionEvent { - - private int taskInstanceId; - - private int workflowInstanceId; - - private String taskInstanceHost; +@NoArgsConstructor +public class TaskExecutorContainerBuilder { - private String workflowInstanceHost; + private String taskExecutorContainerName; - private long eventCreateTime; + private TaskExecutorContainerConfig taskExecutorContainerConfig; - private long eventSendTime; + private ITaskExecutorRepository taskExecutorRepository; - @Override - public TaskInstanceExecutionEventType getEventType() { - return TaskInstanceExecutionEventType.DISPATCH; - } + private TaskExecutorEventBusCoordinator taskExecutorEventBusCoordinator; } diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskKillRequest.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/TaskExecutorContainerConfig.java similarity index 78% rename from dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskKillRequest.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/TaskExecutorContainerConfig.java index a1702b588eb3e..bd4de85edad07 100644 --- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskKillRequest.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/TaskExecutorContainerConfig.java @@ -15,17 +15,20 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.extract.master.transportor; +package org.apache.dolphinscheduler.task.executor.container; import lombok.AllArgsConstructor; +import lombok.Builder; import lombok.Data; import lombok.NoArgsConstructor; @Data +@Builder @NoArgsConstructor @AllArgsConstructor -public class LogicTaskKillRequest { +public class TaskExecutorContainerConfig { - private int taskInstanceId; + @Builder.Default + private int taskExecutorThreadPoolSize = Runtime.getRuntime().availableProcessors() * 2; } diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/TaskExecutorRouterTable.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/TaskExecutorRouterTable.java new file mode 100644 index 0000000000000..331a0c1223510 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/container/TaskExecutorRouterTable.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.task.executor.container; + +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; +import org.apache.dolphinscheduler.task.executor.worker.ITaskExecutorWorker; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class TaskExecutorRouterTable { + + private final Map assignedTaskExecutorIdToWorkerIdMapping = new ConcurrentHashMap<>(); + + public void registerTaskExecutor(final ITaskExecutor taskExecutor, final ITaskExecutorWorker taskExecutorWorker) { + assignedTaskExecutorIdToWorkerIdMapping.put(taskExecutor.getId(), taskExecutorWorker.getId()); + } + + public boolean isTaskExecutorRegistered(final ITaskExecutor taskExecutor) { + return assignedTaskExecutorIdToWorkerIdMapping.containsKey(taskExecutor.getId()); + } + + public void unregisterTaskExecutor(final ITaskExecutor taskExecutor) { + assignedTaskExecutorIdToWorkerIdMapping.remove(taskExecutor.getId()); + } + + public Integer getTaskExecutorWorkerId(final ITaskExecutor taskExecutor) { + return assignedTaskExecutorIdToWorkerIdMapping.get(taskExecutor.getId()); + } +} diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionKilledEvent.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/AbstractTaskExecutorLifecycleEvent.java similarity index 61% rename from dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionKilledEvent.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/AbstractTaskExecutorLifecycleEvent.java index 527f019c759b8..07ba8e1edcfdb 100644 --- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionKilledEvent.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/AbstractTaskExecutorLifecycleEvent.java @@ -15,35 +15,29 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.extract.master.transportor; +package org.apache.dolphinscheduler.task.executor.event; + +import org.apache.dolphinscheduler.eventbus.AbstractDelayEvent; -import lombok.AllArgsConstructor; import lombok.Builder; import lombok.Data; +import lombok.EqualsAndHashCode; import lombok.NoArgsConstructor; +import lombok.experimental.SuperBuilder; @Data -@Builder +@EqualsAndHashCode(callSuper = true) +@SuperBuilder @NoArgsConstructor -@AllArgsConstructor -public class TaskExecutionKilledEvent implements ITaskExecutionEvent { - - private int taskInstanceId; - - private int workflowInstanceId; - - private String taskInstanceHost; - - private String workflowInstanceHost; +public abstract class AbstractTaskExecutorLifecycleEvent extends AbstractDelayEvent + implements + ITaskExecutorLifecycleEvent { - private long endTime; + protected int taskInstanceId; - private long eventCreateTime; + @Builder.Default + protected long eventCreateTime = System.currentTimeMillis(); - private long eventSendTime; + protected TaskExecutorLifecycleEventType type; - @Override - public TaskInstanceExecutionEventType getEventType() { - return TaskInstanceExecutionEventType.KILLED; - } } diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/IOperableTaskExecutorLifecycleEvent.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/IOperableTaskExecutorLifecycleEvent.java new file mode 100644 index 0000000000000..d7f27129a9809 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/IOperableTaskExecutorLifecycleEvent.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.task.executor.event; + +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; + +/** + * The lifecycle event which represent the operation to {@link ITaskExecutor}. + */ +public interface IOperableTaskExecutorLifecycleEvent extends ITaskExecutorLifecycleEvent { + +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/IReportableTaskExecutorLifecycleEvent.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/IReportableTaskExecutorLifecycleEvent.java new file mode 100644 index 0000000000000..246f72ae2258b --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/IReportableTaskExecutorLifecycleEvent.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.task.executor.event; + +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; + +/** + * The lifecycle event of {@link ITaskExecutor} which should be report to master. + */ +public interface IReportableTaskExecutorLifecycleEvent extends ITaskExecutorLifecycleEvent { + + /** + * The id of the workflow instance which the event should report to. + */ + int getWorkflowInstanceId(); + + /** + * The host of the workflow instance which the event should report to. + */ + String getWorkflowInstanceHost(); + + /** + * Set the host of the workflow instance which the event should report to. + */ + void setWorkflowInstanceHost(String workflowInstanceHost); + + /** + * Get the latest report time of the event, if the event is never reported, return null. + */ + Long getLatestReportTime(); + + /** + * Set the latest report time of the event. + */ + void setLatestReportTime(Long latestReportTime); + +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/ITaskExecutorLifecycleEvent.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/ITaskExecutorLifecycleEvent.java new file mode 100644 index 0000000000000..0c55a8d6de334 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/ITaskExecutorLifecycleEvent.java @@ -0,0 +1,33 @@ +/* + * 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.task.executor.event; + +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; + +/** + * The lifecycle event of {@link ITaskExecutor}, all the lifecycle event should implement this interface. + */ +public interface ITaskExecutorLifecycleEvent { + + int getTaskInstanceId(); + + long getEventCreateTime(); + + TaskExecutorLifecycleEventType getType(); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionKilledEventSender.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorDispatchedLifecycleEvent.java similarity index 55% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionKilledEventSender.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorDispatchedLifecycleEvent.java index 250a98ab19ea5..9ba201bbbf76b 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionKilledEventSender.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorDispatchedLifecycleEvent.java @@ -15,36 +15,40 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.message; +package org.apache.dolphinscheduler.task.executor.event; -import org.apache.dolphinscheduler.extract.base.client.Clients; -import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; -import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionKilledEvent; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; -import org.springframework.stereotype.Component; +import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.NoArgsConstructor; +import lombok.experimental.SuperBuilder; -@Component -public class LogicTaskInstanceExecutionKilledEventSender +@Data +@EqualsAndHashCode(callSuper = true) +@NoArgsConstructor +@SuperBuilder +public class TaskExecutorDispatchedLifecycleEvent extends AbstractTaskExecutorLifecycleEvent implements - LogicTaskInstanceExecutionEventSender { - - @Override - public void sendMessage(final TaskExecutionKilledEvent message) { - Clients - .withService(ITaskExecutionEventListener.class) - .withHost(message.getWorkflowInstanceHost()) - .onTaskInstanceExecutionKilled(message); - } + IReportableTaskExecutorLifecycleEvent { - @Override - public TaskExecutionKilledEvent buildMessage(final TaskExecutionContext taskExecutionContext) { - return TaskExecutionKilledEvent.builder() - .workflowInstanceId(taskExecutionContext.getWorkflowInstanceId()) + private int workflowInstanceId; + + private String workflowInstanceHost; + + private String taskInstanceHost; + + private Long latestReportTime; + + public static TaskExecutorDispatchedLifecycleEvent of(final ITaskExecutor taskExecutor) { + final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext(); + return TaskExecutorDispatchedLifecycleEvent.builder() .taskInstanceId(taskExecutionContext.getTaskInstanceId()) - .workflowInstanceHost(taskExecutionContext.getWorkflowInstanceHost()) + .workflowInstanceId(taskExecutionContext.getWorkflowInstanceId()) .taskInstanceHost(taskExecutionContext.getHost()) - .endTime(taskExecutionContext.getEndTime()) + .workflowInstanceHost(taskExecutionContext.getWorkflowInstanceHost()) + .type(TaskExecutorLifecycleEventType.DISPATCHED) .build(); } diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorFailedLifecycleEvent.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorFailedLifecycleEvent.java new file mode 100644 index 0000000000000..324f75a657ceb --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorFailedLifecycleEvent.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.task.executor.event; + +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; + +import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.NoArgsConstructor; +import lombok.experimental.SuperBuilder; + +@Data +@SuperBuilder +@EqualsAndHashCode(callSuper = true) +@NoArgsConstructor +public class TaskExecutorFailedLifecycleEvent extends AbstractTaskExecutorLifecycleEvent + implements + IReportableTaskExecutorLifecycleEvent { + + private int workflowInstanceId; + + private String workflowInstanceHost; + + private String taskInstanceHost; + + private String appIds; + + private long endTime; + + private Long latestReportTime; + + public static TaskExecutorFailedLifecycleEvent of(final ITaskExecutor taskExecutor) { + final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext(); + return TaskExecutorFailedLifecycleEvent.builder() + .taskInstanceId(taskExecutor.getId()) + .workflowInstanceId(taskExecutionContext.getWorkflowInstanceId()) + .taskInstanceHost(taskExecutionContext.getHost()) + .workflowInstanceHost(taskExecutionContext.getWorkflowInstanceHost()) + .appIds(taskExecutionContext.getAppIds()) + .endTime(taskExecutionContext.getEndTime()) + .type(TaskExecutorLifecycleEventType.FAILED) + .build(); + } + +} diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskPauseResponse.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorFinalizeLifecycleEvent.java similarity index 58% rename from dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskPauseResponse.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorFinalizeLifecycleEvent.java index b91c2750744f6..4b658719d58dd 100644 --- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskPauseResponse.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorFinalizeLifecycleEvent.java @@ -15,25 +15,26 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.extract.master.transportor; +package org.apache.dolphinscheduler.task.executor.event; + +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; -import lombok.AllArgsConstructor; import lombok.Data; +import lombok.EqualsAndHashCode; import lombok.NoArgsConstructor; +import lombok.experimental.SuperBuilder; @Data +@EqualsAndHashCode(callSuper = true) +@SuperBuilder @NoArgsConstructor -@AllArgsConstructor -public class LogicTaskPauseResponse { - - private boolean success; - private String message; +public class TaskExecutorFinalizeLifecycleEvent extends AbstractTaskExecutorLifecycleEvent { - public static LogicTaskPauseResponse success() { - return new LogicTaskPauseResponse(true, null); + public static TaskExecutorFinalizeLifecycleEvent of(final ITaskExecutor taskExecutor) { + return TaskExecutorFinalizeLifecycleEvent.builder() + .taskInstanceId(taskExecutor.getId()) + .type(TaskExecutorLifecycleEventType.FINALIZE) + .build(); } - public static LogicTaskPauseResponse fail(String message) { - return new LogicTaskPauseResponse(false, message); - } } diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionFailedEvent.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorKillLifecycleEvent.java similarity index 58% rename from dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionFailedEvent.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorKillLifecycleEvent.java index c1c460acd609a..8277a5bea31af 100644 --- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/TaskExecutionFailedEvent.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorKillLifecycleEvent.java @@ -15,37 +15,28 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.extract.master.transportor; +package org.apache.dolphinscheduler.task.executor.event; + +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; -import lombok.AllArgsConstructor; -import lombok.Builder; import lombok.Data; +import lombok.EqualsAndHashCode; import lombok.NoArgsConstructor; +import lombok.experimental.SuperBuilder; @Data -@Builder +@EqualsAndHashCode(callSuper = true) @NoArgsConstructor -@AllArgsConstructor -public class TaskExecutionFailedEvent implements ITaskExecutionEvent { - - private int taskInstanceId; - - private int workflowInstanceId; - - private String taskInstanceHost; - - private String workflowInstanceHost; - - private String appIds; - - private long endTime; - - private long eventCreateTime; - - private long eventSendTime; - - @Override - public TaskInstanceExecutionEventType getEventType() { - return TaskInstanceExecutionEventType.FAILED; +@SuperBuilder +public class TaskExecutorKillLifecycleEvent extends AbstractTaskExecutorLifecycleEvent + implements + IOperableTaskExecutorLifecycleEvent { + + public static TaskExecutorKillLifecycleEvent of(final ITaskExecutor taskExecutor) { + return TaskExecutorKillLifecycleEvent.builder() + .taskInstanceId(taskExecutor.getId()) + .type(TaskExecutorLifecycleEventType.KILL) + .build(); } + } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionFailedEventSender.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorKilledLifecycleEvent.java similarity index 57% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionFailedEventSender.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorKilledLifecycleEvent.java index e1c19e6b6e905..4c65f140ef576 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionFailedEventSender.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorKilledLifecycleEvent.java @@ -15,36 +15,43 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.message; +package org.apache.dolphinscheduler.task.executor.event; -import org.apache.dolphinscheduler.extract.base.client.Clients; -import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; -import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionFailedEvent; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; -import org.springframework.stereotype.Component; +import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.NoArgsConstructor; +import lombok.experimental.SuperBuilder; -@Component -public class LogicTaskInstanceExecutionFailedEventSender +@Data +@EqualsAndHashCode(callSuper = true) +@SuperBuilder +@NoArgsConstructor +public class TaskExecutorKilledLifecycleEvent extends AbstractTaskExecutorLifecycleEvent implements - LogicTaskInstanceExecutionEventSender { - - @Override - public void sendMessage(final TaskExecutionFailedEvent message) { - Clients - .withService(ITaskExecutionEventListener.class) - .withHost(message.getWorkflowInstanceHost()) - .onTaskInstanceExecutionFailed(message); - } + IReportableTaskExecutorLifecycleEvent { - @Override - public TaskExecutionFailedEvent buildMessage(final TaskExecutionContext taskExecutionContext) { - return TaskExecutionFailedEvent.builder() - .workflowInstanceId(taskExecutionContext.getWorkflowInstanceId()) + private int workflowInstanceId; + + private String workflowInstanceHost; + + private String taskInstanceHost; + + private long endTime; + + private Long latestReportTime; + + public static TaskExecutorKilledLifecycleEvent of(final ITaskExecutor taskExecutor) { + final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext(); + return TaskExecutorKilledLifecycleEvent.builder() .taskInstanceId(taskExecutionContext.getTaskInstanceId()) - .workflowInstanceHost(taskExecutionContext.getWorkflowInstanceHost()) + .workflowInstanceId(taskExecutionContext.getWorkflowInstanceId()) .taskInstanceHost(taskExecutionContext.getHost()) + .workflowInstanceHost(taskExecutionContext.getWorkflowInstanceHost()) .endTime(taskExecutionContext.getEndTime()) + .type(TaskExecutorLifecycleEventType.KILLED) .build(); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ISyncLogicTask.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorLifecycleEventType.java similarity index 75% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ISyncLogicTask.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorLifecycleEventType.java index ba0df6fc2ad7a..0d97b167c7f9f 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ISyncLogicTask.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorLifecycleEventType.java @@ -15,12 +15,29 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.task; +package org.apache.dolphinscheduler.task.executor.event; -import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; +public enum TaskExecutorLifecycleEventType { -public interface ISyncLogicTask extends ILogicTask { + DISPATCHED, - void handle() throws MasterTaskExecuteException; + RUNNING, + + RUNTIME_CONTEXT_CHANGE, + + PAUSE, + + PAUSED, + + KILL, + + KILLED, + + SUCCESS, + + FAILED, + + FINALIZE, + ; } diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorPauseLifecycleEvent.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorPauseLifecycleEvent.java new file mode 100644 index 0000000000000..d402470c8d588 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorPauseLifecycleEvent.java @@ -0,0 +1,40 @@ +/* + * 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.task.executor.event; + +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; + +import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.experimental.SuperBuilder; + +@Data +@EqualsAndHashCode(callSuper = true) +@SuperBuilder +public class TaskExecutorPauseLifecycleEvent extends AbstractTaskExecutorLifecycleEvent + implements + IOperableTaskExecutorLifecycleEvent { + + public static TaskExecutorPauseLifecycleEvent of(final ITaskExecutor taskExecutor) { + return TaskExecutorPauseLifecycleEvent.builder() + .taskInstanceId(taskExecutor.getId()) + .eventCreateTime(System.currentTimeMillis()) + .type(TaskExecutorLifecycleEventType.PAUSE) + .build(); + } +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorPausedLifecycleEvent.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorPausedLifecycleEvent.java new file mode 100644 index 0000000000000..151b6c21afc12 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorPausedLifecycleEvent.java @@ -0,0 +1,57 @@ +/* + * 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.task.executor.event; + +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; + +import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.NoArgsConstructor; +import lombok.experimental.SuperBuilder; + +@Data +@SuperBuilder +@NoArgsConstructor +@EqualsAndHashCode(callSuper = true) +public class TaskExecutorPausedLifecycleEvent extends AbstractTaskExecutorLifecycleEvent + implements + IReportableTaskExecutorLifecycleEvent { + + private int workflowInstanceId; + + private String workflowInstanceHost; + + private String taskInstanceHost; + + private long endTime; + + private Long latestReportTime; + + public static TaskExecutorPausedLifecycleEvent of(final ITaskExecutor taskExecutor) { + final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext(); + return TaskExecutorPausedLifecycleEvent.builder() + .workflowInstanceId(taskExecutionContext.getWorkflowInstanceId()) + .taskInstanceHost(taskExecutionContext.getHost()) + .workflowInstanceHost(taskExecutionContext.getWorkflowInstanceHost()) + .endTime(taskExecutionContext.getEndTime()) + .type(TaskExecutorLifecycleEventType.PAUSED) + .build(); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionSuccessEventSender.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorRuntimeContextChangedLifecycleEvent.java similarity index 56% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionSuccessEventSender.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorRuntimeContextChangedLifecycleEvent.java index 81e2a8c374ad5..ecba893bd4b00 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionSuccessEventSender.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorRuntimeContextChangedLifecycleEvent.java @@ -15,39 +15,47 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.message; +package org.apache.dolphinscheduler.task.executor.event; -import org.apache.dolphinscheduler.extract.base.client.Clients; -import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; -import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionSuccessEvent; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; -import org.springframework.stereotype.Component; +import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.NoArgsConstructor; +import lombok.experimental.SuperBuilder; -@Component -public class LogicTaskInstanceExecutionSuccessEventSender +@Data +@EqualsAndHashCode(callSuper = true) +@SuperBuilder +@NoArgsConstructor +public class TaskExecutorRuntimeContextChangedLifecycleEvent extends AbstractTaskExecutorLifecycleEvent implements - LogicTaskInstanceExecutionEventSender { - - @Override - public void sendMessage(final TaskExecutionSuccessEvent message) { - Clients - .withService(ITaskExecutionEventListener.class) - .withHost(message.getWorkflowInstanceHost()) - .onTaskInstanceExecutionSuccess(message); - } + IReportableTaskExecutorLifecycleEvent { - @Override - public TaskExecutionSuccessEvent buildMessage(final TaskExecutionContext taskExecutionContext) { - return TaskExecutionSuccessEvent.builder() - .workflowInstanceId(taskExecutionContext.getWorkflowInstanceId()) + private int workflowInstanceId; + + private String workflowInstanceHost; + + private String taskInstanceHost; + + @Deprecated + private int processId; + + private String appIds; + + private Long latestReportTime; + + public static TaskExecutorRuntimeContextChangedLifecycleEvent of(final ITaskExecutor taskExecutor) { + final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext(); + return TaskExecutorRuntimeContextChangedLifecycleEvent.builder() .taskInstanceId(taskExecutionContext.getTaskInstanceId()) - .appIds(taskExecutionContext.getAppIds()) - .processId(taskExecutionContext.getProcessId()) + .workflowInstanceId(taskExecutionContext.getWorkflowInstanceId()) .workflowInstanceHost(taskExecutionContext.getWorkflowInstanceHost()) .taskInstanceHost(taskExecutionContext.getHost()) - .endTime(taskExecutionContext.getEndTime()) - .varPool(taskExecutionContext.getVarPool()) + .processId(taskExecutionContext.getProcessId()) + .appIds(taskExecutionContext.getAppIds()) + .type(TaskExecutorLifecycleEventType.RUNTIME_CONTEXT_CHANGE) .build(); } diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorStartedLifecycleEvent.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorStartedLifecycleEvent.java new file mode 100644 index 0000000000000..e147bcb0950f5 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorStartedLifecycleEvent.java @@ -0,0 +1,65 @@ +/* + * 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.task.executor.event; + +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; + +import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.NoArgsConstructor; +import lombok.experimental.SuperBuilder; + +@Data +@EqualsAndHashCode(callSuper = true) +@SuperBuilder +@NoArgsConstructor +public class TaskExecutorStartedLifecycleEvent extends AbstractTaskExecutorLifecycleEvent + implements + IReportableTaskExecutorLifecycleEvent { + + private int workflowInstanceId; + + private String workflowInstanceHost; + + private String taskInstanceHost; + + private long startTime; + + private String logPath; + + private String executePath; + + private Long latestReportTime; + + public static TaskExecutorStartedLifecycleEvent of(final ITaskExecutor taskExecutor) { + final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext(); + + return TaskExecutorStartedLifecycleEvent.builder() + .taskInstanceId(taskExecutor.getId()) + .workflowInstanceId(taskExecutionContext.getWorkflowInstanceId()) + .taskInstanceHost(taskExecutionContext.getHost()) + .startTime(taskExecutor.getTaskExecutionContext().getStartTime()) + .workflowInstanceHost(taskExecutionContext.getWorkflowInstanceHost()) + .logPath(taskExecutionContext.getLogPath()) + .executePath(taskExecutionContext.getExecutePath()) + .type(TaskExecutorLifecycleEventType.RUNNING) + .build(); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionPausedEventSender.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorSuccessLifecycleEvent.java similarity index 54% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionPausedEventSender.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorSuccessLifecycleEvent.java index 9704c4380ff5b..22d2605e5f7fc 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/LogicTaskInstanceExecutionPausedEventSender.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/event/TaskExecutorSuccessLifecycleEvent.java @@ -15,36 +15,45 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.message; +package org.apache.dolphinscheduler.task.executor.event; -import org.apache.dolphinscheduler.extract.base.client.Clients; -import org.apache.dolphinscheduler.extract.master.ITaskExecutionEventListener; -import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionPausedEvent; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; -import org.springframework.stereotype.Component; +import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.NoArgsConstructor; +import lombok.experimental.SuperBuilder; -@Component -public class LogicTaskInstanceExecutionPausedEventSender +@Data +@SuperBuilder +@EqualsAndHashCode(callSuper = true) +@NoArgsConstructor +public class TaskExecutorSuccessLifecycleEvent extends AbstractTaskExecutorLifecycleEvent implements - LogicTaskInstanceExecutionEventSender { - - @Override - public void sendMessage(final TaskExecutionPausedEvent taskExecutionPausedEvent) { - Clients - .withService(ITaskExecutionEventListener.class) - .withHost(taskExecutionPausedEvent.getWorkflowInstanceHost()) - .onTaskInstanceExecutionPaused(taskExecutionPausedEvent); - } + IReportableTaskExecutorLifecycleEvent { + + private int workflowInstanceId; + + private String workflowInstanceHost; + + private String taskInstanceHost; + + private long endTime; - @Override - public TaskExecutionPausedEvent buildMessage(final TaskExecutionContext taskExecutionContext) { - return TaskExecutionPausedEvent.builder() + private String varPool; + + private Long latestReportTime; + + public static TaskExecutorSuccessLifecycleEvent of(final ITaskExecutor taskExecutor) { + final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext(); + return TaskExecutorSuccessLifecycleEvent.builder() .workflowInstanceId(taskExecutionContext.getWorkflowInstanceId()) - .taskInstanceId(taskExecutionContext.getTaskInstanceId()) .workflowInstanceHost(taskExecutionContext.getWorkflowInstanceHost()) + .taskInstanceId(taskExecutionContext.getTaskInstanceId()) .taskInstanceHost(taskExecutionContext.getHost()) - .endTime(taskExecutionContext.getEndTime()) + .varPool(taskExecutionContext.getVarPool()) + .type(TaskExecutorLifecycleEventType.SUCCESS) .build(); } diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/exceptions/TaskExecutorRuntimeException.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/exceptions/TaskExecutorRuntimeException.java new file mode 100644 index 0000000000000..74e4e794dc5d5 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/exceptions/TaskExecutorRuntimeException.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.task.executor.exceptions; + +public class TaskExecutorRuntimeException extends RuntimeException { + + public TaskExecutorRuntimeException(String message) { + super(message); + } + + public TaskExecutorRuntimeException(String message, Throwable cause) { + super(message, cause); + } + +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/exceptions/TaskExecutorRuntimeExceptionEnum.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/exceptions/TaskExecutorRuntimeExceptionEnum.java new file mode 100644 index 0000000000000..3618ca868d423 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/exceptions/TaskExecutorRuntimeExceptionEnum.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.task.executor.exceptions; + +public enum TaskExecutorRuntimeExceptionEnum { + + TASK_EXECUTOR_SUBMIT_ERROR(), + + ; + + private int taskId; + + private String message; + +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/listener/ITaskExecutorLifecycleEventListener.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/listener/ITaskExecutorLifecycleEventListener.java new file mode 100644 index 0000000000000..1faa9e3be5912 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/listener/ITaskExecutorLifecycleEventListener.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.task.executor.listener; + +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorDispatchedLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorFailedLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorFinalizeLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorKillLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorKilledLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorPauseLifecycleEvent; +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; + +public interface ITaskExecutorLifecycleEventListener { + + void onTaskExecutorDispatchedLifecycleEvent(final TaskExecutorDispatchedLifecycleEvent event); + + void onTaskExecutorStartedLifecycleEvent(final TaskExecutorStartedLifecycleEvent event); + + void onTaskExecutorRuntimeContextChangedEvent(final TaskExecutorRuntimeContextChangedLifecycleEvent event); + + void onTaskExecutorPauseLifecycleEvent(final TaskExecutorPauseLifecycleEvent event); + + void onTaskExecutorPausedLifecycleEvent(final TaskExecutorPausedLifecycleEvent event); + + void onTaskExecutorKillLifecycleEvent(final TaskExecutorKillLifecycleEvent event); + + void onTaskExecutorKilledLifecycleEvent(final TaskExecutorKilledLifecycleEvent event); + + void onTaskExecutorSuccessLifecycleEvent(final TaskExecutorSuccessLifecycleEvent event); + + void onTaskExecutorFailLifecycleEvent(final TaskExecutorFailedLifecycleEvent event); + + void onTaskExecutorFinalizeLifecycleEvent(final TaskExecutorFinalizeLifecycleEvent event); + +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/listener/TaskExecutorLifecycleEventListener.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/listener/TaskExecutorLifecycleEventListener.java new file mode 100644 index 0000000000000..61edbcd686293 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/listener/TaskExecutorLifecycleEventListener.java @@ -0,0 +1,133 @@ +/* + * 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.task.executor.listener; + +import org.apache.dolphinscheduler.plugin.task.api.log.TaskInstanceLogHeader; +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; +import org.apache.dolphinscheduler.task.executor.ITaskExecutorLifecycleEventReporter; +import org.apache.dolphinscheduler.task.executor.ITaskExecutorRepository; +import org.apache.dolphinscheduler.task.executor.container.ITaskExecutorContainer; +import org.apache.dolphinscheduler.task.executor.container.ITaskExecutorContainerDelegator; +import org.apache.dolphinscheduler.task.executor.event.IReportableTaskExecutorLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.ITaskExecutorLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorDispatchedLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorFailedLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorFinalizeLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorKillLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorKilledLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorPauseLifecycleEvent; +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 TaskExecutorLifecycleEventListener implements ITaskExecutorLifecycleEventListener { + + private final ITaskExecutorContainerDelegator taskExecutorContainerDelegator; + + private final ITaskExecutorRepository taskExecutorRepository; + + private final ITaskExecutorLifecycleEventReporter taskExecutorLifecycleEventReporter; + + public TaskExecutorLifecycleEventListener(final ITaskExecutorContainerDelegator taskExecutorContainerDelegator, + final ITaskExecutorRepository taskExecutorRepository, + final ITaskExecutorLifecycleEventReporter taskExecutorLifecycleEventReporter) { + this.taskExecutorContainerDelegator = taskExecutorContainerDelegator; + this.taskExecutorRepository = taskExecutorRepository; + this.taskExecutorLifecycleEventReporter = taskExecutorLifecycleEventReporter; + } + + @Override + public void onTaskExecutorDispatchedLifecycleEvent(final TaskExecutorDispatchedLifecycleEvent event) { + reportTaskExecutorLifecycleEventToMaster(event); + } + + @Override + public void onTaskExecutorStartedLifecycleEvent(final TaskExecutorStartedLifecycleEvent event) { + reportTaskExecutorLifecycleEventToMaster(event); + } + + @Override + public void onTaskExecutorRuntimeContextChangedEvent(TaskExecutorRuntimeContextChangedLifecycleEvent event) { + reportTaskExecutorLifecycleEventToMaster(event); + } + + @Override + public void onTaskExecutorPauseLifecycleEvent(final TaskExecutorPauseLifecycleEvent event) { + final ITaskExecutor taskExecutor = getTaskExecutor(event); + taskExecutor.pause(); + } + + @Override + public void onTaskExecutorPausedLifecycleEvent(final TaskExecutorPausedLifecycleEvent event) { + taskExecutorLifecycleEventReporter.reportTaskExecutorLifecycleEvent(event); + } + + @Override + public void onTaskExecutorKillLifecycleEvent(final TaskExecutorKillLifecycleEvent event) { + final ITaskExecutor taskExecutor = getTaskExecutor(event); + taskExecutor.kill(); + } + + @Override + public void onTaskExecutorKilledLifecycleEvent(final TaskExecutorKilledLifecycleEvent event) { + taskExecutorLifecycleEventReporter.reportTaskExecutorLifecycleEvent(event); + } + + @Override + public void onTaskExecutorSuccessLifecycleEvent(final TaskExecutorSuccessLifecycleEvent event) { + taskExecutorLifecycleEventReporter.reportTaskExecutorLifecycleEvent(event); + } + + @Override + public void onTaskExecutorFailLifecycleEvent(TaskExecutorFailedLifecycleEvent event) { + taskExecutorLifecycleEventReporter.reportTaskExecutorLifecycleEvent(event); + } + + @Override + public void onTaskExecutorFinalizeLifecycleEvent(final TaskExecutorFinalizeLifecycleEvent event) { + TaskInstanceLogHeader.printFinalizeTaskHeader(); + final ITaskExecutor taskExecutor = getTaskExecutor(event); + + taskExecutorRepository.remove(taskExecutor.getId()); + + final ITaskExecutorContainer executorContainer = getTaskExecutorContainer(taskExecutor); + executorContainer.finalize(taskExecutor); + } + + private void reportTaskExecutorLifecycleEventToMaster(IReportableTaskExecutorLifecycleEvent taskExecutorLifecycleEvent) { + taskExecutorLifecycleEventReporter.reportTaskExecutorLifecycleEvent(taskExecutorLifecycleEvent); + } + + private ITaskExecutorContainer getTaskExecutorContainer(final ITaskExecutor taskExecutor) { + return taskExecutorContainerDelegator.getExecutorContainer(taskExecutor.getTaskType()); + } + + private ITaskExecutor getTaskExecutor(final ITaskExecutorLifecycleEvent taskExecutorLifecycleEvent) { + final ITaskExecutor taskExecutor = taskExecutorRepository.get(taskExecutorLifecycleEvent.getTaskInstanceId()); + if (taskExecutor == null) { + throw new IllegalArgumentException( + "Cannot find TaskExecutor: " + taskExecutorLifecycleEvent.getTaskInstanceId()); + } + return taskExecutor; + } + +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/log/TaskExecutorMDCUtils.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/log/TaskExecutorMDCUtils.java new file mode 100644 index 0000000000000..90638cc7884c4 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/log/TaskExecutorMDCUtils.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.task.executor.log; + +import org.apache.dolphinscheduler.common.constants.Constants; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; + +import org.slf4j.MDC; + +public class TaskExecutorMDCUtils { + + private static final String TASK_INSTANCE_ID_MDC_KEY = "taskInstanceId"; + private static final String TASK_INSTANCE_LOG_FULL_PATH_MDC_KEY = "taskInstanceLogFullPath"; + + public static MDCAutoClosable logWithMDC(final ITaskExecutor taskExecutor) { + final TaskExecutionContext taskExecutionContext = taskExecutor.getTaskExecutionContext(); + + final int taskInstanceId = taskExecutionContext.getTaskInstanceId(); + final String logPath = taskExecutionContext.getLogPath(); + + if (logPath != null) { + MDC.put(TASK_INSTANCE_LOG_FULL_PATH_MDC_KEY, logPath); + } + + MDC.put(Constants.TASK_INSTANCE_ID_MDC_KEY, String.valueOf(taskInstanceId)); + + return () -> { + MDC.remove(TASK_INSTANCE_LOG_FULL_PATH_MDC_KEY); + MDC.remove(TASK_INSTANCE_ID_MDC_KEY); + }; + } + + public interface MDCAutoClosable extends AutoCloseable { + + @Override + void close(); + } + +} diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskDispatchRequest.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorDispatchRequest.java similarity index 79% rename from dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskDispatchRequest.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorDispatchRequest.java index 660ee23de9e13..9f8147971264a 100644 --- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskDispatchRequest.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorDispatchRequest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.extract.master.transportor; +package org.apache.dolphinscheduler.task.executor.operations; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; @@ -28,9 +28,13 @@ @Data @NoArgsConstructor @AllArgsConstructor -public class LogicTaskDispatchRequest implements Serializable { +public class TaskExecutorDispatchRequest implements Serializable { private static final long serialVersionUID = -1L; private TaskExecutionContext taskExecutionContext; + + public static TaskExecutorDispatchRequest of(TaskExecutionContext taskExecutionContext) { + return new TaskExecutorDispatchRequest(taskExecutionContext); + } } diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TakeOverTaskResponse.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorDispatchResponse.java similarity index 71% rename from dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TakeOverTaskResponse.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorDispatchResponse.java index 3b6b7696e36fd..6b8135fc9501f 100644 --- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TakeOverTaskResponse.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorDispatchResponse.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.extract.worker.transportor; +package org.apache.dolphinscheduler.task.executor.operations; import lombok.AllArgsConstructor; import lombok.Data; @@ -24,17 +24,17 @@ @Data @NoArgsConstructor @AllArgsConstructor -public class TakeOverTaskResponse { +public class TaskExecutorDispatchResponse { + + private boolean dispatchSuccess; - private boolean success; private String message; - public static TakeOverTaskResponse success() { - return new TakeOverTaskResponse(true, null); + public static TaskExecutorDispatchResponse success() { + return new TaskExecutorDispatchResponse(true, null); } - public static TakeOverTaskResponse failed(String message) { - return new TakeOverTaskResponse(false, message); + public static TaskExecutorDispatchResponse failed(String message) { + return new TaskExecutorDispatchResponse(false, message); } - } diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceKillRequest.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorKillRequest.java similarity index 76% rename from dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceKillRequest.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorKillRequest.java index f4f8dddf126be..60e9857199265 100644 --- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstanceKillRequest.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorKillRequest.java @@ -15,7 +15,9 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.extract.worker.transportor; +package org.apache.dolphinscheduler.task.executor.operations; + +import java.io.Serializable; import lombok.AllArgsConstructor; import lombok.Data; @@ -24,8 +26,12 @@ @Data @NoArgsConstructor @AllArgsConstructor -public class TaskInstanceKillRequest { +public class TaskExecutorKillRequest implements Serializable { private Integer taskInstanceId; + public static TaskExecutorKillRequest of(Integer taskInstanceId) { + return new TaskExecutorKillRequest(taskInstanceId); + } + } diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskKillResponse.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorKillResponse.java similarity index 74% rename from dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskKillResponse.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorKillResponse.java index a15b9220cab6e..e646c46c95048 100644 --- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/LogicTaskKillResponse.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorKillResponse.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.extract.master.transportor; +package org.apache.dolphinscheduler.task.executor.operations; import lombok.AllArgsConstructor; import lombok.Data; @@ -24,17 +24,18 @@ @Data @NoArgsConstructor @AllArgsConstructor -public class LogicTaskKillResponse { +public class TaskExecutorKillResponse { private boolean success; + private String message; - public static LogicTaskKillResponse success() { - return new LogicTaskKillResponse(true, null); + public static TaskExecutorKillResponse success() { + return new TaskExecutorKillResponse(true, null); } - public static LogicTaskKillResponse fail(String message) { - return new LogicTaskKillResponse(false, message); + public static TaskExecutorKillResponse fail(String message) { + return new TaskExecutorKillResponse(false, message); } } diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstancePauseRequest.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorPauseRequest.java similarity index 76% rename from dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstancePauseRequest.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorPauseRequest.java index 8026f1d85d82c..a90995d24ea97 100644 --- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstancePauseRequest.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorPauseRequest.java @@ -15,7 +15,9 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.extract.worker.transportor; +package org.apache.dolphinscheduler.task.executor.operations; + +import java.io.Serializable; import lombok.AllArgsConstructor; import lombok.Data; @@ -24,7 +26,11 @@ @Data @NoArgsConstructor @AllArgsConstructor -public class TaskInstancePauseRequest { +public class TaskExecutorPauseRequest implements Serializable { private Integer taskInstanceId; + + public static TaskExecutorPauseRequest of(Integer taskInstanceId) { + return new TaskExecutorPauseRequest(taskInstanceId); + } } diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstancePauseResponse.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorPauseResponse.java similarity index 74% rename from dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstancePauseResponse.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorPauseResponse.java index cf263648295f0..b694f9e875ede 100644 --- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TaskInstancePauseResponse.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorPauseResponse.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.extract.worker.transportor; +package org.apache.dolphinscheduler.task.executor.operations; import lombok.AllArgsConstructor; import lombok.Data; @@ -24,18 +24,18 @@ @Data @NoArgsConstructor @AllArgsConstructor -public class TaskInstancePauseResponse { +public class TaskExecutorPauseResponse { private boolean success; private String message; - public static TaskInstancePauseResponse success() { - return new TaskInstancePauseResponse(true, null); + public static TaskExecutorPauseResponse success() { + return new TaskExecutorPauseResponse(true, null); } - public static TaskInstancePauseResponse fail(String message) { - return new TaskInstancePauseResponse(false, message); + public static TaskExecutorPauseResponse fail(String message) { + return new TaskExecutorPauseResponse(false, message); } } diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TakeOverTaskRequest.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorReassignMasterRequest.java similarity index 90% rename from dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TakeOverTaskRequest.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorReassignMasterRequest.java index d28a4d9a3e8a7..518a830506042 100644 --- a/dolphinscheduler-extract/dolphinscheduler-extract-worker/src/main/java/org/apache/dolphinscheduler/extract/worker/transportor/TakeOverTaskRequest.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorReassignMasterRequest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.extract.worker.transportor; +package org.apache.dolphinscheduler.task.executor.operations; import lombok.AllArgsConstructor; import lombok.Builder; @@ -26,7 +26,7 @@ @Builder @NoArgsConstructor @AllArgsConstructor -public class TakeOverTaskRequest { +public class TaskExecutorReassignMasterRequest { private int taskInstanceId; diff --git a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/StreamingTaskTriggerResponse.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorReassignMasterResponse.java similarity index 72% rename from dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/StreamingTaskTriggerResponse.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorReassignMasterResponse.java index d25611aee0046..2f3fdda7853ad 100644 --- a/dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/transportor/StreamingTaskTriggerResponse.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/operations/TaskExecutorReassignMasterResponse.java @@ -15,27 +15,26 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.extract.master.transportor; +package org.apache.dolphinscheduler.task.executor.operations; import lombok.AllArgsConstructor; import lombok.Data; import lombok.NoArgsConstructor; @Data -@AllArgsConstructor @NoArgsConstructor -public class StreamingTaskTriggerResponse { +@AllArgsConstructor +public class TaskExecutorReassignMasterResponse { private boolean success; - private String message; - public static StreamingTaskTriggerResponse success() { - return new StreamingTaskTriggerResponse(true, null); + public static TaskExecutorReassignMasterResponse success() { + return new TaskExecutorReassignMasterResponse(true, null); } - public static StreamingTaskTriggerResponse fail(String message) { - return new StreamingTaskTriggerResponse(false, message); + public static TaskExecutorReassignMasterResponse failed(String message) { + return new TaskExecutorReassignMasterResponse(false, message); } } diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/worker/AbstractTaskExecutorWorker.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/worker/AbstractTaskExecutorWorker.java new file mode 100644 index 0000000000000..61e75ec65e360 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/worker/AbstractTaskExecutorWorker.java @@ -0,0 +1,91 @@ +/* + * 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.task.executor.worker; + +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; +import org.apache.dolphinscheduler.task.executor.TaskExecutorState; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorFailedLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorFinalizeLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorKilledLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorPausedLifecycleEvent; +import org.apache.dolphinscheduler.task.executor.event.TaskExecutorSuccessLifecycleEvent; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public abstract class AbstractTaskExecutorWorker implements ITaskExecutorWorker { + + protected void trackTaskExecutorState(final ITaskExecutor taskExecutor) { + checkNotNull(taskExecutor, "taskExecutor is null"); + + final TaskExecutorState taskExecutorState = taskExecutor.refreshTaskExecutorState(); + + switch (taskExecutorState) { + case SUCCEEDED: + onTaskExecutorSuccess(taskExecutor); + break; + case FAILED: + onTaskExecutorFailed(taskExecutor); + break; + case KILLED: + onTaskExecutorKilled(taskExecutor); + break; + case PAUSED: + onTaskExecutorPaused(taskExecutor); + break; + case RUNNING: + log.debug("TaskExecutor(id={}) is running", taskExecutor.getId()); + break; + default: + throw new IllegalStateException("Unexpected TaskExecutorState: " + taskExecutorState + + " for taskExecutor(id=" + taskExecutor.getId() + ")"); + } + } + + protected void onTaskExecutorSuccess(final ITaskExecutor taskExecutor) { + taskExecutor.getTaskExecutionContext().setEndTime(System.currentTimeMillis()); + taskExecutor.getTaskExecutorEventBus().publish(TaskExecutorSuccessLifecycleEvent.of(taskExecutor)); + onTaskExecutorFinished(taskExecutor); + } + + protected void onTaskExecutorKilled(final ITaskExecutor taskExecutor) { + taskExecutor.getTaskExecutionContext().setEndTime(System.currentTimeMillis()); + taskExecutor.getTaskExecutorEventBus().publish(TaskExecutorKilledLifecycleEvent.of(taskExecutor)); + onTaskExecutorFinished(taskExecutor); + } + + protected void onTaskExecutorPaused(final ITaskExecutor taskExecutor) { + taskExecutor.getTaskExecutionContext().setEndTime(System.currentTimeMillis()); + taskExecutor.getTaskExecutorEventBus().publish(TaskExecutorPausedLifecycleEvent.of(taskExecutor)); + onTaskExecutorFinished(taskExecutor); + } + + protected void onTaskExecutorFailed(final ITaskExecutor taskExecutor) { + taskExecutor.getTaskExecutionContext().setEndTime(System.currentTimeMillis()); + taskExecutor.getTaskExecutorEventBus().publish(TaskExecutorFailedLifecycleEvent.of(taskExecutor)); + onTaskExecutorFinished(taskExecutor); + } + + protected void onTaskExecutorFinished(final ITaskExecutor taskExecutor) { + unFireTaskExecutor(taskExecutor); + taskExecutor.getTaskExecutorEventBus().publish(TaskExecutorFinalizeLifecycleEvent.of(taskExecutor)); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecuteFunction.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/worker/ITaskExecutorWorker.java similarity index 62% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecuteFunction.java rename to dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/worker/ITaskExecutorWorker.java index 2a34c4adc8d4f..c56fe9540a3f2 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecuteFunction.java +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/worker/ITaskExecutorWorker.java @@ -15,33 +15,23 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.execute; +package org.apache.dolphinscheduler.task.executor.worker; -import java.time.Duration; +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; -import lombok.NonNull; +public interface ITaskExecutorWorker { -public interface AsyncTaskExecuteFunction { + int getId(); - @NonNull - AsyncTaskExecutionStatus getAsyncTaskExecutionStatus(); + void start(); - @NonNull - Duration getAsyncTaskStateCheckInterval(); + void fireTaskExecutor(final ITaskExecutor taskExecutor); - enum AsyncTaskExecutionStatus { + void unFireTaskExecutor(final ITaskExecutor taskExecutor); - RUNNING, + void registerTaskExecutor(final ITaskExecutor taskExecutor); - PAUSE, - - KILL, - - SUCCESS, - - FAILED, - ; - - } + void unRegisterTaskExecutor(final ITaskExecutor taskExecutor); + int getRegisteredTaskExecutorSize(); } diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/worker/TaskExecutorWorker.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/worker/TaskExecutorWorker.java new file mode 100644 index 0000000000000..4b5f14e2786ce --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/worker/TaskExecutorWorker.java @@ -0,0 +1,152 @@ +/* + * 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.task.executor.worker; + +import org.apache.dolphinscheduler.common.thread.ThreadUtils; +import org.apache.dolphinscheduler.task.executor.ITaskExecutor; +import org.apache.dolphinscheduler.task.executor.TaskExecutorState; +import org.apache.dolphinscheduler.task.executor.log.TaskExecutorMDCUtils; +import org.apache.dolphinscheduler.task.executor.log.TaskExecutorMDCUtils.MDCAutoClosable; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class TaskExecutorWorker extends AbstractTaskExecutorWorker { + + private final Map registeredTaskExecutors = new ConcurrentHashMap<>(); + + private final Map activeTaskExecutors = new ConcurrentHashMap<>(); + + private final Lock activeTaskExecutorsChangeLock = new ReentrantLock(); + + private final Condition activeTaskExecutorEmptyCondition = activeTaskExecutorsChangeLock.newCondition(); + + @Getter + private final int workerId; + + public TaskExecutorWorker(int workerId) { + this.workerId = workerId; + } + + @Override + public int getId() { + return workerId; + } + + @Override + public void start() { + while (true) { + final long now = System.currentTimeMillis(); + for (final ITaskExecutor taskExecutor : activeTaskExecutors.values()) { + try (final MDCAutoClosable closable = TaskExecutorMDCUtils.logWithMDC(taskExecutor)) { + try { + if (taskExecutor.getTaskExecutorState() == TaskExecutorState.INITIALIZED) { + taskExecutor.start(); + } + + if (now < taskExecutor.getLatestStateRefreshTime() + taskExecutor.getStateRefreshInterval()) { + continue; + } + trackTaskExecutorState(taskExecutor); + } catch (Throwable e) { + log.error("TaskExecutor(id={}) execute failed", taskExecutor.getId(), e); + onTaskExecutorFailed(taskExecutor); + } + } + } + + activeTaskExecutorsChangeLock.lock(); + try { + while (activeTaskExecutors.isEmpty()) { + activeTaskExecutorEmptyCondition.await(); + } + } catch (InterruptedException e) { + log.info("TaskExecutorWorker(id={}) is interrupted", workerId, e); + break; + } finally { + activeTaskExecutorsChangeLock.unlock(); + } + // todo: Optimize the sleep time + ThreadUtils.sleep(100); + } + } + + @Override + public void fireTaskExecutor(final ITaskExecutor taskExecutor) { + activeTaskExecutorsChangeLock.lock(); + try { + final Integer taskExecutorId = taskExecutor.getId(); + if (!registeredTaskExecutors.containsKey(taskExecutorId)) { + throw new IllegalStateException( + "The TaskExecutorWorker has not registered a taskExecutor(id=" + taskExecutorId + ")"); + } + if (activeTaskExecutors.containsKey(taskExecutorId)) { + throw new IllegalStateException( + "The TaskExecutorWorker has already fired a taskExecutor(id=" + taskExecutorId + ")"); + } + activeTaskExecutors.put(taskExecutorId, taskExecutor); + activeTaskExecutorEmptyCondition.signalAll(); + } finally { + activeTaskExecutorsChangeLock.unlock(); + } + } + + @Override + public void unFireTaskExecutor(ITaskExecutor taskExecutor) { + try { + activeTaskExecutorsChangeLock.lock(); + final Integer taskExecutorId = taskExecutor.getId(); + activeTaskExecutors.remove(taskExecutorId); + } finally { + activeTaskExecutorsChangeLock.unlock(); + } + } + + @Override + public void registerTaskExecutor(final ITaskExecutor taskExecutor) { + final Integer taskExecutorId = taskExecutor.getId(); + if (registeredTaskExecutors.containsKey(taskExecutorId)) { + throw new IllegalStateException( + "The TaskExecutorWorker has already registered a ITaskExecutor(id=" + taskExecutorId + ")"); + } + registeredTaskExecutors.put(taskExecutorId, taskExecutor); + } + + @Override + public void unRegisterTaskExecutor(final ITaskExecutor taskExecutor) { + final Integer taskExecutorId = taskExecutor.getId(); + if (!registeredTaskExecutors.containsKey(taskExecutorId)) { + throw new IllegalStateException( + "The TaskExecutorWorker has not registered a ITaskExecutor(id=" + taskExecutorId + ")"); + } + registeredTaskExecutors.remove(taskExecutorId); + } + + @Override + public int getRegisteredTaskExecutorSize() { + return registeredTaskExecutors.size(); + } + +} diff --git a/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/worker/TaskExecutorWorkers.java b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/worker/TaskExecutorWorkers.java new file mode 100644 index 0000000000000..f584b420b59e2 --- /dev/null +++ b/dolphinscheduler-task-executor/src/main/java/org/apache/dolphinscheduler/task/executor/worker/TaskExecutorWorkers.java @@ -0,0 +1,56 @@ +/* + * 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.task.executor.worker; + +import org.apache.commons.lang3.RandomUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +import lombok.Getter; + +public class TaskExecutorWorkers { + + @Getter + private final List taskExecutorWorkers; + + public TaskExecutorWorkers(final int workerSize) { + this.taskExecutorWorkers = new ArrayList<>(); + for (int i = 0; i < workerSize; i++) { + taskExecutorWorkers.add(new TaskExecutorWorker(i)); + } + } + + public TaskExecutorWorker getTaskExecutorWorkerById(final int workerId) { + return taskExecutorWorkers.get(workerId); + } + + public TaskExecutorWorker getRandomTaskExecutorWorker() { + return taskExecutorWorkers.get(RandomUtils.nextInt(0, taskExecutorWorkers.size())); + } + + public Optional getIdleTaskExecutorWorker() { + for (TaskExecutorWorker taskExecutorWorker : taskExecutorWorkers) { + if (taskExecutorWorker.getRegisteredTaskExecutorSize() == 0) { + return Optional.of(taskExecutorWorker); + } + } + return Optional.empty(); + } +} diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskCallBack.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskCallBack.java index b1a85852dc205..2c622cc10897b 100644 --- a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskCallBack.java +++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskCallBack.java @@ -21,7 +21,7 @@ public interface TaskCallBack { - public void updateRemoteApplicationInfo(int taskInstanceId, ApplicationInfo applicationInfo); + void updateRemoteApplicationInfo(int taskInstanceId, ApplicationInfo applicationInfo); - public void updateTaskInstanceInfo(int taskInstanceId); + void updateTaskInstanceInfo(int taskInstanceId); } diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java index 4b146396520de..a69e17ccc8d9c 100644 --- a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java +++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java @@ -65,8 +65,6 @@ public class TaskExecutionContext implements Serializable { private String appInfoPath; - private String taskJson; - private int processId; private Long workflowDefinitionCode; @@ -105,6 +103,8 @@ public class TaskExecutionContext implements Serializable { private Map prepareParamsMap; + // Please use task instanceId + @Deprecated private String taskAppId; private TaskTimeoutStrategy taskTimeoutStrategy; @@ -113,6 +113,7 @@ public class TaskExecutionContext implements Serializable { private String workerGroup; + @Deprecated private TaskExecutionStatus currentExecutionStatus; private ResourceParametersHelper resourceParametersHelper; diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/shell/BaseLinuxShellInterceptorBuilder.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/shell/BaseLinuxShellInterceptorBuilder.java index 8b74ce59d9141..fd8747d3437c6 100644 --- a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/shell/BaseLinuxShellInterceptorBuilder.java +++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/shell/BaseLinuxShellInterceptorBuilder.java @@ -62,10 +62,10 @@ protected void generateShellScript() throws IOException { FileUtils.createFileWith755(shellAbsolutePath); Files.write(shellAbsolutePath, finalScript.getBytes(), StandardOpenOption.APPEND); log.info( - "Final Shell file is: \n****************************** Script Content *****************************************************************" + "Final Shell file is: \n****************************** Script Content *****************************************************************\n" + "{}" + - "\n****************************** Script Content *****************************************************************", + "\n****************************** Script Content *****************************************************************\n", finalScript); } diff --git a/dolphinscheduler-worker/pom.xml b/dolphinscheduler-worker/pom.xml index 5971b4b9ab721..cb9cc36064848 100644 --- a/dolphinscheduler-worker/pom.xml +++ b/dolphinscheduler-worker/pom.xml @@ -87,6 +87,12 @@ ${project.version} + + 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 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