diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/aspect/AccessLogAspect.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/aspect/AccessLogAspect.java index 6fc123a8b7..027301506c 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/aspect/AccessLogAspect.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/aspect/AccessLogAspect.java @@ -18,6 +18,7 @@ package org.apache.dolphinscheduler.api.aspect; import org.apache.dolphinscheduler.common.constants.Constants; +import org.apache.dolphinscheduler.common.utils.CodeGenerateUtils; import org.apache.dolphinscheduler.dao.entity.User; import org.apache.commons.lang3.StringUtils; @@ -26,7 +27,6 @@ import java.lang.reflect.Method; import java.util.Arrays; import java.util.HashMap; import java.util.Set; -import java.util.UUID; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -79,7 +79,7 @@ public class AccessLogAspect { Method method = sign.getMethod(); AccessLogAnnotation annotation = method.getAnnotation(AccessLogAnnotation.class); - String traceId = UUID.randomUUID().toString(); + String traceId = String.valueOf(CodeGenerateUtils.getInstance().genCode()); // log request if (!annotation.ignoreRequest()) { diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkFlowLineageServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkFlowLineageServiceImpl.java index 35a28eccf8..7966b7f391 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkFlowLineageServiceImpl.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkFlowLineageServiceImpl.java @@ -233,7 +233,8 @@ public class WorkFlowLineageServiceImpl extends BaseServiceImpl implements WorkF DependentParameters dependentParameters = JSONUtils.parseObject(taskDefinitionLog.getDependence(), DependentParameters.class); if (dependentParameters != null) { - List dependTaskList = dependentParameters.getDependTaskList(); + List dependTaskList = + dependentParameters.getDependTaskList(); if (!CollectionUtils.isEmpty(dependTaskList)) { for (DependentTaskModel taskModel : dependTaskList) { List dependItemList = taskModel.getDependItemList(); diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/constants/Constants.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/constants/Constants.java index 940d5594ca..8d3c684d79 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/constants/Constants.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/constants/Constants.java @@ -476,13 +476,6 @@ public final class Constants { public static final String BLOCKING_CONDITION = "blockingCondition"; public static final String ALERT_WHEN_BLOCKING = "alertWhenBlocking"; - /** - * master/worker server use for zk - */ - public static final String MASTER_TYPE = "master"; - public static final String WORKER_TYPE = "worker"; - public static final String DELETE_OP = "delete"; - public static final String ADD_OP = "add"; public static final String ALIAS = "alias"; public static final String CONTENT = "content"; public static final String DEPENDENT_SPLIT = ":||"; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/DateUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/DateUtils.java index de16f0e252..11c86ec531 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/DateUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/DateUtils.java @@ -584,6 +584,14 @@ public final class DateUtils { return intervalSeconds - usedTime; } + public static long getRemainTime(Long baseTime, long intervalSeconds) { + if (baseTime == null) { + return 0; + } + long usedTime = (System.currentTimeMillis() - baseTime) / 1000; + return intervalSeconds - usedTime; + } + /** * get current time stamp : yyyyMMddHHmmssSSS * diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java index 02b17ca9e7..cf5eb62ec7 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java @@ -321,6 +321,8 @@ public class TaskInstance implements Serializable { } public SwitchParameters getSwitchDependency() { + // todo: We need to directly use Jackson to deserialize the taskParam, rather than parse the map and get from + // field. if (this.switchDependency == null) { Map taskParamsMap = JSONUtils.parseObject(this.getTaskParams(), new TypeReference>() { @@ -336,6 +338,7 @@ public class TaskInstance implements Serializable { JSONUtils.parseObject(this.getTaskParams(), new TypeReference>() { }); taskParamsMap.put(Constants.SWITCH_RESULT, JSONUtils.toJsonString(switchDependency)); + this.switchDependency = switchDependency; this.setTaskParams(JSONUtils.toJsonString(taskParamsMap)); } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessDefinitionDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessDefinitionDao.java index 8e009f6bfb..febb009446 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessDefinitionDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessDefinitionDao.java @@ -21,6 +21,7 @@ import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.model.PageListingResult; +import java.util.Collection; import java.util.List; import java.util.Optional; @@ -53,4 +54,6 @@ public interface ProcessDefinitionDao { void deleteById(Integer workflowDefinitionId); void deleteByWorkflowDefinitionCode(long workflowDefinitionCode); + + List queryByCodes(Collection processDefinitionCodes); } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessInstanceDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessInstanceDao.java index 81064e7f84..0144356dfe 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessInstanceDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessInstanceDao.java @@ -87,4 +87,6 @@ public interface ProcessInstanceDao { * @return process instance */ ProcessInstance queryFirstStartProcessInstance(@Param("processDefinitionCode") Long definitionCode); + + ProcessInstance findSubProcessInstanceByParentId(Integer processInstanceId, Integer taskInstanceId); } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProjectDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProjectDao.java new file mode 100644 index 0000000000..4069cf7a2d --- /dev/null +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProjectDao.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.dao.repository; + +import org.apache.dolphinscheduler.dao.entity.Project; + +import java.util.Collection; +import java.util.List; + +public interface ProjectDao { + + List queryByCodes(Collection projectCodes); + +} diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/TaskDefinitionDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/TaskDefinitionDao.java index 4dce211eb8..2145b18fbd 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/TaskDefinitionDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/TaskDefinitionDao.java @@ -19,6 +19,7 @@ package org.apache.dolphinscheduler.dao.repository; import org.apache.dolphinscheduler.dao.entity.TaskDefinition; +import java.util.Collection; import java.util.List; import java.util.Set; @@ -45,4 +46,6 @@ public interface TaskDefinitionDao { void deleteByWorkflowDefinitionCodeAndVersion(long workflowDefinitionCode, int workflowDefinitionVersion); void deleteByTaskDefinitionCodes(Set needToDeleteTaskDefinitionCodes); + + List queryByCodes(Collection taskDefinitionCodes); } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/TaskInstanceDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/TaskInstanceDao.java index 8537e86ba1..65d391d4e1 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/TaskInstanceDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/TaskInstanceDao.java @@ -56,7 +56,7 @@ public interface TaskInstanceDao { * @param processInstance process instance * @return task instance */ - TaskInstance submitTaskInstanceToDB(TaskInstance taskInstance, ProcessInstance processInstance); + boolean submitTaskInstanceToDB(TaskInstance taskInstance, ProcessInstance processInstance); /** * Query list of valid task instance by process instance id diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessDefinitionDaoImpl.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessDefinitionDaoImpl.java index d7398453a0..bf29c814af 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessDefinitionDaoImpl.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessDefinitionDaoImpl.java @@ -25,7 +25,11 @@ import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper; import org.apache.dolphinscheduler.dao.model.PageListingResult; import org.apache.dolphinscheduler.dao.repository.ProcessDefinitionDao; +import org.apache.commons.collections4.CollectionUtils; + import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.Optional; @@ -96,4 +100,12 @@ public class ProcessDefinitionDaoImpl implements ProcessDefinitionDao { public void deleteByWorkflowDefinitionCode(long workflowDefinitionCode) { processDefinitionMapper.deleteByCode(workflowDefinitionCode); } + + @Override + public List queryByCodes(Collection processDefinitionCodes) { + if (CollectionUtils.isEmpty(processDefinitionCodes)) { + return Collections.emptyList(); + } + return processDefinitionMapper.queryByCodes(processDefinitionCodes); + } } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessInstanceDaoImpl.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessInstanceDaoImpl.java index d3d6f66280..5117145fd4 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessInstanceDaoImpl.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessInstanceDaoImpl.java @@ -19,6 +19,8 @@ package org.apache.dolphinscheduler.dao.repository.impl; import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.ProcessInstanceMap; +import org.apache.dolphinscheduler.dao.mapper.ProcessInstanceMapMapper; import org.apache.dolphinscheduler.dao.mapper.ProcessInstanceMapper; import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; import org.apache.dolphinscheduler.plugin.task.api.model.DateInterval; @@ -41,6 +43,9 @@ public class ProcessInstanceDaoImpl implements ProcessInstanceDao { @Autowired private ProcessInstanceMapper processInstanceMapper; + @Autowired + private ProcessInstanceMapMapper processInstanceMapMapper; + @Override public int insertProcessInstance(ProcessInstance processInstance) { return processInstanceMapper.insert(processInstance); @@ -147,4 +152,16 @@ public class ProcessInstanceDaoImpl implements ProcessInstanceDao { public ProcessInstance queryFirstStartProcessInstance(Long definitionCode) { return processInstanceMapper.queryFirstStartProcessInstance(definitionCode); } + + @Override + public ProcessInstance findSubProcessInstanceByParentId(Integer processInstanceId, Integer taskInstanceId) { + ProcessInstance processInstance = null; + ProcessInstanceMap processInstanceMap = + processInstanceMapMapper.queryByParentId(processInstanceId, taskInstanceId); + if (processInstanceMap == null || processInstanceMap.getProcessInstanceId() == 0) { + return processInstance; + } + processInstance = queryByWorkflowInstanceId(processInstanceMap.getProcessInstanceId()); + return processInstance; + } } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProjectDaoImpl.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProjectDaoImpl.java new file mode 100644 index 0000000000..1db9d7fb08 --- /dev/null +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProjectDaoImpl.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.dao.repository.impl; + +import org.apache.dolphinscheduler.dao.entity.Project; +import org.apache.dolphinscheduler.dao.mapper.ProjectMapper; +import org.apache.dolphinscheduler.dao.repository.ProjectDao; + +import java.util.Collection; +import java.util.List; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Repository; + +@Repository +public class ProjectDaoImpl implements ProjectDao { + + @Autowired + private ProjectMapper projectMapper; + + @Override + public List queryByCodes(Collection projectCodes) { + return projectMapper.queryByCodes(projectCodes); + } +} diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/TaskDefinitionDaoImpl.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/TaskDefinitionDaoImpl.java index cb0eeb4f24..8327ccff65 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/TaskDefinitionDaoImpl.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/TaskDefinitionDaoImpl.java @@ -30,6 +30,8 @@ import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao; import org.apache.commons.collections4.CollectionUtils; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -100,4 +102,12 @@ public class TaskDefinitionDaoImpl implements TaskDefinitionDao { taskDefinitionMapper.deleteByBatchCodes(new ArrayList<>(needToDeleteTaskDefinitionCodes)); } + @Override + public List queryByCodes(Collection taskDefinitionCodes) { + if (CollectionUtils.isEmpty(taskDefinitionCodes)) { + return Collections.emptyList(); + } + return taskDefinitionMapper.queryByCodeList(taskDefinitionCodes); + } + } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/TaskInstanceDaoImpl.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/TaskInstanceDaoImpl.java index 48ec1cd15e..6bed7ed900 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/TaskInstanceDaoImpl.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/TaskInstanceDaoImpl.java @@ -78,14 +78,14 @@ public class TaskInstanceDaoImpl implements TaskInstanceDao { } @Override - public TaskInstance submitTaskInstanceToDB(TaskInstance taskInstance, ProcessInstance processInstance) { + public boolean submitTaskInstanceToDB(TaskInstance taskInstance, ProcessInstance processInstance) { WorkflowExecutionStatus processInstanceState = processInstance.getState(); if (processInstanceState.isFinished() || processInstanceState == WorkflowExecutionStatus.READY_STOP) { log.warn("processInstance: {} state was: {}, skip submit this task, taskCode: {}", processInstance.getId(), processInstanceState, taskInstance.getTaskCode()); - return null; + return false; } if (processInstanceState == WorkflowExecutionStatus.READY_PAUSE) { taskInstance.setState(TaskExecutionStatus.PAUSE); @@ -99,11 +99,7 @@ public class TaskInstanceDaoImpl implements TaskInstanceDao { if (taskInstance.getFirstSubmitTime() == null) { taskInstance.setFirstSubmitTime(taskInstance.getSubmitTime()); } - boolean saveResult = upsertTaskInstance(taskInstance); - if (!saveResult) { - return null; - } - return taskInstance; + return upsertTaskInstance(taskInstance); } private TaskExecutionStatus getSubmitTaskState(TaskInstance taskInstance, ProcessInstance processInstance) { diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java index c8c8f887fa..dadd02fbf0 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java @@ -93,7 +93,6 @@ public class MasterServer implements IStoppable { this.masterRegistryClient.start(); this.masterRegistryClient.setRegistryStoppable(this); - this.masterSchedulerBootstrap.init(); this.masterSchedulerBootstrap.start(); this.eventExecuteService.start(); 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 b878b2a4fe..d65a3fe52a 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 @@ -60,6 +60,11 @@ public class MasterConfig implements Validator { * Will create two thread poll to execute {@link WorkflowExecuteRunnable} and {@link TaskExecuteRunnable}. */ private int execThreads = 10; + + // todo: change to sync thread pool/ async thread pool ? + private int masterTaskExecuteThreadPoolSize = Runtime.getRuntime().availableProcessors(); + + private int masterAsyncTaskStateCheckThreadPoolSize = Runtime.getRuntime().availableProcessors(); /** * The task dispatch thread pool size. */ diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java deleted file mode 100644 index 9a34ff1bb2..0000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java +++ /dev/null @@ -1,353 +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.consumer; - -import org.apache.dolphinscheduler.common.constants.Constants; -import org.apache.dolphinscheduler.common.enums.Flag; -import org.apache.dolphinscheduler.common.enums.TaskEventType; -import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager; -import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; -import org.apache.dolphinscheduler.common.thread.ThreadUtils; -import org.apache.dolphinscheduler.common.utils.DateUtils; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; -import org.apache.dolphinscheduler.dao.utils.TaskCacheUtils; -import org.apache.dolphinscheduler.plugin.storage.api.StorageOperate; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; -import org.apache.dolphinscheduler.remote.command.Message; -import org.apache.dolphinscheduler.remote.command.task.TaskDispatchMessage; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; -import org.apache.dolphinscheduler.server.master.config.MasterConfig; -import org.apache.dolphinscheduler.server.master.dispatch.ExecutorDispatcher; -import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext; -import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType; -import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException; -import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; -import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics; -import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; -import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; -import org.apache.dolphinscheduler.service.exceptions.TaskPriorityQueueException; -import org.apache.dolphinscheduler.service.queue.TaskPriority; -import org.apache.dolphinscheduler.service.queue.TaskPriorityQueue; - -import org.apache.commons.collections4.CollectionUtils; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Date; -import java.util.List; -import java.util.Objects; -import java.util.Optional; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -import javax.annotation.PostConstruct; - -import lombok.extern.slf4j.Slf4j; - -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Component; - -/** - * TaskUpdateQueue consumer - */ -@Component -@Slf4j -public class TaskPriorityQueueConsumer extends BaseDaemonThread { - - /** - * taskUpdateQueue - */ - @Autowired - private TaskPriorityQueue taskPriorityQueue; - - @Autowired - private TaskInstanceDao taskInstanceDao; - - /** - * executor dispatcher - */ - @Autowired - private ExecutorDispatcher dispatcher; - - /** - * processInstance cache manager - */ - @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; - - /** - * master config - */ - @Autowired - private MasterConfig masterConfig; - - /** - * task response service - */ - @Autowired - private TaskEventService taskEventService; - - /** - * storage operator - */ - @Autowired(required = false) - private StorageOperate storageOperate; - - /** - * consumer thread pool - */ - private ThreadPoolExecutor consumerThreadPoolExecutor; - - protected TaskPriorityQueueConsumer() { - super("TaskPriorityQueueConsumeThread"); - } - - @PostConstruct - public void init() { - this.consumerThreadPoolExecutor = (ThreadPoolExecutor) ThreadUtils - .newDaemonFixedThreadExecutor("TaskUpdateQueueConsumerThread", masterConfig.getDispatchTaskNumber()); - log.info("Task priority queue consume thread staring"); - super.start(); - log.info("Task priority queue consume thread started"); - } - - @Override - public void run() { - int fetchTaskNum = masterConfig.getDispatchTaskNumber(); - while (!ServerLifeCycleManager.isStopped()) { - try { - List failedDispatchTasks = this.batchDispatch(fetchTaskNum); - - if (CollectionUtils.isNotEmpty(failedDispatchTasks)) { - log.info("{} tasks dispatch failed, will retry to dispatch", failedDispatchTasks.size()); - TaskMetrics.incTaskDispatchFailed(failedDispatchTasks.size()); - for (TaskPriority dispatchFailedTask : failedDispatchTasks) { - taskPriorityQueue.put(dispatchFailedTask); - } - // If the all task dispatch failed, will sleep for 1s to avoid the master cpu higher. - if (fetchTaskNum == failedDispatchTasks.size()) { - log.info("All tasks dispatch failed, will sleep a while to avoid the master cpu higher"); - TimeUnit.MILLISECONDS.sleep(Constants.SLEEP_TIME_MILLIS); - } - } - } catch (Exception e) { - TaskMetrics.incTaskDispatchError(); - log.error("dispatcher task error", e); - } - } - } - - /** - * batch dispatch with thread pool - */ - public List batchDispatch(int fetchTaskNum) throws TaskPriorityQueueException, InterruptedException { - List failedDispatchTasks = Collections.synchronizedList(new ArrayList<>()); - CountDownLatch latch = new CountDownLatch(fetchTaskNum); - - for (int i = 0; i < fetchTaskNum; i++) { - TaskPriority taskPriority = taskPriorityQueue.poll(Constants.SLEEP_TIME_MILLIS, TimeUnit.MILLISECONDS); - if (Objects.isNull(taskPriority)) { - latch.countDown(); - continue; - } - - consumerThreadPoolExecutor.submit(() -> { - try { - try { - this.dispatchTask(taskPriority); - } catch (WorkerGroupNotFoundException e) { - // If the worker group not found, will not try to dispatch again. - // The task instance will be failed - // todo: - addDispatchFailedEvent(taskPriority); - } catch (ExecuteException e) { - failedDispatchTasks.add(taskPriority); - } catch (Exception e) { - log.error("Dispatch task error, meet an unknown exception", e); - failedDispatchTasks.add(taskPriority); - } - } finally { - // make sure the latch countDown - latch.countDown(); - } - }); - } - - latch.await(); - - return failedDispatchTasks; - } - - /** - * Dispatch task to worker. - * - * @param taskPriority taskPriority - * @return dispatch result, return true if dispatch success, return false if dispatch failed. - */ - protected void dispatchTask(TaskPriority taskPriority) throws ExecuteException { - TaskMetrics.incTaskDispatch(); - WorkflowExecuteRunnable workflowExecuteRunnable = - processInstanceExecCacheManager.getByProcessInstanceId(taskPriority.getProcessInstanceId()); - if (workflowExecuteRunnable == null) { - log.error("Cannot find the related processInstance of the task, taskPriority: {}", taskPriority); - return; - } - Optional taskInstanceOptional = - workflowExecuteRunnable.getTaskInstance(taskPriority.getTaskId()); - if (!taskInstanceOptional.isPresent()) { - log.error("Cannot find the task instance from related processInstance, taskPriority: {}", - taskPriority); - // we return true, so that we will drop this task. - return; - } - TaskInstance taskInstance = taskInstanceOptional.get(); - TaskExecutionContext context = taskPriority.getTaskExecutionContext(); - ExecutionContext executionContext = ExecutionContext.builder() - .taskInstance(taskInstance) - .workerGroup(context.getWorkerGroup()) - .executorType(ExecutorType.WORKER) - .message(toCommand(context)) - .build(); - - if (isTaskNeedToCheck(taskPriority)) { - if (taskInstanceIsFinalState(taskPriority.getTaskId())) { - // when task finish, ignore this task, there is no need to dispatch anymore - log.info("Task {} is already finished, no need to dispatch, task instance id: {}", - taskInstance.getName(), taskInstance.getId()); - return; - } - } - - // check task is cache execution, and decide whether to dispatch - if (checkIsCacheExecution(taskInstance, context)) { - return; - } - - dispatcher.dispatch(executionContext); - log.info("Master success dispatch task to worker, taskInstanceId: {}, worker: {}", - taskPriority.getTaskId(), - executionContext.getHost()); - addDispatchEvent(context, executionContext); - } - - /** - * add dispatch event - */ - private void addDispatchEvent(TaskExecutionContext context, ExecutionContext executionContext) { - TaskEvent taskEvent = TaskEvent.newDispatchEvent(context.getProcessInstanceId(), context.getTaskInstanceId(), - executionContext.getHost().getAddress()); - taskEventService.addEvent(taskEvent); - } - - private void addDispatchFailedEvent(TaskPriority taskPriority) { - TaskExecutionContext taskExecutionContext = taskPriority.getTaskExecutionContext(); - TaskEvent taskEvent = TaskEvent.builder() - .processInstanceId(taskPriority.getProcessInstanceId()) - .taskInstanceId(taskPriority.getTaskId()) - .state(TaskExecutionStatus.FAILURE) - .logPath(taskExecutionContext.getLogPath()) - .executePath(taskExecutionContext.getExecutePath()) - .appIds(taskExecutionContext.getAppIds()) - .processId(taskExecutionContext.getProcessId()) - .varPool(taskExecutionContext.getVarPool()) - .startTime(DateUtils.timeStampToDate(taskExecutionContext.getStartTime())) - .endTime(new Date()) - .event(TaskEventType.RESULT) - .build(); - taskEventService.addEvent(taskEvent); - } - - private Message toCommand(TaskExecutionContext taskExecutionContext) { - // todo: we didn't set the host here, since right now we didn't need to retry this message. - TaskDispatchMessage requestCommand = new TaskDispatchMessage(taskExecutionContext, - masterConfig.getMasterAddress(), - taskExecutionContext.getHost(), - System.currentTimeMillis()); - return requestCommand.convert2Command(); - } - - /** - * taskInstance is final state - * success,failure,kill,stop,pause,threadwaiting is final state - * - * @param taskInstanceId taskInstanceId - * @return taskInstance is final state - */ - public boolean taskInstanceIsFinalState(int taskInstanceId) { - TaskInstance taskInstance = taskInstanceDao.findTaskInstanceById(taskInstanceId); - return taskInstance.getState().isFinished(); - } - - /** - * check if task need to check state, if true, refresh the checkpoint - */ - private boolean isTaskNeedToCheck(TaskPriority taskPriority) { - long now = System.currentTimeMillis(); - if (now - taskPriority.getCheckpoint() > Constants.SECOND_TIME_MILLIS) { - taskPriority.setCheckpoint(now); - return true; - } - return false; - } - - /** - * check if task is cache execution - * if the task is defined as cache execution, and we find the cache task instance is finished yet, we will not dispatch this task - * @param taskInstance taskInstance - * @param context context - * @return true if we will not dispatch this task, false if we will dispatch this task - */ - private boolean checkIsCacheExecution(TaskInstance taskInstance, TaskExecutionContext context) { - try { - // check if task is defined as a cache task - if (taskInstance.getIsCache().equals(Flag.NO)) { - return false; - } - // check if task is cache execution - String cacheKey = TaskCacheUtils.generateCacheKey(taskInstance, context, storageOperate); - TaskInstance cacheTaskInstance = taskInstanceDao.findTaskInstanceByCacheKey(cacheKey); - // if we can find the cache task instance, we will add cache event, and return true. - if (cacheTaskInstance != null) { - log.info("Task {} is cache, no need to dispatch, task instance id: {}", - taskInstance.getName(), taskInstance.getId()); - addCacheEvent(taskInstance, cacheTaskInstance); - taskInstance.setCacheKey(TaskCacheUtils.generateTagCacheKey(cacheTaskInstance.getId(), cacheKey)); - return true; - } else { - // if we can not find cache task, update cache key, and return false. the task will be dispatched - taskInstance.setCacheKey(TaskCacheUtils.generateTagCacheKey(taskInstance.getId(), cacheKey)); - } - } catch (Exception e) { - log.error("checkIsCacheExecution error", e); - } - return false; - } - - private void addCacheEvent(TaskInstance taskInstance, TaskInstance cacheTaskInstance) { - if (cacheTaskInstance == null) { - return; - } - TaskEvent taskEvent = TaskEvent.newCacheEvent(taskInstance.getProcessInstanceId(), taskInstance.getId(), - cacheTaskInstance.getId()); - taskEventService.addEvent(taskEvent); - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/ExecutorDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/ExecutorDispatcher.java deleted file mode 100644 index f247e9db31..0000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/ExecutorDispatcher.java +++ /dev/null @@ -1,118 +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.dispatch; - -import org.apache.dolphinscheduler.remote.utils.Host; -import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext; -import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType; -import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException; -import org.apache.dolphinscheduler.server.master.dispatch.executor.ExecutorManager; -import org.apache.dolphinscheduler.server.master.dispatch.executor.NettyExecutorManager; -import org.apache.dolphinscheduler.server.master.dispatch.host.HostManager; - -import org.apache.commons.lang3.StringUtils; - -import java.util.concurrent.ConcurrentHashMap; - -import lombok.extern.slf4j.Slf4j; - -import org.springframework.beans.factory.InitializingBean; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Service; - -/** - * executor dispatcher - */ -@Service -@Slf4j -public class ExecutorDispatcher implements InitializingBean { - - /** - * netty executor manager - */ - @Autowired - private NettyExecutorManager nettyExecutorManager; - - /** - * round robin host manager - */ - @Autowired - private HostManager hostManager; - - /** - * executor manager - */ - private final ConcurrentHashMap> executorManagers; - - /** - * constructor - */ - public ExecutorDispatcher() { - this.executorManagers = new ConcurrentHashMap<>(); - } - - /** - * task dispatch - * - * @param context context - * @return result - * @throws ExecuteException if error throws ExecuteException - */ - public void dispatch(final ExecutionContext context) throws ExecuteException { - // get executor manager - ExecutorManager executorManager = this.executorManagers.get(context.getExecutorType()); - if (executorManager == null) { - throw new ExecuteException("no ExecutorManager for type : " + context.getExecutorType()); - } - - // host select - Host host = hostManager.select(context); - if (StringUtils.isEmpty(host.getAddress())) { - log.warn("fail to execute : {} due to no suitable worker, current task needs worker group {} to execute", - context.getMessage(), context.getWorkerGroup()); - throw new ExecuteException("no suitable worker"); - } - context.setHost(host); - executorManager.beforeExecute(context); - try { - // task execute - executorManager.execute(context); - } finally { - executorManager.afterExecute(context); - } - } - - /** - * register init - * @throws Exception if error throws Exception - */ - @Override - public void afterPropertiesSet() throws Exception { - register(ExecutorType.WORKER, nettyExecutorManager); - register(ExecutorType.CLIENT, nettyExecutorManager); - } - - /** - * register - * @param type executor type - * @param executorManager executorManager - */ - public void register(ExecutorType type, ExecutorManager executorManager) { - executorManagers.put(type, executorManager); - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/enums/ExecutorType.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/enums/ExecutorType.java index 0f1a79011b..6f6ce615fe 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/enums/ExecutorType.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/enums/ExecutorType.java @@ -17,12 +17,12 @@ package org.apache.dolphinscheduler.server.master.dispatch.enums; -/** - * executor type - */ +// todo: refactor this enum public enum ExecutorType { WORKER, - CLIENT; + CLIENT, + MASTER, + ; } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/CommonHostManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/CommonHostManager.java index dd9b78fe37..91fadf31bd 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/CommonHostManager.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/CommonHostManager.java @@ -18,8 +18,6 @@ package org.apache.dolphinscheduler.server.master.dispatch.host; import org.apache.dolphinscheduler.remote.utils.Host; -import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext; -import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType; import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostWorker; import org.apache.dolphinscheduler.server.master.registry.ServerNodeManager; @@ -29,6 +27,7 @@ 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 org.springframework.beans.factory.annotation.Autowired; @@ -44,32 +43,13 @@ public abstract class CommonHostManager implements HostManager { @Autowired protected ServerNodeManager serverNodeManager; - /** - * select host - * - * @param context context - * @return host - * @throws WorkerGroupNotFoundException If the worker group not found - */ @Override - public Host select(ExecutionContext context) throws WorkerGroupNotFoundException { - List candidates = null; - String workerGroup = context.getWorkerGroup(); - ExecutorType executorType = context.getExecutorType(); - switch (executorType) { - case WORKER: - candidates = getWorkerCandidates(workerGroup); - break; - case CLIENT: - break; - default: - throw new IllegalArgumentException("invalid executorType : " + executorType); - } - + public Optional select(String workerGroup) throws WorkerGroupNotFoundException { + List candidates = getWorkerCandidates(workerGroup); if (CollectionUtils.isEmpty(candidates)) { - return new Host(); + return Optional.empty(); } - return select(candidates); + return Optional.ofNullable(select(candidates)); } protected abstract HostWorker select(Collection nodes); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/HostManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/HostManager.java index 31de26bdb9..9ce087197f 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/HostManager.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/HostManager.java @@ -18,21 +18,15 @@ package org.apache.dolphinscheduler.server.master.dispatch.host; import org.apache.dolphinscheduler.remote.utils.Host; -import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext; import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; +import java.util.Optional; + /** * host manager */ public interface HostManager { - /** - * select host - * - * @param context context - * @return host - * @throws WorkerGroupNotFoundException If the worker group does exist - */ - Host select(ExecutionContext context) throws WorkerGroupNotFoundException; + Optional select(String workerGroup) throws WorkerGroupNotFoundException; } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/LowerWeightHostManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/LowerWeightHostManager.java index 5c615563ab..0daaebd10c 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/LowerWeightHostManager.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/LowerWeightHostManager.java @@ -20,7 +20,6 @@ package org.apache.dolphinscheduler.server.master.dispatch.host; import org.apache.dolphinscheduler.common.constants.Constants; import org.apache.dolphinscheduler.common.model.WorkerHeartBeat; import org.apache.dolphinscheduler.remote.utils.Host; -import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext; import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostWeight; import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostWorker; @@ -70,20 +69,13 @@ public class LowerWeightHostManager extends CommonHostManager { serverNodeManager.addWorkerInfoChangeListener(new WorkerWeightListener()); } - /** - * select host - * - * @param context context - * @return host - * @throws WorkerGroupNotFoundException If the worker group not found - */ @Override - public Host select(ExecutionContext context) throws WorkerGroupNotFoundException { - Set workerHostWeights = getWorkerHostWeights(context.getWorkerGroup()); + public Optional select(String workerGroup) throws WorkerGroupNotFoundException { + Set workerHostWeights = getWorkerHostWeights(workerGroup); if (CollectionUtils.isNotEmpty(workerHostWeights)) { - return selector.select(workerHostWeights).getHost(); + return Optional.ofNullable(selector.select(workerHostWeights).getHost()); } - return new Host(); + return Optional.empty(); } @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRejectByWorkerEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRejectByWorkerEventHandler.java deleted file mode 100644 index 77c6f77f85..0000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRejectByWorkerEventHandler.java +++ /dev/null @@ -1,84 +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.event; - -import org.apache.dolphinscheduler.common.enums.TaskEventType; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.remote.command.task.TaskRejectMessageAck; -import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; -import org.apache.dolphinscheduler.server.master.config.MasterConfig; -import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; -import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; - -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Component; - -@Component -public class TaskRejectByWorkerEventHandler implements TaskEventHandler { - - @Autowired - private ProcessInstanceExecCacheManager processInstanceExecCacheManager; - - @Autowired - private MasterConfig masterConfig; - - @Override - public void handleTaskEvent(TaskEvent taskEvent) throws TaskEventHandleError { - int taskInstanceId = taskEvent.getTaskInstanceId(); - int processInstanceId = taskEvent.getProcessInstanceId(); - - WorkflowExecuteRunnable workflowExecuteRunnable = this.processInstanceExecCacheManager.getByProcessInstanceId( - processInstanceId); - if (workflowExecuteRunnable == null) { - sendAckToWorker(taskEvent); - throw new TaskEventHandleError( - "Handle task reject event error, cannot find related workflow instance from cache, will discard this event"); - } - TaskInstance taskInstance = workflowExecuteRunnable.getTaskInstance(taskInstanceId).orElseThrow(() -> { - sendAckToWorker(taskEvent); - return new TaskEventHandleError( - "Handle task reject event error, cannot find the taskInstance from cache, will discord this event"); - }); - try { - // todo: If the worker submit multiple reject response to master, the task instance may be dispatch - // multiple, - // we need to control the worker overload by master rather than worker - // if the task resubmit and the worker failover, this task may be dispatch twice? - // todo: we need to clear the taskInstance host and rollback the status to submit. - workflowExecuteRunnable.resubmit(taskInstance.getTaskCode()); - sendAckToWorker(taskEvent); - } catch (Exception ex) { - throw new TaskEventHandleError("Handle task reject event error", ex); - } - - } - - public void sendAckToWorker(TaskEvent taskEvent) { - TaskRejectMessageAck taskRejectMessageAck = new TaskRejectMessageAck(true, - taskEvent.getTaskInstanceId(), - masterConfig.getMasterAddress(), - taskEvent.getWorkerAddress(), - System.currentTimeMillis()); - taskEvent.getChannel().writeAndFlush(taskRejectMessageAck.convert2Command()); - } - - @Override - public TaskEventType getHandleEventType() { - return TaskEventType.WORKER_REJECT; - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEventHandler.java index 28c048d05c..524d0d4a6d 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEventHandler.java @@ -22,8 +22,6 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.remote.exceptions.RemotingException; import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; -import org.apache.dolphinscheduler.server.master.runner.task.ITaskProcessor; -import org.apache.dolphinscheduler.server.master.runner.task.TaskAction; import java.util.Map; import java.util.Optional; @@ -55,15 +53,15 @@ public class TaskStateEventHandler implements StateEventHandler { log.info( "Handle task instance state event, the current task instance state {} will be changed to {}", - task.getState(), taskStateEvent.getStatus()); + task.getState().name(), taskStateEvent.getStatus().name()); Map completeTaskMap = workflowExecuteRunnable.getCompleteTaskMap(); if (task.getState().isFinished() && (taskStateEvent.getStatus() != null && taskStateEvent.getStatus().isRunning())) { String errorMessage = String.format( "The current task instance state is %s, but the task state event status is %s, so the task state event will be ignored", - task.getState(), - taskStateEvent.getStatus()); + task.getState().name(), + taskStateEvent.getStatus().name()); log.warn(errorMessage); throw new StateEventHandleError(errorMessage); } @@ -85,21 +83,7 @@ public class TaskStateEventHandler implements StateEventHandler { } return true; } - Map activeTaskProcessMap = workflowExecuteRunnable.getActiveTaskProcessMap(); - if (activeTaskProcessMap.containsKey(task.getTaskCode())) { - ITaskProcessor iTaskProcessor = activeTaskProcessMap.get(task.getTaskCode()); - iTaskProcessor.action(TaskAction.RUN); - - if (iTaskProcessor.taskInstance().getState().isFinished()) { - if (iTaskProcessor.taskInstance().getState() != task.getState()) { - task.setState(iTaskProcessor.taskInstance().getState()); - } - workflowExecuteRunnable.taskFinished(task); - } - return true; - } - throw new StateEventHandleError( - "Task state event handle error, due to the task is not in activeTaskProcessorMaps"); + return true; } @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskTimeoutStateEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskTimeoutStateEventHandler.java index dfa60f87ad..e01955d81a 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskTimeoutStateEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskTimeoutStateEventHandler.java @@ -23,8 +23,7 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy; import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; -import org.apache.dolphinscheduler.server.master.runner.task.ITaskProcessor; -import org.apache.dolphinscheduler.server.master.runner.task.TaskAction; +import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable; import java.util.Map; @@ -57,13 +56,12 @@ public class TaskTimeoutStateEventHandler implements StateEventHandler { } TaskTimeoutStrategy taskTimeoutStrategy = taskInstance.getTaskDefine() .getTimeoutNotifyStrategy(); - Map activeTaskProcessMap = workflowExecuteRunnable - .getActiveTaskProcessMap(); + Map taskExecuteRunnableMap = + workflowExecuteRunnable.getTaskExecuteRunnableMap(); if ((TaskTimeoutStrategy.FAILED == taskTimeoutStrategy || TaskTimeoutStrategy.WARNFAILED == taskTimeoutStrategy)) { - if (activeTaskProcessMap.containsKey(taskInstance.getTaskCode())) { - ITaskProcessor taskProcessor = activeTaskProcessMap.get(taskInstance.getTaskCode()); - taskProcessor.action(TaskAction.TIMEOUT); + if (taskExecuteRunnableMap.containsKey(taskInstance.getTaskCode())) { + taskExecuteRunnableMap.get(taskInstance.getTaskCode()).timeout(); } else { log.warn( "cannot find the task processor for task {}, so skip task processor action.", diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskUpdatePidEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskUpdatePidEventHandler.java index 3ade584ac9..b15da0d64f 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskUpdatePidEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskUpdatePidEventHandler.java @@ -21,7 +21,7 @@ import org.apache.dolphinscheduler.common.enums.TaskEventType; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.dao.utils.TaskInstanceUtils; -import org.apache.dolphinscheduler.remote.command.task.TaskUpdatePidAckMessage; +import org.apache.dolphinscheduler.remote.command.task.TaskUpdateRuntimeAckMessage; import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; @@ -91,9 +91,9 @@ public class TaskUpdatePidEventHandler implements TaskEventHandler { private void sendAckToWorker(TaskEvent taskEvent) { // If event handle success, send ack to worker to otherwise the worker will retry this event - TaskUpdatePidAckMessage taskUpdatePidAckMessage = - new TaskUpdatePidAckMessage(true, taskEvent.getTaskInstanceId()); - taskEvent.getChannel().writeAndFlush(taskUpdatePidAckMessage.convert2Command()); + TaskUpdateRuntimeAckMessage taskUpdateRuntimeAckMessage = + new TaskUpdateRuntimeAckMessage(true, taskEvent.getTaskInstanceId()); + taskEvent.getChannel().writeAndFlush(taskUpdateRuntimeAckMessage.convert2Command()); } @Override diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStartEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStartEventHandler.java index 2f5b4a28a9..2c29acd13f 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStartEventHandler.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStartEventHandler.java @@ -47,9 +47,6 @@ public class WorkflowStartEventHandler implements WorkflowEventHandler { @Autowired private WorkflowExecuteThreadPool workflowExecuteThreadPool; - @Autowired - private WorkflowEventQueue workflowEventQueue; - @Override public void handleWorkflowEvent(final WorkflowEvent workflowEvent) throws WorkflowEventHandleError { log.info("Handle workflow start event, begin to start a workflow, event: {}", workflowEvent); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/LogicTaskFactoryNotFoundException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/LogicTaskFactoryNotFoundException.java new file mode 100644 index 0000000000..876a3930e9 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/LogicTaskFactoryNotFoundException.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.exception; + +public class LogicTaskFactoryNotFoundException extends MasterException { + + public LogicTaskFactoryNotFoundException(String message) { + super(message); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/LogicTaskInitializeException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/LogicTaskInitializeException.java new file mode 100644 index 0000000000..d37a187f61 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/LogicTaskInitializeException.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.exception; + +public class LogicTaskInitializeException extends MasterException { + + public LogicTaskInitializeException(String message) { + super(message); + } + + public LogicTaskInitializeException(String message, Throwable cause) { + super(message, cause); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/MasterTaskExecuteException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/MasterTaskExecuteException.java new file mode 100644 index 0000000000..af1b0aa3db --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/MasterTaskExecuteException.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.exception; + +public class MasterTaskExecuteException extends MasterException { + + public MasterTaskExecuteException(String message) { + super(message); + } + + public MasterTaskExecuteException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskDispatchException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskDispatchException.java new file mode 100644 index 0000000000..259822e80b --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskDispatchException.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.exception; + +public class TaskDispatchException extends MasterException { + + public TaskDispatchException(String message) { + super(message); + } + + public TaskDispatchException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecuteRunnableCreateException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecuteRunnableCreateException.java new file mode 100644 index 0000000000..c90c508ec8 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecuteRunnableCreateException.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.exception; + +public class TaskExecuteRunnableCreateException extends MasterException { + + public TaskExecuteRunnableCreateException(String message) { + super(message); + } + + public TaskExecuteRunnableCreateException(String message, Throwable throwable) { + super(message, throwable); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecutionContextCreateException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecutionContextCreateException.java new file mode 100644 index 0000000000..ac37c94438 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecutionContextCreateException.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.exception; + +public class TaskExecutionContextCreateException extends MasterException { + + public TaskExecutionContextCreateException(String message) { + super(message); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/WorkflowCreateException.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/WorkflowCreateException.java new file mode 100644 index 0000000000..a2f94984ed --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/WorkflowCreateException.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.exception; + +public class WorkflowCreateException extends MasterException { + + public WorkflowCreateException(String message) { + super(message); + } + + public WorkflowCreateException(String message, Throwable throwable) { + super(message, throwable); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/CacheProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/CacheProcessor.java index 3f6789a523..ee49520a80 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/CacheProcessor.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/CacheProcessor.java @@ -22,7 +22,7 @@ import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.remote.command.Message; import org.apache.dolphinscheduler.remote.command.MessageType; import org.apache.dolphinscheduler.remote.command.cache.CacheExpireRequest; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; import lombok.extern.slf4j.Slf4j; @@ -38,7 +38,7 @@ import io.netty.channel.Channel; */ @Component @Slf4j -public class CacheProcessor implements NettyRequestProcessor { +public class CacheProcessor implements MasterRpcProcessor { @Autowired private CacheManager cacheManager; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/MasterTaskDispatchProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/MasterTaskDispatchProcessor.java new file mode 100644 index 0000000000..34c8a15d52 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/MasterTaskDispatchProcessor.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.processor; + +import org.apache.dolphinscheduler.common.utils.DateUtils; +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.utils.LogUtils; +import org.apache.dolphinscheduler.remote.command.Message; +import org.apache.dolphinscheduler.remote.command.MessageType; +import org.apache.dolphinscheduler.remote.command.task.TaskDispatchRequest; +import org.apache.dolphinscheduler.remote.command.task.TaskDispatchResponse; +import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; +import org.apache.dolphinscheduler.server.master.runner.MasterDelayTaskExecuteRunnableDelayQueue; +import org.apache.dolphinscheduler.server.master.runner.execute.MasterDelayTaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecuteRunnableFactoryBuilder; +import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutionContextHolder; +import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager; + +import java.util.concurrent.TimeUnit; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +import io.netty.channel.Channel; + +@Slf4j +@Component +public class MasterTaskDispatchProcessor implements MasterRpcProcessor { + + @Autowired + private MasterTaskExecuteRunnableFactoryBuilder masterTaskExecuteRunnableFactoryBuilder; + + @Autowired + private MasterMessageSenderManager masterMessageSenderManager; + + @Autowired + private MasterDelayTaskExecuteRunnableDelayQueue masterDelayTaskExecuteRunnableDelayQueue; + + @Override + public void process(Channel channel, Message message) { + TaskDispatchRequest taskDispatchRequest = JSONUtils.parseObject(message.getBody(), TaskDispatchRequest.class); + log.info("Receive task dispatch request, command: {}", taskDispatchRequest); + TaskExecutionContext taskExecutionContext = taskDispatchRequest.getTaskExecutionContext(); + taskExecutionContext.setLogPath(LogUtils.getTaskInstanceLogFullPath(taskExecutionContext)); + try ( + final LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setWorkflowAndTaskInstanceIDMDC( + taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId()); + final LogUtils.MDCAutoClosableContext mdcAutoClosableContext1 = + LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath())) { + MasterTaskExecutionContextHolder.putTaskExecutionContext(taskExecutionContext); + // todo: calculate the delay in master dispatcher then we don't need to use a queue to store the task + long remainTime = + DateUtils.getRemainTime(DateUtils.timeStampToDate(taskExecutionContext.getFirstSubmitTime()), + TimeUnit.SECONDS.toMillis(taskExecutionContext.getDelayTime())); + + // todo: move this to the master delay queue + if (remainTime > 0) { + log.info("Current taskInstance: {} is choose delay execution, delay time: {}ms, remainTime: {}ms", + taskExecutionContext.getTaskName(), + TimeUnit.SECONDS.toMillis(taskExecutionContext.getDelayTime()), remainTime); + taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.DELAY_EXECUTION); + masterMessageSenderManager.getMasterTaskExecuteResultMessageSender().sendMessage(taskExecutionContext); + } + + MasterDelayTaskExecuteRunnable masterDelayTaskExecuteRunnable = masterTaskExecuteRunnableFactoryBuilder + .createWorkerDelayTaskExecuteRunnableFactory(taskExecutionContext.getTaskType()) + .createWorkerTaskExecuteRunnable(taskExecutionContext); + if (masterDelayTaskExecuteRunnableDelayQueue + .submitMasterDelayTaskExecuteRunnable(masterDelayTaskExecuteRunnable)) { + log.info( + "Submit task: {} to MasterDelayTaskExecuteRunnableDelayQueue success", + taskExecutionContext.getTaskName()); + sendDispatchSuccessResult(channel, message, taskExecutionContext); + } else { + log.error( + "Submit task: {} to MasterDelayTaskExecuteRunnableDelayQueue failed, current task waiting queue size: {} is full", + taskExecutionContext.getTaskName(), masterDelayTaskExecuteRunnableDelayQueue.size()); + sendDispatchRejectResult(channel, message, taskExecutionContext); + } + } catch (Exception ex) { + log.error("Handle task dispatch request error, command: {}", taskDispatchRequest, ex); + sendDispatchFailedResult(channel, message, taskExecutionContext, ex); + } + } + + private void sendDispatchSuccessResult(Channel channel, Message dispatchRequest, + TaskExecutionContext taskExecutionContext) { + TaskDispatchResponse taskDispatchResponse = + TaskDispatchResponse.success(taskExecutionContext.getTaskInstanceId()); + channel.writeAndFlush(taskDispatchResponse.convert2Command(dispatchRequest.getOpaque())); + } + + private void sendDispatchRejectResult(Channel channel, Message dispatchRequest, + TaskExecutionContext taskExecutionContext) { + TaskDispatchResponse taskDispatchResponse = + TaskDispatchResponse.failed(taskExecutionContext.getTaskInstanceId(), "Task dispatch queue is full"); + channel.writeAndFlush(taskDispatchResponse.convert2Command(dispatchRequest.getOpaque())); + } + + private void sendDispatchFailedResult(Channel channel, Message dispatchRequest, + TaskExecutionContext taskExecutionContext, Throwable throwable) { + TaskDispatchResponse taskDispatchResponse = + TaskDispatchResponse.failed(taskExecutionContext.getTaskInstanceId(), throwable.getMessage()); + channel.writeAndFlush(taskDispatchResponse.convert2Command(dispatchRequest.getOpaque())); + } + + @Override + public MessageType getCommandType() { + return MessageType.TASK_DISPATCH_REQUEST; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/MasterTaskKillProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/MasterTaskKillProcessor.java new file mode 100644 index 0000000000..1ad56cedd3 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/MasterTaskKillProcessor.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.processor; + +import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; +import org.apache.dolphinscheduler.remote.command.Message; +import org.apache.dolphinscheduler.remote.command.MessageType; +import org.apache.dolphinscheduler.remote.command.task.TaskKillRequest; +import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; +import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; +import org.apache.dolphinscheduler.server.master.runner.MasterDelayTaskExecuteRunnableDelayQueue; +import org.apache.dolphinscheduler.server.master.runner.MasterTaskExecuteRunnableThreadPool; +import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutionContextHolder; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +import io.netty.channel.Channel; + +@Slf4j +@Component +public class MasterTaskKillProcessor implements MasterRpcProcessor { + + @Autowired + private MasterTaskExecuteRunnableThreadPool masterTaskExecuteRunnableThreadPool; + + @Autowired + private MasterDelayTaskExecuteRunnableDelayQueue masterDelayTaskExecuteRunnableDelayQueue; + + @Override + public void process(Channel channel, Message message) { + TaskKillRequest taskKillRequest = JSONUtils.parseObject(message.getBody(), TaskKillRequest.class); + log.info("Master receive task kill request: {}", taskKillRequest); + int taskInstanceId = taskKillRequest.getTaskInstanceId(); + try (LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setTaskInstanceIdMDC(taskInstanceId)) { + TaskExecutionContext taskExecutionContext = + MasterTaskExecutionContextHolder.getTaskExecutionContext(taskInstanceId); + if (taskExecutionContext == null) { + log.error("Cannot find the TaskExecutionContext, this task may already been killed"); + return; + } + MasterTaskExecuteRunnable masterTaskExecuteRunnable = + masterTaskExecuteRunnableThreadPool.getMasterTaskExecuteRunnable(taskInstanceId); + if (masterTaskExecuteRunnable == null) { + log.error("Cannot find the MasterTaskExecuteRunnable, this task may already been killed"); + return; + } + try { + masterTaskExecuteRunnable.cancelTask(); + masterDelayTaskExecuteRunnableDelayQueue + .removeMasterDelayTaskExecuteRunnable(masterTaskExecuteRunnable); + } catch (MasterTaskExecuteException e) { + log.error("Cancel MasterTaskExecuteRunnable failed ", e); + } + } + } + + @Override + public MessageType getCommandType() { + return MessageType.TASK_KILL_REQUEST; + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/MasterTaskPauseProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/MasterTaskPauseProcessor.java new file mode 100644 index 0000000000..8e53b26396 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/MasterTaskPauseProcessor.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.processor; + +import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; +import org.apache.dolphinscheduler.remote.command.Message; +import org.apache.dolphinscheduler.remote.command.MessageType; +import org.apache.dolphinscheduler.remote.command.task.TaskPauseRequest; +import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; +import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; +import org.apache.dolphinscheduler.server.master.runner.MasterTaskExecuteRunnableThreadPool; +import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecuteRunnable; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +import io.netty.channel.Channel; + +@Slf4j +@Component +public class MasterTaskPauseProcessor implements MasterRpcProcessor { + + @Autowired + private MasterTaskExecuteRunnableThreadPool masterTaskExecuteRunnableThreadPool; + + @Override + public void process(Channel channel, Message message) { + TaskPauseRequest taskPauseRequest = JSONUtils.parseObject(message.getBody(), TaskPauseRequest.class); + MasterTaskExecuteRunnable masterTaskExecuteRunnable = + masterTaskExecuteRunnableThreadPool.getMasterTaskExecuteRunnable(taskPauseRequest.getTaskInstanceId()); + if (masterTaskExecuteRunnable == null) { + log.info("Cannot find the MasterTaskExecuteRunnable"); + return; + } + TaskExecutionContext taskExecutionContext = masterTaskExecuteRunnable.getTaskExecutionContext(); + try ( + LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setWorkflowAndTaskInstanceIDMDC( + taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId())) { + masterTaskExecuteRunnable.pauseTask(); + } catch (MasterTaskExecuteException e) { + log.error("Pause MasterTaskExecuteRunnable failed", e); + } + } + + @Override + public MessageType getCommandType() { + return MessageType.PAUSE_TASK_INSTANCE; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/StateEventProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/StateEventProcessor.java index d4a784f120..bfee81dc48 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/StateEventProcessor.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/StateEventProcessor.java @@ -24,7 +24,7 @@ import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; import org.apache.dolphinscheduler.remote.command.Message; import org.apache.dolphinscheduler.remote.command.MessageType; import org.apache.dolphinscheduler.remote.command.workflow.WorkflowStateEventChangeRequest; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; import org.apache.dolphinscheduler.server.master.event.StateEvent; import org.apache.dolphinscheduler.server.master.event.TaskStateEvent; import org.apache.dolphinscheduler.server.master.event.WorkflowStateEvent; @@ -42,7 +42,7 @@ import io.netty.channel.Channel; */ @Component @Slf4j -public class StateEventProcessor implements NettyRequestProcessor { +public class StateEventProcessor implements MasterRpcProcessor { @Autowired private StateEventResponseService stateEventResponseService; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteResultProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteResultProcessor.java index 81436e3c70..bf48d62b69 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteResultProcessor.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteResultProcessor.java @@ -22,7 +22,7 @@ import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; import org.apache.dolphinscheduler.remote.command.Message; import org.apache.dolphinscheduler.remote.command.MessageType; import org.apache.dolphinscheduler.remote.command.task.TaskExecuteResultMessage; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; @@ -38,7 +38,7 @@ import io.netty.channel.Channel; */ @Component @Slf4j -public class TaskExecuteResultProcessor implements NettyRequestProcessor { +public class TaskExecuteResultProcessor implements MasterRpcProcessor { @Autowired private TaskEventService taskEventService; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteRunningProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteRunningProcessor.java index 55f15a31cf..e768693bea 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteRunningProcessor.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteRunningProcessor.java @@ -21,7 +21,7 @@ import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.remote.command.Message; import org.apache.dolphinscheduler.remote.command.MessageType; import org.apache.dolphinscheduler.remote.command.task.TaskExecuteRunningMessage; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; @@ -37,7 +37,7 @@ import io.netty.channel.Channel; */ @Component @Slf4j -public class TaskExecuteRunningProcessor implements NettyRequestProcessor { +public class TaskExecuteRunningProcessor implements MasterRpcProcessor { @Autowired private TaskEventService taskEventService; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteStartProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteStartProcessor.java index 63f07f5b3d..67c69555d1 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteStartProcessor.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteStartProcessor.java @@ -23,7 +23,7 @@ import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao; import org.apache.dolphinscheduler.remote.command.Message; import org.apache.dolphinscheduler.remote.command.MessageType; import org.apache.dolphinscheduler.remote.command.task.TaskExecuteStartMessage; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; import org.apache.dolphinscheduler.server.master.runner.StreamTaskExecuteRunnable; import org.apache.dolphinscheduler.server.master.runner.StreamTaskExecuteThreadPool; @@ -39,7 +39,7 @@ import io.netty.channel.Channel; */ @Component @Slf4j -public class TaskExecuteStartProcessor implements NettyRequestProcessor { +public class TaskExecuteStartProcessor implements MasterRpcProcessor { @Autowired private StreamTaskExecuteThreadPool streamTaskExecuteThreadPool; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskForceStartProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskForceStartProcessor.java index 9185a0c700..5a1dfb93a6 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskForceStartProcessor.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskForceStartProcessor.java @@ -23,7 +23,7 @@ import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; import org.apache.dolphinscheduler.remote.command.Message; import org.apache.dolphinscheduler.remote.command.MessageType; import org.apache.dolphinscheduler.remote.command.task.TaskForceStartRequest; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; import org.apache.dolphinscheduler.server.master.event.TaskStateEvent; import org.apache.dolphinscheduler.server.master.processor.queue.StateEventResponseService; @@ -36,7 +36,7 @@ import io.netty.channel.Channel; @Component @Slf4j -public class TaskForceStartProcessor implements NettyRequestProcessor { +public class TaskForceStartProcessor implements MasterRpcProcessor { @Autowired private StateEventResponseService stateEventResponseService; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskKillResponseProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskKillResponseProcessor.java index 7b88626665..76ec751103 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskKillResponseProcessor.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskKillResponseProcessor.java @@ -21,7 +21,7 @@ import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.remote.command.Message; import org.apache.dolphinscheduler.remote.command.MessageType; import org.apache.dolphinscheduler.remote.command.task.TaskKillResponse; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; import lombok.extern.slf4j.Slf4j; @@ -34,7 +34,7 @@ import io.netty.channel.Channel; */ @Component @Slf4j -public class TaskKillResponseProcessor implements NettyRequestProcessor { +public class TaskKillResponseProcessor implements MasterRpcProcessor { /** * task final result response diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskRecallProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskRecallProcessor.java deleted file mode 100644 index 9a8967a12f..0000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskRecallProcessor.java +++ /dev/null @@ -1,68 +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.processor; - -import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; -import org.apache.dolphinscheduler.remote.command.Message; -import org.apache.dolphinscheduler.remote.command.MessageType; -import org.apache.dolphinscheduler.remote.command.task.TaskRejectMessage; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; -import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; -import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; - -import lombok.extern.slf4j.Slf4j; - -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Component; - -import io.netty.channel.Channel; - -/** - * task recall processor - */ -@Component -@Slf4j -public class TaskRecallProcessor implements NettyRequestProcessor { - - @Autowired - private TaskEventService taskEventService; - - /** - * task ack process - * - * @param channel channel channel - * @param message command TaskExecuteAckCommand - */ - @Override - public void process(Channel channel, Message message) { - TaskRejectMessage recallCommand = JSONUtils.parseObject(message.getBody(), TaskRejectMessage.class); - TaskEvent taskEvent = TaskEvent.newRecallEvent(recallCommand, channel); - try ( - final LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setWorkflowAndTaskInstanceIDMDC( - recallCommand.getProcessInstanceId(), recallCommand.getTaskInstanceId())) { - log.info("Receive task recall command: {}", recallCommand); - taskEventService.addEvent(taskEvent); - } - } - - @Override - public MessageType getCommandType() { - return MessageType.TASK_REJECT; - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskUpdatePidProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskUpdateRuntimeProcessor.java similarity index 88% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskUpdatePidProcessor.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskUpdateRuntimeProcessor.java index 414012900e..5edf3acf6e 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskUpdatePidProcessor.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskUpdateRuntimeProcessor.java @@ -20,8 +20,8 @@ package org.apache.dolphinscheduler.server.master.processor; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.remote.command.Message; import org.apache.dolphinscheduler.remote.command.MessageType; -import org.apache.dolphinscheduler.remote.command.task.TaskUpdatePidMessage; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.command.task.TaskUpdateRuntimeMessage; +import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; @@ -37,7 +37,7 @@ import io.netty.channel.Channel; */ @Component @Slf4j -public class TaskUpdatePidProcessor implements NettyRequestProcessor { +public class TaskUpdateRuntimeProcessor implements MasterRpcProcessor { @Autowired private TaskEventService taskEventService; @@ -50,8 +50,8 @@ public class TaskUpdatePidProcessor implements NettyRequestProcessor { */ @Override public void process(Channel channel, Message message) { - TaskUpdatePidMessage taskUpdatePidRequest = - JSONUtils.parseObject(message.getBody(), TaskUpdatePidMessage.class); + TaskUpdateRuntimeMessage taskUpdatePidRequest = + JSONUtils.parseObject(message.getBody(), TaskUpdateRuntimeMessage.class); log.info("taskUpdatePidCommand: {}", taskUpdatePidRequest); TaskEvent taskEvent = TaskEvent.newUpdatePidEvent(taskUpdatePidRequest, @@ -62,7 +62,7 @@ public class TaskUpdatePidProcessor implements NettyRequestProcessor { @Override public MessageType getCommandType() { - return MessageType.TASK_UPDATE_PID_MESSAGE; + return MessageType.TASK_UPDATE_RUNTIME_MESSAGE; } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskWakeupProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskWakeupProcessor.java index 4799024e79..d7cf7b5a8f 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskWakeupProcessor.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskWakeupProcessor.java @@ -23,7 +23,7 @@ import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; import org.apache.dolphinscheduler.remote.command.Message; import org.apache.dolphinscheduler.remote.command.MessageType; import org.apache.dolphinscheduler.remote.command.task.TaskWakeupRequest; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; import org.apache.dolphinscheduler.server.master.event.TaskStateEvent; import org.apache.dolphinscheduler.server.master.processor.queue.StateEventResponseService; @@ -39,7 +39,7 @@ import io.netty.channel.Channel; */ @Component @Slf4j -public class TaskWakeupProcessor implements NettyRequestProcessor { +public class TaskWakeupProcessor implements MasterRpcProcessor { @Autowired private StateEventResponseService stateEventResponseService; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/WorkflowExecutingDataRequestProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/WorkflowExecutingDataRequestProcessor.java index b3fb6f2518..036cc315ff 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/WorkflowExecutingDataRequestProcessor.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/WorkflowExecutingDataRequestProcessor.java @@ -23,7 +23,7 @@ import org.apache.dolphinscheduler.remote.command.MessageType; import org.apache.dolphinscheduler.remote.command.workflow.WorkflowExecutingDataRequest; import org.apache.dolphinscheduler.remote.command.workflow.WorkflowExecutingDataResponse; import org.apache.dolphinscheduler.remote.dto.WorkflowExecuteDto; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; import org.apache.dolphinscheduler.server.master.service.ExecutingService; import java.util.Optional; @@ -40,7 +40,7 @@ import io.netty.channel.Channel; */ @Component @Slf4j -public class WorkflowExecutingDataRequestProcessor implements NettyRequestProcessor { +public class WorkflowExecutingDataRequestProcessor implements MasterRpcProcessor { @Autowired private ExecutingService executingService; diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/WorkflowMetricsCleanUpProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/WorkflowMetricsCleanUpProcessor.java index ea08426266..dae969c4ad 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/WorkflowMetricsCleanUpProcessor.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/WorkflowMetricsCleanUpProcessor.java @@ -21,7 +21,7 @@ import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.remote.command.Message; import org.apache.dolphinscheduler.remote.command.MessageType; import org.apache.dolphinscheduler.remote.command.workflow.WorkflowMetricsCleanUpRequest; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics; import org.springframework.stereotype.Component; @@ -29,7 +29,7 @@ import org.springframework.stereotype.Component; import io.netty.channel.Channel; @Component -public class WorkflowMetricsCleanUpProcessor implements NettyRequestProcessor { +public class WorkflowMetricsCleanUpProcessor implements MasterRpcProcessor { @Override public void process(Channel channel, Message message) { diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/StateEventResponseService.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/StateEventResponseService.java index bfa5d00945..11329732c9 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/StateEventResponseService.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/StateEventResponseService.java @@ -114,7 +114,7 @@ public class StateEventResponseService { try { stateEvent = eventQueue.take(); } catch (InterruptedException e) { - log.warn("State event loop service interrupted, will stop this loop", e); + log.warn("State event loop service interrupted, will stop loop"); Thread.currentThread().interrupt(); break; } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskEvent.java index 5f2097f58d..ff250ed403 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskEvent.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskEvent.java @@ -22,8 +22,7 @@ import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; import org.apache.dolphinscheduler.remote.command.task.TaskExecuteResultMessage; import org.apache.dolphinscheduler.remote.command.task.TaskExecuteRunningMessage; -import org.apache.dolphinscheduler.remote.command.task.TaskRejectMessage; -import org.apache.dolphinscheduler.remote.command.task.TaskUpdatePidMessage; +import org.apache.dolphinscheduler.remote.command.task.TaskUpdateRuntimeMessage; import java.util.Date; @@ -145,15 +144,6 @@ public class TaskEvent { return event; } - public static TaskEvent newRecallEvent(TaskRejectMessage command, Channel channel) { - TaskEvent event = new TaskEvent(); - event.setTaskInstanceId(command.getTaskInstanceId()); - event.setProcessInstanceId(command.getProcessInstanceId()); - event.setChannel(channel); - event.setEvent(TaskEventType.WORKER_REJECT); - return event; - } - public static TaskEvent newCacheEvent(int processInstanceId, int taskInstanceId, int cacheTaskInstanceId) { TaskEvent event = new TaskEvent(); event.setProcessInstanceId(processInstanceId); @@ -163,7 +153,7 @@ public class TaskEvent { return event; } - public static TaskEvent newUpdatePidEvent(TaskUpdatePidMessage command, Channel channel, String workerAddress) { + public static TaskEvent newUpdatePidEvent(TaskUpdateRuntimeMessage command, Channel channel, String workerAddress) { TaskEvent event = new TaskEvent(); event.setProcessInstanceId(command.getProcessInstanceId()); event.setTaskInstanceId(command.getTaskInstanceId()); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterRPCServer.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterRPCServer.java index 56df578461..ac9d4da19d 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterRPCServer.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterRPCServer.java @@ -19,7 +19,7 @@ package org.apache.dolphinscheduler.server.master.rpc; import org.apache.dolphinscheduler.remote.NettyRemotingServer; import org.apache.dolphinscheduler.remote.config.NettyServerConfig; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import java.util.List; @@ -42,7 +42,7 @@ public class MasterRPCServer implements AutoCloseable { private MasterConfig masterConfig; @Autowired - private List nettyRequestProcessors; + private List masterRpcProcessors; public void start() { log.info("Starting Master RPC Server..."); @@ -50,9 +50,9 @@ public class MasterRPCServer implements AutoCloseable { NettyServerConfig serverConfig = new NettyServerConfig(); serverConfig.setListenPort(masterConfig.getListenPort()); this.nettyRemotingServer = new NettyRemotingServer(serverConfig); - for (NettyRequestProcessor nettyRequestProcessor : nettyRequestProcessors) { - this.nettyRemotingServer.registerProcessor(nettyRequestProcessor); - log.info("Success register netty processor: {}", nettyRequestProcessor.getClass().getName()); + for (MasterRpcProcessor masterRpcProcessor : masterRpcProcessors) { + this.nettyRemotingServer.registerProcessor(masterRpcProcessor); + log.info("Success register netty processor: {}", masterRpcProcessor.getClass().getName()); } this.nettyRemotingServer.start(); log.info("Started Master RPC Server..."); diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterRpcClient.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterRpcClient.java index a8f587f347..413523d527 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterRpcClient.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterRpcClient.java @@ -46,4 +46,7 @@ public class MasterRpcClient { return client.sendSync(host, rpcMessage, DEFAULT_TIME_OUT_MILLS); } + public void send(Host of, Message message) throws RemotingException { + client.send(of, message); + } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseTaskDispatcher.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseTaskDispatcher.java new file mode 100644 index 0000000000..9878391ea5 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseTaskDispatcher.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.dolphinscheduler.common.enums.TaskEventType; +import org.apache.dolphinscheduler.common.utils.DateUtils; +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.remote.command.Message; +import org.apache.dolphinscheduler.remote.command.task.TaskDispatchRequest; +import org.apache.dolphinscheduler.remote.command.task.TaskDispatchResponse; +import org.apache.dolphinscheduler.remote.exceptions.RemotingException; +import org.apache.dolphinscheduler.remote.utils.Host; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; +import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; +import org.apache.dolphinscheduler.server.master.exception.TaskDispatchException; +import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; +import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; +import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; +import org.apache.dolphinscheduler.server.master.runner.dispatcher.TaskDispatcher; +import org.apache.dolphinscheduler.server.master.runner.execute.TaskExecuteRunnable; + +import java.util.Date; +import java.util.Optional; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public abstract class BaseTaskDispatcher implements TaskDispatcher { + + protected final TaskEventService taskEventService; + protected final MasterConfig masterConfig; + protected final MasterRpcClient masterRpcClient; + + protected BaseTaskDispatcher(TaskEventService taskEventService, + MasterConfig masterConfig, + MasterRpcClient masterRpcClient) { + this.taskEventService = checkNotNull(taskEventService); + this.masterConfig = checkNotNull(masterConfig); + this.masterRpcClient = checkNotNull(masterRpcClient); + } + + @Override + public void dispatchTask(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException { + Host taskInstanceDispatchHost; + try { + taskInstanceDispatchHost = getTaskInstanceDispatchHost(taskExecuteRunnable) + .orElseThrow(() -> new TaskDispatchException("Cannot find the host to execute task.")); + } catch (WorkerGroupNotFoundException workerGroupNotFoundException) { + log.error("Dispatch task: {} failed, worker group not found.", + taskExecuteRunnable.getTaskExecutionContext().getTaskName(), workerGroupNotFoundException); + addDispatchFailedEvent(taskExecuteRunnable); + return; + } + taskExecuteRunnable.getTaskExecutionContext().setHost(taskInstanceDispatchHost.getAddress()); + doDispatch(taskExecuteRunnable); + taskExecuteRunnable.getTaskInstance().setHost(taskInstanceDispatchHost.getAddress()); + log.info("Success dispatch task {} to {}.", taskExecuteRunnable.getTaskExecutionContext().getTaskName(), + taskInstanceDispatchHost.getAddress()); + addDispatchEvent(taskExecuteRunnable); + } + + protected abstract Optional getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecutionContext) throws TaskDispatchException, WorkerGroupNotFoundException; + + protected void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException { + TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext(); + TaskDispatchRequest taskDispatchRequest = new TaskDispatchRequest(taskExecutionContext); + try { + Message message = masterRpcClient.sendSyncCommand(Host.of(taskExecutionContext.getHost()), + taskDispatchRequest.convert2Command()); + TaskDispatchResponse taskDispatchResponse = + JSONUtils.parseObject(message.getBody(), TaskDispatchResponse.class); + if (!taskDispatchResponse.isDispatchSuccess()) { + throw new TaskDispatchException(String.format("Dispatch task to %s failed, response is: %s", + taskExecutionContext.getHost(), taskDispatchResponse)); + } + } catch (InterruptedException e) { + // This exception should only happen when we close the server. + Thread.currentThread().interrupt(); + throw new TaskDispatchException(String.format("Dispatch task to %s failed, get response failed", + taskExecutionContext.getHost()), e); + } catch (RemotingException e) { + throw new TaskDispatchException(String.format("Dispatch task to %s failed", + taskExecutionContext.getHost()), e); + } + } + + protected void addDispatchEvent(TaskExecuteRunnable taskExecuteRunnable) { + TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext(); + TaskEvent taskEvent = TaskEvent.newDispatchEvent( + taskExecutionContext.getProcessInstanceId(), + taskExecutionContext.getTaskInstanceId(), + taskExecutionContext.getHost()); + taskEventService.addEvent(taskEvent); + } + + private void addDispatchFailedEvent(TaskExecuteRunnable taskExecuteRunnable) { + TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext(); + TaskEvent taskEvent = TaskEvent.builder() + .processInstanceId(taskExecutionContext.getProcessInstanceId()) + .taskInstanceId(taskExecutionContext.getTaskInstanceId()) + .state(TaskExecutionStatus.FAILURE) + .logPath(taskExecutionContext.getLogPath()) + .executePath(taskExecutionContext.getExecutePath()) + .appIds(taskExecutionContext.getAppIds()) + .processId(taskExecutionContext.getProcessId()) + .varPool(taskExecutionContext.getVarPool()) + .startTime(DateUtils.timeStampToDate(taskExecutionContext.getStartTime())) + .endTime(new Date()) + .event(TaskEventType.RESULT) + .build(); + taskEventService.addEvent(taskEvent); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueue.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueue.java new file mode 100644 index 0000000000..a8f8f88498 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueue.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.DefaultTaskExecuteRunnable; + +import java.util.concurrent.PriorityBlockingQueue; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class GlobalTaskDispatchWaitingQueue { + + private final PriorityBlockingQueue queue = new PriorityBlockingQueue<>(); + + public void submitNeedToDispatchTaskExecuteRunnable(DefaultTaskExecuteRunnable priorityTaskExecuteRunnable) { + queue.put(priorityTaskExecuteRunnable); + } + + public DefaultTaskExecuteRunnable takeNeedToDispatchTaskExecuteRunnable() throws InterruptedException { + return queue.take(); + } + + public int getWaitingDispatchTaskNumber() { + return queue.size(); + } + +} 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 new file mode 100644 index 0000000000..53ad4e6ef7 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooper.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.server.master.runner; + +import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; +import org.apache.dolphinscheduler.common.thread.ThreadUtils; +import org.apache.dolphinscheduler.server.master.runner.dispatcher.TaskDispatchFactory; +import org.apache.dolphinscheduler.server.master.runner.dispatcher.TaskDispatcher; +import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable; + +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class GlobalTaskDispatchWaitingQueueLooper extends BaseDaemonThread implements AutoCloseable { + + @Autowired + private GlobalTaskDispatchWaitingQueue globalTaskDispatchWaitingQueue; + + @Autowired + private TaskDispatchFactory taskDispatchFactory; + + private final AtomicBoolean RUNNING_FLAG = new AtomicBoolean(false); + + private final AtomicInteger DISPATCHED_TIMES = new AtomicInteger(); + + private static final Integer MAX_DISPATCHED_FAILED_TIMES = 100; + + public GlobalTaskDispatchWaitingQueueLooper() { + super("GlobalTaskDispatchWaitingQueueLooper"); + } + + @Override + public synchronized void start() { + if (!RUNNING_FLAG.compareAndSet(false, true)) { + log.error("The GlobalTaskDispatchWaitingQueueLooper already started, will not start again"); + return; + } + log.info("GlobalTaskDispatchWaitingQueueLooper starting..."); + super.start(); + log.info("GlobalTaskDispatchWaitingQueueLooper started..."); + } + + @Override + public void run() { + DefaultTaskExecuteRunnable defaultTaskExecuteRunnable; + while (RUNNING_FLAG.get()) { + try { + defaultTaskExecuteRunnable = globalTaskDispatchWaitingQueue.takeNeedToDispatchTaskExecuteRunnable(); + } catch (InterruptedException e) { + log.warn("Get waiting dispatch task failed, the current thread has been interrupted, will stop loop"); + Thread.currentThread().interrupt(); + break; + } + try { + final TaskDispatcher taskDispatcher = taskDispatchFactory + .getTaskDispatcher(defaultTaskExecuteRunnable.getTaskInstance().getTaskType()); + taskDispatcher.dispatchTask(defaultTaskExecuteRunnable); + DISPATCHED_TIMES.set(0); + } catch (Exception e) { + globalTaskDispatchWaitingQueue.submitNeedToDispatchTaskExecuteRunnable(defaultTaskExecuteRunnable); + if (DISPATCHED_TIMES.incrementAndGet() > MAX_DISPATCHED_FAILED_TIMES) { + ThreadUtils.sleep(10 * 1000L); + } + log.error("Dispatch task failed", e); + } + } + log.info("GlobalTaskDispatchWaitingQueueLooper started..."); + } + + @Override + public void close() throws Exception { + if (RUNNING_FLAG.compareAndSet(true, false)) { + log.info("GlobalTaskDispatchWaitingQueueLooper stopping..."); + log.info("GlobalTaskDispatchWaitingQueueLooper stopped..."); + } + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterDelayTaskExecuteRunnableDelayQueue.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterDelayTaskExecuteRunnableDelayQueue.java new file mode 100644 index 0000000000..bdd1510527 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterDelayTaskExecuteRunnableDelayQueue.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.server.master.runner; + +import org.apache.dolphinscheduler.server.master.runner.execute.MasterDelayTaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecuteRunnable; + +import java.util.concurrent.DelayQueue; + +import org.springframework.stereotype.Component; + +/** + * + */ +@Component +public class MasterDelayTaskExecuteRunnableDelayQueue { + + private final DelayQueue masterDelayTaskExecuteRunnableDelayQueue = + new DelayQueue<>(); + + public boolean submitMasterDelayTaskExecuteRunnable(MasterDelayTaskExecuteRunnable masterDelayTaskExecuteRunnable) { + return masterDelayTaskExecuteRunnableDelayQueue.offer(masterDelayTaskExecuteRunnable); + } + + public MasterDelayTaskExecuteRunnable takeMasterDelayTaskExecuteRunnable() throws InterruptedException { + return masterDelayTaskExecuteRunnableDelayQueue.take(); + } + + // todo: if we move the delay process to master, than we don't need this method, since dispatchProcess can directly + // submit to thread pool + public boolean removeMasterDelayTaskExecuteRunnable(MasterTaskExecuteRunnable masterTaskExecuteRunnable) { + return masterDelayTaskExecuteRunnableDelayQueue.remove(masterTaskExecuteRunnable); + } + + public int size() { + return masterDelayTaskExecuteRunnableDelayQueue.size(); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterDelayTaskExecuteRunnableDelayQueueLooper.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterDelayTaskExecuteRunnableDelayQueueLooper.java new file mode 100644 index 0000000000..3b429e5d76 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterDelayTaskExecuteRunnableDelayQueueLooper.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.common.thread.BaseDaemonThread; +import org.apache.dolphinscheduler.server.master.runner.execute.MasterDelayTaskExecuteRunnable; + +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 MasterDelayTaskExecuteRunnableDelayQueueLooper extends BaseDaemonThread implements AutoCloseable { + + @Autowired + private MasterDelayTaskExecuteRunnableDelayQueue masterDelayTaskExecuteRunnableDelayQueue; + + @Autowired + private MasterTaskExecuteRunnableThreadPool masterTaskExecuteRunnableThreadPool; + + private final AtomicBoolean RUNNING_FLAG = new AtomicBoolean(false); + + public MasterDelayTaskExecuteRunnableDelayQueueLooper() { + super("MasterDelayTaskExecuteRunnableDelayQueueLooper"); + } + + @Override + public synchronized void start() { + if (!RUNNING_FLAG.compareAndSet(false, true)) { + log.error("The MasterDelayTaskExecuteRunnableDelayQueueLooper already started, will not start again"); + return; + } + log.info("MasterDelayTaskExecuteRunnableDelayQueueLooper starting..."); + super.start(); + masterTaskExecuteRunnableThreadPool.start(); + log.info("MasterDelayTaskExecuteRunnableDelayQueueLooper started..."); + } + + @Override + public void run() { + while (RUNNING_FLAG.get()) { + try { + final MasterDelayTaskExecuteRunnable masterDelayTaskExecuteRunnable = + masterDelayTaskExecuteRunnableDelayQueue.takeMasterDelayTaskExecuteRunnable(); + masterTaskExecuteRunnableThreadPool.submitMasterTaskExecuteRunnable(masterDelayTaskExecuteRunnable); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + log.warn("MasterDelayTaskExecuteRunnableDelayQueueLooper has been interrupted, will stop loop"); + break; + } + } + log.info("MasterDelayTaskExecuteRunnableDelayQueueLooper stop loop..."); + } + + @Override + public void close() throws Exception { + if (RUNNING_FLAG.compareAndSet(true, false)) { + log.info("MasterDelayTaskExecuteRunnableDelayQueueLooper stopping..."); + log.info("MasterDelayTaskExecuteRunnableDelayQueueLooper stopped..."); + } + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerBootstrap.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerBootstrap.java index 0a8ab85ee3..cde57407a2 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerBootstrap.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerBootstrap.java @@ -18,40 +18,28 @@ package org.apache.dolphinscheduler.server.master.runner; import org.apache.dolphinscheduler.common.constants.Constants; -import org.apache.dolphinscheduler.common.enums.SlotCheckState; import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager; import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; import org.apache.dolphinscheduler.common.thread.ThreadUtils; -import org.apache.dolphinscheduler.common.utils.NetUtils; import org.apache.dolphinscheduler.common.utils.OSUtils; import org.apache.dolphinscheduler.dao.entity.Command; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; -import org.apache.dolphinscheduler.dao.repository.TaskDefinitionLogDao; -import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; -import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.event.WorkflowEvent; import org.apache.dolphinscheduler.server.master.event.WorkflowEventQueue; import org.apache.dolphinscheduler.server.master.event.WorkflowEventType; import org.apache.dolphinscheduler.server.master.exception.MasterException; +import org.apache.dolphinscheduler.server.master.exception.WorkflowCreateException; import org.apache.dolphinscheduler.server.master.metrics.MasterServerMetrics; import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics; import org.apache.dolphinscheduler.server.master.registry.ServerNodeManager; -import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; -import org.apache.dolphinscheduler.service.alert.ProcessAlertManager; import org.apache.dolphinscheduler.service.command.CommandService; -import org.apache.dolphinscheduler.service.expand.CuringParamsService; -import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.commons.collections4.CollectionUtils; -import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ThreadPoolExecutor; import lombok.extern.slf4j.Slf4j; @@ -65,43 +53,17 @@ import org.springframework.stereotype.Service; @Slf4j public class MasterSchedulerBootstrap extends BaseDaemonThread implements AutoCloseable { - @Autowired - private ProcessService processService; - @Autowired private CommandService commandService; - @Autowired - private ProcessInstanceDao processInstanceDao; - - @Autowired - private TaskInstanceDao taskInstanceDao; - - @Autowired - private TaskDefinitionLogDao taskDefinitionLogDao; - @Autowired private MasterConfig masterConfig; - @Autowired - private ProcessAlertManager processAlertManager; - - @Autowired - private MasterRpcClient masterRpcClient; - - /** - * master prepare exec service - */ - private ThreadPoolExecutor masterPrepareExecService; - @Autowired private ProcessInstanceExecCacheManager processInstanceExecCacheManager; @Autowired - private StateWheelExecuteThread stateWheelExecuteThread; - - @Autowired - private CuringParamsService curingGlobalParamsService; + private WorkflowExecuteRunnableFactory workflowExecuteRunnableFactory; @Autowired private WorkflowEventQueue workflowEventQueue; @@ -112,33 +74,31 @@ public class MasterSchedulerBootstrap extends BaseDaemonThread implements AutoCl @Autowired private ServerNodeManager serverNodeManager; - private String masterAddress; + @Autowired + private MasterTaskExecutorBootstrap masterTaskExecutorBootstrap; protected MasterSchedulerBootstrap() { super("MasterCommandLoopThread"); } - /** - * constructor of MasterSchedulerService - */ - public void init() { - this.masterPrepareExecService = (ThreadPoolExecutor) ThreadUtils - .newDaemonFixedThreadExecutor("MasterPreExecThread", masterConfig.getPreExecThreads()); - this.masterAddress = NetUtils.getAddr(masterConfig.getListenPort()); - } - @Override public synchronized void start() { - log.info("Master schedule bootstrap starting.."); + log.info("MasterSchedulerBootstrap starting.."); super.start(); workflowEventLooper.start(); - log.info("Master schedule bootstrap started..."); + masterTaskExecutorBootstrap.start(); + log.info("MasterSchedulerBootstrap started..."); } @Override - public void close() { - log.info("Master schedule bootstrap stopping..."); - log.info("Master schedule bootstrap stopped..."); + public void close() throws Exception { + log.info("MasterSchedulerBootstrap stopping..."); + try ( + final WorkflowEventLooper workflowEventLooper1 = workflowEventLooper; + final MasterTaskExecutorBootstrap masterTaskExecutorBootstrap1 = masterTaskExecutorBootstrap) { + // closed the resource + } + log.info("MasterSchedulerBootstrap stopped..."); } /** @@ -150,15 +110,14 @@ public class MasterSchedulerBootstrap extends BaseDaemonThread implements AutoCl try { if (!ServerLifeCycleManager.isRunning()) { // the current server is not at running status, cannot consume command. - log.warn("The current server {} is not at running status, cannot consumes commands.", - this.masterAddress); + log.warn("The current server is not at running status, cannot consumes commands."); Thread.sleep(Constants.SLEEP_TIME_MILLIS); } // todo: if the workflow event queue is much, we need to handle the back pressure boolean isOverload = OSUtils.isOverload(masterConfig.getMaxCpuLoadAvg(), masterConfig.getReservedMemory()); if (isOverload) { - log.warn("The current server {} is overload, cannot consumes commands.", this.masterAddress); + log.warn("The current server is overload, cannot consumes commands."); MasterServerMetrics.incMasterOverload(); Thread.sleep(Constants.SLEEP_TIME_MILLIS); continue; @@ -170,39 +129,25 @@ public class MasterSchedulerBootstrap extends BaseDaemonThread implements AutoCl continue; } - List processInstances = command2ProcessInstance(commands); - if (CollectionUtils.isEmpty(processInstances)) { - // indicate that the command transform to processInstance error, sleep for 1s - Thread.sleep(Constants.SLEEP_TIME_MILLIS); - continue; - } + commands.parallelStream() + .forEach(command -> { + try { + WorkflowExecuteRunnable workflowExecuteRunnable = + workflowExecuteRunnableFactory.createWorkflowExecuteRunnable(command); + ProcessInstance processInstance = workflowExecuteRunnable.getProcessInstance(); + if (processInstanceExecCacheManager.contains(processInstance.getId())) { + log.error( + "The workflow instance is already been cached, this case shouldn't be happened"); + } + processInstanceExecCacheManager.cache(processInstance.getId(), workflowExecuteRunnable); + workflowEventQueue.addEvent( + new WorkflowEvent(WorkflowEventType.START_WORKFLOW, processInstance.getId())); + } catch (WorkflowCreateException workflowCreateException) { + log.error("Master handle command {} error ", command.getId(), workflowCreateException); + commandService.moveToErrorCommand(command, workflowCreateException.toString()); + } + }); MasterServerMetrics.incMasterConsumeCommand(commands.size()); - - processInstances.forEach(processInstance -> { - try { - LogUtils.setWorkflowInstanceIdMDC(processInstance.getId()); - if (processInstanceExecCacheManager.contains(processInstance.getId())) { - log.error( - "The workflow instance is already been cached, this case shouldn't be happened"); - } - WorkflowExecuteRunnable workflowRunnable = new WorkflowExecuteRunnable(processInstance, - commandService, - processService, - processInstanceDao, - masterRpcClient, - processAlertManager, - masterConfig, - stateWheelExecuteThread, - curingGlobalParamsService, - taskInstanceDao, - taskDefinitionLogDao); - processInstanceExecCacheManager.cache(processInstance.getId(), workflowRunnable); - workflowEventQueue.addEvent(new WorkflowEvent(WorkflowEventType.START_WORKFLOW, - processInstance.getId())); - } finally { - LogUtils.removeWorkflowInstanceIdMDC(); - } - }); } catch (InterruptedException interruptedException) { log.warn("Master schedule bootstrap interrupted, close the loop", interruptedException); Thread.currentThread().interrupt(); @@ -215,50 +160,6 @@ public class MasterSchedulerBootstrap extends BaseDaemonThread implements AutoCl } } - private List command2ProcessInstance(List commands) throws InterruptedException { - long commandTransformStartTime = System.currentTimeMillis(); - log.info("Master schedule bootstrap transforming command to ProcessInstance, commandSize: {}", - commands.size()); - List processInstances = Collections.synchronizedList(new ArrayList<>(commands.size())); - CountDownLatch latch = new CountDownLatch(commands.size()); - for (final Command command : commands) { - masterPrepareExecService.execute(() -> { - try { - // Note: this check is not safe, the slot may change after command transform. - // We use the database transaction in `handleCommand` so that we can guarantee the command will - // always be executed - // by only one master - SlotCheckState slotCheckState = slotCheck(command); - if (slotCheckState.equals(SlotCheckState.CHANGE) || slotCheckState.equals(SlotCheckState.INJECT)) { - log.info("Master handle command {} skip, slot check state: {}", command.getId(), - slotCheckState); - return; - } - ProcessInstance processInstance = processService.handleCommand(masterAddress, command); - if (processInstance != null) { - processInstances.add(processInstance); - log.info("Master handle command {} end, create process instance {}", command.getId(), - processInstance.getId()); - } - } catch (Exception e) { - log.error("Master handle command {} error ", command.getId(), e); - commandService.moveToErrorCommand(command, e.toString()); - } finally { - latch.countDown(); - } - }); - } - - // make sure to finish handling command each time before next scan - latch.await(); - log.info( - "Master schedule bootstrap transformed command to ProcessInstance, commandSize: {}, processInstanceSize: {}", - commands.size(), processInstances.size()); - ProcessInstanceMetrics - .recordProcessInstanceGenerateTime(System.currentTimeMillis() - commandTransformStartTime); - return processInstances; - } - private List findCommands() throws MasterException { try { long scheduleStartTime = System.currentTimeMillis(); @@ -284,18 +185,4 @@ public class MasterSchedulerBootstrap extends BaseDaemonThread implements AutoCl } } - private SlotCheckState slotCheck(Command command) { - int slot = serverNodeManager.getSlot(); - int masterSize = serverNodeManager.getMasterSize(); - SlotCheckState state; - if (masterSize <= 0) { - state = SlotCheckState.CHANGE; - } else if (command.getId() % masterSize == slot) { - state = SlotCheckState.PASS; - } else { - state = SlotCheckState.INJECT; - } - return state; - } - } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecuteRunnableThreadPool.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecuteRunnableThreadPool.java new file mode 100644 index 0000000000..86c95308d8 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecuteRunnableThreadPool.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.common.thread.ThreadUtils; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; +import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecuteRunnable; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; + +@Slf4j +@Component +public class MasterTaskExecuteRunnableThreadPool { + + @Autowired + private MasterConfig masterConfig; + + private static final Map SUBMITTED_MASTER_TASK_MAP = new ConcurrentHashMap<>(); + + private ListeningExecutorService listeningExecutorService; + + public synchronized void start() { + log.info("MasterTaskExecuteRunnableThreadPool starting..."); + this.listeningExecutorService = MoreExecutors.listeningDecorator(ThreadUtils.newDaemonFixedThreadExecutor( + "MasterTaskExecuteRunnableThread", masterConfig.getMasterTaskExecuteThreadPoolSize())); + log.info("MasterTaskExecuteRunnableThreadPool started..."); + } + + public void submitMasterTaskExecuteRunnable(MasterTaskExecuteRunnable masterTaskExecuteRunnable) { + ListenableFuture future = listeningExecutorService.submit(masterTaskExecuteRunnable); + Futures.addCallback(future, new MasterTaskExecuteCallback(masterTaskExecuteRunnable), + this.listeningExecutorService); + SUBMITTED_MASTER_TASK_MAP.put(masterTaskExecuteRunnable.getTaskExecutionContext().getTaskInstanceId(), + masterTaskExecuteRunnable); + } + + public MasterTaskExecuteRunnable getMasterTaskExecuteRunnable(Integer taskInstanceId) { + return SUBMITTED_MASTER_TASK_MAP.get(taskInstanceId); + } + + private static class MasterTaskExecuteCallback implements FutureCallback { + + private MasterTaskExecuteRunnable masterTaskExecuteRunnable; + + public MasterTaskExecuteCallback(MasterTaskExecuteRunnable masterTaskExecuteRunnable) { + this.masterTaskExecuteRunnable = masterTaskExecuteRunnable; + } + + @Override + public void onSuccess(Object result) { + log.info("MasterTaskExecuteRunnable execute success, will remove this task"); + SUBMITTED_MASTER_TASK_MAP.remove(masterTaskExecuteRunnable.getTaskExecutionContext().getTaskInstanceId()); + } + + @Override + public void onFailure(Throwable t) { + log.info("MasterTaskExecuteRunnable execute failed, will remove this task"); + SUBMITTED_MASTER_TASK_MAP.remove(masterTaskExecuteRunnable.getTaskExecutionContext().getTaskInstanceId()); + } + } + +} 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 new file mode 100644 index 0000000000..744e560c00 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecutorBootstrap.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 MasterDelayTaskExecuteRunnableDelayQueueLooper masterDelayTaskExecuteRunnableDelayQueueLooper; + + @Autowired + private AsyncMasterTaskDelayQueueLooper asyncMasterTaskDelayQueueLooper; + + public synchronized void start() { + log.info("MasterTaskExecutorBootstrap starting..."); + globalTaskDispatchWaitingQueueLooper.start(); + masterDelayTaskExecuteRunnableDelayQueueLooper.start(); + asyncMasterTaskDelayQueueLooper.start(); + log.info("MasterTaskExecutorBootstrap started..."); + } + + @Override + public void close() throws Exception { + log.info("MasterTaskExecutorBootstrap closing..."); + try ( + final GlobalTaskDispatchWaitingQueueLooper globalTaskDispatchWaitingQueueLooper1 = + globalTaskDispatchWaitingQueueLooper; + final MasterDelayTaskExecuteRunnableDelayQueueLooper masterDelayTaskExecuteRunnableDelayQueueLooper1 = + masterDelayTaskExecuteRunnableDelayQueueLooper; + 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/StateWheelExecuteThread.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StateWheelExecuteThread.java index fc171e4ccc..5576f13050 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StateWheelExecuteThread.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StateWheelExecuteThread.java @@ -136,8 +136,9 @@ public class StateWheelExecuteThread extends BaseDaemonThread { return; } for (Integer processInstanceId : processInstanceTimeoutCheckList) { - try { - LogUtils.setWorkflowInstanceIdMDC(processInstanceId); + try ( + LogUtils.MDCAutoClosableContext mdcAutoClosableContext = + LogUtils.setWorkflowInstanceIdMDC(processInstanceId)) { WorkflowExecuteRunnable workflowExecuteThread = processInstanceExecCacheManager.getByProcessInstanceId( processInstanceId); if (workflowExecuteThread == null) { @@ -162,8 +163,6 @@ public class StateWheelExecuteThread extends BaseDaemonThread { } } catch (Exception ex) { log.error("Check workflow instance timeout error"); - } finally { - LogUtils.removeWorkflowInstanceIdMDC(); } } } @@ -246,9 +245,10 @@ public class StateWheelExecuteThread extends BaseDaemonThread { return; } for (TaskInstanceKey taskInstanceKey : taskInstanceTimeoutCheckList) { - try { + try ( + LogUtils.MDCAutoClosableContext mdcAutoClosableContext = + LogUtils.setWorkflowInstanceIdMDC(taskInstanceKey.getProcessInstanceId())) { int processInstanceId = taskInstanceKey.getProcessInstanceId(); - LogUtils.setWorkflowInstanceIdMDC(processInstanceId); long taskCode = taskInstanceKey.getTaskCode(); WorkflowExecuteRunnable workflowExecuteThread = @@ -282,8 +282,6 @@ public class StateWheelExecuteThread extends BaseDaemonThread { } } catch (Exception ex) { log.error("Check task timeout error, taskInstanceKey: {}", taskInstanceKey, ex); - } finally { - LogUtils.removeWorkflowInstanceIdMDC(); } } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java index 9689519940..a8359b6069 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java @@ -42,20 +42,18 @@ import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode; import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper; import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils; import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; -import org.apache.dolphinscheduler.remote.command.task.TaskDispatchMessage; import org.apache.dolphinscheduler.remote.command.task.TaskExecuteRunningMessageAck; import org.apache.dolphinscheduler.remote.command.task.TaskExecuteStartMessage; import org.apache.dolphinscheduler.server.master.builder.TaskExecutionContextBuilder; import org.apache.dolphinscheduler.server.master.cache.StreamTaskInstanceExecCacheManager; import org.apache.dolphinscheduler.server.master.config.MasterConfig; -import org.apache.dolphinscheduler.server.master.dispatch.ExecutorDispatcher; -import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext; -import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType; -import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException; import org.apache.dolphinscheduler.server.master.event.StateEventHandleError; import org.apache.dolphinscheduler.server.master.event.StateEventHandleException; import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics; import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; +import org.apache.dolphinscheduler.server.master.runner.dispatcher.WorkerTaskDispatcher; +import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnableFactory; import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.spi.enums.ResourceType; @@ -88,7 +86,9 @@ public class StreamTaskExecuteRunnable implements Runnable { protected TaskInstanceDao taskInstanceDao; - protected ExecutorDispatcher dispatcher; + protected DefaultTaskExecuteRunnableFactory defaultTaskExecuteRunnableFactory; + + protected WorkerTaskDispatcher workerTaskDispatcher; protected ProcessTaskRelationMapper processTaskRelationMapper; @@ -114,7 +114,7 @@ public class StreamTaskExecuteRunnable implements Runnable { public StreamTaskExecuteRunnable(TaskDefinition taskDefinition, TaskExecuteStartMessage taskExecuteStartMessage) { this.processService = SpringApplicationContext.getBean(ProcessService.class); this.masterConfig = SpringApplicationContext.getBean(MasterConfig.class); - this.dispatcher = SpringApplicationContext.getBean(ExecutorDispatcher.class); + this.workerTaskDispatcher = SpringApplicationContext.getBean(WorkerTaskDispatcher.class); this.taskPluginManager = SpringApplicationContext.getBean(TaskPluginManager.class); this.processTaskRelationMapper = SpringApplicationContext.getBean(ProcessTaskRelationMapper.class); this.taskInstanceDao = SpringApplicationContext.getBean(TaskInstanceDao.class); @@ -144,48 +144,20 @@ public class StreamTaskExecuteRunnable implements Runnable { int processDefinitionVersion = processTaskRelationList.get(0).getProcessDefinitionVersion(); processDefinition = processService.findProcessDefinition(processDefinitionCode, processDefinitionVersion); - // dispatch task - TaskExecutionContext taskExecutionContext = getTaskExecutionContext(taskInstance); - if (taskExecutionContext == null) { - taskInstance.setState(TaskExecutionStatus.FAILURE); - taskInstanceDao.upsertTaskInstance(taskInstance); - return; - } - - TaskDispatchMessage dispatchCommand = new TaskDispatchMessage(taskExecutionContext, - masterConfig.getMasterAddress(), - taskExecutionContext.getHost(), - System.currentTimeMillis()); - - ExecutionContext executionContext = new ExecutionContext(dispatchCommand.convert2Command(), ExecutorType.WORKER, - taskExecutionContext.getWorkerGroup(), taskInstance); - Boolean dispatchSuccess = false; try { - dispatcher.dispatch(executionContext); - dispatchSuccess = true; - } catch (ExecuteException e) { - log.error("Master dispatch task to worker error, taskInstanceId: {}, worker: {}", - taskInstance.getId(), - executionContext.getHost(), - e); - } - if (!dispatchSuccess) { - log.info("Master failed to dispatch task to worker, taskInstanceId: {}, worker: {}", - taskInstance.getId(), - executionContext.getHost()); - - // set task instance fail + DefaultTaskExecuteRunnable taskExecuteRunnable = + defaultTaskExecuteRunnableFactory.createTaskExecuteRunnable(taskInstance); + workerTaskDispatcher.dispatchTask(taskExecuteRunnable); + } catch (Exception e) { + log.error("Master dispatch task to worker error, taskInstanceName: {}", taskInstance.getName(), e); taskInstance.setState(TaskExecutionStatus.FAILURE); taskInstanceDao.upsertTaskInstance(taskInstance); return; } - // set started flag taskRunnableStatus = TaskRunnableStatus.STARTED; - - log.info("Master success dispatch task to worker, taskInstanceId: {}, worker: {}", - taskInstance.getId(), - executionContext.getHost()); + log.info("Master success dispatch task to worker, taskInstanceName: {}, worker: {}", taskInstance.getId(), + taskInstance.getHost()); } public boolean isStart() { @@ -344,8 +316,11 @@ public class StreamTaskExecuteRunnable implements Runnable { TaskChannel taskChannel = taskPluginManager.getTaskChannel(taskInstance.getTaskType()); ResourceParametersHelper resources = taskChannel.getResources(taskInstance.getTaskParams()); - AbstractParameters baseParam = taskPluginManager.getParameters(ParametersNode.builder() - .taskType(taskInstance.getTaskType()).taskParams(taskInstance.getTaskParams()).build()); + AbstractParameters baseParam = taskPluginManager.getParameters( + ParametersNode.builder() + .taskType(taskInstance.getTaskType()) + .taskParams(taskInstance.getTaskParams()) + .build()); Map propertyMap = paramParsingPreparation(taskInstance, baseParam); TaskExecutionContext taskExecutionContext = TaskExecutionContextBuilder.get() .buildWorkflowInstanceHost(masterConfig.getMasterAddress()) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowEventLooper.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowEventLooper.java index 2e3b5844d5..af9eafbec3 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowEventLooper.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowEventLooper.java @@ -18,7 +18,6 @@ package org.apache.dolphinscheduler.server.master.runner; 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.common.thread.ThreadUtils; import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; @@ -32,6 +31,7 @@ import org.apache.dolphinscheduler.server.master.event.WorkflowEventType; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; import javax.annotation.PostConstruct; @@ -42,7 +42,7 @@ import org.springframework.stereotype.Component; @Component @Slf4j -public class WorkflowEventLooper extends BaseDaemonThread { +public class WorkflowEventLooper extends BaseDaemonThread implements AutoCloseable { @Autowired private WorkflowEventQueue workflowEventQueue; @@ -52,6 +52,8 @@ public class WorkflowEventLooper extends BaseDaemonThread { private final Map workflowEventHandlerMap = new HashMap<>(); + private final AtomicBoolean RUNNING_FLAG = new AtomicBoolean(false); + protected WorkflowEventLooper() { super("WorkflowEventLooper"); } @@ -65,44 +67,59 @@ public class WorkflowEventLooper extends BaseDaemonThread { @Override public synchronized void start() { - log.info("WorkflowEventLooper thread starting"); + if (!RUNNING_FLAG.compareAndSet(false, true)) { + log.error("WorkflowEventLooper thread has already started, will not start again"); + return; + } + log.info("WorkflowEventLooper starting..."); super.start(); - log.info("WorkflowEventLooper thread started"); + log.info("WorkflowEventLooper started..."); } public void run() { - WorkflowEvent workflowEvent = null; - while (!ServerLifeCycleManager.isStopped()) { + WorkflowEvent workflowEvent; + while (RUNNING_FLAG.get()) { try { workflowEvent = workflowEventQueue.poolEvent(); - LogUtils.setWorkflowInstanceIdMDC(workflowEvent.getWorkflowInstanceId()); - log.info("Workflow event looper receive a workflow event: {}, will handle this", workflowEvent); - WorkflowEventHandler workflowEventHandler = - workflowEventHandlerMap.get(workflowEvent.getWorkflowEventType()); - workflowEventHandler.handleWorkflowEvent(workflowEvent); } catch (InterruptedException e) { - log.warn("WorkflowEventLooper thread is interrupted, will close this loop", e); + log.warn("WorkflowEventLooper thread is interrupted, will close this loop"); Thread.currentThread().interrupt(); break; + } + try ( + LogUtils.MDCAutoClosableContext mdcAutoClosableContext = + LogUtils.setWorkflowInstanceIdMDC(workflowEvent.getWorkflowInstanceId())) { + log.info("Begin to handle WorkflowEvent: {}", workflowEvent); + WorkflowEventHandler workflowEventHandler = + workflowEventHandlerMap.get(workflowEvent.getWorkflowEventType()); + workflowEventHandler.handleWorkflowEvent(workflowEvent); + log.info("Success handle WorkflowEvent: {}", workflowEvent); } catch (WorkflowEventHandleException workflowEventHandleException) { - log.error("Handle workflow event failed, will add this event to event queue again, event: {}", - workflowEvent, workflowEventHandleException); + log.error("Handle workflow event failed, will retry again: {}", workflowEvent, + workflowEventHandleException); workflowEventQueue.addEvent(workflowEvent); ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS); } catch (WorkflowEventHandleError workflowEventHandleError) { - log.error("Handle workflow event error, will drop this event, event: {}", + log.error("Handle workflow event error, will drop this event: {}", workflowEvent, workflowEventHandleError); } catch (Exception unknownException) { - log.error( - "Handle workflow event failed, get a unknown exception, will add this event to event queue again, event: {}", - workflowEvent, unknownException); + log.error("Handle workflow event failed, get a unknown exception, will retry again: {}", workflowEvent, + unknownException); workflowEventQueue.addEvent(workflowEvent); ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS); - } finally { - LogUtils.removeWorkflowInstanceIdMDC(); } } } + @Override + public void close() throws Exception { + if (!RUNNING_FLAG.compareAndSet(true, false)) { + log.info("WorkflowEventLooper thread is not start, no need to close"); + return; + } + log.info("WorkflowEventLooper is closing..."); + this.interrupt(); + log.info("WorkflowEventLooper closed..."); + } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java index 98d267383d..4cab7ea0b0 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java @@ -66,6 +66,7 @@ import org.apache.dolphinscheduler.plugin.task.api.enums.DependResult; 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.plugin.task.api.parameters.SwitchParameters; import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; import org.apache.dolphinscheduler.remote.command.Message; import org.apache.dolphinscheduler.remote.command.task.TaskWakeupRequest; @@ -82,11 +83,12 @@ import org.apache.dolphinscheduler.server.master.event.StateEventHandler; import org.apache.dolphinscheduler.server.master.event.StateEventHandlerManager; import org.apache.dolphinscheduler.server.master.event.TaskStateEvent; import org.apache.dolphinscheduler.server.master.event.WorkflowStateEvent; +import org.apache.dolphinscheduler.server.master.exception.TaskExecuteRunnableCreateException; import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics; import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; -import org.apache.dolphinscheduler.server.master.runner.task.ITaskProcessor; -import org.apache.dolphinscheduler.server.master.runner.task.TaskAction; -import org.apache.dolphinscheduler.server.master.runner.task.TaskProcessorFactory; +import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnableFactory; +import org.apache.dolphinscheduler.server.master.utils.TaskUtils; import org.apache.dolphinscheduler.service.alert.ProcessAlertManager; import org.apache.dolphinscheduler.service.command.CommandService; import org.apache.dolphinscheduler.service.cron.CronUtils; @@ -103,7 +105,6 @@ import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.math.NumberUtils; import org.apache.commons.lang3.tuple.Pair; -import java.lang.reflect.InvocationTargetException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -118,6 +119,7 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicBoolean; @@ -181,10 +183,9 @@ public class WorkflowExecuteRunnable implements Callable { private final Map taskInstanceMap = new ConcurrentHashMap<>(); /** - * running taskProcessor, taskCode as key, taskProcessor as value - * only on taskProcessor per taskCode + * TaskCode as Key, TaskExecuteRunnable as Value */ - private final Map activeTaskProcessorMaps = new ConcurrentHashMap<>(); + private final Map taskExecuteRunnableMap = new ConcurrentHashMap<>(); /** * valid task map, taskCode as key, taskId as value @@ -246,6 +247,8 @@ public class WorkflowExecuteRunnable implements Callable { private final String masterAddress; + private final DefaultTaskExecuteRunnableFactory defaultTaskExecuteRunnableFactory; + /** * @param processInstance processInstance * @param processService processService @@ -266,7 +269,8 @@ public class WorkflowExecuteRunnable implements Callable { @NonNull StateWheelExecuteThread stateWheelExecuteThread, @NonNull CuringParamsService curingParamsService, @NonNull TaskInstanceDao taskInstanceDao, - @NonNull TaskDefinitionLogDao taskDefinitionLogDao) { + @NonNull TaskDefinitionLogDao taskDefinitionLogDao, + @NonNull DefaultTaskExecuteRunnableFactory defaultTaskExecuteRunnableFactory) { this.processService = processService; this.commandService = commandService; this.processInstanceDao = processInstanceDao; @@ -278,6 +282,10 @@ public class WorkflowExecuteRunnable implements Callable { this.taskInstanceDao = taskInstanceDao; this.taskDefinitionLogDao = taskDefinitionLogDao; this.masterAddress = NetUtils.getAddr(masterConfig.getListenPort()); + this.defaultTaskExecuteRunnableFactory = defaultTaskExecuteRunnableFactory; + this.processDefinition = processService.findProcessDefinition(processInstance.getProcessDefinitionCode(), + processInstance.getProcessDefinitionVersion()); + this.processInstance.setProcessDefinition(processDefinition); TaskMetrics.registerTaskPrepared(readyToSubmitTaskQueue::size); } @@ -377,24 +385,30 @@ public class WorkflowExecuteRunnable implements Callable { } public boolean checkForceStartAndWakeUp(StateEvent stateEvent) { - TaskGroupQueue taskGroupQueue = this.processService.loadTaskGroupQueue(stateEvent.getTaskInstanceId()); + TaskGroupQueue taskGroupQueue = processService.loadTaskGroupQueue(stateEvent.getTaskInstanceId()); if (taskGroupQueue.getForceStart() == Flag.YES.getCode()) { log.info("Begin to force start taskGroupQueue: {}", taskGroupQueue.getId()); - TaskInstance taskInstance = this.taskInstanceDao.findTaskInstanceById(stateEvent.getTaskInstanceId()); - ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskInstance.getTaskCode()); - taskProcessor.action(TaskAction.DISPATCH); - this.processService.updateTaskGroupQueueStatus(taskGroupQueue.getTaskId(), - TaskGroupQueueStatus.ACQUIRE_SUCCESS.getCode()); - log.info("Success force start taskGroupQueue: {}", taskGroupQueue.getId()); + TaskInstance taskInstance = taskInstanceDao.findTaskInstanceById(stateEvent.getTaskInstanceId()); + + DefaultTaskExecuteRunnable defaultTaskExecuteRunnable = + taskExecuteRunnableMap.get(taskInstance.getTaskCode()); + if (defaultTaskExecuteRunnable != null) { + defaultTaskExecuteRunnable.dispatch(); + this.processService.updateTaskGroupQueueStatus(taskGroupQueue.getTaskId(), + TaskGroupQueueStatus.ACQUIRE_SUCCESS.getCode()); + log.info("Success force start task: {}, taskGroup: {}", taskGroupQueue.getTaskName(), + taskGroupQueue.getGroupId()); + } else { + log.warn("Cannot find the TaskExecuteRunnable: {}", taskGroupQueue.getTaskName()); + } return true; } if (taskGroupQueue.getInQueue() == Flag.YES.getCode()) { log.info("Begin to wake up taskGroupQueue: {}", taskGroupQueue.getId()); boolean acquireTaskGroup = processService.robTaskGroupResource(taskGroupQueue); if (acquireTaskGroup) { - TaskInstance taskInstance = this.taskInstanceDao.findTaskInstanceById(stateEvent.getTaskInstanceId()); - ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskInstance.getTaskCode()); - taskProcessor.action(TaskAction.DISPATCH); + TaskInstance taskInstance = taskInstanceDao.findTaskInstanceById(stateEvent.getTaskInstanceId()); + taskExecuteRunnableMap.get(taskInstance.getTaskCode()).dispatch(); log.info("Success wake up taskGroupQueue: {}", taskGroupQueue.getId()); return true; } @@ -422,7 +436,8 @@ public class WorkflowExecuteRunnable implements Callable { log.info("TaskInstance finished task code:{} state:{}", taskInstance.getTaskCode(), taskInstance.getState()); try { - activeTaskProcessorMaps.remove(taskInstance.getTaskCode()); + // Do we need to remove? + taskExecuteRunnableMap.remove(taskInstance.getTaskCode()); stateWheelExecuteThread.removeTask4TimeoutCheck(processInstance, taskInstance); stateWheelExecuteThread.removeTask4RetryCheck(processInstance, taskInstance); stateWheelExecuteThread.removeTask4StateCheck(processInstance, taskInstance); @@ -721,14 +736,15 @@ public class WorkflowExecuteRunnable implements Callable { */ @Override public WorkflowSubmitStatus call() { - if (isStart()) { - // This case should not been happened - log.warn("[WorkflowInstance-{}] The workflow has already been started", processInstance.getId()); - return WorkflowSubmitStatus.DUPLICATED_SUBMITTED; - } - try { - LogUtils.setWorkflowInstanceIdMDC(processInstance.getId()); + try ( + LogUtils.MDCAutoClosableContext mdcAutoClosableContext = + LogUtils.setWorkflowInstanceIdMDC(processInstance.getId())) { + if (isStart()) { + // This case should not been happened + log.warn("The workflow has already been started, current state: {}", workflowRunnableStatus); + return WorkflowSubmitStatus.DUPLICATED_SUBMITTED; + } if (workflowRunnableStatus == WorkflowRunnableStatus.CREATED) { buildFlowDag(); workflowRunnableStatus = WorkflowRunnableStatus.INITIALIZE_DAG; @@ -748,8 +764,6 @@ public class WorkflowExecuteRunnable implements Callable { } catch (Exception e) { log.error("Start workflow error", e); return WorkflowSubmitStatus.FAILED; - } finally { - LogUtils.removeWorkflowInstanceIdMDC(); } } @@ -805,15 +819,8 @@ public class WorkflowExecuteRunnable implements Callable { commandService.createCommand(command); } - /** - * Generate process dag - * - * @throws Exception exception - */ + // todo: move the initialize code to constructor private void buildFlowDag() throws Exception { - processDefinition = processService.findProcessDefinition(processInstance.getProcessDefinitionCode(), - processInstance.getProcessDefinitionVersion()); - processInstance.setProcessDefinition(processDefinition); List recoverNodeList = getRecoverTaskInstanceList(processInstance.getCommandParam()); @@ -852,7 +859,8 @@ public class WorkflowExecuteRunnable implements Callable { private void initTaskQueue() throws StateEventHandleException, CronParseException { taskFailedSubmit = false; - activeTaskProcessorMaps.clear(); + // do we need to clear? + taskExecuteRunnableMap.clear(); dependFailedTaskSet.clear(); completeTaskMap.clear(); errorTaskMap.clear(); @@ -967,100 +975,68 @@ public class WorkflowExecuteRunnable implements Callable { errorTaskMap); } - /** - * submit task to execute - * - * @param taskInstance task instance - * @return TaskInstance - */ - private Optional submitTaskExec(TaskInstance taskInstance) { + private boolean executeTask(TaskInstance taskInstance) { try { - // package task instance before submit + // package task instance before submit, inject the process instance to task instance + // todo: we need to use task execute context rather than packege a lot of pojo into task instance + // 1. submit the task instance to db processService.packageTaskInstance(taskInstance, processInstance); - - ITaskProcessor taskProcessor = TaskProcessorFactory.getTaskProcessor(taskInstance.getTaskType()); - taskProcessor.init(taskInstance, processInstance); - - boolean submit = taskProcessor.action(TaskAction.SUBMIT); - if (!submit) { - log.error("Submit standby task failed!, taskCode: {}, taskName: {}", - taskInstance.getTaskCode(), - taskInstance.getName()); - return Optional.empty(); + // todo: remove this method + if (!processService.submitTask(processInstance, taskInstance)) { + log.error("Submit standby task: {} failed", taskInstance.getName()); + return true; } - + // 2. create task execute runnable // in a dag, only one taskInstance is valid per taskCode, so need to set the old taskInstance invalid - LogUtils.setWorkflowAndTaskInstanceIDMDC(taskInstance.getProcessInstanceId(), taskInstance.getId()); - if (validTaskMap.containsKey(taskInstance.getTaskCode())) { - int oldTaskInstanceId = validTaskMap.get(taskInstance.getTaskCode()); - if (taskInstance.getId() != oldTaskInstanceId) { - TaskInstance oldTaskInstance = taskInstanceMap.get(oldTaskInstanceId); - oldTaskInstance.setFlag(Flag.NO); - taskInstanceDao.updateTaskInstance(oldTaskInstance); - validTaskMap.remove(taskInstance.getTaskCode()); - activeTaskProcessorMaps.remove(taskInstance.getTaskCode()); + try ( + LogUtils.MDCAutoClosableContext mdcAutoClosableContext = + LogUtils.setTaskInstanceIdMDC(taskInstance.getId())) { + DefaultTaskExecuteRunnable taskExecuteRunnable = + defaultTaskExecuteRunnableFactory.createTaskExecuteRunnable(taskInstance); + if (validTaskMap.containsKey(taskInstance.getTaskCode())) { + int oldTaskInstanceId = validTaskMap.get(taskInstance.getTaskCode()); + if (taskInstance.getId() != oldTaskInstanceId) { + TaskInstance oldTaskInstance = taskInstanceMap.get(oldTaskInstanceId); + oldTaskInstance.setFlag(Flag.NO); + taskInstanceDao.updateTaskInstance(oldTaskInstance); + validTaskMap.remove(taskInstance.getTaskCode()); + taskExecuteRunnableMap.remove(taskInstance.getTaskCode()); + } } - } - validTaskMap.put(taskInstance.getTaskCode(), taskInstance.getId()); - taskInstanceMap.put(taskInstance.getId(), taskInstance); - activeTaskProcessorMaps.put(taskInstance.getTaskCode(), taskProcessor); - - // if we use task group, then need to acquire the task group resource - // if there is no resource the current task instance will not be dispatched - // it will be wakeup when other tasks release the resource. - int taskGroupId = taskInstance.getTaskGroupId(); - if (taskGroupId > 0) { - boolean acquireTaskGroup = processService.acquireTaskGroup(taskInstance.getId(), - taskInstance.getName(), - taskGroupId, - taskInstance.getProcessInstanceId(), - taskInstance.getTaskGroupPriority()); - if (!acquireTaskGroup) { - log.info( - "Submitted task will not be dispatch right now because the first time to try to acquire" + - " task group failed, taskInstanceName: {}, taskGroupId: {}", - taskInstance.getName(), taskGroupId); - return Optional.of(taskInstance); + validTaskMap.put(taskInstance.getTaskCode(), taskInstance.getId()); + taskInstanceMap.put(taskInstance.getId(), taskInstance); + taskExecuteRunnableMap.put(taskInstance.getTaskCode(), taskExecuteRunnable); + // 3. acquire the task group. + // if we use task group, then need to acquire the task group resource + // if there is no resource the current task instance will not be dispatched + // it will be wakeup when other tasks release the resource. + int taskGroupId = taskInstance.getTaskGroupId(); + if (taskGroupId > 0) { + boolean acquireTaskGroup = processService.acquireTaskGroup(taskInstance.getId(), + taskInstance.getName(), + taskGroupId, + taskInstance.getProcessInstanceId(), + taskInstance.getTaskGroupPriority()); + if (!acquireTaskGroup) { + log.info( + "Submitted task will not be dispatch right now because the first time to try to acquire" + + + " task group failed, taskInstanceName: {}, taskGroupId: {}", + taskInstance.getName(), taskGroupId); + return true; + } } - } - - boolean dispatchSuccess = taskProcessor.action(TaskAction.DISPATCH); - if (!dispatchSuccess) { - log.error("Dispatch standby process {} task {} failed", processInstance.getName(), - taskInstance.getName()); - return Optional.empty(); - } - taskProcessor.action(TaskAction.RUN); + // 4. submit to dispatch queue + taskExecuteRunnable.dispatch(); - stateWheelExecuteThread.addTask4TimeoutCheck(processInstance, taskInstance); - stateWheelExecuteThread.addTask4StateCheck(processInstance, taskInstance); - - if (taskProcessor.taskInstance().getState().isFinished()) { - if (processInstance.isBlocked()) { - TaskStateEvent processBlockEvent = TaskStateEvent.builder() - .processInstanceId(processInstance.getId()) - .taskInstanceId(taskInstance.getId()) - .status(taskProcessor.taskInstance().getState()) - .type(StateEventType.PROCESS_BLOCKED) - .build(); - this.stateEvents.add(processBlockEvent); - } - TaskStateEvent taskStateChangeEvent = TaskStateEvent.builder() - .processInstanceId(processInstance.getId()) - .taskInstanceId(taskInstance.getId()) - .status(taskProcessor.taskInstance().getState()) - .type(StateEventType.TASK_STATE_CHANGE) - .build(); - this.stateEvents.add(taskStateChangeEvent); + stateWheelExecuteThread.addTask4TimeoutCheck(processInstance, taskInstance); + stateWheelExecuteThread.addTask4StateCheck(processInstance, taskInstance); + return true; } - return Optional.of(taskInstance); } catch (Exception e) { - log.error("Submit standby task {} error, taskCode: {}", taskInstance.getName(), - taskInstance.getTaskCode(), e); - return Optional.empty(); - } finally { - LogUtils.removeWorkflowAndTaskInstanceIdMDC(); + log.error("Submit standby task {} error", taskInstance.getName(), e); + return false; } } @@ -1372,10 +1348,9 @@ public class WorkflowExecuteRunnable implements Callable { } } taskInstances.add(existTaskInstance); - continue; + } else { + taskInstances.add(createTaskInstance(processInstance, taskNodeObject)); } - TaskInstance task = createTaskInstance(processInstance, taskNodeObject); - taskInstances.add(task); } // the end node of the branch of the dag if (StringUtils.isNotEmpty(parentNodeCode) && dag.getEndNode().contains(parentNodeCode)) { @@ -1398,7 +1373,7 @@ public class WorkflowExecuteRunnable implements Callable { for (TaskInstance task : taskInstances) { if (readyToSubmitTaskQueue.contains(task)) { - log.warn("Task is already at submit queue, taskInstanceId: {}", task.getId()); + log.warn("Task is already at submit queue, taskInstanceName: {}", task.getName()); continue; } @@ -1418,7 +1393,7 @@ public class WorkflowExecuteRunnable implements Callable { } private boolean tryToTakeOverTaskInstance(TaskInstance taskInstance) { - if (TaskProcessorFactory.isMasterTask(taskInstance.getTaskType())) { + if (TaskUtils.isMasterTask(taskInstance.getTaskType())) { return false; } try { @@ -1440,16 +1415,15 @@ public class WorkflowExecuteRunnable implements Callable { return false; } - ITaskProcessor taskProcessor = TaskProcessorFactory.getTaskProcessor(taskInstance.getTaskType()); - taskProcessor.init(taskInstance, processInstance); + // todo: create the takeover task execute runnable. + taskExecuteRunnableMap.put(taskInstance.getTaskCode(), + defaultTaskExecuteRunnableFactory.createTaskExecuteRunnable(taskInstance)); taskInstanceMap.put(taskInstance.getId(), taskInstance); stateWheelExecuteThread.addTask4TimeoutCheck(processInstance, taskInstance); stateWheelExecuteThread.addTask4RetryCheck(processInstance, taskInstance); - activeTaskProcessorMaps.put(taskInstance.getTaskCode(), taskProcessor); return true; - } catch (RemotingException | InterruptedException | InstantiationException | IllegalAccessException - | InvocationTargetException e) { + } catch (RemotingException | InterruptedException | TaskExecuteRunnableCreateException e) { log.error( "Takeover task instance failed, the worker {} might not be alive, will try to create a new task instance", taskInstance.getHost(), e); @@ -1529,25 +1503,31 @@ public class WorkflowExecuteRunnable implements Callable { /** * depend node is completed, but here need check the condition task branch is the next node */ - private boolean dependTaskSuccess(String dependNodeName, String nextNodeName) { - if (dag.getNode(dependNodeName).isConditionsTask()) { + private boolean dependTaskSuccess(String dependNodeCode, String nextNodeCode) { + TaskNode dependentNode = dag.getNode(dependNodeCode); + if (dependentNode.isConditionsTask()) { // condition task need check the branch to run List nextTaskList = - DagHelper.parseConditionTask(dependNodeName, skipTaskNodeMap, dag, getCompleteTaskInstanceMap()); - if (!nextTaskList.contains(nextNodeName)) { + DagHelper.parseConditionTask(dependNodeCode, skipTaskNodeMap, dag, getCompleteTaskInstanceMap()); + if (!nextTaskList.contains(nextNodeCode)) { log.info( "DependTask is a condition task, and its next condition branch does not hava current task, " + "dependTaskCode: {}, currentTaskCode: {}", - dependNodeName, nextNodeName); + dependNodeCode, nextNodeCode); return false; } - } else { - long taskCode = Long.parseLong(dependNodeName); - Integer taskInstanceId = completeTaskMap.get(taskCode); - TaskExecutionStatus depTaskState = taskInstanceMap.get(taskInstanceId).getState(); - return !depTaskState.isFailure(); + return true; } - return true; + if (dependentNode.isSwitchTask()) { + TaskInstance dependentTaskInstance = taskInstanceMap.get(validTaskMap.get(dependentNode.getCode())); + SwitchParameters switchParameters = dependentTaskInstance.getSwitchDependency(); + return switchParameters.getDependTaskList().get(switchParameters.getResultConditionLocation()).getNextNode() + .contains(nextNodeCode); + } + long taskCode = Long.parseLong(dependNodeCode); + Integer taskInstanceId = completeTaskMap.get(taskCode); + TaskExecutionStatus depTaskState = taskInstanceMap.get(taskInstanceId).getState(); + return !depTaskState.isFailure(); } /** @@ -1612,7 +1592,7 @@ public class WorkflowExecuteRunnable implements Callable { return true; } if (processInstance.getFailureStrategy() == FailureStrategy.CONTINUE) { - return readyToSubmitTaskQueue.size() == 0 && activeTaskProcessorMaps.size() == 0 + return readyToSubmitTaskQueue.size() == 0 && taskExecuteRunnableMap.size() == 0 && waitToRetryTaskInstanceMap.size() == 0; } } @@ -1650,10 +1630,10 @@ public class WorkflowExecuteRunnable implements Callable { * @return ExecutionStatus */ private WorkflowExecutionStatus processReadyBlock() { - if (activeTaskProcessorMaps.size() > 0) { - for (ITaskProcessor taskProcessor : activeTaskProcessorMaps.values()) { - if (!TASK_TYPE_BLOCKING.equals(taskProcessor.getType())) { - taskProcessor.action(TaskAction.PAUSE); + if (taskExecuteRunnableMap.size() > 0) { + for (DefaultTaskExecuteRunnable taskExecuteRunnable : taskExecuteRunnableMap.values()) { + if (!TASK_TYPE_BLOCKING.equals(taskExecuteRunnable.getTaskInstance().getTaskType())) { + taskExecuteRunnable.pause(); } } } @@ -1673,7 +1653,7 @@ public class WorkflowExecuteRunnable implements Callable { private WorkflowExecutionStatus getProcessInstanceState(ProcessInstance instance) { WorkflowExecutionStatus state = instance.getState(); - if (activeTaskProcessorMaps.size() > 0 || hasRetryTaskInStandBy()) { + if (taskExecuteRunnableMap.size() > 0 || hasRetryTaskInStandBy()) { // active task and retry task exists WorkflowExecutionStatus executionStatus = runningState(state); log.info("The workflowInstance has task running, the workflowInstance status is {}", executionStatus); @@ -1861,14 +1841,13 @@ public class WorkflowExecuteRunnable implements Callable { public void killAllTasks() { log.info("kill called on process instance id: {}, num: {}", processInstance.getId(), - activeTaskProcessorMaps.size()); + taskExecuteRunnableMap.size()); if (readyToSubmitTaskQueue.size() > 0) { readyToSubmitTaskQueue.clear(); } - for (long taskCode : activeTaskProcessorMaps.keySet()) { - ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskCode); + for (long taskCode : taskExecuteRunnableMap.keySet()) { Integer taskInstanceId = validTaskMap.get(taskCode); if (taskInstanceId == null || taskInstanceId.equals(0)) { continue; @@ -1880,16 +1859,19 @@ public class WorkflowExecuteRunnable implements Callable { if (taskInstance == null || taskInstance.getState().isFinished()) { continue; } - taskProcessor.action(TaskAction.STOP); - if (taskProcessor.taskInstance().getState().isFinished()) { - TaskStateEvent taskStateEvent = TaskStateEvent.builder() - .processInstanceId(processInstance.getId()) - .taskInstanceId(taskInstance.getId()) - .status(taskProcessor.taskInstance().getState()) - .type(StateEventType.TASK_STATE_CHANGE) - .build(); - this.addStateEvent(taskStateEvent); - } + DefaultTaskExecuteRunnable defaultTaskExecuteRunnable = taskExecuteRunnableMap.get(taskCode); + CompletableFuture.runAsync(defaultTaskExecuteRunnable::kill) + .thenRun(() -> { + if (defaultTaskExecuteRunnable.getTaskInstance().getState().isFinished()) { + TaskStateEvent taskStateEvent = TaskStateEvent.builder() + .processInstanceId(processInstance.getId()) + .taskInstanceId(taskInstance.getId()) + .status(defaultTaskExecuteRunnable.getTaskInstance().getState()) + .type(StateEventType.TASK_STATE_CHANGE) + .build(); + this.addStateEvent(taskStateEvent); + } + }); } } } @@ -1902,12 +1884,8 @@ public class WorkflowExecuteRunnable implements Callable { * handling the list of tasks to be submitted */ public void submitStandByTask() throws StateEventHandleException { - int length = readyToSubmitTaskQueue.size(); - for (int i = 0; i < length; i++) { - TaskInstance task = readyToSubmitTaskQueue.peek(); - if (task == null) { - continue; - } + TaskInstance task; + while ((task = readyToSubmitTaskQueue.peek()) != null) { // stop tasks which is retrying if forced success happens if (task.taskCanRetry()) { TaskInstance retryTask = taskInstanceDao.findTaskInstanceById(task.getId()); @@ -1934,8 +1912,8 @@ public class WorkflowExecuteRunnable implements Callable { DependResult dependResult = getDependResultForTask(task); if (DependResult.SUCCESS == dependResult) { log.info("The dependResult of task {} is success, so ready to submit to execute", task.getName()); - Optional taskInstanceOptional = submitTaskExec(task); - if (!taskInstanceOptional.isPresent()) { + if (!executeTask(task)) { + // todo: don't ste 0 here task.setId(0); this.taskFailedSubmit = true; // Remove and add to complete map and error map @@ -1948,7 +1926,9 @@ public class WorkflowExecuteRunnable implements Callable { completeTaskMap.put(task.getTaskCode(), task.getId()); taskInstanceMap.put(task.getId(), task); errorTaskMap.put(task.getTaskCode(), task.getId()); - activeTaskProcessorMaps.remove(task.getTaskCode()); + + taskExecuteRunnableMap.remove(task.getTaskCode()); + log.error("Task submitted failed, workflowInstanceId: {}, taskInstanceId: {}, taskCode: {}", task.getProcessInstanceId(), task.getId(), @@ -1960,13 +1940,13 @@ public class WorkflowExecuteRunnable implements Callable { // if the dependency fails, the current node is not submitted and the state changes to failure. dependFailedTaskSet.add(task.getTaskCode()); removeTaskFromStandbyList(task); - log.info("Task dependent result is failed, taskInstanceId:{} depend result : {}", task.getId(), + log.info("Task dependent result is failed, taskInstanceName: {} depend result : {}", task.getName(), dependResult); } else if (DependResult.NON_EXEC == dependResult) { // for some reasons(depend task pause/stop) this task would not be submit removeTaskFromStandbyList(task); - log.info("Remove task due to depend result not executed, taskInstanceId:{} depend result : {}", - task.getId(), dependResult); + log.info("Remove task due to depend result not executed, taskInstanceName:{} depend result : {}", + task.getName(), dependResult); } } } @@ -2078,22 +2058,12 @@ public class WorkflowExecuteRunnable implements Callable { return false; } - public void resubmit(long taskCode) throws Exception { - ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskCode); - if (taskProcessor != null) { - taskProcessor.action(TaskAction.RESUBMIT); - log.debug("RESUBMIT: task code:{}", taskCode); - } else { - throw new Exception("resubmit error, taskProcessor is null, task code: " + taskCode); - } - } - public Map getCompleteTaskMap() { return completeTaskMap; } - public Map getActiveTaskProcessMap() { - return activeTaskProcessorMaps; + public Map getTaskExecuteRunnableMap() { + return taskExecuteRunnableMap; } public Map getWaitToRetryTaskInstanceMap() { diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableFactory.java new file mode 100644 index 0000000000..00510756b7 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableFactory.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.common.enums.SlotCheckState; +import org.apache.dolphinscheduler.dao.entity.Command; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; +import org.apache.dolphinscheduler.dao.repository.TaskDefinitionLogDao; +import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; +import org.apache.dolphinscheduler.server.master.exception.WorkflowCreateException; +import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics; +import org.apache.dolphinscheduler.server.master.registry.ServerNodeManager; +import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; +import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnableFactory; +import org.apache.dolphinscheduler.service.alert.ProcessAlertManager; +import org.apache.dolphinscheduler.service.command.CommandService; +import org.apache.dolphinscheduler.service.expand.CuringParamsService; +import org.apache.dolphinscheduler.service.process.ProcessService; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class WorkflowExecuteRunnableFactory { + + @Autowired + private ServerNodeManager serverNodeManager; + + @Autowired + private CommandService commandService; + + @Autowired + private ProcessService processService; + + @Autowired + private ProcessInstanceDao processInstanceDao; + + @Autowired + private MasterRpcClient masterRpcClient; + + @Autowired + private ProcessAlertManager processAlertManager; + + @Autowired + private StateWheelExecuteThread stateWheelExecuteThread; + + @Autowired + private CuringParamsService curingGlobalParamsService; + + @Autowired + private TaskInstanceDao taskInstanceDao; + + @Autowired + private MasterConfig masterConfig; + + @Autowired + private TaskDefinitionLogDao taskDefinitionLogDao; + + @Autowired + private DefaultTaskExecuteRunnableFactory defaultTaskExecuteRunnableFactory; + + public WorkflowExecuteRunnable createWorkflowExecuteRunnable(Command command) throws WorkflowCreateException { + try { + ProcessInstance workflowInstance = createWorkflowInstance(command); + return new WorkflowExecuteRunnable(workflowInstance, + commandService, + processService, + processInstanceDao, + masterRpcClient, + processAlertManager, + masterConfig, + stateWheelExecuteThread, + curingGlobalParamsService, + taskInstanceDao, + taskDefinitionLogDao, + defaultTaskExecuteRunnableFactory); + } catch (Exception ex) { + throw new WorkflowCreateException("Create workflow execute runnable failed", ex); + } + } + + private ProcessInstance createWorkflowInstance(Command command) throws Exception { + long commandTransformStartTime = System.currentTimeMillis(); + // Note: this check is not safe, the slot may change after command transform. + // We use the database transaction in `handleCommand` so that we can guarantee the command will + // always be executed + // by only one master + SlotCheckState slotCheckState = slotCheck(command); + if (slotCheckState.equals(SlotCheckState.CHANGE) || slotCheckState.equals(SlotCheckState.INJECT)) { + log.info("Master handle command {} skip, slot check state: {}", command.getId(), slotCheckState); + throw new RuntimeException("Slot check failed the current state: " + slotCheckState); + } + ProcessInstance processInstance = processService.handleCommand(masterConfig.getMasterAddress(), command); + log.info("Master handle command {} end, create process instance {}", command.getId(), processInstance.getId()); + ProcessInstanceMetrics + .recordProcessInstanceGenerateTime(System.currentTimeMillis() - commandTransformStartTime); + return processInstance; + } + + private SlotCheckState slotCheck(Command command) { + int slot = serverNodeManager.getSlot(); + int masterSize = serverNodeManager.getMasterSize(); + SlotCheckState state; + if (masterSize <= 0) { + state = SlotCheckState.CHANGE; + } else if (command.getId() % masterSize == slot) { + state = SlotCheckState.PASS; + } else { + state = SlotCheckState.INJECT; + } + return state; + } + +} 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 new file mode 100644 index 0000000000..f4fb3a908d --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcher.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.runner.dispatcher; + +import org.apache.dolphinscheduler.remote.utils.Host; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; +import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; +import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; +import org.apache.dolphinscheduler.server.master.runner.BaseTaskDispatcher; +import org.apache.dolphinscheduler.server.master.runner.execute.TaskExecuteRunnable; + +import java.util.Optional; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class MasterTaskDispatcher extends BaseTaskDispatcher { + + private final Optional masterTaskExecuteHost; + + public MasterTaskDispatcher(TaskEventService taskEventService, + MasterConfig masterConfig, + MasterRpcClient masterRpcClient) { + super(taskEventService, masterConfig, masterRpcClient); + masterTaskExecuteHost = Optional.of(Host.of(masterConfig.getMasterAddress())); + } + + @Override + protected Optional getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecutionContext) { + return masterTaskExecuteHost; + } +} 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/runner/dispatcher/TaskDispatchFactory.java new file mode 100644 index 0000000000..1979b48de5 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatchFactory.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.server.master.runner.dispatcher; + +import org.apache.dolphinscheduler.server.master.utils.TaskUtils; + +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) { + return TaskUtils.isMasterTask(taskType) ? masterTaskDispatcher : workerTaskDispatcher; + } + +} 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 new file mode 100644 index 0000000000..32a195fb5a --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatcher.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.runner.dispatcher; + +import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; +import org.apache.dolphinscheduler.server.master.exception.TaskDispatchException; +import org.apache.dolphinscheduler.server.master.runner.execute.TaskExecuteRunnable; + +/** + * Used to do task dispatcher. + */ +public interface TaskDispatcher { + + void dispatchTask(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException, WorkerGroupNotFoundException; + +} 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 new file mode 100644 index 0000000000..b2c256cd26 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.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.server.master.runner.dispatcher; + +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.dolphinscheduler.remote.utils.Host; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; +import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; +import org.apache.dolphinscheduler.server.master.dispatch.host.HostManager; +import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; +import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; +import org.apache.dolphinscheduler.server.master.runner.BaseTaskDispatcher; +import org.apache.dolphinscheduler.server.master.runner.execute.TaskExecuteRunnable; + +import java.util.Optional; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class WorkerTaskDispatcher extends BaseTaskDispatcher { + + private final HostManager hostManager; + + public WorkerTaskDispatcher(TaskEventService taskEventService, + MasterConfig masterConfig, + MasterRpcClient masterRpcClient, + HostManager hostManager) { + super(taskEventService, masterConfig, masterRpcClient); + this.hostManager = checkNotNull(hostManager); + } + + @Override + protected Optional getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecuteRunnable) throws WorkerGroupNotFoundException { + String workerGroup = taskExecuteRunnable.getTaskExecutionContext().getWorkerGroup(); + return hostManager.select(workerGroup); + + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterDelayTaskExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterDelayTaskExecuteRunnable.java new file mode 100644 index 0000000000..5b41481714 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterDelayTaskExecuteRunnable.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.MasterMessageSenderManager; +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 AsyncMasterDelayTaskExecuteRunnable extends MasterDelayTaskExecuteRunnable { + + private final AsyncMasterTaskDelayQueue asyncMasterTaskDelayQueue; + + public AsyncMasterDelayTaskExecuteRunnable(TaskExecutionContext taskExecutionContext, + LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder, + MasterMessageSenderManager masterMessageSenderManager, + AsyncMasterTaskDelayQueue asyncTaskDelayQueue) { + super(taskExecutionContext, logicTaskPluginFactoryBuilder, masterMessageSenderManager); + 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/AsyncMasterDelayTaskExecuteRunnableFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterDelayTaskExecuteRunnableFactory.java new file mode 100644 index 0000000000..169d57a2e4 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterDelayTaskExecuteRunnableFactory.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.runner.execute; + +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager; +import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Component +public class AsyncMasterDelayTaskExecuteRunnableFactory + implements + MasterDelayTaskExecuteRunnableFactory { + + @Autowired + private LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder; + + @Autowired + private MasterMessageSenderManager masterMessageSenderManager; + + @Autowired + private AsyncMasterTaskDelayQueue asyncTaskDelayQueue; + + @Override + public AsyncMasterDelayTaskExecuteRunnable createWorkerTaskExecuteRunnable(TaskExecutionContext taskExecutionContext) { + return new AsyncMasterDelayTaskExecuteRunnable(taskExecutionContext, + logicTaskPluginFactoryBuilder, + masterMessageSenderManager, + asyncTaskDelayQueue); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueue.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueue.java new file mode 100644 index 0000000000..51bda7edbe --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueue.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.concurrent.DelayQueue; + +import javax.annotation.Nullable; + +import lombok.NonNull; + +import org.springframework.stereotype.Component; + +@Component +public class AsyncMasterTaskDelayQueue { + + private final DelayQueue asyncTaskCheckDelayQueue = new DelayQueue<>(); + + public void addAsyncTask(@NonNull AsyncTaskExecutionContext asyncTaskExecutionContext) { + asyncTaskExecutionContext.refreshStartTime(); + asyncTaskCheckDelayQueue.add(asyncTaskExecutionContext); + } + + public @Nullable AsyncTaskExecutionContext pollAsyncTask() throws InterruptedException { + return asyncTaskCheckDelayQueue.take(); + } + + public int getAsyncTaskRunningNum() { + return asyncTaskCheckDelayQueue.size(); + } + +} 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 new file mode 100644 index 0000000000..d71a116f91 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueueLooper.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.common.thread.ThreadUtils; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; + +import java.util.concurrent.ExecutorService; +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 MasterConfig masterConfig; + + private static final AtomicBoolean RUNNING_FLAG = new AtomicBoolean(false); + + private ExecutorService asyncTaskStateCheckThreadPool; + + 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() { + asyncTaskStateCheckThreadPool = ThreadUtils.newDaemonFixedThreadExecutor("AsyncTaskStateCheckThreadPool", + masterConfig.getMasterAsyncTaskStateCheckThreadPoolSize()); + 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.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setWorkflowAndTaskInstanceIDMDC( + taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId()); + LogUtils.MDCAutoClosableContext mdcAutoClosableContext1 = + 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; + } + asyncTaskStateCheckThreadPool.submit(() -> { + final AsyncTaskExecuteFunction asyncTaskExecuteFunction = + asyncTaskExecutionContext.getAsyncTaskExecuteFunction(); + final AsyncTaskCallbackFunction asyncTaskCallbackFunction = + asyncTaskExecutionContext.getAsyncTaskCallbackFunction(); + try ( + LogUtils.MDCAutoClosableContext mdcAutoClosableContext2 = + LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath())) { + 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 SUCCESS: + asyncTaskCallbackFunction.executeSuccess(); + break; + case FAILED: + asyncTaskCallbackFunction.executeFailed(); + break; + } + } catch (Exception ex) { + asyncTaskCallbackFunction.executeThrowing(ex); + } + }); + } + } + 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; + } + log.info("AsyncMasterTaskDelayQueueLooper closing..."); + asyncTaskStateCheckThreadPool.shutdown(); + log.info("AsyncMasterTaskDelayQueueLooper closed..."); + } +} 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 new file mode 100644 index 0000000000..2cc23c1248 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunction.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.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); + +} 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 new file mode 100644 index 0000000000..cb5d07fc20 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunctionImpl.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.master.runner.execute; + +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContextCacheManager; +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 AsyncMasterDelayTaskExecuteRunnable asyncMasterDelayTaskExecuteRunnable; + + public AsyncTaskCallbackFunctionImpl(@NonNull AsyncMasterDelayTaskExecuteRunnable asyncMasterDelayTaskExecuteRunnable) { + this.asyncMasterDelayTaskExecuteRunnable = asyncMasterDelayTaskExecuteRunnable; + } + + @Override + public void executeSuccess() { + asyncMasterDelayTaskExecuteRunnable.getTaskExecutionContext() + .setCurrentExecutionStatus(TaskExecutionStatus.SUCCESS); + executeFinished(); + } + + @Override + public void executeFailed() { + asyncMasterDelayTaskExecuteRunnable.getTaskExecutionContext() + .setCurrentExecutionStatus(TaskExecutionStatus.FAILURE); + executeFinished(); + } + + @Override + public void executeThrowing(Throwable throwable) { + asyncMasterDelayTaskExecuteRunnable.afterThrowing(throwable); + } + + private void executeFinished() { + TaskInstanceLogHeader.printFinalizeTaskHeader(); + TaskExecutionContextCacheManager.removeByTaskInstanceId( + asyncMasterDelayTaskExecuteRunnable.getTaskExecutionContext().getTaskInstanceId()); + log.info("Task execute finished, removed the TaskExecutionContext"); + asyncMasterDelayTaskExecuteRunnable.sendTaskResult(); + log.info( + "Execute task finished, will send the task execute result to master, the current task execute result is {}", + asyncMasterDelayTaskExecuteRunnable.getTaskExecutionContext().getCurrentExecutionStatus().name()); + asyncMasterDelayTaskExecuteRunnable.closeLogAppender(); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecuteFunction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecuteFunction.java new file mode 100644 index 0000000000..3b2f62a2d1 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecuteFunction.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.server.master.runner.execute; + +import java.time.Duration; + +import lombok.NonNull; + +public interface AsyncTaskExecuteFunction { + + @NonNull + AsyncTaskExecutionStatus getAsyncTaskExecutionStatus(); + + @NonNull + Duration getAsyncTaskStateCheckInterval(); + + enum AsyncTaskExecutionStatus { + + RUNNING, + + SUCCESS, + + FAILED, + ; + + } + +} 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 new file mode 100644 index 0000000000..ac254d210e --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecutionContext.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.enums.TaskTimeoutStrategy; + +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; + private long timeout; + + 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; + if (TaskTimeoutStrategy.FAILED.equals(taskExecutionContext.getTaskTimeoutStrategy()) + || TaskTimeoutStrategy.WARNFAILED.equals(taskExecutionContext.getTaskTimeoutStrategy())) { + // will timeout + this.timeout = taskExecutionContext.getStartTime() + + TimeUnit.SECONDS.toMillis(taskExecutionContext.getTaskTimeout()); + } else { + this.timeout = TimeUnit.SECONDS.toMillis(Integer.MAX_VALUE); + } + this.executeInterval = Math.max(asyncTaskExecuteFunction.getAsyncTaskStateCheckInterval().toMillis(), 1000L); + } + + public void refreshStartTime() { + if (executeTimes == 0) { + // The first time doesn't have delay + executeTimes++; + } else { + currentStartTime = System.currentTimeMillis(); + } + } + + @Override + public long getDelay(TimeUnit unit) { + long nextExecuteTimeDelay = Math.min(currentStartTime + executeInterval, timeout) - 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/DefaultTaskExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnable.java new file mode 100644 index 0000000000..4e9a5aed9f --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnable.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 com.google.common.base.Preconditions.checkNotNull; + +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.runner.operator.TaskOperatorManager; + +public class DefaultTaskExecuteRunnable extends PriorityTaskExecuteRunnable { + + private final TaskOperatorManager taskOperatorManager; + + public DefaultTaskExecuteRunnable(ProcessInstance workflowInstance, + TaskInstance taskInstance, + TaskExecutionContext taskExecutionContext, + TaskOperatorManager taskOperatorManager) { + super(workflowInstance, taskInstance, taskExecutionContext); + this.taskOperatorManager = checkNotNull(taskOperatorManager); + } + + @Override + public void dispatch() { + taskOperatorManager.getTaskDispatchOperator().handle(this); + } + + @Override + public void kill() { + taskOperatorManager.getTaskKillOperator().handle(this); + } + + @Override + public void pause() { + taskOperatorManager.getTaskPauseOperator().handle(this); + } + + @Override + public void timeout() { + taskOperatorManager.getTaskTimeoutOperator().handle(this); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnableFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnableFactory.java new file mode 100644 index 0000000000..d5c53d0c35 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnableFactory.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.exception.TaskExecuteRunnableCreateException; +import org.apache.dolphinscheduler.server.master.exception.TaskExecutionContextCreateException; +import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.operator.TaskOperatorManager; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class DefaultTaskExecuteRunnableFactory implements TaskExecuteRunnableFactory { + + @Autowired + private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + + @Autowired + private TaskExecutionContextFactory taskExecutionContextFactory; + + @Autowired + private TaskOperatorManager taskOperatorManager; + + @Override + public DefaultTaskExecuteRunnable createTaskExecuteRunnable(TaskInstance taskInstance) throws TaskExecuteRunnableCreateException { + WorkflowExecuteRunnable workflowExecuteRunnable = + processInstanceExecCacheManager.getByProcessInstanceId(taskInstance.getProcessInstanceId()); + try { + return new DefaultTaskExecuteRunnable( + workflowExecuteRunnable.getProcessInstance(), + taskInstance, + taskExecutionContextFactory.createTaskExecutionContext(taskInstance), + taskOperatorManager); + } catch (TaskExecutionContextCreateException ex) { + throw new TaskExecuteRunnableCreateException("Create DefaultTaskExecuteRunnable failed", ex); + } + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterDelayTaskExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterDelayTaskExecuteRunnable.java new file mode 100644 index 0000000000..653901f4de --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterDelayTaskExecuteRunnable.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.utils.DateUtils; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager; +import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder; + +import java.util.concurrent.Delayed; +import java.util.concurrent.TimeUnit; + +public abstract class MasterDelayTaskExecuteRunnable extends MasterTaskExecuteRunnable implements Delayed { + + public MasterDelayTaskExecuteRunnable(TaskExecutionContext taskExecutionContext, + LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder, + MasterMessageSenderManager masterMessageSenderManager) { + super(taskExecutionContext, logicTaskPluginFactoryBuilder, masterMessageSenderManager); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof MasterDelayTaskExecuteRunnable)) { + return false; + } + MasterDelayTaskExecuteRunnable other = (MasterDelayTaskExecuteRunnable) obj; + return other.getTaskExecutionContext().getTaskInstanceId() == this.getTaskExecutionContext() + .getTaskInstanceId(); + } + + @Override + public int hashCode() { + return this.getTaskExecutionContext().getTaskInstanceId(); + } + + @Override + public long getDelay(TimeUnit unit) { + TaskExecutionContext taskExecutionContext = getTaskExecutionContext(); + return unit.convert( + DateUtils.getRemainTime( + taskExecutionContext.getFirstSubmitTime(), taskExecutionContext.getDelayTime() * 60L), + TimeUnit.SECONDS); + } + + @Override + public int compareTo(Delayed o) { + if (o == null) { + return 1; + } + return Long.compare(this.getDelay(TimeUnit.MILLISECONDS), o.getDelay(TimeUnit.MILLISECONDS)); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterDelayTaskExecuteRunnableFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterDelayTaskExecuteRunnableFactory.java new file mode 100644 index 0000000000..0fd79dfabe --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterDelayTaskExecuteRunnableFactory.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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; + +public interface MasterDelayTaskExecuteRunnableFactory { + + T createWorkerTaskExecuteRunnable(TaskExecutionContext taskExecutionContext); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecuteRunnable.java new file mode 100644 index 0000000000..4dbdcfce9d --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecuteRunnable.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 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.TaskExecutionContextCacheManager; +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.remote.exceptions.RemotingException; +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.MasterMessageSenderManager; +import org.apache.dolphinscheduler.server.master.runner.task.ILogicTask; +import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public abstract class MasterTaskExecuteRunnable implements Runnable { + + protected final TaskExecutionContext taskExecutionContext; + protected final LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder; + protected final MasterMessageSenderManager masterMessageSenderManager; + protected ILogicTask logicTask; + + public MasterTaskExecuteRunnable(TaskExecutionContext taskExecutionContext, + LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder, + MasterMessageSenderManager masterMessageSenderManager) { + this.taskExecutionContext = taskExecutionContext; + this.logicTaskPluginFactoryBuilder = logicTaskPluginFactoryBuilder; + this.masterMessageSenderManager = masterMessageSenderManager; + } + + protected abstract void executeTask() throws MasterTaskExecuteException; + + protected abstract void afterExecute() throws MasterTaskExecuteException; + + protected void afterThrowing(Throwable throwable) { + try { + cancelTask(); + log.info("Get a exception when execute the task, canceled the task"); + } 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()); + TaskExecutionContextCacheManager.removeByTaskInstanceId(taskExecutionContext.getTaskInstanceId()); + log.info("Get a exception when execute the task, removed the TaskExecutionContext"); + } + + public void cancelTask() throws MasterTaskExecuteException { + if (logicTask != null) { + logicTask.kill(); + } + } + + public void pauseTask() throws MasterTaskExecuteException { + if (logicTask != null) { + logicTask.pause(); + } + } + + public TaskExecutionContext getTaskExecutionContext() { + return taskExecutionContext; + } + + @Override + public void run() { + try ( + final LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setWorkflowAndTaskInstanceIDMDC( + taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId()); + final LogUtils.MDCAutoClosableContext mdcAutoClosableContext1 = + LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath())) { + TaskInstanceLogHeader.printInitializeTaskContextHeader(); + initializeTask(); + + TaskInstanceLogHeader.printLoadTaskInstancePluginHeader(); + beforeExecute(); + + TaskInstanceLogHeader.printExecuteTaskHeader(); + executeTask(); + + afterExecute(); + } catch (Throwable ex) { + log.error("Task execute failed, due to meet an exception", ex); + afterThrowing(ex); + closeLogAppender(); + } + } + + 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.getProcessInstanceId(), + 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, RemotingException { + taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.RUNNING_EXECUTION); + masterMessageSenderManager.getMasterTaskExecuteRunningMessageSender().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())); + masterMessageSenderManager.getMasterTaskExecuteResultMessageSender().sendMessage(taskExecutionContext); + 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/MasterTaskExecuteRunnableFactoryBuilder.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecuteRunnableFactoryBuilder.java new file mode 100644 index 0000000000..f196d85153 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecuteRunnableFactoryBuilder.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.runner.execute; + +import org.apache.dolphinscheduler.server.master.runner.task.dependent.DependentLogicTask; +import org.apache.dolphinscheduler.server.master.runner.task.subworkflow.SubWorkflowLogicTask; + +import java.util.Set; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +import com.google.common.collect.Sets; + +@Component +public class MasterTaskExecuteRunnableFactoryBuilder { + + @Autowired + private AsyncMasterDelayTaskExecuteRunnableFactory asyncMasterDelayTaskExecuteRunnableFactory; + + @Autowired + private SyncMasterDelayTaskExecuteRunnableFactory syncMasterDelayTaskExecuteRunnableFactory; + + private static final Set ASYNC_TASK_TYPE = Sets.newHashSet( + DependentLogicTask.TASK_TYPE, + SubWorkflowLogicTask.TASK_TYPE); + + public MasterDelayTaskExecuteRunnableFactory createWorkerDelayTaskExecuteRunnableFactory(String taskType) { + if (ASYNC_TASK_TYPE.contains(taskType)) { + return asyncMasterDelayTaskExecuteRunnableFactory; + } + return syncMasterDelayTaskExecuteRunnableFactory; + } +} 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 new file mode 100644 index 0000000000..edcbbe6708 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutionContextHolder.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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/PriorityTaskExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/PriorityTaskExecuteRunnable.java new file mode 100644 index 0000000000..b78762b21d --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/PriorityTaskExecuteRunnable.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 com.google.common.base.Preconditions.checkNotNull; + +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; + +import org.jetbrains.annotations.NotNull; + +public abstract class PriorityTaskExecuteRunnable implements TaskExecuteRunnable, Comparable { + + private final ProcessInstance workflowInstance; + private final TaskInstance taskInstance; + private final TaskExecutionContext taskExecutionContext; + + public PriorityTaskExecuteRunnable(ProcessInstance workflowInstance, + TaskInstance taskInstance, + TaskExecutionContext taskExecutionContext) { + this.taskInstance = checkNotNull(taskInstance); + this.workflowInstance = checkNotNull(workflowInstance); + this.taskExecutionContext = checkNotNull(taskExecutionContext); + } + + @Override + public ProcessInstance getWorkflowInstance() { + return workflowInstance; + } + + @Override + public TaskInstance getTaskInstance() { + return taskInstance; + } + + @Override + public TaskExecutionContext getTaskExecutionContext() { + return taskExecutionContext; + } + + @Override + public int compareTo(@NotNull TaskExecuteRunnable other) { + int workflowInstancePriorityCompareResult = workflowInstance.getProcessInstancePriority().getCode() + - other.getWorkflowInstance().getProcessInstancePriority().getCode(); + if (workflowInstancePriorityCompareResult != 0) { + return workflowInstancePriorityCompareResult; + } + int workflowInstanceIdCompareResult = workflowInstance.getId() - other.getWorkflowInstance().getId(); + if (workflowInstanceIdCompareResult != 0) { + return workflowInstanceIdCompareResult; + } + int taskInstancePriorityCompareResult = taskInstance.getTaskInstancePriority().getCode() + - other.getTaskInstance().getTaskInstancePriority().getCode(); + if (taskInstancePriorityCompareResult > 0) { + return taskInstancePriorityCompareResult; + } + // larger number, higher priority + int taskGroupPriorityCompareResult = + taskInstance.getTaskGroupPriority() - other.getTaskInstance().getTaskGroupPriority(); + if (taskGroupPriorityCompareResult != 0) { + return -taskGroupPriorityCompareResult; + } + // The task instance shouldn't be equals + return taskInstance.getId() - other.getTaskInstance().getId(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof PriorityTaskExecuteRunnable) { + PriorityTaskExecuteRunnable other = (PriorityTaskExecuteRunnable) obj; + return compareTo(other) == 0; + } + return false; + } + + @Override + public int hashCode() { + return taskInstance.getId(); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/SyncMasterDelayTaskExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/SyncMasterDelayTaskExecuteRunnable.java new file mode 100644 index 0000000000..03d3c0ea6f --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/SyncMasterDelayTaskExecuteRunnable.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.MasterMessageSenderManager; +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 SyncMasterDelayTaskExecuteRunnable extends MasterDelayTaskExecuteRunnable { + + public SyncMasterDelayTaskExecuteRunnable(TaskExecutionContext taskExecutionContext, + LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder, + MasterMessageSenderManager masterMessageSenderManager) { + super(taskExecutionContext, logicTaskPluginFactoryBuilder, masterMessageSenderManager); + } + + @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(); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/SyncMasterDelayTaskExecuteRunnableFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/SyncMasterDelayTaskExecuteRunnableFactory.java new file mode 100644 index 0000000000..8c254485ae --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/SyncMasterDelayTaskExecuteRunnableFactory.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.runner.execute; + +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager; +import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class SyncMasterDelayTaskExecuteRunnableFactory + implements + MasterDelayTaskExecuteRunnableFactory { + + @Autowired + private LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder; + @Autowired + private MasterMessageSenderManager masterMessageSenderManager; + + @Override + public SyncMasterDelayTaskExecuteRunnable createWorkerTaskExecuteRunnable(TaskExecutionContext taskExecutionContext) { + return new SyncMasterDelayTaskExecuteRunnable(taskExecutionContext, logicTaskPluginFactoryBuilder, + masterMessageSenderManager); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/TaskExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/TaskExecuteRunnable.java new file mode 100644 index 0000000000..02980aff49 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/TaskExecuteRunnable.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.server.master.runner.execute; + +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; + +/** + * This interface is used to define a task which is executing. + * todo: split to MasterTaskExecuteRunnable and WorkerTaskExecuteRunnable + */ +public interface TaskExecuteRunnable extends Comparable { + + void dispatch(); + + void kill(); + + void pause(); + + void timeout(); + + ProcessInstance getWorkflowInstance(); + + TaskInstance getTaskInstance(); + + TaskExecutionContext getTaskExecutionContext(); +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/TaskExecuteRunnableFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/TaskExecuteRunnableFactory.java new file mode 100644 index 0000000000..e3afb2d6b3 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/TaskExecuteRunnableFactory.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.runner.execute; + +import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.server.master.exception.TaskExecuteRunnableCreateException; + +public interface TaskExecuteRunnableFactory { + + T createTaskExecuteRunnable(TaskInstance taskInstance) throws TaskExecuteRunnableCreateException; + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseTaskProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/TaskExecutionContextFactory.java similarity index 71% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseTaskProcessor.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/TaskExecutionContextFactory.java index 9925e36b5e..03005ff70c 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseTaskProcessor.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/TaskExecutionContextFactory.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.task; +package org.apache.dolphinscheduler.server.master.runner.execute; import static org.apache.dolphinscheduler.common.constants.Constants.ADDRESS; import static org.apache.dolphinscheduler.common.constants.Constants.DATABASE; @@ -46,15 +46,10 @@ import org.apache.dolphinscheduler.dao.entity.DqRuleInputEntry; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.entity.UdfFunc; -import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; -import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; -import org.apache.dolphinscheduler.plugin.storage.api.StorageOperate; import org.apache.dolphinscheduler.plugin.task.api.DataQualityTaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.K8sTaskExecutionContext; -import org.apache.dolphinscheduler.plugin.task.api.TaskChannel; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager; -import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; import org.apache.dolphinscheduler.plugin.task.api.enums.dp.ConnectorType; import org.apache.dolphinscheduler.plugin.task.api.enums.dp.ExecuteSqlType; import org.apache.dolphinscheduler.plugin.task.api.model.JdbcInfo; @@ -69,17 +64,15 @@ import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.DataSourc import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper; import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.UdfFuncParameters; import org.apache.dolphinscheduler.plugin.task.api.utils.JdbcUrlParser; -import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils; import org.apache.dolphinscheduler.plugin.task.spark.SparkParameters; import org.apache.dolphinscheduler.server.master.builder.TaskExecutionContextBuilder; import org.apache.dolphinscheduler.server.master.config.MasterConfig; -import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; +import org.apache.dolphinscheduler.server.master.exception.TaskExecutionContextCreateException; import org.apache.dolphinscheduler.service.expand.CuringParamsService; import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.ResourceType; -import org.apache.dolphinscheduler.spi.plugin.SPIIdentify; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; @@ -89,252 +82,60 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Properties; -import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; import com.zaxxer.hikari.HikariDataSource; -public abstract class BaseTaskProcessor implements ITaskProcessor { +@Slf4j +@Component +public class TaskExecutionContextFactory { - protected final Logger log = - LoggerFactory.getLogger(BaseTaskProcessor.class); + @Autowired + private ProcessService processService; - private String tenantCode; + @Autowired + private TaskPluginManager taskPluginManager; - protected boolean killed = false; + @Autowired + private CuringParamsService curingParamsService; - protected boolean paused = false; + @Autowired + private MasterConfig masterConfig; - protected boolean timeout = false; + @Autowired + private HikariDataSource hikariDataSource; - protected TaskInstance taskInstance = null; - - protected ProcessInstance processInstance; - - protected int maxRetryTimes; - - protected long commitInterval; - - protected ProcessService processService; - - protected TaskInstanceDao taskInstanceDao; - - protected ProcessInstanceDao processInstanceDao; - - protected StorageOperate storageOperate; - - protected MasterConfig masterConfig; - - protected TaskPluginManager taskPluginManager; - - protected CuringParamsService curingParamsService; - - protected String threadLoggerInfoName; - - @Override - public void init(@NonNull TaskInstance taskInstance, @NonNull ProcessInstance processInstance) { - processService = SpringApplicationContext.getBean(ProcessService.class); - processInstanceDao = SpringApplicationContext.getBean(ProcessInstanceDao.class); - masterConfig = SpringApplicationContext.getBean(MasterConfig.class); - taskPluginManager = SpringApplicationContext.getBean(TaskPluginManager.class); - curingParamsService = SpringApplicationContext.getBean(CuringParamsService.class); - taskInstanceDao = SpringApplicationContext.getBean(TaskInstanceDao.class); - storageOperate = SpringApplicationContext.getBean(StorageOperate.class, null); - this.taskInstance = taskInstance; - this.processInstance = processInstance; - this.maxRetryTimes = masterConfig.getTaskCommitRetryTimes(); - this.commitInterval = masterConfig.getTaskCommitInterval().toMillis(); - this.tenantCode = getTenantCode(); - } - - protected javax.sql.DataSource defaultDataSource = - SpringApplicationContext.getBean(javax.sql.DataSource.class); - - /** - * pause task, common tasks donot need this. - */ - protected abstract boolean pauseTask(); - - /** - * kill task, all tasks need to realize this function - */ - protected abstract boolean killTask(); - - /** - * task timeout process - */ - protected abstract boolean taskTimeout(); - - /** - * submit task - */ - protected abstract boolean submitTask(); - - /* - * resubmit task - */ - protected abstract boolean resubmitTask(); - - /** - * run task - */ - protected abstract boolean runTask(); - - /** - * dispatch task - */ - protected abstract boolean dispatchTask(); - - @Override - public boolean action(TaskAction taskAction) { - String oldTaskInstanceLogPathMdc = LogUtils.getTaskInstanceLogFullPathMdc(); - if (StringUtils.isNotEmpty(threadLoggerInfoName)) { - LogUtils.setTaskInstanceLogFullPathMDC(threadLoggerInfoName); - } - boolean result = false; - try { - switch (taskAction) { - case STOP: - result = stop(); - break; - case PAUSE: - result = pause(); - break; - case TIMEOUT: - result = timeout(); - break; - case SUBMIT: - result = submit(); - break; - case RUN: - result = run(); - break; - case DISPATCH: - result = dispatch(); - break; - case RESUBMIT: - result = resubmit(); - break; - default: - log.error("unknown task action: {}", taskAction); - } - return result; - } finally { - LogUtils.removeTaskInstanceLogFullPathMDC(); - // reset MDC value, this should be removed. - if (oldTaskInstanceLogPathMdc != null) { - LogUtils.setTaskInstanceLogFullPathMDC(oldTaskInstanceLogPathMdc); - } - } - } - - protected boolean resubmit() { - return resubmitTask(); - } - - protected boolean submit() { - return submitTask(); - } - - protected boolean run() { - return runTask(); - } - - protected boolean dispatch() { - return dispatchTask(); - } - - protected boolean timeout() { - if (timeout) { - return true; - } - timeout = taskTimeout(); - return timeout; - } - - protected boolean pause() { - if (paused) { - return true; - } - paused = pauseTask(); - return paused; - } - - protected boolean stop() { - if (killed) { - return true; - } - killed = killTask(); - return killed; - } - - @Override - public String getType() { - throw new UnsupportedOperationException("This abstract class doesn's has type"); - } - - @Override - public SPIIdentify getIdentify() { - return SPIIdentify.builder().name(getType()).build(); - } - - @Override - public TaskInstance taskInstance() { - return this.taskInstance; - } - - /** - * set master task running log. - */ - public void setTaskExecutionLogger() { - threadLoggerInfoName = LogUtils.getTaskInstanceLogFullPath( - taskInstance.getFirstSubmitTime(), - processInstance.getProcessDefinitionCode(), - processInstance.getProcessDefinitionVersion(), - taskInstance.getProcessInstanceId(), - taskInstance.getId()); - LogUtils.setTaskInstanceLogFullPathMDC(threadLoggerInfoName); - } - - /** - * get TaskExecutionContext - * - * @param taskInstance taskInstance - * @return TaskExecutionContext - */ - protected TaskExecutionContext getTaskExecutionContext(TaskInstance taskInstance) { - if (tenantCode == null) { - log.info("Task state changes to {}", TaskExecutionStatus.FAILURE); - taskInstance.setState(TaskExecutionStatus.FAILURE); - taskInstanceDao.upsertTaskInstance(taskInstance); - return null; - } - taskInstance.getProcessInstance().setTenantCode(tenantCode); + public TaskExecutionContext createTaskExecutionContext(TaskInstance taskInstance) throws TaskExecutionContextCreateException { + ProcessInstance workflowInstance = taskInstance.getProcessInstance(); taskInstance.setResources(getResourceFullNames(taskInstance)); - TaskChannel taskChannel = taskPluginManager.getTaskChannel(taskInstance.getTaskType()); - ResourceParametersHelper resources = taskChannel.getResources(taskInstance.getTaskParams()); - this.setTaskResourceInfo(resources); + ResourceParametersHelper resources = + Optional.ofNullable(taskPluginManager.getTaskChannel(taskInstance.getTaskType())) + .map(taskChannel -> taskChannel.getResources(taskInstance.getTaskParams())) + .orElse(null); + setTaskResourceInfo(resources); // TODO to be optimized DataQualityTaskExecutionContext dataQualityTaskExecutionContext = null; if (TASK_TYPE_DATA_QUALITY.equalsIgnoreCase(taskInstance.getTaskType())) { dataQualityTaskExecutionContext = new DataQualityTaskExecutionContext(); - setDataQualityTaskRelation(dataQualityTaskExecutionContext, taskInstance, tenantCode); + setDataQualityTaskRelation(dataQualityTaskExecutionContext, taskInstance, workflowInstance.getTenantCode()); } K8sTaskExecutionContext k8sTaskExecutionContext = setK8sTaskRelation(taskInstance); - Map businessParamsMap = curingParamsService.preBuildBusinessParams(processInstance); + Map businessParamsMap = curingParamsService.preBuildBusinessParams(workflowInstance); AbstractParameters baseParam = taskPluginManager.getParameters(ParametersNode.builder() .taskType(taskInstance.getTaskType()).taskParams(taskInstance.getTaskParams()).build()); Map propertyMap = - curingParamsService.paramParsingPreparation(taskInstance, baseParam, processInstance); + curingParamsService.paramParsingPreparation(taskInstance, baseParam, workflowInstance); return TaskExecutionContextBuilder.get() .buildWorkflowInstanceHost(masterConfig.getMasterAddress()) .buildTaskInstanceRelatedInfo(taskInstance) @@ -349,17 +150,33 @@ public abstract class BaseTaskProcessor implements ITaskProcessor { .create(); } - public void setTaskResourceInfo(ResourceParametersHelper resourceParametersHelper) { + private Map getResourceFullNames(TaskInstance taskInstance) { + Map resourcesMap = new HashMap<>(); + AbstractParameters baseParam = taskPluginManager.getParameters(ParametersNode.builder() + .taskType(taskInstance.getTaskType()).taskParams(taskInstance.getTaskParams()).build()); + if (baseParam != null) { + List projectResourceFiles = baseParam.getResourceFilesList(); + if (CollectionUtils.isNotEmpty(projectResourceFiles)) { + // TODO: Modify this part to accomodate(migrate) oldversionresources in the future. + projectResourceFiles.forEach(file -> resourcesMap.put(file.getResourceName(), + processService.queryTenantCodeByResName(file.getResourceName(), ResourceType.FILE))); + } + } + + return resourcesMap; + } + + private void setTaskResourceInfo(ResourceParametersHelper resourceParametersHelper) { if (Objects.isNull(resourceParametersHelper)) { return; } resourceParametersHelper.getResourceMap().forEach((type, map) -> { switch (type) { case DATASOURCE: - this.setTaskDataSourceResourceInfo(map); + setTaskDataSourceResourceInfo(map); break; case UDF: - this.setTaskUdfFuncResourceInfo(map); + setTaskUdfFuncResourceInfo(map); break; default: break; @@ -382,7 +199,6 @@ public abstract class BaseTaskProcessor implements ITaskProcessor { dataSourceParameters.setConnectionParams(datasource.getConnectionParams()); map.put(code, dataSourceParameters); }); - } private void setTaskUdfFuncResourceInfo(Map map) { @@ -401,12 +217,6 @@ public abstract class BaseTaskProcessor implements ITaskProcessor { }); } - /** - * set data quality task relation - * - * @param dataQualityTaskExecutionContext dataQualityTaskExecutionContext - * @param taskInstance taskInstance - */ private void setDataQualityTaskRelation(DataQualityTaskExecutionContext dataQualityTaskExecutionContext, TaskInstance taskInstance, String tenantCode) { DataQualityParameters dataQualityParameters = @@ -451,10 +261,64 @@ public abstract class BaseTaskProcessor implements ITaskProcessor { setStatisticsValueWriterConfig(dataQualityTaskExecutionContext); } + private K8sTaskExecutionContext setK8sTaskRelation(TaskInstance taskInstance) { + K8sTaskExecutionContext k8sTaskExecutionContext = null; + String namespace = ""; + switch (taskInstance.getTaskType()) { + case "K8S": + case "KUBEFLOW": + K8sTaskParameters k8sTaskParameters = + JSONUtils.parseObject(taskInstance.getTaskParams(), K8sTaskParameters.class); + namespace = k8sTaskParameters.getNamespace(); + break; + case "SPARK": + SparkParameters sparkParameters = + JSONUtils.parseObject(taskInstance.getTaskParams(), SparkParameters.class); + if (StringUtils.isNotEmpty(sparkParameters.getNamespace())) { + namespace = sparkParameters.getNamespace(); + } + break; + default: + break; + } + + if (StringUtils.isNotEmpty(namespace)) { + String clusterName = JSONUtils.toMap(namespace).get(CLUSTER); + String configYaml = processService.findConfigYamlByName(clusterName); + if (configYaml != null) { + k8sTaskExecutionContext = + new K8sTaskExecutionContext(configYaml, JSONUtils.toMap(namespace).get(NAMESPACE_NAME)); + } + } + return k8sTaskExecutionContext; + } + + /** + * The SourceConfig will be used in DataQualityApplication that + * get the data which be used to get the statistics value + * + * @param dataQualityTaskExecutionContext + * @param config + */ + private void setSourceConfig(DataQualityTaskExecutionContext dataQualityTaskExecutionContext, + Map config) { + if (StringUtils.isNotEmpty(config.get(SRC_DATASOURCE_ID))) { + DataSource dataSource = processService.findDataSourceById(Integer.parseInt(config.get(SRC_DATASOURCE_ID))); + if (dataSource != null) { + ConnectorType srcConnectorType = ConnectorType.of( + DbType.of(Integer.parseInt(config.get(SRC_CONNECTOR_TYPE))).isHive() ? 1 : 0); + dataQualityTaskExecutionContext.setSourceConnectorType(srcConnectorType.getDescription()); + dataQualityTaskExecutionContext.setSourceType(dataSource.getType().getCode()); + dataQualityTaskExecutionContext.setSourceConnectionParams(dataSource.getConnectionParams()); + } + } + } + /** * It is used to get comparison params, the param contains * comparison name、comparison table and execute sql. * When the type is fixed_value, params will be null. + * * @param dataQualityTaskExecutionContext * @param config * @param ruleInputEntryList @@ -501,15 +365,52 @@ public abstract class BaseTaskProcessor implements ITaskProcessor { } } + /** + * The TargetConfig will be used in DataQualityApplication that + * get the data which be used to compare to src value + * + * @param dataQualityTaskExecutionContext + * @param config + */ + private void setTargetConfig(DataQualityTaskExecutionContext dataQualityTaskExecutionContext, + Map config) { + if (StringUtils.isNotEmpty(config.get(TARGET_DATASOURCE_ID))) { + DataSource dataSource = + processService.findDataSourceById(Integer.parseInt(config.get(TARGET_DATASOURCE_ID))); + if (dataSource != null) { + ConnectorType targetConnectorType = ConnectorType.of( + DbType.of(Integer.parseInt(config.get(TARGET_CONNECTOR_TYPE))).isHive() ? 1 : 0); + dataQualityTaskExecutionContext.setTargetConnectorType(targetConnectorType.getDescription()); + dataQualityTaskExecutionContext.setTargetType(dataSource.getType().getCode()); + dataQualityTaskExecutionContext.setTargetConnectionParams(dataSource.getConnectionParams()); + } + } + } + + /** + * The WriterConfig will be used in DataQualityApplication that + * writes the data quality check result into dolphin scheduler datasource + * + * @param dataQualityTaskExecutionContext + */ + private void setWriterConfig(DataQualityTaskExecutionContext dataQualityTaskExecutionContext) { + DataSource dataSource = getDefaultDataSource(); + ConnectorType writerConnectorType = ConnectorType.of(dataSource.getType().isHive() ? 1 : 0); + dataQualityTaskExecutionContext.setWriterConnectorType(writerConnectorType.getDescription()); + dataQualityTaskExecutionContext.setWriterType(dataSource.getType().getCode()); + dataQualityTaskExecutionContext.setWriterConnectionParams(dataSource.getConnectionParams()); + dataQualityTaskExecutionContext.setWriterTable("t_ds_dq_execute_result"); + } + /** * The default datasource is used to get the dolphinscheduler datasource info, * and the info will be used in StatisticsValueConfig and WriterConfig + * * @return DataSource */ public DataSource getDefaultDataSource() { DataSource dataSource = new DataSource(); - HikariDataSource hikariDataSource = (HikariDataSource) defaultDataSource; dataSource.setUserName(hikariDataSource.getUsername()); JdbcInfo jdbcInfo = JdbcUrlParser.getJdbcInfo(hikariDataSource.getJdbcUrl()); if (jdbcInfo != null) { @@ -541,118 +442,4 @@ public abstract class BaseTaskProcessor implements ITaskProcessor { dataQualityTaskExecutionContext.setStatisticsValueTable("t_ds_dq_task_statistics_value"); } - /** - * The WriterConfig will be used in DataQualityApplication that - * writes the data quality check result into dolphin scheduler datasource - * @param dataQualityTaskExecutionContext - */ - private void setWriterConfig(DataQualityTaskExecutionContext dataQualityTaskExecutionContext) { - DataSource dataSource = getDefaultDataSource(); - ConnectorType writerConnectorType = ConnectorType.of(dataSource.getType().isHive() ? 1 : 0); - dataQualityTaskExecutionContext.setWriterConnectorType(writerConnectorType.getDescription()); - dataQualityTaskExecutionContext.setWriterType(dataSource.getType().getCode()); - dataQualityTaskExecutionContext.setWriterConnectionParams(dataSource.getConnectionParams()); - dataQualityTaskExecutionContext.setWriterTable("t_ds_dq_execute_result"); - } - - /** - * The TargetConfig will be used in DataQualityApplication that - * get the data which be used to compare to src value - * @param dataQualityTaskExecutionContext - * @param config - */ - private void setTargetConfig(DataQualityTaskExecutionContext dataQualityTaskExecutionContext, - Map config) { - if (StringUtils.isNotEmpty(config.get(TARGET_DATASOURCE_ID))) { - DataSource dataSource = - processService.findDataSourceById(Integer.parseInt(config.get(TARGET_DATASOURCE_ID))); - if (dataSource != null) { - ConnectorType targetConnectorType = ConnectorType.of( - DbType.of(Integer.parseInt(config.get(TARGET_CONNECTOR_TYPE))).isHive() ? 1 : 0); - dataQualityTaskExecutionContext.setTargetConnectorType(targetConnectorType.getDescription()); - dataQualityTaskExecutionContext.setTargetType(dataSource.getType().getCode()); - dataQualityTaskExecutionContext.setTargetConnectionParams(dataSource.getConnectionParams()); - } - } - } - - /** - * The SourceConfig will be used in DataQualityApplication that - * get the data which be used to get the statistics value - * @param dataQualityTaskExecutionContext - * @param config - */ - private void setSourceConfig(DataQualityTaskExecutionContext dataQualityTaskExecutionContext, - Map config) { - if (StringUtils.isNotEmpty(config.get(SRC_DATASOURCE_ID))) { - DataSource dataSource = processService.findDataSourceById(Integer.parseInt(config.get(SRC_DATASOURCE_ID))); - if (dataSource != null) { - ConnectorType srcConnectorType = ConnectorType.of( - DbType.of(Integer.parseInt(config.get(SRC_CONNECTOR_TYPE))).isHive() ? 1 : 0); - dataQualityTaskExecutionContext.setSourceConnectorType(srcConnectorType.getDescription()); - dataQualityTaskExecutionContext.setSourceType(dataSource.getType().getCode()); - dataQualityTaskExecutionContext.setSourceConnectionParams(dataSource.getConnectionParams()); - } - } - } - - /** - * get resource map key is full name and value is tenantCode - */ - public Map getResourceFullNames(TaskInstance taskInstance) { - Map resourcesMap = new HashMap<>(); - AbstractParameters baseParam = taskPluginManager.getParameters(ParametersNode.builder() - .taskType(taskInstance.getTaskType()).taskParams(taskInstance.getTaskParams()).build()); - if (baseParam != null) { - List projectResourceFiles = baseParam.getResourceFilesList(); - if (CollectionUtils.isNotEmpty(projectResourceFiles)) { - projectResourceFiles.forEach(file -> resourcesMap.put(file.getResourceName(), - storageOperate.getResourceFileName(file.getResourceName()))); - } - } - - return resourcesMap; - } - - /** - * get k8s task execution context based on task type and deploy mode - * - * @param taskInstance taskInstance - */ - private K8sTaskExecutionContext setK8sTaskRelation(TaskInstance taskInstance) { - K8sTaskExecutionContext k8sTaskExecutionContext = null; - String namespace = ""; - switch (taskInstance.getTaskType()) { - case "K8S": - case "KUBEFLOW": - K8sTaskParameters k8sTaskParameters = - JSONUtils.parseObject(taskInstance.getTaskParams(), K8sTaskParameters.class); - namespace = k8sTaskParameters.getNamespace(); - break; - case "SPARK": - SparkParameters sparkParameters = - JSONUtils.parseObject(taskInstance.getTaskParams(), SparkParameters.class); - if (StringUtils.isNotEmpty(sparkParameters.getNamespace())) { - namespace = sparkParameters.getNamespace(); - } - break; - default: - break; - } - - if (StringUtils.isNotEmpty(namespace)) { - String clusterName = JSONUtils.toMap(namespace).get(CLUSTER); - String configYaml = processService.findConfigYamlByName(clusterName); - if (configYaml != null) { - k8sTaskExecutionContext = - new K8sTaskExecutionContext(configYaml, JSONUtils.toMap(namespace).get(NAMESPACE_NAME)); - } - } - return k8sTaskExecutionContext; - } - - private String getTenantCode() { - int userId = taskInstance.getProcessDefine() == null ? 0 : taskInstance.getProcessDefine().getUserId(); - return processService.getTenantForProcess(taskInstance.getProcessInstance().getTenantCode(), userId); - } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterMessageSender.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterMessageSender.java new file mode 100644 index 0000000000..8c2693b0c6 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterMessageSender.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.runner.message; + +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.remote.command.BaseMessage; +import org.apache.dolphinscheduler.remote.command.MessageType; +import org.apache.dolphinscheduler.remote.exceptions.RemotingException; + +// todo: unify with WorkerMessageSender +public interface MasterMessageSender { + + /** + * Send the message + * + * @throws RemotingException Cannot connect to the target host. + */ + void sendMessage(T message) throws RemotingException; + + default void sendMessage(TaskExecutionContext taskExecutionContext) throws RemotingException { + T message = buildMessage(taskExecutionContext); + sendMessage(message); + } + + /** + * Build the message from task context and message received address. + */ + T buildMessage(TaskExecutionContext taskExecutionContext); + + /** + * The message type can be sent by this sender. + */ + MessageType getMessageType(); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterMessageSenderManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterMessageSenderManager.java new file mode 100644 index 0000000000..82fbb00c41 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterMessageSenderManager.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.runner.message; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class MasterMessageSenderManager { + + @Autowired + private MasterTaskExecuteResultMessageSender masterTaskExecuteResultMessageSender; + + @Autowired + private MasterTaskExecuteRunningMessageSender masterTaskExecuteRunningMessageSender; + + @Autowired + private MasterTaskUpdateRuntimeMessageSender masterTaskUpdateRuntimeMessageSender; + + public MasterTaskExecuteResultMessageSender getMasterTaskExecuteResultMessageSender() { + return masterTaskExecuteResultMessageSender; + } + + public MasterTaskExecuteRunningMessageSender getMasterTaskExecuteRunningMessageSender() { + return masterTaskExecuteRunningMessageSender; + } + + public MasterTaskUpdateRuntimeMessageSender getMasterTaskUpdatePidMessageSender() { + return masterTaskUpdateRuntimeMessageSender; + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterTaskExecuteResultMessageSender.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterTaskExecuteResultMessageSender.java new file mode 100644 index 0000000000..90986a59eb --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterTaskExecuteResultMessageSender.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.remote.command.MessageType; +import org.apache.dolphinscheduler.remote.command.task.TaskExecuteResultMessage; +import org.apache.dolphinscheduler.remote.exceptions.RemotingException; +import org.apache.dolphinscheduler.remote.utils.Host; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; +import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Component +public class MasterTaskExecuteResultMessageSender implements MasterMessageSender { + + @Autowired + private MasterConfig masterConfig; + + @Autowired + private MasterRpcClient masterRpcClient; + + @Override + public void sendMessage(TaskExecuteResultMessage message) throws RemotingException { + masterRpcClient.send(Host.of(message.getMessageReceiverAddress()), message.convert2Command()); + } + + @Override + public TaskExecuteResultMessage buildMessage(TaskExecutionContext taskExecutionContext) { + TaskExecuteResultMessage taskExecuteResultMessage = + new TaskExecuteResultMessage(masterConfig.getMasterAddress(), + taskExecutionContext.getWorkflowInstanceHost(), + System.currentTimeMillis()); + taskExecuteResultMessage.setProcessInstanceId(taskExecutionContext.getProcessInstanceId()); + taskExecuteResultMessage.setTaskInstanceId(taskExecutionContext.getTaskInstanceId()); + taskExecuteResultMessage.setStatus(taskExecutionContext.getCurrentExecutionStatus().getCode()); + taskExecuteResultMessage.setLogPath(taskExecutionContext.getLogPath()); + taskExecuteResultMessage.setExecutePath(taskExecutionContext.getExecutePath()); + taskExecuteResultMessage.setAppIds(taskExecutionContext.getAppIds()); + taskExecuteResultMessage.setProcessId(taskExecutionContext.getProcessId()); + taskExecuteResultMessage.setHost(taskExecutionContext.getHost()); + taskExecuteResultMessage.setStartTime(taskExecutionContext.getStartTime()); + taskExecuteResultMessage.setEndTime(taskExecutionContext.getEndTime()); + taskExecuteResultMessage.setVarPool(taskExecutionContext.getVarPool()); + taskExecuteResultMessage.setExecutePath(taskExecutionContext.getExecutePath()); + return taskExecuteResultMessage; + } + + @Override + public MessageType getMessageType() { + return MessageType.TASK_EXECUTE_RESULT_MESSAGE; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterTaskExecuteRunningMessageSender.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterTaskExecuteRunningMessageSender.java new file mode 100644 index 0000000000..ca846bbb57 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterTaskExecuteRunningMessageSender.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.remote.command.MessageType; +import org.apache.dolphinscheduler.remote.command.task.TaskExecuteRunningMessage; +import org.apache.dolphinscheduler.remote.exceptions.RemotingException; +import org.apache.dolphinscheduler.remote.utils.Host; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; +import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; + +import lombok.NonNull; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Component +public class MasterTaskExecuteRunningMessageSender implements MasterMessageSender { + + @Autowired + private MasterRpcClient masterRpcClient; + + @Autowired + private MasterConfig masterConfig; + + @Override + public void sendMessage(TaskExecuteRunningMessage message) throws RemotingException { + masterRpcClient.send(Host.of(message.getMessageReceiverAddress()), message.convert2Command()); + } + + @Override + public TaskExecuteRunningMessage buildMessage(@NonNull TaskExecutionContext taskExecutionContext) { + TaskExecuteRunningMessage taskExecuteRunningMessage = + new TaskExecuteRunningMessage(masterConfig.getMasterAddress(), + taskExecutionContext.getWorkflowInstanceHost(), + System.currentTimeMillis()); + taskExecuteRunningMessage.setTaskInstanceId(taskExecutionContext.getTaskInstanceId()); + taskExecuteRunningMessage.setProcessInstanceId(taskExecutionContext.getProcessInstanceId()); + taskExecuteRunningMessage.setStatus(taskExecutionContext.getCurrentExecutionStatus()); + taskExecuteRunningMessage.setLogPath(taskExecutionContext.getLogPath()); + taskExecuteRunningMessage.setHost(taskExecutionContext.getHost()); + taskExecuteRunningMessage.setStartTime(taskExecutionContext.getStartTime()); + taskExecuteRunningMessage.setExecutePath(taskExecutionContext.getExecutePath()); + taskExecuteRunningMessage.setAppIds(taskExecutionContext.getAppIds()); + return taskExecuteRunningMessage; + } + + @Override + public MessageType getMessageType() { + return MessageType.TASK_EXECUTE_RUNNING_MESSAGE; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterTaskUpdateRuntimeMessageSender.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterTaskUpdateRuntimeMessageSender.java new file mode 100644 index 0000000000..6af8f3f391 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterTaskUpdateRuntimeMessageSender.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.server.master.runner.message; + +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.remote.command.MessageType; +import org.apache.dolphinscheduler.remote.command.task.TaskUpdateRuntimeMessage; +import org.apache.dolphinscheduler.remote.exceptions.RemotingException; +import org.apache.dolphinscheduler.remote.utils.Host; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; +import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; + +import lombok.NonNull; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Component +public class MasterTaskUpdateRuntimeMessageSender implements MasterMessageSender { + + @Autowired + private MasterRpcClient masterRpcClient; + + @Autowired + private MasterConfig masterConfig; + + @Override + public void sendMessage(TaskUpdateRuntimeMessage message) throws RemotingException { + masterRpcClient.send(Host.of(message.getMessageReceiverAddress()), message.convert2Command()); + } + + @Override + public TaskUpdateRuntimeMessage buildMessage(@NonNull TaskExecutionContext taskExecutionContext) { + TaskUpdateRuntimeMessage taskUpdatePidRequest = + new TaskUpdateRuntimeMessage(masterConfig.getMasterAddress(), + taskExecutionContext.getWorkflowInstanceHost(), + System.currentTimeMillis()); + taskUpdatePidRequest.setTaskInstanceId(taskExecutionContext.getTaskInstanceId()); + taskUpdatePidRequest.setProcessInstanceId(taskExecutionContext.getProcessInstanceId()); + taskUpdatePidRequest.setHost(taskExecutionContext.getHost()); + taskUpdatePidRequest.setStartTime(taskExecutionContext.getStartTime()); + return taskUpdatePidRequest; + } + + @Override + public MessageType getMessageType() { + return MessageType.TASK_UPDATE_RUNTIME_MESSAGE; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskDispatchOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskDispatchOperator.java new file mode 100644 index 0000000000..1280b772e7 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskDispatchOperator.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.runner.operator; + +import org.apache.dolphinscheduler.server.master.runner.GlobalTaskDispatchWaitingQueue; +import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Component +public class TaskDispatchOperator implements TaskOperator { + + @Autowired + private GlobalTaskDispatchWaitingQueue globalTaskDispatchWaitingQueue; + + @Override + public void handle(DefaultTaskExecuteRunnable taskExecuteRunnable) { + globalTaskDispatchWaitingQueue.submitNeedToDispatchTaskExecuteRunnable(taskExecuteRunnable); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskKillOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskKillOperator.java new file mode 100644 index 0000000000..7b95ad767e --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskKillOperator.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.runner.operator; + +import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; +import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; +import org.apache.dolphinscheduler.remote.command.task.TaskKillRequest; +import org.apache.dolphinscheduler.remote.exceptions.RemotingException; +import org.apache.dolphinscheduler.remote.utils.Host; +import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; +import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable; + +import org.apache.commons.lang3.StringUtils; + +import java.util.Date; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class TaskKillOperator implements TaskOperator { + + @Autowired + private TaskInstanceDao taskInstanceDao; + + @Autowired + private MasterRpcClient masterRpcClient; + + @Override + public void handle(DefaultTaskExecuteRunnable taskExecuteRunnable) { + TaskInstance taskInstance = taskExecuteRunnable.getTaskInstance(); + log.info("Begin to kill task instance: {}", taskInstance.getName()); + if (taskInstance.getState().isFinished()) { + log.info("The task stance {} is finished, no need to kill", taskInstance.getName()); + return; + } + try { + killTaskInstanceInDB(taskInstance); + killRemoteTaskInstanceInThreadPool(taskInstance); + } catch (Exception ex) { + // todo: do we need to throw this exception? + log.error("Kill task instance {} failed", taskInstance.getName(), ex); + } + } + + private void killTaskInstanceInDB(TaskInstance taskInstance) { + taskInstance.setState(TaskExecutionStatus.KILL); + taskInstance.setEndTime(new Date()); + taskInstanceDao.updateTaskInstance(taskInstance); + } + + private void killRemoteTaskInstanceInThreadPool(TaskInstance taskInstance) throws RemotingException { + if (StringUtils.isEmpty(taskInstance.getHost())) { + return; + } + TaskKillRequest killCommand = new TaskKillRequest(taskInstance.getId()); + masterRpcClient.send(Host.of(taskInstance.getHost()), killCommand.convert2Command()); + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskOperator.java new file mode 100644 index 0000000000..6667b364e1 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskOperator.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.operator; + +import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable; + +public interface TaskOperator { + + void handle(DefaultTaskExecuteRunnable taskExecuteRunnable); + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskOperatorManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskOperatorManager.java new file mode 100644 index 0000000000..fd647240e0 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskOperatorManager.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.server.master.runner.operator; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Component +public class TaskOperatorManager { + + @Autowired + private TaskKillOperator taskKillOperator; + + @Autowired + private TaskPauseOperator taskPauseOperator; + + @Autowired + private TaskDispatchOperator taskDispatchOperator; + + @Autowired + private TaskTimeoutOperator taskTimeoutOperator; + + public TaskOperator getTaskKillOperator() { + return taskKillOperator; + } + + public TaskPauseOperator getTaskPauseOperator() { + return taskPauseOperator; + } + + public TaskDispatchOperator getTaskDispatchOperator() { + return taskDispatchOperator; + } + + public TaskTimeoutOperator getTaskTimeoutOperator() { + return taskTimeoutOperator; + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskPauseOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskPauseOperator.java new file mode 100644 index 0000000000..656e402adf --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskPauseOperator.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.operator; + +import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.remote.command.task.TaskPauseRequest; +import org.apache.dolphinscheduler.remote.exceptions.RemotingException; +import org.apache.dolphinscheduler.remote.utils.Host; +import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; +import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable; + +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 TaskPauseOperator implements TaskOperator { + + @Autowired + private MasterRpcClient masterRpcClient; + + @Override + public void handle(DefaultTaskExecuteRunnable taskExecuteRunnable) { + try { + pauseRemoteTaskInstanceInThreadPool(taskExecuteRunnable.getTaskInstance()); + } catch (Exception e) { + log.error("Pause MasterTaskExecuteRunnable failed", e); + } + } + + private void pauseRemoteTaskInstanceInThreadPool(TaskInstance taskInstance) throws RemotingException { + if (StringUtils.isEmpty(taskInstance.getHost())) { + log.info("The task instance: {}'s host is null", taskInstance.getName()); + return; + } + masterRpcClient.send(Host.of(taskInstance.getHost()), + new TaskPauseRequest(taskInstance.getId()).convert2Command()); + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskTimeoutOperator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskTimeoutOperator.java new file mode 100644 index 0000000000..741a681d8a --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskTimeoutOperator.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.operator; + +import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; +import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; +import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy; +import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable; + +import java.util.Date; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class TaskTimeoutOperator implements TaskOperator { + + @Autowired + private TaskInstanceDao taskInstanceDao; + + @Override + public void handle(DefaultTaskExecuteRunnable taskExecuteRunnable) { + // Right now, if the task is running in worker, the timeout strategy will be handled at worker side. + // if the task is in master, the timeout strategy will be handled at master side. + // todo: we should unify this, the master only need to handle the timeout strategy. and send request to worker + // to kill the task, if the strategy is timeout_failed. + TaskInstance taskInstance = taskExecuteRunnable.getTaskInstance(); + TaskTimeoutStrategy taskTimeoutStrategy = taskInstance.getTaskDefine().getTimeoutNotifyStrategy(); + if (TaskTimeoutStrategy.FAILED != taskTimeoutStrategy + && TaskTimeoutStrategy.WARNFAILED != taskTimeoutStrategy) { + log.warn("TaskInstance: {} timeout, the current timeout strategy is {}, will continue running", + taskInstance.getName(), taskTimeoutStrategy.name()); + return; + } + log.info("TaskInstance: {} timeout, will kill the task instance", taskInstance.getName()); + taskInstance.setState(TaskExecutionStatus.FAILURE); + taskInstance.setEndTime(new Date()); + taskInstanceDao.upsertTaskInstance(taskInstance); + } +} 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 new file mode 100644 index 0000000000..0d190659f7 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseAsyncLogicTask.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; +import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutionContextHolder; + +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 void kill() { + MasterTaskExecutionContextHolder.removeTaskExecutionContext(taskExecutionContext.getTaskInstanceId()); + } + + public void pause() throws MasterTaskExecuteException { + MasterTaskExecutionContextHolder.removeTaskExecutionContext(taskExecutionContext.getTaskInstanceId()); + } + + @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 new file mode 100644 index 0000000000..10a4ec1e7c --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseSyncLogicTask.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 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 T taskParameters; + + protected BaseSyncLogicTask(TaskExecutionContext taskExecutionContext, T taskParameters) { + this.taskExecutionContext = taskExecutionContext; + 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/BlockingTaskProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BlockingTaskProcessor.java deleted file mode 100644 index 81739e7642..0000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BlockingTaskProcessor.java +++ /dev/null @@ -1,212 +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 static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_BLOCKING; - -import org.apache.dolphinscheduler.common.constants.Constants; -import org.apache.dolphinscheduler.common.enums.BlockingOpportunity; -import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; -import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.common.utils.NetUtils; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -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.parameters.BlockingParameters; -import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameters; -import org.apache.dolphinscheduler.plugin.task.api.utils.DependentUtils; -import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; - -import java.util.ArrayList; -import java.util.Date; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -import com.google.auto.service.AutoService; - -/** - * blocking task processor - */ -@AutoService(ITaskProcessor.class) -public class BlockingTaskProcessor extends BaseTaskProcessor { - - /** - * dependent parameters - */ - private DependentParameters dependentParameters; - - /** - * condition result - */ - private DependResult conditionResult = DependResult.WAITING; - - /** - * blocking parameters - */ - private BlockingParameters blockingParam; - - /** - * complete task map - */ - private Map completeTaskList = new ConcurrentHashMap<>(); - - private void initTaskParameters() { - taskInstance.setLogPath( - LogUtils.getTaskInstanceLogFullPath( - taskInstance.getFirstSubmitTime(), - processInstance.getProcessDefinitionCode(), - processInstance.getProcessDefinitionVersion(), - taskInstance.getProcessInstanceId(), - taskInstance.getId())); - this.taskInstance.setHost(NetUtils.getAddr(masterConfig.getListenPort())); - this.taskInstance.setState(TaskExecutionStatus.RUNNING_EXECUTION); - this.taskInstance.setStartTime(new Date()); - this.taskInstanceDao.upsertTaskInstance(taskInstance); - this.dependentParameters = taskInstance.getDependency(); - this.blockingParam = JSONUtils.parseObject(taskInstance.getTaskParams(), BlockingParameters.class); - } - - @Override - protected boolean pauseTask() { - // todo: task cannot be pause - taskInstance.setState(TaskExecutionStatus.PAUSE); - taskInstance.setEndTime(new Date()); - taskInstanceDao.upsertTaskInstance(taskInstance); - log.info("blocking task has been paused"); - return true; - } - - @Override - protected boolean killTask() { - taskInstance.setState(TaskExecutionStatus.KILL); - taskInstance.setEndTime(new Date()); - taskInstanceDao.upsertTaskInstance(taskInstance); - log.info("blocking task has been killed"); - return true; - } - - @Override - protected boolean taskTimeout() { - return true; - } - - @Override - protected boolean submitTask() { - this.taskInstance = - processService.submitTaskWithRetry(processInstance, taskInstance, maxRetryTimes, commitInterval); - if (this.taskInstance == null) { - return false; - } - this.setTaskExecutionLogger(); - log.info("blocking task submit success"); - return true; - } - - @Override - protected boolean runTask() { - log.info("blocking task starting"); - initTaskParameters(); - if (conditionResult.equals(DependResult.WAITING)) { - setConditionResult(); - endTask(); - } else { - endTask(); - } - log.info("blocking task finished"); - return true; - } - - @Override - protected boolean resubmitTask() { - return true; - } - - @Override - protected boolean dispatchTask() { - return false; - } - - @Override - public String getType() { - return TASK_TYPE_BLOCKING; - } - - /** - * depend result for depend item - */ - private DependResult getDependResultForItem(DependentItem item) { - - DependResult dependResult = DependResult.SUCCESS; - if (!completeTaskList.containsKey(item.getDepTaskCode())) { - log.info("depend item: {} have not completed yet.", item.getDepTaskCode()); - dependResult = DependResult.FAILED; - return dependResult; - } - TaskExecutionStatus executionStatus = completeTaskList.get(item.getDepTaskCode()); - if (executionStatus != item.getStatus()) { - log.info("depend item : {} expect status: {}, actual status: {}", item.getDepTaskCode(), - item.getStatus(), executionStatus); - dependResult = DependResult.FAILED; - } - log.info("dependent item complete {} {},{}", - Constants.DEPENDENT_SPLIT, item.getDepTaskCode(), dependResult); - return dependResult; - } - - private void setConditionResult() { - - List taskInstances = taskInstanceDao - .findValidTaskListByProcessId(taskInstance.getProcessInstanceId(), processInstance.getTestFlag()); - for (TaskInstance task : taskInstances) { - completeTaskList.putIfAbsent(task.getTaskCode(), task.getState()); - } - - List tempResultList = new ArrayList<>(); - for (DependentTaskModel dependentTaskModel : dependentParameters.getDependTaskList()) { - List itemDependResult = new ArrayList<>(); - for (DependentItem item : dependentTaskModel.getDependItemList()) { - itemDependResult.add(getDependResultForItem(item)); - } - DependResult tempResult = - DependentUtils.getDependResultForRelation(dependentTaskModel.getRelation(), itemDependResult); - tempResultList.add(tempResult); - } - conditionResult = DependentUtils.getDependResultForRelation(dependentParameters.getRelation(), tempResultList); - log.info("the blocking task depend result : {}", conditionResult); - } - - private void endTask() { - DependResult expected = this.blockingParam.getBlockingOpportunity() - .equals(BlockingOpportunity.BLOCKING_ON_SUCCESS.getDesc()) - ? DependResult.SUCCESS - : DependResult.FAILED; - boolean isBlocked = (expected == this.conditionResult); - log.info("blocking opportunity: expected-->{}, actual-->{}", expected, this.conditionResult); - processInstance.setBlocked(isBlocked); - if (isBlocked) { - processInstance.setStateWithDesc(WorkflowExecutionStatus.READY_BLOCK, "ready block"); - } - taskInstance.setState(TaskExecutionStatus.SUCCESS); - taskInstance.setEndTime(new Date()); - taskInstanceDao.updateTaskInstance(taskInstance); - log.info("blocking task execute complete, blocking:{}", isBlocked); - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessor.java deleted file mode 100644 index 068e8be084..0000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessor.java +++ /dev/null @@ -1,211 +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.constants.Constants; -import org.apache.dolphinscheduler.common.constants.DataSourceConstants; -import org.apache.dolphinscheduler.common.utils.JSONUtils; -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 org.apache.dolphinscheduler.remote.command.task.TaskKillRequest; -import org.apache.dolphinscheduler.remote.utils.Host; -import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext; -import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType; -import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException; -import org.apache.dolphinscheduler.server.master.dispatch.executor.NettyExecutorManager; -import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; -import org.apache.dolphinscheduler.service.queue.TaskPriority; -import org.apache.dolphinscheduler.service.queue.TaskPriorityQueue; -import org.apache.dolphinscheduler.service.queue.TaskPriorityQueueImpl; - -import org.apache.commons.lang3.StringUtils; - -import java.util.Date; -import java.util.Map; - -import com.fasterxml.jackson.core.type.TypeReference; -import com.google.auto.service.AutoService; - -/** - * common task processor - */ -@AutoService(ITaskProcessor.class) -public class CommonTaskProcessor extends BaseTaskProcessor { - - private TaskPriorityQueue taskUpdateQueue; - - private NettyExecutorManager nettyExecutorManager = SpringApplicationContext.getBean(NettyExecutorManager.class); - - @Override - protected boolean submitTask() { - if (this.taskInstance.getTestFlag() == Constants.TEST_FLAG_YES) { - convertExeEnvironmentOnlineToTest(); - } - - this.taskInstance = - processService.submitTaskWithRetry(processInstance, taskInstance, maxRetryTimes, commitInterval); - - return this.taskInstance != null; - } - - @Override - protected boolean resubmitTask() { - if (this.taskInstance == null) { - return false; - } - setTaskExecutionLogger(); - return dispatchTask(); - } - - @Override - public boolean runTask() { - return true; - } - - @Override - protected boolean taskTimeout() { - return true; - } - - /** - * common task cannot be paused - */ - @Override - protected boolean pauseTask() { - return true; - } - - @Override - public String getType() { - return Constants.COMMON_TASK_TYPE; - } - - @Override - public boolean dispatchTask() { - try { - if (taskUpdateQueue == null) { - this.initQueue(); - } - if (taskInstance.getState().isFinished()) { - log.info("Task {} has already finished, no need to submit to task queue, taskState: {}", - taskInstance.getName(), taskInstance.getState()); - return true; - } - // task cannot be submitted because its execution state is RUNNING or DELAY. - if (taskInstance.getState() == TaskExecutionStatus.RUNNING_EXECUTION - || taskInstance.getState() == TaskExecutionStatus.DELAY_EXECUTION) { - log.info("Task {} is already running or delayed, no need to submit to task queue, taskState: {}", - taskInstance.getName(), taskInstance.getState()); - return true; - } - log.info("Task {} is ready to dispatch to worker", taskInstance.getName()); - - TaskPriority taskPriority = new TaskPriority(processInstance.getProcessInstancePriority().getCode(), - processInstance.getId(), taskInstance.getProcessInstancePriority().getCode(), - taskInstance.getId(), taskInstance.getTaskGroupPriority(), - Constants.DEFAULT_WORKER_GROUP); - - TaskExecutionContext taskExecutionContext = getTaskExecutionContext(taskInstance); - if (taskExecutionContext == null) { - log.error("Get taskExecutionContext fail, task: {}", taskInstance); - return false; - } - - taskPriority.setTaskExecutionContext(taskExecutionContext); - - taskUpdateQueue.put(taskPriority); - log.info("Task {} is submitted to priority queue success by master", taskInstance.getName()); - return true; - } catch (Exception e) { - log.error("Task {} is submitted to priority queue error", taskInstance.getName(), e); - return false; - } - } - - public void initQueue() { - this.taskUpdateQueue = SpringApplicationContext.getBean(TaskPriorityQueueImpl.class); - } - - @Override - public boolean killTask() { - - try { - log.info("Begin to kill task: {}", taskInstance.getName()); - if (taskInstance == null) { - log.warn("Kill task failed, the task instance is not exist"); - return true; - } - if (taskInstance.getState().isFinished()) { - log.warn("Kill task failed, the task instance is already finished"); - return true; - } - // we don't wait the kill response - taskInstance.setState(TaskExecutionStatus.KILL); - taskInstance.setEndTime(new Date()); - taskInstanceDao.updateTaskInstance(taskInstance); - if (StringUtils.isNotEmpty(taskInstance.getHost())) { - killRemoteTask(); - } - } catch (Exception e) { - log.error("Master kill task: {} error, taskInstance id: {}", taskInstance.getName(), - taskInstance.getId(), e); - return false; - } - - log.info("Master success kill task: {}, taskInstanceId: {}", taskInstance.getName(), taskInstance.getId()); - return true; - } - - private void killRemoteTask() throws ExecuteException { - TaskKillRequest killCommand = new TaskKillRequest(); - killCommand.setTaskInstanceId(taskInstance.getId()); - - ExecutionContext executionContext = - new ExecutionContext(killCommand.convert2Command(), ExecutorType.WORKER, taskInstance); - - Host host = Host.of(taskInstance.getHost()); - executionContext.setHost(host); - - nettyExecutorManager.executeDirectly(executionContext); - } - - protected void convertExeEnvironmentOnlineToTest() { - // SQL taskType - if (TaskConstants.TASK_TYPE_SQL.equals(taskInstance.getTaskType())) { - // replace test data source - Map taskDefinitionParams = JSONUtils.parseObject( - taskInstance.getTaskDefine().getTaskParams(), new TypeReference>() { - }); - Map taskInstanceParams = - JSONUtils.parseObject(taskInstance.getTaskParams(), new TypeReference>() { - }); - Integer onlineDataSourceId = (Integer) taskDefinitionParams.get(DataSourceConstants.DATASOURCE); - Integer testDataSourceId = processService.queryTestDataSourceId(onlineDataSourceId); - taskDefinitionParams.put(DataSourceConstants.DATASOURCE, testDataSourceId); - taskInstanceParams.put(DataSourceConstants.DATASOURCE, testDataSourceId); - taskInstance.getTaskDefine().setTaskParams(JSONUtils.toJsonString(taskDefinitionParams)); - taskInstance.setTaskParams(JSONUtils.toJsonString(taskInstanceParams)); - if (null == testDataSourceId) { - log.warn("task name :{}, test data source replacement failed", taskInstance.getName()); - } else { - log.info("task name :{}, test data source replacement succeeded", taskInstance.getName()); - } - } - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ConditionTaskProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ConditionTaskProcessor.java deleted file mode 100644 index 7bdd48d793..0000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ConditionTaskProcessor.java +++ /dev/null @@ -1,199 +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 static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_CONDITIONS; - -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -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.enums.TaskTimeoutStrategy; -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.parameters.DependentParameters; -import org.apache.dolphinscheduler.plugin.task.api.utils.DependentUtils; -import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; - -import java.util.ArrayList; -import java.util.Date; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -import com.google.auto.service.AutoService; - -/** - * condition task processor - */ -@AutoService(ITaskProcessor.class) -public class ConditionTaskProcessor extends BaseTaskProcessor { - - /** - * dependent parameters - */ - private DependentParameters dependentParameters; - - /** - * condition result - */ - private DependResult conditionResult = DependResult.WAITING; - - /** - * complete task map - */ - private Map completeTaskList = new ConcurrentHashMap<>(); - - @Override - public boolean submitTask() { - this.taskInstance = - processService.submitTaskWithRetry(processInstance, taskInstance, maxRetryTimes, commitInterval); - if (this.taskInstance == null) { - return false; - } - this.setTaskExecutionLogger(); - log.info("condition task submit success"); - return true; - } - - @Override - public boolean runTask() { - initTaskParameters(); - log.info("condition task start"); - if (conditionResult.equals(DependResult.WAITING)) { - setConditionResult(); - endTask(); - } else { - endTask(); - } - log.info("condition task finished"); - return true; - } - - @Override - protected boolean resubmitTask() { - return true; - } - - @Override - protected boolean dispatchTask() { - return true; - } - - @Override - protected boolean pauseTask() { - this.taskInstance.setState(TaskExecutionStatus.PAUSE); - this.taskInstance.setEndTime(new Date()); - taskInstanceDao.upsertTaskInstance(taskInstance); - return true; - } - - @Override - protected boolean taskTimeout() { - TaskTimeoutStrategy taskTimeoutStrategy = taskInstance.getTaskDefine().getTimeoutNotifyStrategy(); - if (taskTimeoutStrategy == TaskTimeoutStrategy.WARN) { - return true; - } - log.info("condition task {} timeout, strategy {} ", - taskInstance.getId(), taskTimeoutStrategy.getDescp()); - conditionResult = DependResult.FAILED; - endTask(); - return true; - } - - @Override - protected boolean killTask() { - this.taskInstance.setState(TaskExecutionStatus.KILL); - this.taskInstance.setEndTime(new Date()); - taskInstanceDao.upsertTaskInstance(taskInstance); - return true; - } - - @Override - public String getType() { - return TASK_TYPE_CONDITIONS; - } - - private void initTaskParameters() { - taskInstance.setLogPath( - LogUtils.getTaskInstanceLogFullPath(taskInstance.getFirstSubmitTime(), - processInstance.getProcessDefinitionCode(), - processInstance.getProcessDefinitionVersion(), - taskInstance.getProcessInstanceId(), - taskInstance.getId())); - this.taskInstance.setHost(masterConfig.getMasterAddress()); - taskInstance.setState(TaskExecutionStatus.RUNNING_EXECUTION); - taskInstance.setStartTime(new Date()); - this.taskInstanceDao.upsertTaskInstance(taskInstance); - this.dependentParameters = taskInstance.getDependency(); - } - - private void setConditionResult() { - - List taskInstances = taskInstanceDao - .findValidTaskListByProcessId(taskInstance.getProcessInstanceId(), processInstance.getTestFlag()); - for (TaskInstance task : taskInstances) { - completeTaskList.putIfAbsent(task.getTaskCode(), task.getState()); - } - - List modelResultList = new ArrayList<>(); - for (DependentTaskModel dependentTaskModel : dependentParameters.getDependTaskList()) { - List itemDependResult = new ArrayList<>(); - for (DependentItem item : dependentTaskModel.getDependItemList()) { - itemDependResult.add(getDependResultForItem(item)); - } - DependResult modelResult = - DependentUtils.getDependResultForRelation(dependentTaskModel.getRelation(), itemDependResult); - modelResultList.add(modelResult); - } - conditionResult = DependentUtils.getDependResultForRelation(dependentParameters.getRelation(), modelResultList); - log.info("the conditions task depend result : {}", conditionResult); - } - - /** - * depend result for depend item - */ - private DependResult getDependResultForItem(DependentItem item) { - - DependResult dependResult = DependResult.SUCCESS; - if (!completeTaskList.containsKey(item.getDepTaskCode())) { - log.info("depend item: {} have not completed yet.", item.getDepTaskCode()); - dependResult = DependResult.FAILED; - return dependResult; - } - TaskExecutionStatus executionStatus = completeTaskList.get(item.getDepTaskCode()); - if (executionStatus != item.getStatus()) { - log.info("depend item : {} expect status: {}, actual status: {}", item.getDepTaskCode(), - item.getStatus(), executionStatus); - dependResult = DependResult.FAILED; - } - log.info("dependent item complete, dependentTaskCode: {}, dependResult: {}", item.getDepTaskCode(), - dependResult); - return dependResult; - } - - /** - * - */ - private void endTask() { - TaskExecutionStatus status = - (conditionResult == DependResult.SUCCESS) ? TaskExecutionStatus.SUCCESS : TaskExecutionStatus.FAILURE; - taskInstance.setState(status); - taskInstance.setEndTime(new Date()); - taskInstanceDao.updateTaskInstance(taskInstance); - } -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/DependentTaskProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/DependentTaskProcessor.java deleted file mode 100644 index a7d807cba3..0000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/DependentTaskProcessor.java +++ /dev/null @@ -1,301 +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 static org.apache.dolphinscheduler.common.constants.Constants.DEPENDENT_SPLIT; -import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_DEPENDENT; - -import org.apache.dolphinscheduler.common.constants.Constants; -import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; -import org.apache.dolphinscheduler.dao.entity.Project; -import org.apache.dolphinscheduler.dao.entity.TaskDefinition; -import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper; -import org.apache.dolphinscheduler.dao.mapper.ProjectMapper; -import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper; -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.enums.TaskTimeoutStrategy; -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.parameters.DependentParameters; -import org.apache.dolphinscheduler.plugin.task.api.utils.DependentUtils; -import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; -import org.apache.dolphinscheduler.server.master.utils.DependentExecute; -import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; - -import java.util.ArrayList; -import java.util.Date; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.function.Function; -import java.util.stream.Collectors; - -import com.google.auto.service.AutoService; - -/** - * dependent task processor - */ -@AutoService(ITaskProcessor.class) -public class DependentTaskProcessor extends BaseTaskProcessor { - - private DependentParameters dependentParameters; - - private final ProcessDefinitionMapper processDefinitionMapper = - SpringApplicationContext.getBean(ProcessDefinitionMapper.class); - - private final TaskDefinitionMapper taskDefinitionMapper = - SpringApplicationContext.getBean(TaskDefinitionMapper.class); - - private final ProjectMapper projectMapper = SpringApplicationContext.getBean(ProjectMapper.class); - - /** - * dependent task list - */ - private List dependentTaskList = new ArrayList<>(); - - /** - * depend item result map - * save the result to log file - */ - private Map dependResultMap = new HashMap<>(); - - private Map projectCodeMap = new HashMap<>(); - private Map processDefinitionMap = new HashMap<>(); - private Map taskDefinitionMap = new HashMap<>(); - - /** - * dependent date - */ - private Date dependentDate; - - DependResult result; - - /** - * test flag - */ - private int testFlag; - - boolean allDependentItemFinished; - - @Override - public boolean submitTask() { - try { - this.taskInstance = - processService.submitTaskWithRetry(processInstance, taskInstance, maxRetryTimes, commitInterval); - - if (this.taskInstance == null) { - return false; - } - this.setTaskExecutionLogger(); - log.info("Dependent task submit success"); - taskInstance.setLogPath(LogUtils.getTaskInstanceLogFullPath(taskInstance.getFirstSubmitTime(), - processInstance.getProcessDefinitionCode(), - processInstance.getProcessDefinitionVersion(), - taskInstance.getProcessInstanceId(), - taskInstance.getId())); - taskInstance.setHost(masterConfig.getMasterAddress()); - taskInstance.setState(TaskExecutionStatus.RUNNING_EXECUTION); - taskInstance.setStartTime(new Date()); - taskInstanceDao.updateTaskInstance(taskInstance); - initDependParameters(); - log.info("Success initialize dependent task parameters, the dependent data is: {}", dependentDate); - return true; - } catch (Exception ex) { - log.error("Submit/Initialize dependent task error", ex); - return false; - } - } - - @Override - public boolean runTask() { - if (!allDependentItemFinished) { - allDependentItemFinished = allDependentTaskFinish(); - } - if (allDependentItemFinished) { - getTaskDependResult(); - endTask(); - } - return true; - } - - @Override - protected boolean resubmitTask() { - return true; - } - - @Override - protected boolean dispatchTask() { - return true; - } - - @Override - protected boolean taskTimeout() { - TaskTimeoutStrategy taskTimeoutStrategy = taskInstance.getTaskDefine().getTimeoutNotifyStrategy(); - if (TaskTimeoutStrategy.FAILED != taskTimeoutStrategy - && TaskTimeoutStrategy.WARNFAILED != taskTimeoutStrategy) { - return true; - } - log.info("dependent taskInstanceId: {} timeout, taskName: {}, strategy: {} ", - taskInstance.getId(), taskInstance.getName(), taskTimeoutStrategy.getDescp()); - result = DependResult.FAILED; - endTask(); - return true; - } - - /** - * init dependent parameters - */ - private void initDependParameters() { - this.dependentParameters = taskInstance.getDependency(); - if (processInstance.getScheduleTime() != null) { - this.dependentDate = this.processInstance.getScheduleTime(); - } else { - this.dependentDate = new Date(); - } - this.testFlag = processInstance.getTestFlag(); - // check dependent project is exist - List dependTaskList = dependentParameters.getDependTaskList(); - Set projectCodes = new HashSet<>(); - Set processDefinitionCodes = new HashSet<>(); - Set taskDefinitionCodes = new HashSet<>(); - dependTaskList.forEach(dependentTaskModel -> { - dependentTaskModel.getDependItemList().forEach(dependentItem -> { - projectCodes.add(dependentItem.getProjectCode()); - processDefinitionCodes.add(dependentItem.getDefinitionCode()); - taskDefinitionCodes.add(dependentItem.getDepTaskCode()); - }); - }); - projectCodeMap = projectMapper.queryByCodes(projectCodes).stream() - .collect(Collectors.toMap(Project::getCode, Function.identity())); - processDefinitionMap = processDefinitionMapper.queryByCodes(processDefinitionCodes).stream() - .collect(Collectors.toMap(ProcessDefinition::getCode, Function.identity())); - taskDefinitionMap = taskDefinitionMapper.queryByCodeList(taskDefinitionCodes).stream() - .collect(Collectors.toMap(TaskDefinition::getCode, Function.identity())); - - for (DependentTaskModel taskModel : dependentParameters.getDependTaskList()) { - log.info("Add sub dependent check tasks, dependent relation: {}", taskModel.getRelation()); - for (DependentItem dependentItem : taskModel.getDependItemList()) { - Project project = projectCodeMap.get(dependentItem.getProjectCode()); - if (project == null) { - log.error("The dependent task's project is not exist, dependentItem: {}", dependentItem); - throw new RuntimeException( - "The dependent task's project is not exist, dependentItem: " + dependentItem); - } - ProcessDefinition processDefinition = processDefinitionMap.get(dependentItem.getDefinitionCode()); - if (processDefinition == null) { - log.error("The dependent task's workflow is not exist, dependentItem: {}", dependentItem); - throw new RuntimeException( - "The dependent task's workflow is not exist, dependentItem: " + dependentItem); - } - if (dependentItem.getDepTaskCode() == Constants.DEPENDENT_ALL_TASK_CODE) { - log.info( - "Add dependent task: projectName: {}, workflowName: {}, taskName: ALL, dependentKey: {}", - project.getName(), processDefinition.getName(), dependentItem.getKey()); - - } else { - TaskDefinition taskDefinition = taskDefinitionMap.get(dependentItem.getDepTaskCode()); - if (taskDefinition == null) { - log.error("The dependent task's taskDefinition is not exist, dependentItem: {}", - dependentItem); - throw new RuntimeException( - "The dependent task's taskDefinition is not exist, dependentItem: " + dependentItem); - } - log.info("Add dependent task: projectName: {}, workflowName: {}, taskName: {}, dependentKey: {}", - project.getName(), processDefinition.getName(), taskDefinition.getName(), - dependentItem.getKey()); - } - } - this.dependentTaskList.add(new DependentExecute(taskModel.getDependItemList(), taskModel.getRelation(), - processInstance, taskInstance)); - } - } - - @Override - protected boolean pauseTask() { - this.taskInstance.setState(TaskExecutionStatus.PAUSE); - this.taskInstance.setEndTime(new Date()); - taskInstanceDao.upsertTaskInstance(taskInstance); - return true; - } - - @Override - protected boolean killTask() { - this.taskInstance.setState(TaskExecutionStatus.KILL); - this.taskInstance.setEndTime(new Date()); - taskInstanceDao.upsertTaskInstance(taskInstance); - return true; - } - - /** - * judge all dependent tasks finish - * - * @return whether all dependent tasks finish - */ - private boolean allDependentTaskFinish() { - boolean finish = true; - for (DependentExecute dependentExecute : dependentTaskList) { - for (Map.Entry entry : dependentExecute.getDependResultMap().entrySet()) { - if (!dependResultMap.containsKey(entry.getKey())) { - dependResultMap.put(entry.getKey(), entry.getValue()); - // save depend result to log - log.info("dependent item complete, {} dependentKey: {}, result: {}, dependentDate: {}", - DEPENDENT_SPLIT, entry.getKey(), entry.getValue(), dependentDate); - } - } - if (!dependentExecute.finish(dependentDate, testFlag)) { - finish = false; - } - } - return finish; - } - - /** - * get dependent result - * - * @return DependResult - */ - private DependResult getTaskDependResult() { - List dependResultList = new ArrayList<>(); - for (DependentExecute dependentExecute : dependentTaskList) { - DependResult dependResult = dependentExecute.getModelDependResult(dependentDate, testFlag); - dependResultList.add(dependResult); - } - result = DependentUtils.getDependResultForRelation(this.dependentParameters.getRelation(), dependResultList); - log.info("Dependent task completed, dependent result: {}", result); - return result; - } - - /** - * - */ - private void endTask() { - TaskExecutionStatus status; - status = (result == DependResult.SUCCESS) ? TaskExecutionStatus.SUCCESS : TaskExecutionStatus.FAILURE; - taskInstance.setState(status); - taskInstance.setEndTime(new Date()); - taskInstanceDao.upsertTaskInstance(taskInstance); - } - - @Override - public String getType() { - return TASK_TYPE_DEPENDENT; - } -} 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 new file mode 100644 index 0000000000..88d06ed1e4 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/IAsyncLogicTask.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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/test/java/org/apache/dolphinscheduler/server/master/runner/task/TaskProcessorFactoryTest.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ILogicTask.java similarity index 56% rename from dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/task/TaskProcessorFactoryTest.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ILogicTask.java index 63784f96f4..806792f45a 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/task/TaskProcessorFactoryTest.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ILogicTask.java @@ -17,26 +17,18 @@ package org.apache.dolphinscheduler.server.master.runner.task; -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.exception.MasterTaskExecuteException; -import java.lang.reflect.InvocationTargetException; +public interface ILogicTask { -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Disabled; -import org.junit.jupiter.api.Test; + void pause() throws MasterTaskExecuteException; -@Disabled -public class TaskProcessorFactoryTest { + void kill() throws MasterTaskExecuteException; - @Test - public void testFactory() throws InvocationTargetException, InstantiationException, IllegalAccessException { + AbstractParameters getTaskParameters(); - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setTaskType("shell"); - - ITaskProcessor iTaskProcessor = TaskProcessorFactory.getTaskProcessor(taskInstance.getTaskType()); - - Assertions.assertNotNull(iTaskProcessor); - } + TaskExecutionContext getTaskExecutionContext(); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ITaskProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ILogicTaskPluginFactory.java similarity index 65% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ITaskProcessor.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ILogicTaskPluginFactory.java index de0a0a7c23..08b2285184 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ITaskProcessor.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ILogicTaskPluginFactory.java @@ -17,21 +17,13 @@ package org.apache.dolphinscheduler.server.master.runner.task; -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.spi.plugin.PrioritySPI; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; -/** - * interface of task processor in master - */ -public interface ITaskProcessor extends PrioritySPI { - - void init(TaskInstance taskInstance, ProcessInstance processInstance); - - boolean action(TaskAction taskAction); +public interface ILogicTaskPluginFactory { - String getType(); + T createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException; - TaskInstance taskInstance(); + String getTaskType(); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/TaskAction.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ISyncLogicTask.java similarity index 81% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/TaskAction.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ISyncLogicTask.java index d292cb1d34..ba0df6fc2a 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/TaskAction.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ISyncLogicTask.java @@ -17,15 +17,10 @@ package org.apache.dolphinscheduler.server.master.runner.task; -/** - * task action - */ -public enum TaskAction { - PAUSE, - STOP, - TIMEOUT, - SUBMIT, - RUN, - DISPATCH, - RESUBMIT +import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; + +public interface ISyncLogicTask extends ILogicTask { + + void handle() throws MasterTaskExecuteException; + } 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/runner/task/LogicTaskPluginFactoryBuilder.java new file mode 100644 index 0000000000..7d6cc7260c --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/LogicTaskPluginFactoryBuilder.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.runner.task; + +import org.apache.dolphinscheduler.server.master.exception.LogicTaskFactoryNotFoundException; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.stereotype.Component; + +@Slf4j +@Component +@SuppressWarnings("unchecked") +public class LogicTaskPluginFactoryBuilder { + + private final Map logicTaskPluginFactoryMap = new ConcurrentHashMap<>(); + + public LogicTaskPluginFactoryBuilder(List logicTaskPluginFactories) { + logicTaskPluginFactories.forEach( + logicTaskPluginFactory -> logicTaskPluginFactoryMap.put(logicTaskPluginFactory.getTaskType(), + logicTaskPluginFactory)); + } + + public ILogicTaskPluginFactory createILogicTaskPluginFactory(String taskType) throws LogicTaskFactoryNotFoundException { + ILogicTaskPluginFactory logicTaskPluginFactory = logicTaskPluginFactoryMap.get(taskType); + if (logicTaskPluginFactory == null) { + throw new LogicTaskFactoryNotFoundException("Cannot find the logic task factory: " + taskType); + } + return logicTaskPluginFactory; + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/SubTaskProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/SubTaskProcessor.java deleted file mode 100644 index 76d7356484..0000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/SubTaskProcessor.java +++ /dev/null @@ -1,239 +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 static org.apache.dolphinscheduler.common.constants.Constants.LOCAL_PARAMS; -import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_SUB_PROCESS; - -import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; -import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.common.utils.NetUtils; -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.plugin.task.api.enums.Direct; -import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; -import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy; -import org.apache.dolphinscheduler.plugin.task.api.model.Property; -import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; -import org.apache.dolphinscheduler.remote.command.workflow.WorkflowStateEventChangeRequest; -import org.apache.dolphinscheduler.remote.processor.StateEventCallbackService; -import org.apache.dolphinscheduler.remote.utils.Host; -import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; - -import org.apache.commons.lang3.StringUtils; - -import java.util.Date; -import java.util.List; -import java.util.Map; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; -import java.util.stream.Collectors; - -import com.fasterxml.jackson.core.type.TypeReference; -import com.google.auto.service.AutoService; - -/** - * subtask processor - */ -@AutoService(ITaskProcessor.class) -public class SubTaskProcessor extends BaseTaskProcessor { - - private ProcessInstance subProcessInstance = null; - - /** - * run lock - */ - private final Lock runLock = new ReentrantLock(); - - private StateEventCallbackService stateEventCallbackService = - SpringApplicationContext.getBean(StateEventCallbackService.class); - - @Override - public boolean submitTask() { - this.taskInstance = - processService.submitTaskWithRetry(processInstance, taskInstance, maxRetryTimes, commitInterval); - - if (this.taskInstance == null) { - return false; - } - this.setTaskExecutionLogger(); - taskInstance.setLogPath(LogUtils.getTaskInstanceLogFullPath(taskInstance.getFirstSubmitTime(), - processInstance.getProcessDefinitionCode(), - processInstance.getProcessDefinitionVersion(), - taskInstance.getProcessInstanceId(), - taskInstance.getId())); - - return true; - } - - @Override - public boolean runTask() { - try { - this.runLock.lock(); - if (setSubWorkFlow()) { - updateTaskState(); - } - } catch (Exception e) { - log.error("work flow {} sub task {} exceptions", - this.processInstance.getId(), - this.taskInstance.getId(), - e); - } finally { - this.runLock.unlock(); - } - return true; - } - - @Override - protected boolean resubmitTask() { - return true; - } - - @Override - protected boolean dispatchTask() { - return true; - } - - @Override - protected boolean taskTimeout() { - TaskTimeoutStrategy taskTimeoutStrategy = taskInstance.getTaskDefine().getTimeoutNotifyStrategy(); - if (TaskTimeoutStrategy.FAILED != taskTimeoutStrategy - && TaskTimeoutStrategy.WARNFAILED != taskTimeoutStrategy) { - return true; - } - log.info("sub process task {} timeout, strategy {} ", - taskInstance.getId(), taskTimeoutStrategy.getDescp()); - killTask(); - return true; - } - - private void updateTaskState() { - subProcessInstance = processService.findSubProcessInstance(processInstance.getId(), taskInstance.getId()); - log.info("work flow {} task {}, sub work flow: {} state: {}", - this.processInstance.getId(), - this.taskInstance.getId(), - subProcessInstance.getId(), - subProcessInstance.getState()); - if (subProcessInstance != null && subProcessInstance.getState().isFinished()) { - // todo: check the status and transform - taskInstance.setState(TaskExecutionStatus.of(subProcessInstance.getState().getCode())); - taskInstance.setEndTime(new Date()); - dealFinish(); - taskInstanceDao.upsertTaskInstance(taskInstance); - } - } - - /** - * get the params from subProcessInstance to this subProcessTask - */ - private void dealFinish() { - String thisTaskInstanceVarPool = taskInstance.getVarPool(); - if (StringUtils.isNotEmpty(thisTaskInstanceVarPool)) { - String subProcessInstanceVarPool = subProcessInstance.getVarPool(); - if (StringUtils.isNotEmpty(subProcessInstanceVarPool)) { - List varPoolProperties = JSONUtils.toList(thisTaskInstanceVarPool, Property.class); - Map taskParams = - JSONUtils.parseObject(taskInstance.getTaskParams(), new TypeReference>() { - }); - Object localParams = taskParams.get(LOCAL_PARAMS); - if (localParams != null) { - List properties = JSONUtils.toList(JSONUtils.toJsonString(localParams), Property.class); - Map subProcessParam = - JSONUtils.toList(subProcessInstanceVarPool, Property.class).stream() - .collect(Collectors.toMap(Property::getProp, Property::getValue)); - List outProperties = - properties.stream().filter(r -> Direct.OUT == r.getDirect()).collect(Collectors.toList()); - for (Property info : outProperties) { - info.setValue(subProcessParam.get(info.getProp())); - varPoolProperties.add(info); - } - taskInstance.setVarPool(JSONUtils.toJsonString(varPoolProperties)); - // deal with localParam for show in the page - processService.changeOutParam(taskInstance); - } - } - } - } - - @Override - protected boolean pauseTask() { - pauseSubWorkFlow(); - return true; - } - - private boolean pauseSubWorkFlow() { - ProcessInstance subProcessInstance = - processService.findSubProcessInstance(processInstance.getId(), taskInstance.getId()); - if (subProcessInstance == null || taskInstance.getState().isFinished()) { - return false; - } - subProcessInstance.setStateWithDesc(WorkflowExecutionStatus.READY_PAUSE, "ready pause sub workflow"); - processInstanceDao.updateProcessInstance(subProcessInstance); - sendToSubProcess(); - return true; - } - - private boolean setSubWorkFlow() { - log.info("set work flow {} task {} running", - this.processInstance.getId(), - this.taskInstance.getId()); - if (this.subProcessInstance != null) { - return true; - } - subProcessInstance = processService.findSubProcessInstance(processInstance.getId(), taskInstance.getId()); - if (subProcessInstance == null || taskInstance.getState().isFinished()) { - return false; - } - taskInstance.setHost(NetUtils.getAddr(masterConfig.getListenPort())); - taskInstance.setState(TaskExecutionStatus.RUNNING_EXECUTION); - taskInstance.setStartTime(new Date()); - taskInstanceDao.updateTaskInstance(taskInstance); - log.info("set sub work flow {} task {} state: {}", - processInstance.getId(), - taskInstance.getId(), - taskInstance.getState()); - return true; - - } - - @Override - protected boolean killTask() { - ProcessInstance subProcessInstance = - processService.findSubProcessInstance(processInstance.getId(), taskInstance.getId()); - if (subProcessInstance == null || taskInstance.getState().isFinished()) { - return false; - } - subProcessInstance.setStateWithDesc(WorkflowExecutionStatus.READY_STOP, "ready stop by kill task"); - processInstanceDao.updateProcessInstance(subProcessInstance); - sendToSubProcess(); - return true; - } - - private void sendToSubProcess() { - WorkflowStateEventChangeRequest workflowStateEventChangeRequest = new WorkflowStateEventChangeRequest( - processInstance.getId(), taskInstance.getId(), subProcessInstance.getState(), - subProcessInstance.getId(), 0); - Host host = new Host(subProcessInstance.getHost()); - this.stateEventCallbackService.sendResult(host, workflowStateEventChangeRequest.convert2Command()); - } - - @Override - public String getType() { - return TASK_TYPE_SUB_PROCESS; - } - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/TaskProcessorFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/TaskProcessorFactory.java deleted file mode 100644 index 6da322167a..0000000000 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/TaskProcessorFactory.java +++ /dev/null @@ -1,82 +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 static org.apache.dolphinscheduler.common.constants.Constants.COMMON_TASK_TYPE; - -import org.apache.dolphinscheduler.spi.plugin.PrioritySPIFactory; - -import org.apache.commons.lang3.StringUtils; - -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -import lombok.experimental.UtilityClass; -import lombok.extern.slf4j.Slf4j; - -/** - * the factory to create task processor - */ -@UtilityClass -@Slf4j -public final class TaskProcessorFactory { - - private static final Map> PROCESS_MAP = new ConcurrentHashMap<>(); - - private static final String DEFAULT_PROCESSOR = COMMON_TASK_TYPE; - - static { - PrioritySPIFactory prioritySPIFactory = new PrioritySPIFactory<>(ITaskProcessor.class); - for (Map.Entry entry : prioritySPIFactory.getSPIMap().entrySet()) { - try { - log.info("Registering task processor: {} - {}", entry.getKey(), entry.getValue().getClass()); - PROCESS_MAP.put(entry.getKey(), - (Constructor) entry.getValue().getClass().getConstructor()); - log.info("Registered task processor: {} - {}", entry.getKey(), entry.getValue().getClass()); - } catch (NoSuchMethodException e) { - throw new IllegalArgumentException( - String.format("The task processor: %s should has a no args constructor", entry.getKey())); - } - } - } - - public static ITaskProcessor getTaskProcessor(String type) throws InvocationTargetException, InstantiationException, IllegalAccessException { - if (StringUtils.isEmpty(type)) { - type = DEFAULT_PROCESSOR; - } - Constructor iTaskProcessorConstructor = PROCESS_MAP.get(type); - if (iTaskProcessorConstructor == null) { - iTaskProcessorConstructor = PROCESS_MAP.get(DEFAULT_PROCESSOR); - } - - return iTaskProcessorConstructor.newInstance(); - } - - /** - * if match master processor, then this task type is processed on the master - * - * @param type - * @return - */ - public static boolean isMasterTask(String type) { - return PROCESS_MAP.containsKey(type); - } - -} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/blocking/BlockingLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/blocking/BlockingLogicTask.java new file mode 100644 index 0000000000..c77dc671fd --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/blocking/BlockingLogicTask.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.blocking; + +import org.apache.dolphinscheduler.common.constants.Constants; +import org.apache.dolphinscheduler.common.enums.BlockingOpportunity; +import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; +import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; +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.model.DependentTaskModel; +import org.apache.dolphinscheduler.plugin.task.api.parameters.BlockingParameters; +import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameters; +import org.apache.dolphinscheduler.plugin.task.api.utils.DependentUtils; +import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; +import org.apache.dolphinscheduler.server.master.runner.task.BaseSyncLogicTask; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +import lombok.extern.slf4j.Slf4j; + +import com.fasterxml.jackson.core.type.TypeReference; + +@Slf4j +public class BlockingLogicTask extends BaseSyncLogicTask { + + public static final String TASK_TYPE = "BLOCKING"; + + private final ProcessInstanceExecCacheManager processInstanceExecCacheManager; + + private final ProcessInstanceDao processInstanceDao; + + private final TaskInstanceDao taskInstanceDao; + + public BlockingLogicTask(TaskExecutionContext taskExecutionContext, + ProcessInstanceExecCacheManager processInstanceExecCacheManager, + ProcessInstanceDao processInstanceDao, + TaskInstanceDao taskInstanceDao) { + super(taskExecutionContext, + JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference() { + })); + this.processInstanceExecCacheManager = processInstanceExecCacheManager; + this.processInstanceDao = processInstanceDao; + this.taskInstanceDao = taskInstanceDao; + } + + @Override + public void handle() throws MasterTaskExecuteException { + DependResult conditionResult = calculateConditionResult(); + DependResult expected = taskParameters.getBlockingOpportunity() + .equals(BlockingOpportunity.BLOCKING_ON_SUCCESS.getDesc()) + ? DependResult.SUCCESS + : DependResult.FAILED; + boolean isBlocked = (expected == conditionResult); + log.info("blocking opportunity: expected-->{}, actual-->{}", expected, conditionResult); + ProcessInstance workflowInstance = processInstanceExecCacheManager + .getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()).getProcessInstance(); + workflowInstance.setBlocked(isBlocked); + if (isBlocked) { + workflowInstance.setStateWithDesc(WorkflowExecutionStatus.READY_BLOCK, "ready block"); + } + taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.SUCCESS); + } + + private DependResult calculateConditionResult() throws MasterTaskExecuteException { + // todo: Directly get the task instance from the cache + Map completeTaskList = taskInstanceDao + .findValidTaskListByProcessId(taskExecutionContext.getProcessInstanceId(), + taskExecutionContext.getTestFlag()) + .stream() + .collect(Collectors.toMap(TaskInstance::getTaskCode, Function.identity())); + + // todo: we need to parse the task parameter from TaskExecutionContext + TaskInstance taskInstance = + processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()) + .getTaskInstance(taskExecutionContext.getTaskInstanceId()) + .orElseThrow(() -> new MasterTaskExecuteException("Task instance not found")); + DependentParameters dependentParameters = taskInstance.getDependency(); + + List tempResultList = new ArrayList<>(); + for (DependentTaskModel dependentTaskModel : dependentParameters.getDependTaskList()) { + List itemDependResult = new ArrayList<>(); + for (DependentItem item : dependentTaskModel.getDependItemList()) { + itemDependResult.add(getDependResultForItem(item, completeTaskList)); + } + DependResult tempResult = + DependentUtils.getDependResultForRelation(dependentTaskModel.getRelation(), itemDependResult); + tempResultList.add(tempResult); + } + return DependentUtils.getDependResultForRelation(dependentParameters.getRelation(), tempResultList); + } + + private DependResult getDependResultForItem(DependentItem item, Map completeTaskList) { + + DependResult dependResult = DependResult.SUCCESS; + if (!completeTaskList.containsKey(item.getDepTaskCode())) { + log.info("depend item: {} have not completed yet.", item.getDepTaskCode()); + dependResult = DependResult.FAILED; + return dependResult; + } + TaskInstance taskInstance = completeTaskList.get(item.getDepTaskCode()); + if (taskInstance.getState() != item.getStatus()) { + log.info("depend item : {} expect status: {}, actual status: {}", item.getDepTaskCode(), item.getStatus(), + taskInstance.getState().name()); + dependResult = DependResult.FAILED; + } + log.info("Dependent item complete {} {},{}", + Constants.DEPENDENT_SPLIT, item.getDepTaskCode(), dependResult); + return dependResult; + } + +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/blocking/BlockingLogicTaskPluginFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/blocking/BlockingLogicTaskPluginFactory.java new file mode 100644 index 0000000000..b4fdd56c10 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/blocking/BlockingLogicTaskPluginFactory.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.runner.task.blocking; + +import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; +import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Component +public class BlockingLogicTaskPluginFactory implements ILogicTaskPluginFactory { + + @Autowired + private ProcessInstanceDao processInstanceDao; + + @Autowired + private TaskInstanceDao taskInstanceDao; + + @Autowired + private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + + @Override + public BlockingLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) { + return new BlockingLogicTask(taskExecutionContext, processInstanceExecCacheManager, processInstanceDao, + taskInstanceDao); + } + + @Override + public String getTaskType() { + return BlockingLogicTask.TASK_TYPE; + } +} 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/runner/task/condition/ConditionLogicTask.java new file mode 100644 index 0000000000..b0d989ee2d --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/condition/ConditionLogicTask.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.runner.task.condition; + +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; +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.DependentParameters; +import org.apache.dolphinscheduler.plugin.task.api.utils.DependentUtils; +import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; +import org.apache.dolphinscheduler.server.master.runner.task.BaseSyncLogicTask; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.function.Function; +import java.util.stream.Collectors; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class ConditionLogicTask extends BaseSyncLogicTask { + + public static final String TASK_TYPE = "CONDITIONS"; + + private final TaskInstanceDao taskInstanceDao; + private final ProcessInstanceDao workflowInstanceDao; + + public ConditionLogicTask(TaskExecutionContext taskExecutionContext, + ProcessInstanceExecCacheManager processInstanceExecCacheManager, + TaskInstanceDao taskInstanceDao, + ProcessInstanceDao workflowInstanceDao) throws LogicTaskInitializeException { + // todo: we need to change the parameter in front-end, so that we can directly use json to parse + super(taskExecutionContext, + processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()) + .getTaskInstance(taskExecutionContext.getTaskInstanceId()) + .orElseThrow(() -> new LogicTaskInitializeException( + "Cannot find the task instance in workflow execute runnable")) + .getDependency()); + // todo:check the parameters, why we don't use conditionTask? taskInstance.getDependency(); + this.taskInstanceDao = taskInstanceDao; + this.workflowInstanceDao = workflowInstanceDao; + } + + @Override + public void handle() { + // calculate the conditionResult + DependResult conditionResult = calculateConditionResult(); + TaskExecutionStatus taskExecutionStatus = + (conditionResult == DependResult.SUCCESS) ? TaskExecutionStatus.SUCCESS : TaskExecutionStatus.FAILURE; + log.info("The condition result is {}, task instance statue will be: {}", conditionResult, taskExecutionStatus); + taskExecutionContext.setCurrentExecutionStatus(taskExecutionStatus); + } + + private DependResult calculateConditionResult() { + final ProcessInstance processInstance = + workflowInstanceDao.queryByWorkflowInstanceId(taskExecutionContext.getProcessInstanceId()); + final List taskInstances = + taskInstanceDao.findValidTaskListByProcessId(processInstance.getId(), processInstance.getTestFlag()); + final Map taskInstanceMap = + taskInstances.stream().collect(Collectors.toMap(TaskInstance::getTaskCode, Function.identity())); + + List dependResults = taskParameters.getDependTaskList().stream() + .map(dependentTaskModel -> DependentUtils.getDependResultForRelation( + dependentTaskModel.getRelation(), + dependentTaskModel.getDependItemList() + .stream() + .map(dependentItem -> getDependResultForItem(dependentItem, taskInstanceMap)) + .collect(Collectors.toList()))) + .collect(Collectors.toList()); + return DependentUtils.getDependResultForRelation(taskParameters.getRelation(), dependResults); + } + + private DependResult getDependResultForItem(DependentItem item, Map taskInstanceMap) { + TaskInstance taskInstance = taskInstanceMap.get(item.getDepTaskCode()); + if (taskInstance == null) { + log.info("The depend item: {} has not completed yet", DependResult.FAILED); + log.info("The dependent result will be {}", DependResult.FAILED); + return DependResult.FAILED; + } + + DependResult dependResult = + Objects.equals(item.getStatus(), taskInstance.getState()) ? DependResult.SUCCESS : DependResult.FAILED; + log.info("The depend item: {}", item); + log.info("Expect status: {}", item.getStatus()); + log.info("Actual status: {}", taskInstance.getState()); + log.info("The dependent result will be: {}", dependResult); + return dependResult; + } +} 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/runner/task/condition/ConditionLogicTaskPluginFactory.java new file mode 100644 index 0000000000..d6887df6b5 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/condition/ConditionLogicTaskPluginFactory.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.server.master.runner.task.condition; + +import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; +import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; +import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class ConditionLogicTaskPluginFactory implements ILogicTaskPluginFactory { + + @Autowired + private TaskInstanceDao taskInstanceDao; + @Autowired + private ProcessInstanceDao processInstanceDao; + + @Autowired + private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + + @Override + public ConditionLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException { + return new ConditionLogicTask(taskExecutionContext, processInstanceExecCacheManager, taskInstanceDao, + processInstanceDao); + } + + @Override + public String getTaskType() { + return ConditionLogicTask.TASK_TYPE; + } +} 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/runner/task/dependent/DependentAsyncTaskExecuteFunction.java new file mode 100644 index 0000000000..9cfa8f6b06 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentAsyncTaskExecuteFunction.java @@ -0,0 +1,215 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.dependent; + +import org.apache.dolphinscheduler.common.constants.Constants; +import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.Project; +import org.apache.dolphinscheduler.dao.entity.TaskDefinition; +import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.dao.repository.ProcessDefinitionDao; +import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; +import org.apache.dolphinscheduler.dao.repository.ProjectDao; +import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao; +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.model.DependentItem; +import org.apache.dolphinscheduler.plugin.task.api.model.DependentTaskModel; +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; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class DependentAsyncTaskExecuteFunction implements AsyncTaskExecuteFunction { + + private static final Duration DEPENDENT_TASK_STATE_CHECK_INTERVAL = Duration.ofSeconds(10); + + private final TaskExecutionContext taskExecutionContext; + private final DependentParameters dependentParameters; + private final ProjectDao projectDao; + private final ProcessDefinitionDao processDefinitionDao; + private final TaskDefinitionDao taskDefinitionDao; + private final TaskInstanceDao taskInstanceDao; + + private final ProcessInstance processInstance; + private final Date dependentDate; + private final List dependentTaskList; + private final Map dependResultMap; + + public DependentAsyncTaskExecuteFunction(TaskExecutionContext taskExecutionContext, + DependentParameters dependentParameters, + ProjectDao projectDao, + ProcessDefinitionDao processDefinitionDao, + TaskDefinitionDao taskDefinitionDao, + TaskInstanceDao taskInstanceDao, + ProcessInstanceDao processInstanceDao) { + this.taskExecutionContext = taskExecutionContext; + this.dependentParameters = dependentParameters; + this.projectDao = projectDao; + this.processDefinitionDao = processDefinitionDao; + this.taskDefinitionDao = taskDefinitionDao; + this.taskInstanceDao = taskInstanceDao; + this.processInstance = + processInstanceDao.queryByWorkflowInstanceId(taskExecutionContext.getProcessInstanceId()); + this.dependentDate = calculateDependentDate(); + this.dependentTaskList = initializeDependentTaskList(); + log.info("Initialized dependent task list successfully"); + this.dependResultMap = new HashMap<>(); + } + + @Override + public @NonNull AsyncTaskExecutionStatus getAsyncTaskExecutionStatus() { + if (isAllDependentTaskFinished()) { + log.info("All dependent task finished, will calculate the dependent result"); + DependResult dependResult = calculateDependResult(); + log.info("The Dependent result is: {}", dependResult); + return dependResult == DependResult.SUCCESS ? AsyncTaskExecutionStatus.SUCCESS + : AsyncTaskExecutionStatus.FAILED; + } + return AsyncTaskExecutionStatus.RUNNING; + } + + private Date calculateDependentDate() { + if (processInstance.getScheduleTime() != null) { + return processInstance.getScheduleTime(); + } else { + return new Date(); + } + } + + private List initializeDependentTaskList() { + log.info("Begin to initialize dependent task list"); + final Set projectCodes = new HashSet<>(); + final Set processDefinitionCodes = new HashSet<>(); + final Set taskDefinitionCodes = new HashSet<>(); + for (DependentTaskModel taskModel : dependentParameters.getDependTaskList()) { + for (DependentItem dependentItem : taskModel.getDependItemList()) { + projectCodes.add(dependentItem.getProjectCode()); + processDefinitionCodes.add(dependentItem.getDefinitionCode()); + taskDefinitionCodes.add(dependentItem.getDepTaskCode()); + } + } + + final Map projectCodeMap = projectDao.queryByCodes(new ArrayList<>(projectCodes)).stream() + .collect(Collectors.toMap(Project::getCode, Function.identity())); + final Map processDefinitionMap = + processDefinitionDao.queryByCodes(processDefinitionCodes).stream() + .collect(Collectors.toMap(ProcessDefinition::getCode, Function.identity())); + final Map taskDefinitionMap = taskDefinitionDao.queryByCodes(taskDefinitionCodes).stream() + .collect(Collectors.toMap(TaskDefinition::getCode, Function.identity())); + final TaskInstance taskInstance = + taskInstanceDao.findTaskInstanceById(taskExecutionContext.getTaskInstanceId()); + List dependentExecutes = dependentParameters.getDependTaskList() + .stream() + .map(dependentTaskModel -> { + for (DependentItem dependentItem : dependentTaskModel.getDependItemList()) { + Project project = projectCodeMap.get(dependentItem.getProjectCode()); + if (project == null) { + log.error("The dependent task's project is not exist, dependentItem: {}", dependentItem); + throw new RuntimeException( + "The dependent task's project is not exist, dependentItem: " + dependentItem); + } + ProcessDefinition processDefinition = + processDefinitionMap.get(dependentItem.getDefinitionCode()); + if (processDefinition == null) { + log.error("The dependent task's workflow is not exist, dependentItem: {}", dependentItem); + throw new RuntimeException( + "The dependent task's workflow is not exist, dependentItem: " + dependentItem); + } + if (dependentItem.getDepTaskCode() == Constants.DEPENDENT_ALL_TASK_CODE) { + log.info("Add dependent task:"); + log.info("DependentRelation: {}", dependentTaskModel.getRelation()); + log.info("ProjectName: {}", project.getName()); + log.info("WorkflowName: {}", processDefinition.getName()); + log.info("TaskName: {}", "ALL"); + log.info("DependentKey: {}", dependentItem.getKey()); + } else { + TaskDefinition taskDefinition = taskDefinitionMap.get(dependentItem.getDepTaskCode()); + if (taskDefinition == null) { + log.error("The dependent task's taskDefinition is not exist, dependentItem: {}", + dependentItem); + throw new RuntimeException( + "The dependent task's taskDefinition is not exist, dependentItem: " + + dependentItem); + } + log.info("Add dependent task:"); + log.info("DependentRelation: {}", dependentTaskModel.getRelation()); + log.info("ProjectName: {}", project.getName()); + log.info("WorkflowName: {}", processDefinition.getName()); + log.info("TaskName: {}", taskDefinition.getName()); + log.info("DependentKey: {}", dependentItem.getKey()); + } + } + return new DependentExecute(dependentTaskModel.getDependItemList(), + dependentTaskModel.getRelation(), processInstance, taskInstance); + }).collect(Collectors.toList()); + log.info("Initialized dependent task list"); + return dependentExecutes; + } + + private DependResult calculateDependResult() { + List dependResultList = new ArrayList<>(); + for (DependentExecute dependentExecute : dependentTaskList) { + DependResult dependResult = + dependentExecute.getModelDependResult(dependentDate, processInstance.getTestFlag()); + dependResultList.add(dependResult); + } + return DependentUtils.getDependResultForRelation(this.dependentParameters.getRelation(), + dependResultList); + } + + private boolean isAllDependentTaskFinished() { + boolean isAllDependentTaskFinished = true; + for (DependentExecute dependentExecute : dependentTaskList) { + dependentExecute.getDependResultMap().forEach((dependentKey, dependResult) -> { + if (!dependResultMap.containsKey(dependentKey)) { + dependResultMap.put(dependentKey, dependResult); + log.info("Dependent item check finished: dependentKey: {}, result: {}, dependentDate: {}", + dependentKey, + dependResult, dependentDate); + } + }); + if (!dependentExecute.finish(dependentDate, processInstance.getTestFlag())) { + isAllDependentTaskFinished = false; + } + } + return isAllDependentTaskFinished; + } + + @Override + public @NonNull Duration getAsyncTaskStateCheckInterval() { + return DEPENDENT_TASK_STATE_CHECK_INTERVAL; + } +} 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/runner/task/dependent/DependentLogicTask.java new file mode 100644 index 0000000000..e6ee3a78ca --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTask.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.runner.task.dependent; + +import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.dao.repository.ProcessDefinitionDao; +import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; +import org.apache.dolphinscheduler.dao.repository.ProjectDao; +import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao; +import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; +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.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; +import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; +import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction; +import org.apache.dolphinscheduler.server.master.runner.task.BaseAsyncLogicTask; + +import java.util.Date; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class DependentLogicTask extends BaseAsyncLogicTask { + + public static final String TASK_TYPE = "DEPENDENT"; + + private final ProjectDao projectDao; + private final ProcessDefinitionDao processDefinitionDao; + private final TaskDefinitionDao taskDefinitionDao; + private final TaskInstanceDao taskInstanceDao; + private final ProcessInstanceDao processInstanceDao; + + private final ProcessInstanceExecCacheManager processInstanceExecCacheManager; + + public DependentLogicTask(TaskExecutionContext taskExecutionContext, + ProjectDao projectDao, + ProcessDefinitionDao processDefinitionDao, + TaskDefinitionDao taskDefinitionDao, + TaskInstanceDao taskInstanceDao, + ProcessInstanceDao processInstanceDao, + ProcessInstanceExecCacheManager processInstanceExecCacheManager) throws LogicTaskInitializeException { + super(taskExecutionContext, + processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()) + .getTaskInstance(taskExecutionContext.getTaskInstanceId()) + .orElseThrow(() -> new LogicTaskInitializeException( + "Cannot find the task instance in workflow execute runnable")) + .getDependency()); + this.projectDao = projectDao; + this.processDefinitionDao = processDefinitionDao; + this.taskDefinitionDao = taskDefinitionDao; + this.taskInstanceDao = taskInstanceDao; + this.processInstanceDao = processInstanceDao; + this.processInstanceExecCacheManager = processInstanceExecCacheManager; + + } + + @Override + public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() { + return new DependentAsyncTaskExecuteFunction(taskExecutionContext, + taskParameters, + projectDao, + processDefinitionDao, + taskDefinitionDao, + taskInstanceDao, + processInstanceDao); + } + + @Override + public void pause() throws MasterTaskExecuteException { + WorkflowExecuteRunnable workflowExecuteRunnable = + processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()); + if (workflowExecuteRunnable == null) { + log.error("Cannot find the WorkflowExecuteRunnable"); + return; + } + TaskInstance taskInstance = + workflowExecuteRunnable.getTaskInstance(taskExecutionContext.getTaskInstanceId()).orElse(null); + if (taskInstance == null) { + log.error("Cannot find the TaskInstance in workflowExecuteRunnable"); + return; + } + taskInstance.setState(TaskExecutionStatus.PAUSE); + taskInstance.setEndTime(new Date()); + taskInstanceDao.upsertTaskInstance(taskInstance); + super.pause(); + } + +} 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/runner/task/dependent/DependentLogicTaskPluginFactory.java new file mode 100644 index 0000000000..e8012ac3f9 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/dependent/DependentLogicTaskPluginFactory.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.dependent; + +import org.apache.dolphinscheduler.dao.repository.ProcessDefinitionDao; +import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; +import org.apache.dolphinscheduler.dao.repository.ProjectDao; +import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao; +import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; +import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class DependentLogicTaskPluginFactory implements ILogicTaskPluginFactory { + + @Autowired + private ProjectDao projectDao; + @Autowired + private ProcessDefinitionDao processDefinitionDao; + @Autowired + private TaskDefinitionDao taskDefinitionDao; + @Autowired + private TaskInstanceDao taskInstanceDao; + @Autowired + private ProcessInstanceDao processInstanceDao; + + @Autowired + private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + + @Override + public DependentLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException { + return new DependentLogicTask( + taskExecutionContext, + projectDao, + processDefinitionDao, + taskDefinitionDao, + taskInstanceDao, + processInstanceDao, + processInstanceExecCacheManager); + } + + @Override + public String getTaskType() { + return DependentLogicTask.TASK_TYPE; + } +} 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/runner/task/subworkflow/SubWorkflowAsyncTaskExecuteFunction.java new file mode 100644 index 0000000000..08a4f6836e --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowAsyncTaskExecuteFunction.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.subworkflow; + +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.runner.execute.AsyncTaskExecuteFunction; + +import java.time.Duration; + +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); + + private final TaskExecutionContext taskExecutionContext; + private final ProcessInstanceDao processInstanceDao; + private ProcessInstance subWorkflowInstance; + + public SubWorkflowAsyncTaskExecuteFunction(TaskExecutionContext taskExecutionContext, + ProcessInstanceDao processInstanceDao) { + this.taskExecutionContext = taskExecutionContext; + this.processInstanceDao = processInstanceDao; + } + + @Override + public @NonNull AsyncTaskExecutionStatus getAsyncTaskExecutionStatus() { + // query the status of sub workflow instance + if (subWorkflowInstance == null) { + subWorkflowInstance = processInstanceDao.findSubProcessInstanceByParentId( + taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId()); + } + if (subWorkflowInstance == null) { + log.info("The sub workflow instance doesn't created"); + return AsyncTaskExecutionStatus.RUNNING; + } + subWorkflowInstance = processInstanceDao.queryByWorkflowInstanceId(subWorkflowInstance.getId()); + if (subWorkflowInstance != null && subWorkflowInstance.getState().isFinished()) { + return subWorkflowInstance.getState().isSuccess() ? AsyncTaskExecutionStatus.SUCCESS + : AsyncTaskExecutionStatus.FAILED; + } + return AsyncTaskExecutionStatus.RUNNING; + } + + @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/SubWorkflowLogicTask.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java new file mode 100644 index 0000000000..366fe53cec --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTask.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.subworkflow; + +import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; +import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.plugin.task.api.parameters.SubProcessParameters; +import org.apache.dolphinscheduler.remote.command.workflow.WorkflowStateEventChangeRequest; +import org.apache.dolphinscheduler.remote.exceptions.RemotingException; +import org.apache.dolphinscheduler.remote.utils.Host; +import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; +import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; +import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +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 SubWorkflowLogicTask extends BaseAsyncLogicTask { + + public static final String TASK_TYPE = "SUB_PROCESS"; + private final ProcessInstanceExecCacheManager processInstanceExecCacheManager; + private final ProcessInstanceDao processInstanceDao; + private final MasterRpcClient masterRpcClient; + + public SubWorkflowLogicTask(TaskExecutionContext taskExecutionContext, + ProcessInstanceExecCacheManager processInstanceExecCacheManager, + ProcessInstanceDao processInstanceDao, + MasterRpcClient masterRpcClient) { + super(taskExecutionContext, + JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference() { + })); + this.processInstanceExecCacheManager = processInstanceExecCacheManager; + this.processInstanceDao = processInstanceDao; + this.masterRpcClient = masterRpcClient; + } + + @Override + public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() throws MasterTaskExecuteException { + // todo: create sub workflow instance here? + return new SubWorkflowAsyncTaskExecuteFunction(taskExecutionContext, processInstanceDao); + } + + @Override + public void pause() throws MasterTaskExecuteException { + WorkflowExecuteRunnable workflowExecuteRunnable = + processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()); + if (workflowExecuteRunnable == null) { + log.warn("Cannot find WorkflowExecuteRunnable"); + return; + } + ProcessInstance subProcessInstance = + processInstanceDao.findSubProcessInstanceByParentId(taskExecutionContext.getProcessInstanceId(), + taskExecutionContext.getTaskInstanceId()); + if (subProcessInstance == null) { + log.info("SubWorkflow instance is null"); + return; + } + TaskInstance taskInstance = + workflowExecuteRunnable.getTaskInstance(taskExecutionContext.getTaskInstanceId()).orElse(null); + if (taskInstance == null) { + // we don't need to do this check, the task instance shouldn't be null + log.info("TaskInstance is null"); + return; + } + if (taskInstance.getState().isFinished()) { + log.info("The task instance is finished, no need to pause"); + return; + } + subProcessInstance.setStateWithDesc(WorkflowExecutionStatus.READY_PAUSE, "ready pause sub workflow"); + processInstanceDao.updateProcessInstance(subProcessInstance); + try { + sendToSubProcess(taskExecutionContext, subProcessInstance); + log.info("Success send pause request to SubWorkflow's master: {}", subProcessInstance.getHost()); + } catch (RemotingException e) { + throw new MasterTaskExecuteException(String.format("Send pause request to SubWorkflow's master: %s failed", + subProcessInstance.getHost()), e); + } + } + + @Override + public void kill() { + WorkflowExecuteRunnable workflowExecuteRunnable = + processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()); + if (workflowExecuteRunnable == null) { + log.warn("Cannot find WorkflowExecuteRunnable"); + return; + } + ProcessInstance subProcessInstance = + processInstanceDao.findSubProcessInstanceByParentId(taskExecutionContext.getProcessInstanceId(), + taskExecutionContext.getTaskInstanceId()); + if (subProcessInstance == null) { + log.info("SubWorkflow instance is null"); + return; + } + TaskInstance taskInstance = + workflowExecuteRunnable.getTaskInstance(taskExecutionContext.getTaskInstanceId()).orElse(null); + if (taskInstance == null) { + // we don't need to do this check, the task instance shouldn't be null + log.info("TaskInstance is null"); + return; + } + if (taskInstance.getState().isFinished()) { + log.info("The task instance is finished, no need to pause"); + return; + } + subProcessInstance.setStateWithDesc(WorkflowExecutionStatus.READY_STOP, "ready stop by kill task"); + processInstanceDao.updateProcessInstance(subProcessInstance); + try { + sendToSubProcess(taskExecutionContext, subProcessInstance); + log.info("Success send kill request to SubWorkflow's master: {}", subProcessInstance.getHost()); + } catch (RemotingException e) { + log.error("Send kill request to SubWorkflow's master: {} failed", subProcessInstance.getHost(), e); + } + } + + private void sendToSubProcess(TaskExecutionContext taskExecutionContext, + ProcessInstance subProcessInstance) throws RemotingException { + WorkflowStateEventChangeRequest stateEventChangeCommand = new WorkflowStateEventChangeRequest( + taskExecutionContext.getProcessInstanceId(), + taskExecutionContext.getTaskInstanceId(), + subProcessInstance.getState(), + subProcessInstance.getId(), + 0); + Host host = new Host(subProcessInstance.getHost()); + masterRpcClient.send(host, stateEventChangeCommand.convert2Command()); + } +} 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/runner/task/subworkflow/SubWorkflowLogicTaskPluginFactory.java new file mode 100644 index 0000000000..48b3850d65 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/subworkflow/SubWorkflowLogicTaskPluginFactory.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.server.master.runner.task.subworkflow; + +import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; +import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class SubWorkflowLogicTaskPluginFactory implements ILogicTaskPluginFactory { + + @Autowired + private ProcessInstanceDao processInstanceDao; + @Autowired + private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + @Autowired + private MasterRpcClient masterRpcClient; + + @Override + public SubWorkflowLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) { + return new SubWorkflowLogicTask(taskExecutionContext, processInstanceExecCacheManager, processInstanceDao, + masterRpcClient); + } + + @Override + public String getTaskType() { + return SubWorkflowLogicTask.TASK_TYPE; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/SwitchTaskProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTask.java similarity index 51% rename from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/SwitchTaskProcessor.java rename to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTask.java index c2d78be5bc..47f978ce76 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/SwitchTaskProcessor.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTask.java @@ -15,180 +15,119 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.master.runner.task; - -import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_SWITCH; +package org.apache.dolphinscheduler.server.master.runner.task.switchtask; import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance; +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.Property; import org.apache.dolphinscheduler.plugin.task.api.model.SwitchResultVo; import org.apache.dolphinscheduler.plugin.task.api.parameters.SwitchParameters; -import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; +import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; +import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; +import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; +import org.apache.dolphinscheduler.server.master.runner.task.BaseSyncLogicTask; import org.apache.dolphinscheduler.server.master.utils.SwitchTaskUtils; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; -import java.util.Date; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; -import com.google.auto.service.AutoService; - -/** - * switch task processor - */ -@AutoService(ITaskProcessor.class) -public class SwitchTaskProcessor extends BaseTaskProcessor { - - protected final String rgex = "['\"]*\\$\\{(.*?)\\}['\"]*"; - - /** - * switch result - */ - private DependResult conditionResult; - - @Override - public boolean submitTask() { - this.taskInstance = - processService.submitTaskWithRetry(processInstance, taskInstance, maxRetryTimes, commitInterval); - - if (this.taskInstance == null) { - return false; - } - this.setTaskExecutionLogger(); - log.info("switch task submit success"); - return true; - } - - @Override - public boolean runTask() { - log.info("switch task starting"); - taskInstance.setLogPath( - LogUtils.getTaskInstanceLogFullPath(taskInstance.getFirstSubmitTime(), - processInstance.getProcessDefinitionCode(), - processInstance.getProcessDefinitionVersion(), - taskInstance.getProcessInstanceId(), - taskInstance.getId())); - taskInstance.setHost(masterConfig.getMasterAddress()); - taskInstance.setState(TaskExecutionStatus.RUNNING_EXECUTION); - taskInstance.setStartTime(new Date()); - taskInstanceDao.updateTaskInstance(taskInstance); - - if (!this.taskInstance().getState().isFinished()) { - setSwitchResult(); - } - endTaskState(); - log.info("switch task finished"); - return true; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class SwitchLogicTask extends BaseSyncLogicTask { + + public static final String TASK_TYPE = "SWITCH"; + + private static final String rgex = "['\"]*\\$\\{(.*?)\\}['\"]*"; + + private final ProcessInstance processInstance; + private final TaskInstance taskInstance; + + public SwitchLogicTask(TaskExecutionContext taskExecutionContext, + ProcessInstanceExecCacheManager processInstanceExecCacheManager) throws LogicTaskInitializeException { + super(taskExecutionContext, + // todo: we need to refactor the logic task parameter........ + processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()) + .getTaskInstance(taskExecutionContext.getTaskInstanceId()) + .orElseThrow(() -> new LogicTaskInitializeException( + "Cannot find the task instance in workflow execute runnable")) + .getSwitchDependency()); + WorkflowExecuteRunnable workflowExecuteRunnable = + processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId()); + this.processInstance = workflowExecuteRunnable.getProcessInstance(); + this.taskInstance = workflowExecuteRunnable.getTaskInstance(taskExecutionContext.getTaskInstanceId()) + .orElseThrow(() -> new LogicTaskInitializeException( + "Cannot find the task instance in workflow execute runnable")); } @Override - protected boolean resubmitTask() { - return true; - } - - @Override - protected boolean dispatchTask() { - return true; - } - - @Override - protected boolean pauseTask() { - this.taskInstance.setState(TaskExecutionStatus.PAUSE); - this.taskInstance.setEndTime(new Date()); - taskInstanceDao.upsertTaskInstance(taskInstance); - return true; + public void handle() throws MasterTaskExecuteException { + DependResult conditionResult = calculateConditionResult(); + TaskExecutionStatus status = + (conditionResult == DependResult.SUCCESS) ? TaskExecutionStatus.SUCCESS : TaskExecutionStatus.FAILURE; + log.info("Switch task execute finished, condition result is: {}, task status is: {}", conditionResult, + status.name()); + taskExecutionContext.setCurrentExecutionStatus(status); } - @Override - protected boolean killTask() { - this.taskInstance.setState(TaskExecutionStatus.KILL); - this.taskInstance.setEndTime(new Date()); - taskInstanceDao.upsertTaskInstance(taskInstance); - return true; - } + // todo: don't use depend result, use switch result + private DependResult calculateConditionResult() { + DependResult conditionResult = DependResult.SUCCESS; - @Override - protected boolean taskTimeout() { - return true; - } + List switchResultVos = taskParameters.getDependTaskList(); - @Override - public String getType() { - return TASK_TYPE_SWITCH; - } - - private boolean setSwitchResult() { - List taskInstances = taskInstanceDao.findValidTaskListByProcessId( - taskInstance.getProcessInstanceId(), processInstance.getTestFlag()); - Map completeTaskList = new HashMap<>(); - for (TaskInstance task : taskInstances) { - completeTaskList.putIfAbsent(task.getName(), task.getState()); - } - SwitchParameters switchParameters = taskInstance.getSwitchDependency(); - List switchResultVos = switchParameters.getDependTaskList(); SwitchResultVo switchResultVo = new SwitchResultVo(); - switchResultVo.setNextNode(switchParameters.getNextNode()); + switchResultVo.setNextNode(taskParameters.getNextNode()); switchResultVos.add(switchResultVo); + // todo: refactor these calculate code int finalConditionLocation = switchResultVos.size() - 1; int i = 0; - conditionResult = DependResult.SUCCESS; for (SwitchResultVo info : switchResultVos) { - log.info("the {} execution ", (i + 1)); - log.info("original condition sentence:{}", info.getCondition()); + log.info("Begin to execute {} condition: {} ", (i + 1), info.getCondition()); if (StringUtils.isEmpty(info.getCondition())) { finalConditionLocation = i; break; } String content = setTaskParams(info.getCondition().replaceAll("'", "\""), rgex); - log.info("format condition sentence::{}", content); - Boolean result = null; + log.info("Format condition sentence::{} successfully", content); + Boolean result; try { result = SwitchTaskUtils.evaluate(content); + log.info("Execute condition sentence: {} successfully: {}", content, result); } catch (Exception e) { - log.info("error sentence : {}", content); + log.info("Execute condition sentence: {} failed", content, e); conditionResult = DependResult.FAILED; break; } - log.info("condition result : {}", result); if (result) { finalConditionLocation = i; break; } i++; } - switchParameters.setDependTaskList(switchResultVos); - switchParameters.setResultConditionLocation(finalConditionLocation); - taskInstance.setSwitchDependency(switchParameters); + taskParameters.setDependTaskList(switchResultVos); + taskParameters.setResultConditionLocation(finalConditionLocation); + taskInstance.setSwitchDependency(taskParameters); if (!isValidSwitchResult(switchResultVos.get(finalConditionLocation))) { conditionResult = DependResult.FAILED; - log.error("the switch task depend result is invalid, result:{}, switch branch:{}", conditionResult, + log.error("The switch task depend result is invalid, result:{}, switch branch:{}", conditionResult, finalConditionLocation); - return true; } - log.info("the switch task depend result:{}, switch branch:{}", conditionResult, finalConditionLocation); - return true; - } - - /** - * update task state - */ - private void endTaskState() { - TaskExecutionStatus status = - (conditionResult == DependResult.SUCCESS) ? TaskExecutionStatus.SUCCESS : TaskExecutionStatus.FAILURE; - taskInstance.setEndTime(new Date()); - taskInstance.setState(status); - taskInstanceDao.updateTaskInstance(taskInstance); + log.info("The switch task depend result:{}, switch branch:{}", conditionResult, finalConditionLocation); + return conditionResult; } public String setTaskParams(String content, String rgex) { @@ -222,9 +161,6 @@ public class SwitchTaskProcessor extends BaseTaskProcessor { return content; } - /** - * check whether switch result is valid - */ private boolean isValidSwitchResult(SwitchResultVo switchResult) { if (CollectionUtils.isEmpty(switchResult.getNextNode())) { return false; @@ -236,4 +172,5 @@ public class SwitchTaskProcessor extends BaseTaskProcessor { } return true; } + } 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/runner/task/switchtask/SwitchLogicTaskPluginFactory.java new file mode 100644 index 0000000000..dfb47aa92d --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/switchtask/SwitchLogicTaskPluginFactory.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.switchtask; + +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; +import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; +import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Slf4j +@Component +public class SwitchLogicTaskPluginFactory implements ILogicTaskPluginFactory { + + @Autowired + private ProcessInstanceExecCacheManager processInstanceExecCacheManager; + + @Override + public SwitchLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException { + return new SwitchLogicTask(taskExecutionContext, processInstanceExecCacheManager); + } + + @Override + public String getTaskType() { + return SwitchLogicTask.TASK_TYPE; + } +} diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/WorkerFailoverService.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/WorkerFailoverService.java index 9fa001a222..e75d19d7a8 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/WorkerFailoverService.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/WorkerFailoverService.java @@ -36,7 +36,7 @@ import org.apache.dolphinscheduler.server.master.event.TaskStateEvent; import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThreadPool; -import org.apache.dolphinscheduler.server.master.runner.task.TaskProcessorFactory; +import org.apache.dolphinscheduler.server.master.utils.TaskUtils; import org.apache.dolphinscheduler.service.log.LogClient; import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.utils.ProcessUtils; @@ -163,11 +163,10 @@ public class WorkerFailoverService { */ private void failoverTaskInstance(@NonNull ProcessInstance processInstance, @NonNull TaskInstance taskInstance) { TaskMetrics.incTaskInstanceByState("failover"); - boolean isMasterTask = TaskProcessorFactory.isMasterTask(taskInstance.getTaskType()); taskInstance.setProcessInstance(processInstance); - if (!isMasterTask) { + if (!TaskUtils.isMasterTask(taskInstance.getTaskType())) { log.info("The failover taskInstance is not master task"); TaskExecutionContext taskExecutionContext = TaskExecutionContextBuilder.get() .buildWorkflowInstanceHost(masterConfig.getMasterAddress()) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/SwitchTaskUtils.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/SwitchTaskUtils.java index f63f99d5c6..89a6533fbe 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/SwitchTaskUtils.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/SwitchTaskUtils.java @@ -23,8 +23,8 @@ import javax.script.ScriptException; public class SwitchTaskUtils { - private static ScriptEngineManager manager; - private static ScriptEngine engine; + private static final ScriptEngineManager manager; + private static final ScriptEngine engine; static { manager = new ScriptEngineManager(); @@ -33,7 +33,7 @@ public class SwitchTaskUtils { public static boolean evaluate(String expression) throws ScriptException { Object result = engine.eval(expression); - return (Boolean) result; + return Boolean.TRUE.equals(result); } } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/TaskUtils.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/TaskUtils.java new file mode 100644 index 0000000000..d717e69362 --- /dev/null +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/utils/TaskUtils.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.utils; + +import org.apache.dolphinscheduler.server.master.runner.task.blocking.BlockingLogicTask; +import org.apache.dolphinscheduler.server.master.runner.task.condition.ConditionLogicTask; +import org.apache.dolphinscheduler.server.master.runner.task.dependent.DependentLogicTask; +import org.apache.dolphinscheduler.server.master.runner.task.subworkflow.SubWorkflowLogicTask; +import org.apache.dolphinscheduler.server.master.runner.task.switchtask.SwitchLogicTask; + +import java.util.Set; + +import lombok.experimental.UtilityClass; + +import com.google.common.collect.Sets; + +@UtilityClass +public class TaskUtils { + + // todo: Add to SPI + private final Set MASTER_TASK_TYPES = Sets.newHashSet( + BlockingLogicTask.TASK_TYPE, + ConditionLogicTask.TASK_TYPE, + DependentLogicTask.TASK_TYPE, + SubWorkflowLogicTask.TASK_TYPE, + SwitchLogicTask.TASK_TYPE); + + public boolean isMasterTask(String taskType) { + return MASTER_TASK_TYPES.contains(taskType); + } +} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/BlockingTaskTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/BlockingTaskTest.java index bdf3b0b21a..386e93b156 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/BlockingTaskTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/BlockingTaskTest.java @@ -17,267 +17,229 @@ package org.apache.dolphinscheduler.server.master; -import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_BLOCKING; - -import org.apache.dolphinscheduler.common.enums.TimeoutFlag; -import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; -import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.dao.entity.TaskDefinition; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao; -import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; -import org.apache.dolphinscheduler.plugin.task.api.enums.DependentRelation; -import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; -import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy; -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.parameters.BlockingParameters; -import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameters; -import org.apache.dolphinscheduler.server.master.config.MasterConfig; -import org.apache.dolphinscheduler.server.master.runner.task.BlockingTaskProcessor; -import org.apache.dolphinscheduler.server.master.runner.task.TaskAction; -import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; -import org.apache.dolphinscheduler.service.model.TaskNode; -import org.apache.dolphinscheduler.service.process.ProcessService; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.Date; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.MockedStatic; -import org.mockito.Mockito; - -public class BlockingTaskTest { - - /** - * TaskNode.runFlag : task can be run normally - */ - public static final String FLOW_NODE_RUN_FLAG_NORMAL = "NORMAL"; - - private ProcessService processService; - - private TaskInstanceDao taskInstanceDao; - - private TaskDefinitionDao taskDefinitionDao; - - private ProcessInstance processInstance; - - private MasterConfig config; - - private MockedStatic mockedStaticSpringApplicationContext; - - @BeforeEach - public void before() { - // mock master - config = new MasterConfig(); - config.setTaskCommitRetryTimes(3); - config.setTaskCommitInterval(Duration.ofSeconds(1)); - - mockedStaticSpringApplicationContext = Mockito.mockStatic(SpringApplicationContext.class); - Mockito.when(SpringApplicationContext.getBean(MasterConfig.class)).thenReturn(config); - - // mock process service - processService = Mockito.mock(ProcessService.class); - Mockito.when(SpringApplicationContext.getBean(ProcessService.class)).thenReturn(processService); - - taskInstanceDao = Mockito.mock(TaskInstanceDao.class); - Mockito.when(SpringApplicationContext.getBean(TaskInstanceDao.class)).thenReturn(taskInstanceDao); - - taskDefinitionDao = Mockito.mock(TaskDefinitionDao.class); - Mockito.when(SpringApplicationContext.getBean(TaskDefinitionDao.class)).thenReturn(taskDefinitionDao); - - // mock process instance - processInstance = getProcessInstance(); - Mockito.when(processService - .findProcessInstanceById(processInstance.getId())) - .thenReturn(processInstance); - - TaskDefinition taskDefinition = new TaskDefinition(); - taskDefinition.setTimeoutFlag(TimeoutFlag.OPEN); - taskDefinition.setTimeoutNotifyStrategy(TaskTimeoutStrategy.WARN); - taskDefinition.setTimeout(0); - Mockito.when(taskDefinitionDao.findTaskDefinition(1L, 1)) - .thenReturn(taskDefinition); - } - - @AfterEach - public void after() { - mockedStaticSpringApplicationContext.close(); - } - - private ProcessInstance getProcessInstance() { - // mock process instance - ProcessInstance processInstance = new ProcessInstance(); - processInstance.setId(1000); - processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION); - processInstance.setProcessDefinitionCode(1L); - - return processInstance; - } - - private TaskInstance getTaskInstance(TaskNode taskNode, ProcessInstance processInstance) { - // wrap taskNode - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(100); - taskInstance.setName(taskNode.getName()); - taskInstance.setTaskType(taskNode.getType().toUpperCase()); - taskInstance.setTaskCode(taskNode.getCode()); - taskInstance.setTaskDefinitionVersion(taskNode.getVersion()); - taskInstance.setProcessInstanceId(processInstance.getId()); - taskInstance.setTaskParams(taskNode.getTaskParams()); - taskInstance.setState(TaskExecutionStatus.RUNNING_EXECUTION); - taskInstance.setFirstSubmitTime(new Date()); - Mockito.when(processService - .submitTaskWithRetry(Mockito.any(ProcessInstance.class), Mockito.any(TaskInstance.class), - Mockito.any(Integer.class), Mockito.any(Long.class))) - .thenReturn(taskInstance); - return taskInstance; - } - - private TaskNode getTaskNode(String blockingCondition) { - // mock task nodes - // 1----\ - // 2-----4(Blocking Node) - // 3----/ - // blocking logic: 1-->SUCCESS 2-->SUCCESS 3-->SUCCESS - TaskNode taskNode = new TaskNode(); - taskNode.setId("tasks-1000"); - taskNode.setName("4"); - taskNode.setCode(1L); - taskNode.setVersion(1); - taskNode.setType(TASK_TYPE_BLOCKING); - taskNode.setRunFlag(FLOW_NODE_RUN_FLAG_NORMAL); - - DependentItem dependentItemA = new DependentItem(); - dependentItemA.setDepTaskCode(1L); - dependentItemA.setStatus(TaskExecutionStatus.SUCCESS); - - DependentItem dependentItemB = new DependentItem(); - dependentItemB.setDepTaskCode(2L); - dependentItemB.setStatus(TaskExecutionStatus.SUCCESS); - - DependentItem dependentItemC = new DependentItem(); - dependentItemC.setDepTaskCode(3L); - dependentItemC.setStatus(TaskExecutionStatus.SUCCESS); - - // build relation - DependentTaskModel dependentTaskModel = new DependentTaskModel(); - dependentTaskModel.setDependItemList(Stream.of(dependentItemA, dependentItemB, dependentItemC) - .collect(Collectors.toList())); - dependentTaskModel.setRelation(DependentRelation.AND); - - DependentParameters dependentParameters = new DependentParameters(); - dependentParameters.setDependTaskList(Stream.of(dependentTaskModel).collect(Collectors.toList())); - dependentParameters.setRelation(DependentRelation.AND); - - taskNode.setDependence(JSONUtils.toJsonString(dependentParameters)); - - // set blocking node params - BlockingParameters blockingParameters = new BlockingParameters(); - blockingParameters.setAlertWhenBlocking(false); - blockingParameters.setBlockingCondition(blockingCondition); - - taskNode.setParams(JSONUtils.toJsonString(blockingParameters)); - - return taskNode; - } - - private TaskInstance testBasicInit(String blockingCondition, TaskExecutionStatus... expectResults) { - - TaskInstance taskInstance = getTaskInstance(getTaskNode(blockingCondition), processInstance); - - Mockito.when(processService - .submitTask(processInstance, taskInstance)) - .thenReturn(taskInstance); - - Mockito.when(taskInstanceDao - .findTaskInstanceById(taskInstance.getId())) - .thenReturn(taskInstance); - - // for BlockingTaskExecThread.initTaskParameters - Mockito.when(taskInstanceDao.upsertTaskInstance(taskInstance)) - .thenReturn(true); - - // for BlockingTaskExecThread.updateTaskState - Mockito.when(taskInstanceDao - .updateTaskInstance(taskInstance)) - .thenReturn(true); - - // for BlockingTaskExecThread.waitTaskQuit - List conditions = getTaskInstanceForValidTaskList(expectResults); - Mockito.when( - taskInstanceDao.findValidTaskListByProcessId(processInstance.getId(), processInstance.getTestFlag())) - .thenReturn(conditions); - taskInstance.setProcessInstance(processInstance); - return taskInstance; - } - - /** - * mock task instance and its execution result in front of blocking node - */ - private List getTaskInstanceForValidTaskList(TaskExecutionStatus... status) { - List taskInstanceList = new ArrayList<>(); - for (int i = 1; i <= status.length; i++) { - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(i); - taskInstance.setName(String.valueOf(i)); - taskInstance.setState(status[i - 1]); - taskInstanceList.add(taskInstance); - } - return taskInstanceList; - } - - @Test - public void testBlockingTaskSubmit() { - TaskInstance taskInstance = testBasicInit("BlockingOnFailed", - TaskExecutionStatus.SUCCESS, TaskExecutionStatus.FAILURE, TaskExecutionStatus.SUCCESS); - BlockingTaskProcessor blockingTaskProcessor = new BlockingTaskProcessor(); - blockingTaskProcessor.init(taskInstance, processInstance); - boolean res = blockingTaskProcessor.action(TaskAction.SUBMIT); - Assertions.assertTrue(res); - } - - @Test - public void testPauseTask() { - TaskInstance taskInstance = testBasicInit("BlockingOnFailed", - TaskExecutionStatus.SUCCESS, TaskExecutionStatus.FAILURE, TaskExecutionStatus.SUCCESS); - BlockingTaskProcessor blockingTaskProcessor = new BlockingTaskProcessor(); - blockingTaskProcessor.init(taskInstance, processInstance); - blockingTaskProcessor.action(TaskAction.SUBMIT); - blockingTaskProcessor.action(TaskAction.PAUSE); - TaskExecutionStatus status = taskInstance.getState(); - Assertions.assertEquals(TaskExecutionStatus.PAUSE, status); - } - - @Test - public void testBlocking() { - TaskInstance taskInstance = testBasicInit("BlockingOnFailed", - TaskExecutionStatus.SUCCESS, TaskExecutionStatus.FAILURE, TaskExecutionStatus.SUCCESS); - BlockingTaskProcessor blockingTaskProcessor = new BlockingTaskProcessor(); - blockingTaskProcessor.init(taskInstance, processInstance); - blockingTaskProcessor.action(TaskAction.SUBMIT); - blockingTaskProcessor.action(TaskAction.RUN); - WorkflowExecutionStatus status = processInstance.getState(); - Assertions.assertEquals(WorkflowExecutionStatus.READY_BLOCK, status); - } - - @Test - public void testNoneBlocking() { - TaskInstance taskInstance = testBasicInit("BlockingOnSuccess", - TaskExecutionStatus.SUCCESS, TaskExecutionStatus.SUCCESS, TaskExecutionStatus.SUCCESS); - BlockingTaskProcessor blockingTaskProcessor = new BlockingTaskProcessor(); - blockingTaskProcessor.init(taskInstance, processInstance); - blockingTaskProcessor.action(TaskAction.SUBMIT); - blockingTaskProcessor.action(TaskAction.RUN); - WorkflowExecutionStatus status = processInstance.getState(); - Assertions.assertEquals(WorkflowExecutionStatus.RUNNING_EXECUTION, status); - } -} +// public class BlockingTaskTest { +// +// /** +// * TaskNode.runFlag : task can be run normally +// */ +// public static final String FLOW_NODE_RUN_FLAG_NORMAL = "NORMAL"; +// +// private ProcessService processService; +// +// private TaskInstanceDao taskInstanceDao; +// +// private TaskDefinitionDao taskDefinitionDao; +// +// private ProcessInstance processInstance; +// +// private MasterConfig config; +// +// private MockedStatic mockedStaticSpringApplicationContext; +// +// @BeforeEach +// public void before() { +// // mock master +// config = new MasterConfig(); +// config.setTaskCommitRetryTimes(3); +// config.setTaskCommitInterval(Duration.ofSeconds(1)); +// +// mockedStaticSpringApplicationContext = Mockito.mockStatic(SpringApplicationContext.class); +// Mockito.when(SpringApplicationContext.getBean(MasterConfig.class)).thenReturn(config); +// +// // mock process service +// processService = Mockito.mock(ProcessService.class); +// Mockito.when(SpringApplicationContext.getBean(ProcessService.class)).thenReturn(processService); +// +// taskInstanceDao = Mockito.mock(TaskInstanceDao.class); +// Mockito.when(SpringApplicationContext.getBean(TaskInstanceDao.class)).thenReturn(taskInstanceDao); +// +// taskDefinitionDao = Mockito.mock(TaskDefinitionDao.class); +// Mockito.when(SpringApplicationContext.getBean(TaskDefinitionDao.class)).thenReturn(taskDefinitionDao); +// +// // mock process instance +// processInstance = getProcessInstance(); +// Mockito.when(processService +// .findProcessInstanceById(processInstance.getId())) +// .thenReturn(processInstance); +// +// TaskDefinition taskDefinition = new TaskDefinition(); +// taskDefinition.setTimeoutFlag(TimeoutFlag.OPEN); +// taskDefinition.setTimeoutNotifyStrategy(TaskTimeoutStrategy.WARN); +// taskDefinition.setTimeout(0); +// Mockito.when(taskDefinitionDao.findTaskDefinition(1L, 1)) +// .thenReturn(taskDefinition); +// } +// +// @AfterEach +// public void after() { +// mockedStaticSpringApplicationContext.close(); +// } +// +// private ProcessInstance getProcessInstance() { +// // mock process instance +// ProcessInstance processInstance = new ProcessInstance(); +// processInstance.setId(1000); +// processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION); +// processInstance.setProcessDefinitionCode(1L); +// +// return processInstance; +// } +// +// private TaskInstance getTaskInstance(TaskNode taskNode, ProcessInstance processInstance) { +// // wrap taskNode +// TaskInstance taskInstance = new TaskInstance(); +// taskInstance.setId(100); +// taskInstance.setName(taskNode.getName()); +// taskInstance.setTaskType(taskNode.getType().toUpperCase()); +// taskInstance.setTaskCode(taskNode.getCode()); +// taskInstance.setTaskDefinitionVersion(taskNode.getVersion()); +// taskInstance.setProcessInstanceId(processInstance.getId()); +// taskInstance.setTaskParams(taskNode.getTaskParams()); +// taskInstance.setState(TaskExecutionStatus.RUNNING_EXECUTION); +// taskInstance.setFirstSubmitTime(new Date()); +// Mockito.when(processService +// .submitTaskWithRetry(Mockito.any(ProcessInstance.class), Mockito.any(TaskInstance.class), +// Mockito.any(Integer.class), Mockito.any(Long.class))) +// .thenReturn(taskInstance); +// return taskInstance; +// } +// +// private TaskNode getTaskNode(String blockingCondition) { +// // mock task nodes +// // 1----\ +// // 2-----4(Blocking Node) +// // 3----/ +// // blocking logic: 1-->SUCCESS 2-->SUCCESS 3-->SUCCESS +// TaskNode taskNode = new TaskNode(); +// taskNode.setId("tasks-1000"); +// taskNode.setName("4"); +// taskNode.setCode(1L); +// taskNode.setVersion(1); +// taskNode.setType(TASK_TYPE_BLOCKING); +// taskNode.setRunFlag(FLOW_NODE_RUN_FLAG_NORMAL); +// +// DependentItem dependentItemA = new DependentItem(); +// dependentItemA.setDepTaskCode(1L); +// dependentItemA.setStatus(TaskExecutionStatus.SUCCESS); +// +// DependentItem dependentItemB = new DependentItem(); +// dependentItemB.setDepTaskCode(2L); +// dependentItemB.setStatus(TaskExecutionStatus.SUCCESS); +// +// DependentItem dependentItemC = new DependentItem(); +// dependentItemC.setDepTaskCode(3L); +// dependentItemC.setStatus(TaskExecutionStatus.SUCCESS); +// +// // build relation +// DependentTaskModel dependentTaskModel = new DependentTaskModel(); +// dependentTaskModel.setDependItemList(Stream.of(dependentItemA, dependentItemB, dependentItemC) +// .collect(Collectors.toList())); +// dependentTaskModel.setRelation(DependentRelation.AND); +// +// DependentParameters dependentParameters = new DependentParameters(); +// dependentParameters.setDependTaskList(Stream.of(dependentTaskModel).collect(Collectors.toList())); +// dependentParameters.setRelation(DependentRelation.AND); +// +// taskNode.setDependence(JSONUtils.toJsonString(dependentParameters)); +// +// // set blocking node params +// BlockingParameters blockingParameters = new BlockingParameters(); +// blockingParameters.setAlertWhenBlocking(false); +// blockingParameters.setBlockingCondition(blockingCondition); +// +// taskNode.setParams(JSONUtils.toJsonString(blockingParameters)); +// +// return taskNode; +// } +// +// private TaskInstance testBasicInit(String blockingCondition, TaskExecutionStatus... expectResults) { +// +// TaskInstance taskInstance = getTaskInstance(getTaskNode(blockingCondition), processInstance); +// +// Mockito.when(processService +// .submitTask(processInstance, taskInstance)) +// .thenReturn(taskInstance); +// +// Mockito.when(taskInstanceDao +// .findTaskInstanceById(taskInstance.getId())) +// .thenReturn(taskInstance); +// +// // for BlockingTaskExecThread.initTaskParameters +// Mockito.when(taskInstanceDao.upsertTaskInstance(taskInstance)) +// .thenReturn(true); +// +// // for BlockingTaskExecThread.updateTaskState +// Mockito.when(taskInstanceDao +// .updateTaskInstance(taskInstance)) +// .thenReturn(true); +// +// // for BlockingTaskExecThread.waitTaskQuit +// List conditions = getTaskInstanceForValidTaskList(expectResults); +// Mockito.when( +// taskInstanceDao.findValidTaskListByProcessId(processInstance.getId(), processInstance.getTestFlag())) +// .thenReturn(conditions); +// taskInstance.setProcessInstance(processInstance); +// return taskInstance; +// } +// +// /** +// * mock task instance and its execution result in front of blocking node +// */ +// private List getTaskInstanceForValidTaskList(TaskExecutionStatus... status) { +// List taskInstanceList = new ArrayList<>(); +// for (int i = 1; i <= status.length; i++) { +// TaskInstance taskInstance = new TaskInstance(); +// taskInstance.setId(i); +// taskInstance.setName(String.valueOf(i)); +// taskInstance.setState(status[i - 1]); +// taskInstanceList.add(taskInstance); +// } +// return taskInstanceList; +// } +// +// @Test +// public void testBlockingTaskSubmit() { +// TaskInstance taskInstance = testBasicInit("BlockingOnFailed", +// TaskExecutionStatus.SUCCESS, TaskExecutionStatus.FAILURE, TaskExecutionStatus.SUCCESS); +// BlockingTaskProcessor blockingTaskProcessor = new BlockingTaskProcessor(); +// blockingTaskProcessor.init(taskInstance, processInstance); +// boolean res = blockingTaskProcessor.action(TaskAction.SUBMIT); +// Assertions.assertTrue(res); +// } +// +// @Test +// public void testPauseTask() { +// TaskInstance taskInstance = testBasicInit("BlockingOnFailed", +// TaskExecutionStatus.SUCCESS, TaskExecutionStatus.FAILURE, TaskExecutionStatus.SUCCESS); +// BlockingTaskProcessor blockingTaskProcessor = new BlockingTaskProcessor(); +// blockingTaskProcessor.init(taskInstance, processInstance); +// blockingTaskProcessor.action(TaskAction.SUBMIT); +// blockingTaskProcessor.action(TaskAction.PAUSE); +// TaskExecutionStatus status = taskInstance.getState(); +// Assertions.assertEquals(TaskExecutionStatus.PAUSE, status); +// } +// +// @Test +// public void testBlocking() { +// TaskInstance taskInstance = testBasicInit("BlockingOnFailed", +// TaskExecutionStatus.SUCCESS, TaskExecutionStatus.FAILURE, TaskExecutionStatus.SUCCESS); +// BlockingTaskProcessor blockingTaskProcessor = new BlockingTaskProcessor(); +// blockingTaskProcessor.init(taskInstance, processInstance); +// blockingTaskProcessor.action(TaskAction.SUBMIT); +// blockingTaskProcessor.action(TaskAction.RUN); +// WorkflowExecutionStatus status = processInstance.getState(); +// Assertions.assertEquals(WorkflowExecutionStatus.READY_BLOCK, status); +// } +// +// @Test +// public void testNoneBlocking() { +// TaskInstance taskInstance = testBasicInit("BlockingOnSuccess", +// TaskExecutionStatus.SUCCESS, TaskExecutionStatus.SUCCESS, TaskExecutionStatus.SUCCESS); +// BlockingTaskProcessor blockingTaskProcessor = new BlockingTaskProcessor(); +// blockingTaskProcessor.init(taskInstance, processInstance); +// blockingTaskProcessor.action(TaskAction.SUBMIT); +// blockingTaskProcessor.action(TaskAction.RUN); +// WorkflowExecutionStatus status = processInstance.getState(); +// Assertions.assertEquals(WorkflowExecutionStatus.RUNNING_EXECUTION, status); +// } +// } diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/ConditionsTaskTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/ConditionsTaskTest.java index ae8b6022a0..7b73aa96b6 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/ConditionsTaskTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/ConditionsTaskTest.java @@ -17,176 +17,143 @@ package org.apache.dolphinscheduler.server.master; -import org.apache.dolphinscheduler.common.enums.TimeoutFlag; -import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; -import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.dao.entity.TaskDefinition; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao; -import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; -import org.apache.dolphinscheduler.plugin.task.api.enums.DependentRelation; -import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; -import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy; -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.parameters.ConditionsParameters; -import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameters; -import org.apache.dolphinscheduler.server.master.config.MasterConfig; -import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; -import org.apache.dolphinscheduler.service.model.TaskNode; -import org.apache.dolphinscheduler.service.process.ProcessService; - -import java.time.Duration; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Mockito; -import org.mockito.junit.jupiter.MockitoExtension; -import org.mockito.junit.jupiter.MockitoSettings; -import org.mockito.quality.Strictness; -import org.springframework.context.ApplicationContext; - -@ExtendWith(MockitoExtension.class) -@MockitoSettings(strictness = Strictness.LENIENT) -public class ConditionsTaskTest { - - /** - * TaskNode.runFlag : task can be run normally - */ - public static final String FLOWNODE_RUN_FLAG_NORMAL = "NORMAL"; - - private ProcessService processService; - - private ProcessInstance processInstance; - - private TaskInstanceDao taskInstanceDao; - - private TaskDefinitionDao taskDefinitionDao; - - @BeforeEach - public void before() { - ApplicationContext applicationContext = Mockito.mock(ApplicationContext.class); - SpringApplicationContext springApplicationContext = new SpringApplicationContext(); - springApplicationContext.setApplicationContext(applicationContext); - - MasterConfig config = new MasterConfig(); - Mockito.when(applicationContext.getBean(MasterConfig.class)).thenReturn(config); - config.setTaskCommitRetryTimes(3); - config.setTaskCommitInterval(Duration.ofSeconds(1)); - - processService = Mockito.mock(ProcessService.class); - Mockito.when(applicationContext.getBean(ProcessService.class)).thenReturn(processService); - - taskInstanceDao = Mockito.mock(TaskInstanceDao.class); - Mockito.when(applicationContext.getBean(TaskInstanceDao.class)).thenReturn(taskInstanceDao); - - taskDefinitionDao = Mockito.mock(TaskDefinitionDao.class); - Mockito.when(SpringApplicationContext.getBean(TaskDefinitionDao.class)).thenReturn(taskDefinitionDao); - - processInstance = getProcessInstance(); - Mockito.when(processService - .findProcessInstanceById(processInstance.getId())) - .thenReturn(processInstance); - - TaskDefinition taskDefinition = new TaskDefinition(); - taskDefinition.setTimeoutFlag(TimeoutFlag.OPEN); - taskDefinition.setTimeoutNotifyStrategy(TaskTimeoutStrategy.WARN); - taskDefinition.setTimeout(0); - Mockito.when(taskDefinitionDao.findTaskDefinition(1L, 1)) - .thenReturn(taskDefinition); - } - - private TaskInstance testBasicInit(TaskExecutionStatus expectResult) { - TaskInstance taskInstance = getTaskInstance(getTaskNode(), processInstance); - - // for MasterBaseTaskExecThread.submit - Mockito.when(processService - .submitTask(processInstance, taskInstance)) - .thenReturn(taskInstance); - // for MasterBaseTaskExecThread.call - Mockito.when(taskInstanceDao - .findTaskInstanceById(taskInstance.getId())) - .thenReturn(taskInstance); - // for ConditionsTaskExecThread.initTaskParameters - Mockito.when(taskInstanceDao.upsertTaskInstance(taskInstance)) - .thenReturn(true); - // for ConditionsTaskExecThread.updateTaskState - Mockito.when(taskInstanceDao - .updateTaskInstance(taskInstance)) - .thenReturn(true); - - // for ConditionsTaskExecThread.waitTaskQuit - List conditions = Stream.of( - getTaskInstanceForValidTaskList(expectResult)).collect(Collectors.toList()); - Mockito.when(taskInstanceDao - .findValidTaskListByProcessId(processInstance.getId(), processInstance.getTestFlag())) - .thenReturn(conditions); - return taskInstance; - } - - private TaskNode getTaskNode() { - TaskNode taskNode = new TaskNode(); - taskNode.setId("tasks-1000"); - taskNode.setName("C"); - taskNode.setCode(1L); - taskNode.setVersion(1); - taskNode.setType("CONDITIONS"); - taskNode.setRunFlag(FLOWNODE_RUN_FLAG_NORMAL); - - DependentItem dependentItem = new DependentItem(); - dependentItem.setDepTaskCode(11L); - dependentItem.setStatus(TaskExecutionStatus.SUCCESS); - - DependentTaskModel dependentTaskModel = new DependentTaskModel(); - dependentTaskModel.setDependItemList(Stream.of(dependentItem).collect(Collectors.toList())); - dependentTaskModel.setRelation(DependentRelation.AND); - - DependentParameters dependentParameters = new DependentParameters(); - dependentParameters.setDependTaskList(Stream.of(dependentTaskModel).collect(Collectors.toList())); - dependentParameters.setRelation(DependentRelation.AND); - - // in: AND(AND(1 is SUCCESS)) - taskNode.setDependence(JSONUtils.toJsonString(dependentParameters)); - - ConditionsParameters conditionsParameters = new ConditionsParameters(); - conditionsParameters.setSuccessNode(Stream.of("2").collect(Collectors.toList())); - conditionsParameters.setFailedNode(Stream.of("3").collect(Collectors.toList())); - - // out: SUCCESS => 2, FAILED => 3 - taskNode.setConditionResult(JSONUtils.toJsonString(conditionsParameters)); - - return taskNode; - } - - private ProcessInstance getProcessInstance() { - ProcessInstance processInstance = new ProcessInstance(); - processInstance.setId(1000); - processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION); - - return processInstance; - } - - private TaskInstance getTaskInstance(TaskNode taskNode, ProcessInstance processInstance) { - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(1000); - taskInstance.setName(taskNode.getName()); - taskInstance.setTaskType(taskNode.getType().toUpperCase()); - taskInstance.setTaskCode(taskNode.getCode()); - taskInstance.setTaskDefinitionVersion(taskNode.getVersion()); - taskInstance.setProcessInstanceId(processInstance.getId()); - taskInstance.setTaskParams(taskNode.getTaskParams()); - return taskInstance; - } - - private TaskInstance getTaskInstanceForValidTaskList(TaskExecutionStatus state) { - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(1001); - taskInstance.setName("1"); - taskInstance.setState(state); - return taskInstance; - } -} +// @ExtendWith(MockitoExtension.class) +// @MockitoSettings(strictness = Strictness.LENIENT) +// public class ConditionsTaskTest { +// +// /** +// * TaskNode.runFlag : task can be run normally +// */ +// public static final String FLOWNODE_RUN_FLAG_NORMAL = "NORMAL"; +// +// private ProcessService processService; +// +// private ProcessInstance processInstance; +// +// private TaskInstanceDao taskInstanceDao; +// +// private TaskDefinitionDao taskDefinitionDao; +// +// @BeforeEach +// public void before() { +// ApplicationContext applicationContext = Mockito.mock(ApplicationContext.class); +// SpringApplicationContext springApplicationContext = new SpringApplicationContext(); +// springApplicationContext.setApplicationContext(applicationContext); +// +// MasterConfig config = new MasterConfig(); +// Mockito.when(applicationContext.getBean(MasterConfig.class)).thenReturn(config); +// config.setTaskCommitRetryTimes(3); +// config.setTaskCommitInterval(Duration.ofSeconds(1)); +// +// processService = Mockito.mock(ProcessService.class); +// Mockito.when(applicationContext.getBean(ProcessService.class)).thenReturn(processService); +// +// taskInstanceDao = Mockito.mock(TaskInstanceDao.class); +// Mockito.when(applicationContext.getBean(TaskInstanceDao.class)).thenReturn(taskInstanceDao); +// +// taskDefinitionDao = Mockito.mock(TaskDefinitionDao.class); +// Mockito.when(SpringApplicationContext.getBean(TaskDefinitionDao.class)).thenReturn(taskDefinitionDao); +// +// processInstance = getProcessInstance(); +// Mockito.when(processService +// .findProcessInstanceById(processInstance.getId())) +// .thenReturn(processInstance); +// +// TaskDefinition taskDefinition = new TaskDefinition(); +// taskDefinition.setTimeoutFlag(TimeoutFlag.OPEN); +// taskDefinition.setTimeoutNotifyStrategy(TaskTimeoutStrategy.WARN); +// taskDefinition.setTimeout(0); +// Mockito.when(taskDefinitionDao.findTaskDefinition(1L, 1)) +// .thenReturn(taskDefinition); +// } +// +// private TaskInstance testBasicInit(TaskExecutionStatus expectResult) { +// TaskInstance taskInstance = getTaskInstance(getTaskNode(), processInstance); +// +// // for MasterBaseTaskExecThread.submit +// Mockito.when(processService +// .submitTask(processInstance, taskInstance)) +// .thenReturn(taskInstance); +// // for MasterBaseTaskExecThread.call +// Mockito.when(taskInstanceDao +// .findTaskInstanceById(taskInstance.getId())) +// .thenReturn(taskInstance); +// // for ConditionsTaskExecThread.initTaskParameters +// Mockito.when(taskInstanceDao.upsertTaskInstance(taskInstance)) +// .thenReturn(true); +// // for ConditionsTaskExecThread.updateTaskState +// Mockito.when(taskInstanceDao +// .updateTaskInstance(taskInstance)) +// .thenReturn(true); +// +// // for ConditionsTaskExecThread.waitTaskQuit +// List conditions = Stream.of( +// getTaskInstanceForValidTaskList(expectResult)).collect(Collectors.toList()); +// Mockito.when(taskInstanceDao +// .findValidTaskListByProcessId(processInstance.getId(), processInstance.getTestFlag())) +// .thenReturn(conditions); +// return taskInstance; +// } +// +// private TaskNode getTaskNode() { +// TaskNode taskNode = new TaskNode(); +// taskNode.setId("tasks-1000"); +// taskNode.setName("C"); +// taskNode.setCode(1L); +// taskNode.setVersion(1); +// taskNode.setType("CONDITIONS"); +// taskNode.setRunFlag(FLOWNODE_RUN_FLAG_NORMAL); +// +// DependentItem dependentItem = new DependentItem(); +// dependentItem.setDepTaskCode(11L); +// dependentItem.setStatus(TaskExecutionStatus.SUCCESS); +// +// DependentTaskModel dependentTaskModel = new DependentTaskModel(); +// dependentTaskModel.setDependItemList(Stream.of(dependentItem).collect(Collectors.toList())); +// dependentTaskModel.setRelation(DependentRelation.AND); +// +// DependentParameters dependentParameters = new DependentParameters(); +// dependentParameters.setDependTaskList(Stream.of(dependentTaskModel).collect(Collectors.toList())); +// dependentParameters.setRelation(DependentRelation.AND); +// +// // in: AND(AND(1 is SUCCESS)) +// taskNode.setDependence(JSONUtils.toJsonString(dependentParameters)); +// +// ConditionsParameters conditionsParameters = new ConditionsParameters(); +// conditionsParameters.setSuccessNode(Stream.of("2").collect(Collectors.toList())); +// conditionsParameters.setFailedNode(Stream.of("3").collect(Collectors.toList())); +// +// // out: SUCCESS => 2, FAILED => 3 +// taskNode.setConditionResult(JSONUtils.toJsonString(conditionsParameters)); +// +// return taskNode; +// } +// +// private ProcessInstance getProcessInstance() { +// ProcessInstance processInstance = new ProcessInstance(); +// processInstance.setId(1000); +// processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION); +// +// return processInstance; +// } +// +// private TaskInstance getTaskInstance(TaskNode taskNode, ProcessInstance processInstance) { +// TaskInstance taskInstance = new TaskInstance(); +// taskInstance.setId(1000); +// taskInstance.setName(taskNode.getName()); +// taskInstance.setTaskType(taskNode.getType().toUpperCase()); +// taskInstance.setTaskCode(taskNode.getCode()); +// taskInstance.setTaskDefinitionVersion(taskNode.getVersion()); +// taskInstance.setProcessInstanceId(processInstance.getId()); +// taskInstance.setTaskParams(taskNode.getTaskParams()); +// return taskInstance; +// } +// +// private TaskInstance getTaskInstanceForValidTaskList(TaskExecutionStatus state) { +// TaskInstance taskInstance = new TaskInstance(); +// taskInstance.setId(1001); +// taskInstance.setName("1"); +// taskInstance.setState(state); +// return taskInstance; +// } +// } diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/DependentTaskTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/DependentTaskTest.java index 5a5f04a474..013ae6ae9b 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/DependentTaskTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/DependentTaskTest.java @@ -17,462 +17,424 @@ package org.apache.dolphinscheduler.server.master; -import org.apache.dolphinscheduler.common.constants.Constants; -import org.apache.dolphinscheduler.common.enums.TimeoutFlag; -import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; -import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager; -import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.dao.entity.TaskDefinition; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; -import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao; -import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; -import org.apache.dolphinscheduler.plugin.task.api.enums.DependResult; -import org.apache.dolphinscheduler.plugin.task.api.enums.DependentRelation; -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.parameters.DependentParameters; -import org.apache.dolphinscheduler.server.master.config.MasterConfig; -import org.apache.dolphinscheduler.server.master.utils.DependentExecute; -import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; -import org.apache.dolphinscheduler.service.model.TaskNode; -import org.apache.dolphinscheduler.service.process.ProcessService; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -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.Mockito; -import org.mockito.junit.jupiter.MockitoExtension; -import org.mockito.junit.jupiter.MockitoSettings; -import org.mockito.quality.Strictness; -import org.springframework.context.ApplicationContext; - /** * DependentTaskTest */ -@ExtendWith(MockitoExtension.class) -@MockitoSettings(strictness = Strictness.LENIENT) -public class DependentTaskTest { - - /** - * TaskNode.runFlag : task can be run normally - */ - public static final String FLOWNODE_RUN_FLAG_NORMAL = "NORMAL"; - - public static final Long TASK_CODE = 1111L; - public static final Long DEPEND_TASK_CODE_A = 110L; - public static final Long DEPEND_TASK_CODE_B = 111L; - public static final Long DEPEND_TASK_CODE_C = 112L; - public static final Long DEPEND_TASK_CODE_D = 113L; - public static final int TASK_VERSION = 1; - - private ProcessService processService; - - private ProcessInstanceDao processInstanceDao; - - private TaskInstanceDao taskInstanceDao; - - private TaskDefinitionDao taskDefinitionDao; - - /** - * the dependent task to be tested - * ProcessDefinition id=1 - * Task id=task-10, name=D - * ProcessInstance id=100 - * TaskInstance id=1000 - * notice: must be initialized by setupTaskInstance() on each test case - */ - private ProcessInstance processInstance; - private TaskInstance taskInstance; - - @BeforeEach - public void before() { - ApplicationContext applicationContext = Mockito.mock(ApplicationContext.class); - SpringApplicationContext springApplicationContext = new SpringApplicationContext(); - springApplicationContext.setApplicationContext(applicationContext); - - MasterConfig config = new MasterConfig(); - config.setTaskCommitRetryTimes(3); - config.setTaskCommitInterval(Duration.ofSeconds(1)); - Mockito.when(applicationContext.getBean(MasterConfig.class)).thenReturn(config); - - processService = Mockito.mock(ProcessService.class); - Mockito.when(applicationContext.getBean(ProcessService.class)).thenReturn(processService); - - processInstanceDao = Mockito.mock(ProcessInstanceDao.class); - Mockito.when(applicationContext.getBean(ProcessInstanceDao.class)).thenReturn(processInstanceDao); - - taskInstanceDao = Mockito.mock(TaskInstanceDao.class); - Mockito.when(applicationContext.getBean(TaskInstanceDao.class)).thenReturn(taskInstanceDao); - - taskDefinitionDao = Mockito.mock(TaskDefinitionDao.class); - Mockito.when(SpringApplicationContext.getBean(TaskDefinitionDao.class)).thenReturn(taskDefinitionDao); - - processInstance = getProcessInstance(); - taskInstance = getTaskInstance(); - - // for MasterBaseTaskExecThread.call - // for DependentTaskExecThread.waitTaskQuit - Mockito.when(processService - .findProcessInstanceById(100)) - .thenAnswer(i -> processInstance); - - // for MasterBaseTaskExecThread.submit - Mockito.when(processService - .submitTask(processInstance, taskInstance)) - .thenAnswer(i -> taskInstance); - - // for DependentTaskExecThread.initTaskParameters - Mockito.when(taskInstanceDao - .updateTaskInstance(Mockito.any())) - .thenReturn(true); - // for DependentTaskExecThread.updateTaskState - Mockito.when(taskInstanceDao.upsertTaskInstance(Mockito.any())) - .thenReturn(true); - - // for DependentTaskExecThread.waitTaskQuit - Mockito.when(taskInstanceDao - .findTaskInstanceById(1000)) - .thenAnswer(i -> taskInstance); - - Mockito.when(taskDefinitionDao.findTaskDefinition(TASK_CODE, TASK_VERSION)) - .thenReturn(getTaskDefinition()); - } - - private void testBasicInit() { - TaskNode taskNode = getDependantTaskNode(); - DependentTaskModel dependentTaskModel = new DependentTaskModel(); - dependentTaskModel.setRelation(DependentRelation.AND); - dependentTaskModel.setDependItemList(Stream.of( - getDependentItemFromTaskNode(2L, DEPEND_TASK_CODE_A, "today", "day")).collect(Collectors.toList())); - - DependentParameters dependentParameters = new DependentParameters(); - dependentParameters.setRelation(DependentRelation.AND); - dependentParameters.setDependTaskList(Stream.of(dependentTaskModel).collect(Collectors.toList())); - - // dependence: AND(AND(2-A-day-today)) - taskNode.setDependence(JSONUtils.toJsonString(dependentParameters)); - - setupTaskInstance(taskNode); - } - - @Test - public void testBasicSuccess() { - testBasicInit(); - ProcessInstance dependentProcessInstance = - getProcessInstanceForFindLastRunningProcess(200, WorkflowExecutionStatus.FAILURE); - // for DependentExecute.findLastProcessInterval - Mockito.when(processInstanceDao - .findLastRunningProcess(Mockito.eq(2L), Mockito.any(), Mockito.any(), Mockito.anyInt())) - .thenReturn(dependentProcessInstance); - - // for DependentExecute.getDependTaskResult - Mockito.when(taskInstanceDao - .findValidTaskListByProcessId(200, 0)) - .thenReturn(Stream.of( - getTaskInstanceForValidTaskList(2000, TaskExecutionStatus.SUCCESS, DEPEND_TASK_CODE_A, - dependentProcessInstance), - getTaskInstanceForValidTaskList(2000, TaskExecutionStatus.FAILURE, DEPEND_TASK_CODE_B, - dependentProcessInstance)) - .collect(Collectors.toList())); - - } - - @Test - public void testBasicFailure() { - testBasicInit(); - ProcessInstance dependentProcessInstance = - getProcessInstanceForFindLastRunningProcess(200, WorkflowExecutionStatus.SUCCESS); - // for DependentExecute.findLastProcessInterval - Mockito.when(processInstanceDao - .findLastRunningProcess(Mockito.eq(2L), Mockito.any(), Mockito.any(), Mockito.anyInt())) - .thenReturn(dependentProcessInstance); - - // for DependentExecute.getDependTaskResult - Mockito.when(taskInstanceDao - .findValidTaskListByProcessId(200, 0)) - .thenReturn(Stream.of( - getTaskInstanceForValidTaskList(2000, TaskExecutionStatus.FAILURE, DEPEND_TASK_CODE_A, - dependentProcessInstance), - getTaskInstanceForValidTaskList(2000, TaskExecutionStatus.SUCCESS, DEPEND_TASK_CODE_B, - dependentProcessInstance)) - .collect(Collectors.toList())); - } - - @Test - public void testDependentRelation() { - DependentTaskModel dependentTaskModel1 = new DependentTaskModel(); - dependentTaskModel1.setRelation(DependentRelation.AND); - dependentTaskModel1.setDependItemList(Stream.of( - getDependentItemFromTaskNode(2L, DEPEND_TASK_CODE_A, "today", "day"), - getDependentItemFromTaskNode(3L, DEPEND_TASK_CODE_B, "today", "day")).collect(Collectors.toList())); - - DependentTaskModel dependentTaskModel2 = new DependentTaskModel(); - dependentTaskModel2.setRelation(DependentRelation.OR); - dependentTaskModel2.setDependItemList(Stream.of( - getDependentItemFromTaskNode(2L, DEPEND_TASK_CODE_A, "today", "day"), - getDependentItemFromTaskNode(3L, DEPEND_TASK_CODE_C, "today", "day")).collect(Collectors.toList())); - - /* - * OR AND 2-A-day-today 3-B-day-today OR 2-A-day-today 3-C-day-today - */ - DependentParameters dependentParameters = new DependentParameters(); - dependentParameters.setRelation(DependentRelation.OR); - dependentParameters.setDependTaskList(Stream.of( - dependentTaskModel1, - dependentTaskModel2).collect(Collectors.toList())); - - TaskNode taskNode = getDependantTaskNode(); - taskNode.setDependence(JSONUtils.toJsonString(dependentParameters)); - setupTaskInstance(taskNode); - - ProcessInstance processInstance200 = - getProcessInstanceForFindLastRunningProcess(200, WorkflowExecutionStatus.FAILURE); - ProcessInstance processInstance300 = - getProcessInstanceForFindLastRunningProcess(300, WorkflowExecutionStatus.SUCCESS); - - // for DependentExecute.findLastProcessInterval - Mockito.when(processInstanceDao - .findLastRunningProcess(Mockito.eq(2L), Mockito.any(), Mockito.any(), Mockito.anyInt())) - .thenReturn(processInstance200); - Mockito.when(processInstanceDao - .findLastRunningProcess(Mockito.eq(3L), Mockito.any(), Mockito.any(), Mockito.anyInt())) - .thenReturn(processInstance300); - - // for DependentExecute.getDependTaskResult - Mockito.when(taskInstanceDao - .findValidTaskListByProcessId(200, 0)) - .thenReturn(Stream.of( - getTaskInstanceForValidTaskList(2000, TaskExecutionStatus.FAILURE, DEPEND_TASK_CODE_A, - processInstance200)) - .collect(Collectors.toList())); - Mockito.when(taskInstanceDao - .findValidTaskListByProcessId(300, 0)) - .thenReturn(Stream.of( - getTaskInstanceForValidTaskList(3000, TaskExecutionStatus.SUCCESS, DEPEND_TASK_CODE_B, - processInstance300), - getTaskInstanceForValidTaskList(3001, TaskExecutionStatus.SUCCESS, DEPEND_TASK_CODE_C, - processInstance300)) - .collect(Collectors.toList())); - - } - - /** - * test when dependent on ALL tasks in another process - */ - private void testDependentOnAllInit() { - TaskNode taskNode = getDependantTaskNode(); - DependentTaskModel dependentTaskModel = new DependentTaskModel(); - dependentTaskModel.setRelation(DependentRelation.AND); - dependentTaskModel.setDependItemList(Stream.of( - getDependentItemFromTaskNode(2L, Constants.DEPENDENT_ALL_TASK_CODE, "today", "day")) - .collect(Collectors.toList())); - - DependentParameters dependentParameters = new DependentParameters(); - dependentParameters.setRelation(DependentRelation.AND); - dependentParameters.setDependTaskList(Stream.of(dependentTaskModel).collect(Collectors.toList())); - - // dependence: AND(AND(2:ALL today day)) - taskNode.setDependence(JSONUtils.toJsonString(dependentParameters)); - - setupTaskInstance(taskNode); - } - - @Test - public void testDependentOnAllSuccess() { - testDependentOnAllInit(); - // for DependentExecute.findLastProcessInterval - Mockito.when(processInstanceDao - .findLastRunningProcess(Mockito.eq(2L), Mockito.any(), Mockito.any(), Mockito.anyInt())) - .thenReturn(getProcessInstanceForFindLastRunningProcess(200, WorkflowExecutionStatus.SUCCESS)); - - } - - @Test - public void testDependentOnAllFailure() { - testDependentOnAllInit(); - // for DependentExecute.findLastProcessInterval - Mockito.when(processInstanceDao - .findLastRunningProcess(Mockito.eq(2L), Mockito.any(), Mockito.any(), Mockito.anyInt())) - .thenReturn(getProcessInstanceForFindLastRunningProcess(200, WorkflowExecutionStatus.FAILURE)); - - } - - /** - * test whether waitTaskQuit has been well impl - */ - @Test - public void testWaitAndCancel() { - // for the poor independence of UT, error on other place may causes the condition happens - if (!ServerLifeCycleManager.isRunning()) { - return; - } - - TaskNode taskNode = getDependantTaskNode(); - DependentTaskModel dependentTaskModel = new DependentTaskModel(); - dependentTaskModel.setRelation(DependentRelation.AND); - dependentTaskModel.setDependItemList(Stream.of( - getDependentItemFromTaskNode(2L, DEPEND_TASK_CODE_A, "today", "day")).collect(Collectors.toList())); - - DependentParameters dependentParameters = new DependentParameters(); - dependentParameters.setRelation(DependentRelation.AND); - dependentParameters.setDependTaskList(Stream.of(dependentTaskModel).collect(Collectors.toList())); - - // dependence: AND(AND(2:A today day)) - taskNode.setDependence(JSONUtils.toJsonString(dependentParameters)); - - setupTaskInstance(taskNode); - - ProcessInstance dependentProcessInstance = - getProcessInstanceForFindLastRunningProcess(200, WorkflowExecutionStatus.RUNNING_EXECUTION); - // for DependentExecute.findLastProcessInterval - Mockito.when(processInstanceDao - .findLastRunningProcess(Mockito.eq(2L), Mockito.any(), Mockito.any(), Mockito.anyInt())) - .thenReturn(dependentProcessInstance); - - // DependentTaskExecThread taskExecThread = new DependentTaskExecThread(taskInstance); - - // for DependentExecute.getDependTaskResult - Mockito.when(taskInstanceDao - .findValidTaskListByProcessId(200, 0)) - .thenAnswer(i -> { - processInstance.setState(WorkflowExecutionStatus.READY_STOP); - return Stream.of( - getTaskInstanceForValidTaskList(2000, TaskExecutionStatus.RUNNING_EXECUTION, - DEPEND_TASK_CODE_A, dependentProcessInstance)) - .collect(Collectors.toList()); - }) - .thenThrow(new IllegalStateException("have not been stopped as expected")); - - } - - @Test - public void testIsSelfDependent() { - DependentExecute dependentExecute = - new DependentExecute(new ArrayList<>(), DependentRelation.AND, processInstance, taskInstance); - DependentItem dependentItem = new DependentItem(); - dependentItem.setDefinitionCode(processInstance.getProcessDefinitionCode()); - dependentItem.setDepTaskCode(Constants.DEPENDENT_ALL_TASK_CODE); - Assertions.assertTrue(dependentExecute.isSelfDependent(dependentItem)); - - dependentItem.setDepTaskCode(taskInstance.getTaskCode()); - Assertions.assertTrue(dependentExecute.isSelfDependent(dependentItem)); - - // no self task - dependentItem.setDepTaskCode(12345678); - Assertions.assertFalse(dependentExecute.isSelfDependent(dependentItem)); - - // no self wf - dependentItem.setDefinitionCode(processInstance.getProcessDefinitionCode()); - Assertions.assertFalse(dependentExecute.isSelfDependent(dependentItem)); - } - - @Test - public void testIsFirstProcessInstance() { - Mockito.when(processInstanceDao.queryFirstScheduleProcessInstance(processInstance.getProcessDefinitionCode())) - .thenReturn(processInstance); - DependentExecute dependentExecute = - new DependentExecute(new ArrayList<>(), DependentRelation.AND, processInstance, taskInstance); - DependentItem dependentItem = new DependentItem(); - dependentItem.setDefinitionCode(processInstance.getProcessDefinitionCode()); - Assertions.assertTrue(dependentExecute.isFirstProcessInstance(dependentItem)); - - dependentItem.setDefinitionCode(12345678L); - Assertions.assertFalse(dependentExecute.isFirstProcessInstance(dependentItem)); - } - - private ProcessInstance getProcessInstance() { - ProcessInstance processInstance = new ProcessInstance(); - processInstance.setId(100); - processInstance.setProcessDefinitionCode(10000L); - processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION); - return processInstance; - } - - private TaskInstance getTaskInstance() { - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(1000); - taskInstance.setTaskCode(10000L); - return taskInstance; - } - - /** - * task that dependent on others (and to be tested here) - * notice: should be filled with setDependence() and be passed to setupTaskInstance() - */ - private TaskNode getDependantTaskNode() { - TaskNode taskNode = new TaskNode(); - taskNode.setId("tasks-10"); - taskNode.setName("D"); - taskNode.setCode(DEPEND_TASK_CODE_D); - taskNode.setType("DEPENDENT"); - taskNode.setRunFlag(FLOWNODE_RUN_FLAG_NORMAL); - return taskNode; - } - - private TaskDefinition getTaskDefinition() { - TaskDefinition taskDefinition = new TaskDefinition(); - taskDefinition.setCode(TASK_CODE); - taskDefinition.setVersion(TASK_VERSION); - taskDefinition.setTimeoutFlag(TimeoutFlag.CLOSE); - taskDefinition.setTimeout(0); - return taskDefinition; - } - - private void setupTaskInstance(TaskNode taskNode) { - taskInstance = new TaskInstance(); - taskInstance.setId(1000); - taskInstance.setTaskCode(TASK_CODE); - taskInstance.setTaskDefinitionVersion(TASK_VERSION); - taskInstance.setProcessInstanceId(processInstance.getId()); - taskInstance.setState(TaskExecutionStatus.SUBMITTED_SUCCESS); - taskInstance.setTaskType(taskNode.getType().toUpperCase()); - taskInstance.setDependency(JSONUtils.parseObject(taskNode.getDependence(), DependentParameters.class)); - taskInstance.setName(taskNode.getName()); - } - - /** - * DependentItem defines the condition for the dependent - */ - private DependentItem getDependentItemFromTaskNode(Long processDefinitionCode, long taskCode, String date, - String cycle) { - DependentItem dependentItem = new DependentItem(); - dependentItem.setDefinitionCode(processDefinitionCode); - dependentItem.setDepTaskCode(taskCode); - dependentItem.setDateValue(date); - dependentItem.setCycle(cycle); - // so far, the following fields have no effect - dependentItem.setDependResult(DependResult.SUCCESS); - dependentItem.setStatus(TaskExecutionStatus.SUCCESS); - return dependentItem; - } - - private ProcessInstance getProcessInstanceForFindLastRunningProcess(int processInstanceId, - WorkflowExecutionStatus state) { - ProcessInstance processInstance = new ProcessInstance(); - processInstance.setId(processInstanceId); - processInstance.setState(state); - processInstance.setTestFlag(0); - return processInstance; - } - - private TaskInstance getTaskInstanceForValidTaskList( - int taskInstanceId, TaskExecutionStatus state, - long taskCode, ProcessInstance processInstance) { - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setTaskType("DEPENDENT"); - taskInstance.setId(taskInstanceId); - taskInstance.setTaskCode(taskCode); - taskInstance.setProcessInstanceId(processInstance.getId()); - taskInstance.setState(state); - return taskInstance; - } -} +// @ExtendWith(MockitoExtension.class) +// @MockitoSettings(strictness = Strictness.LENIENT) +// public class DependentTaskTest { +// +// /** +// * TaskNode.runFlag : task can be run normally +// */ +// public static final String FLOWNODE_RUN_FLAG_NORMAL = "NORMAL"; +// +// public static final Long TASK_CODE = 1111L; +// public static final Long DEPEND_TASK_CODE_A = 110L; +// public static final Long DEPEND_TASK_CODE_B = 111L; +// public static final Long DEPEND_TASK_CODE_C = 112L; +// public static final Long DEPEND_TASK_CODE_D = 113L; +// public static final int TASK_VERSION = 1; +// +// private ProcessService processService; +// +// private ProcessInstanceDao processInstanceDao; +// +// private TaskInstanceDao taskInstanceDao; +// +// private TaskDefinitionDao taskDefinitionDao; +// +// /** +// * the dependent task to be tested +// * ProcessDefinition id=1 +// * Task id=task-10, name=D +// * ProcessInstance id=100 +// * TaskInstance id=1000 +// * notice: must be initialized by setupTaskInstance() on each test case +// */ +// private ProcessInstance processInstance; +// private TaskInstance taskInstance; +// +// @BeforeEach +// public void before() { +// ApplicationContext applicationContext = Mockito.mock(ApplicationContext.class); +// SpringApplicationContext springApplicationContext = new SpringApplicationContext(); +// springApplicationContext.setApplicationContext(applicationContext); +// +// MasterConfig config = new MasterConfig(); +// config.setTaskCommitRetryTimes(3); +// config.setTaskCommitInterval(Duration.ofSeconds(1)); +// Mockito.when(applicationContext.getBean(MasterConfig.class)).thenReturn(config); +// +// processService = Mockito.mock(ProcessService.class); +// Mockito.when(applicationContext.getBean(ProcessService.class)).thenReturn(processService); +// +// processInstanceDao = Mockito.mock(ProcessInstanceDao.class); +// Mockito.when(applicationContext.getBean(ProcessInstanceDao.class)).thenReturn(processInstanceDao); +// +// taskInstanceDao = Mockito.mock(TaskInstanceDao.class); +// Mockito.when(applicationContext.getBean(TaskInstanceDao.class)).thenReturn(taskInstanceDao); +// +// taskDefinitionDao = Mockito.mock(TaskDefinitionDao.class); +// Mockito.when(SpringApplicationContext.getBean(TaskDefinitionDao.class)).thenReturn(taskDefinitionDao); +// +// processInstance = getProcessInstance(); +// taskInstance = getTaskInstance(); +// +// // for MasterBaseTaskExecThread.call +// // for DependentTaskExecThread.waitTaskQuit +// Mockito.when(processService +// .findProcessInstanceById(100)) +// .thenAnswer(i -> processInstance); +// +// // for MasterBaseTaskExecThread.submit +// Mockito.when(processService +// .submitTask(processInstance, taskInstance)) +// .thenAnswer(i -> taskInstance); +// +// // for DependentTaskExecThread.initTaskParameters +// Mockito.when(taskInstanceDao +// .updateTaskInstance(Mockito.any())) +// .thenReturn(true); +// // for DependentTaskExecThread.updateTaskState +// Mockito.when(taskInstanceDao.upsertTaskInstance(Mockito.any())) +// .thenReturn(true); +// +// // for DependentTaskExecThread.waitTaskQuit +// Mockito.when(taskInstanceDao +// .findTaskInstanceById(1000)) +// .thenAnswer(i -> taskInstance); +// +// Mockito.when(taskDefinitionDao.findTaskDefinition(TASK_CODE, TASK_VERSION)) +// .thenReturn(getTaskDefinition()); +// } +// +// private void testBasicInit() { +// TaskNode taskNode = getDependantTaskNode(); +// DependentTaskModel dependentTaskModel = new DependentTaskModel(); +// dependentTaskModel.setRelation(DependentRelation.AND); +// dependentTaskModel.setDependItemList(Stream.of( +// getDependentItemFromTaskNode(2L, DEPEND_TASK_CODE_A, "today", "day")).collect(Collectors.toList())); +// +// DependentParameters dependentParameters = new DependentParameters(); +// dependentParameters.setRelation(DependentRelation.AND); +// dependentParameters.setDependTaskList(Stream.of(dependentTaskModel).collect(Collectors.toList())); +// +// // dependence: AND(AND(2-A-day-today)) +// taskNode.setDependence(JSONUtils.toJsonString(dependentParameters)); +// +// setupTaskInstance(taskNode); +// } +// +// @Test +// public void testBasicSuccess() { +// testBasicInit(); +// ProcessInstance dependentProcessInstance = +// getProcessInstanceForFindLastRunningProcess(200, WorkflowExecutionStatus.FAILURE); +// // for DependentExecute.findLastProcessInterval +// Mockito.when(processInstanceDao +// .findLastRunningProcess(Mockito.eq(2L), Mockito.any(), Mockito.any(), Mockito.anyInt())) +// .thenReturn(dependentProcessInstance); +// +// // for DependentExecute.getDependTaskResult +// Mockito.when(taskInstanceDao +// .findValidTaskListByProcessId(200, 0)) +// .thenReturn(Stream.of( +// getTaskInstanceForValidTaskList(2000, TaskExecutionStatus.SUCCESS, DEPEND_TASK_CODE_A, +// dependentProcessInstance), +// getTaskInstanceForValidTaskList(2000, TaskExecutionStatus.FAILURE, DEPEND_TASK_CODE_B, +// dependentProcessInstance)) +// .collect(Collectors.toList())); +// +// } +// +// @Test +// public void testBasicFailure() { +// testBasicInit(); +// ProcessInstance dependentProcessInstance = +// getProcessInstanceForFindLastRunningProcess(200, WorkflowExecutionStatus.SUCCESS); +// // for DependentExecute.findLastProcessInterval +// Mockito.when(processInstanceDao +// .findLastRunningProcess(Mockito.eq(2L), Mockito.any(), Mockito.any(), Mockito.anyInt())) +// .thenReturn(dependentProcessInstance); +// +// // for DependentExecute.getDependTaskResult +// Mockito.when(taskInstanceDao +// .findValidTaskListByProcessId(200, 0)) +// .thenReturn(Stream.of( +// getTaskInstanceForValidTaskList(2000, TaskExecutionStatus.FAILURE, DEPEND_TASK_CODE_A, +// dependentProcessInstance), +// getTaskInstanceForValidTaskList(2000, TaskExecutionStatus.SUCCESS, DEPEND_TASK_CODE_B, +// dependentProcessInstance)) +// .collect(Collectors.toList())); +// } +// +// @Test +// public void testDependentRelation() { +// DependentTaskModel dependentTaskModel1 = new DependentTaskModel(); +// dependentTaskModel1.setRelation(DependentRelation.AND); +// dependentTaskModel1.setDependItemList(Stream.of( +// getDependentItemFromTaskNode(2L, DEPEND_TASK_CODE_A, "today", "day"), +// getDependentItemFromTaskNode(3L, DEPEND_TASK_CODE_B, "today", "day")).collect(Collectors.toList())); +// +// DependentTaskModel dependentTaskModel2 = new DependentTaskModel(); +// dependentTaskModel2.setRelation(DependentRelation.OR); +// dependentTaskModel2.setDependItemList(Stream.of( +// getDependentItemFromTaskNode(2L, DEPEND_TASK_CODE_A, "today", "day"), +// getDependentItemFromTaskNode(3L, DEPEND_TASK_CODE_C, "today", "day")).collect(Collectors.toList())); +// +// /* +// * OR AND 2-A-day-today 3-B-day-today OR 2-A-day-today 3-C-day-today +// */ +// DependentParameters dependentParameters = new DependentParameters(); +// dependentParameters.setRelation(DependentRelation.OR); +// dependentParameters.setDependTaskList(Stream.of( +// dependentTaskModel1, +// dependentTaskModel2).collect(Collectors.toList())); +// +// TaskNode taskNode = getDependantTaskNode(); +// taskNode.setDependence(JSONUtils.toJsonString(dependentParameters)); +// setupTaskInstance(taskNode); +// +// ProcessInstance processInstance200 = +// getProcessInstanceForFindLastRunningProcess(200, WorkflowExecutionStatus.FAILURE); +// ProcessInstance processInstance300 = +// getProcessInstanceForFindLastRunningProcess(300, WorkflowExecutionStatus.SUCCESS); +// +// // for DependentExecute.findLastProcessInterval +// Mockito.when(processInstanceDao +// .findLastRunningProcess(Mockito.eq(2L), Mockito.any(), Mockito.any(), Mockito.anyInt())) +// .thenReturn(processInstance200); +// Mockito.when(processInstanceDao +// .findLastRunningProcess(Mockito.eq(3L), Mockito.any(), Mockito.any(), Mockito.anyInt())) +// .thenReturn(processInstance300); +// +// // for DependentExecute.getDependTaskResult +// Mockito.when(taskInstanceDao +// .findValidTaskListByProcessId(200, 0)) +// .thenReturn(Stream.of( +// getTaskInstanceForValidTaskList(2000, TaskExecutionStatus.FAILURE, DEPEND_TASK_CODE_A, +// processInstance200)) +// .collect(Collectors.toList())); +// Mockito.when(taskInstanceDao +// .findValidTaskListByProcessId(300, 0)) +// .thenReturn(Stream.of( +// getTaskInstanceForValidTaskList(3000, TaskExecutionStatus.SUCCESS, DEPEND_TASK_CODE_B, +// processInstance300), +// getTaskInstanceForValidTaskList(3001, TaskExecutionStatus.SUCCESS, DEPEND_TASK_CODE_C, +// processInstance300)) +// .collect(Collectors.toList())); +// +// } +// +// /** +// * test when dependent on ALL tasks in another process +// */ +// private void testDependentOnAllInit() { +// TaskNode taskNode = getDependantTaskNode(); +// DependentTaskModel dependentTaskModel = new DependentTaskModel(); +// dependentTaskModel.setRelation(DependentRelation.AND); +// dependentTaskModel.setDependItemList(Stream.of( +// getDependentItemFromTaskNode(2L, Constants.DEPENDENT_ALL_TASK_CODE, "today", "day")) +// .collect(Collectors.toList())); +// +// DependentParameters dependentParameters = new DependentParameters(); +// dependentParameters.setRelation(DependentRelation.AND); +// dependentParameters.setDependTaskList(Stream.of(dependentTaskModel).collect(Collectors.toList())); +// +// // dependence: AND(AND(2:ALL today day)) +// taskNode.setDependence(JSONUtils.toJsonString(dependentParameters)); +// +// setupTaskInstance(taskNode); +// } +// +// @Test +// public void testDependentOnAllSuccess() { +// testDependentOnAllInit(); +// // for DependentExecute.findLastProcessInterval +// Mockito.when(processInstanceDao +// .findLastRunningProcess(Mockito.eq(2L), Mockito.any(), Mockito.any(), Mockito.anyInt())) +// .thenReturn(getProcessInstanceForFindLastRunningProcess(200, WorkflowExecutionStatus.SUCCESS)); +// +// } +// +// @Test +// public void testDependentOnAllFailure() { +// testDependentOnAllInit(); +// // for DependentExecute.findLastProcessInterval +// Mockito.when(processInstanceDao +// .findLastRunningProcess(Mockito.eq(2L), Mockito.any(), Mockito.any(), Mockito.anyInt())) +// .thenReturn(getProcessInstanceForFindLastRunningProcess(200, WorkflowExecutionStatus.FAILURE)); +// +// } +// +// /** +// * test whether waitTaskQuit has been well impl +// */ +// @Test +// public void testWaitAndCancel() { +// // for the poor independence of UT, error on other place may causes the condition happens +// if (!ServerLifeCycleManager.isRunning()) { +// return; +// } +// +// TaskNode taskNode = getDependantTaskNode(); +// DependentTaskModel dependentTaskModel = new DependentTaskModel(); +// dependentTaskModel.setRelation(DependentRelation.AND); +// dependentTaskModel.setDependItemList(Stream.of( +// getDependentItemFromTaskNode(2L, DEPEND_TASK_CODE_A, "today", "day")).collect(Collectors.toList())); +// +// DependentParameters dependentParameters = new DependentParameters(); +// dependentParameters.setRelation(DependentRelation.AND); +// dependentParameters.setDependTaskList(Stream.of(dependentTaskModel).collect(Collectors.toList())); +// +// // dependence: AND(AND(2:A today day)) +// taskNode.setDependence(JSONUtils.toJsonString(dependentParameters)); +// +// setupTaskInstance(taskNode); +// +// ProcessInstance dependentProcessInstance = +// getProcessInstanceForFindLastRunningProcess(200, WorkflowExecutionStatus.RUNNING_EXECUTION); +// // for DependentExecute.findLastProcessInterval +// Mockito.when(processInstanceDao +// .findLastRunningProcess(Mockito.eq(2L), Mockito.any(), Mockito.any(), Mockito.anyInt())) +// .thenReturn(dependentProcessInstance); +// +// // DependentTaskExecThread taskExecThread = new DependentTaskExecThread(taskInstance); +// +// // for DependentExecute.getDependTaskResult +// Mockito.when(taskInstanceDao +// .findValidTaskListByProcessId(200, 0)) +// .thenAnswer(i -> { +// processInstance.setState(WorkflowExecutionStatus.READY_STOP); +// return Stream.of( +// getTaskInstanceForValidTaskList(2000, TaskExecutionStatus.RUNNING_EXECUTION, +// DEPEND_TASK_CODE_A, dependentProcessInstance)) +// .collect(Collectors.toList()); +// }) +// .thenThrow(new IllegalStateException("have not been stopped as expected")); +// +// } +// +// @Test +// public void testIsSelfDependent() { +// DependentExecute dependentExecute = +// new DependentExecute(new ArrayList<>(), DependentRelation.AND, processInstance, taskInstance); +// DependentItem dependentItem = new DependentItem(); +// dependentItem.setDefinitionCode(processInstance.getProcessDefinitionCode()); +// dependentItem.setDepTaskCode(Constants.DEPENDENT_ALL_TASK_CODE); +// Assertions.assertTrue(dependentExecute.isSelfDependent(dependentItem)); +// +// dependentItem.setDepTaskCode(taskInstance.getTaskCode()); +// Assertions.assertTrue(dependentExecute.isSelfDependent(dependentItem)); +// +// // no self task +// dependentItem.setDepTaskCode(12345678); +// Assertions.assertFalse(dependentExecute.isSelfDependent(dependentItem)); +// +// // no self wf +// dependentItem.setDefinitionCode(processInstance.getProcessDefinitionCode()); +// Assertions.assertFalse(dependentExecute.isSelfDependent(dependentItem)); +// } +// +// @Test +// public void testIsFirstProcessInstance() { +// Mockito.when(processInstanceDao.queryFirstScheduleProcessInstance(processInstance.getProcessDefinitionCode())) +// .thenReturn(processInstance); +// DependentExecute dependentExecute = +// new DependentExecute(new ArrayList<>(), DependentRelation.AND, processInstance, taskInstance); +// DependentItem dependentItem = new DependentItem(); +// dependentItem.setDefinitionCode(processInstance.getProcessDefinitionCode()); +// Assertions.assertTrue(dependentExecute.isFirstProcessInstance(dependentItem)); +// +// dependentItem.setDefinitionCode(12345678L); +// Assertions.assertFalse(dependentExecute.isFirstProcessInstance(dependentItem)); +// } +// +// private ProcessInstance getProcessInstance() { +// ProcessInstance processInstance = new ProcessInstance(); +// processInstance.setId(100); +// processInstance.setProcessDefinitionCode(10000L); +// processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION); +// return processInstance; +// } +// +// private TaskInstance getTaskInstance() { +// TaskInstance taskInstance = new TaskInstance(); +// taskInstance.setId(1000); +// taskInstance.setTaskCode(10000L); +// return taskInstance; +// } +// +// /** +// * task that dependent on others (and to be tested here) +// * notice: should be filled with setDependence() and be passed to setupTaskInstance() +// */ +// private TaskNode getDependantTaskNode() { +// TaskNode taskNode = new TaskNode(); +// taskNode.setId("tasks-10"); +// taskNode.setName("D"); +// taskNode.setCode(DEPEND_TASK_CODE_D); +// taskNode.setType("DEPENDENT"); +// taskNode.setRunFlag(FLOWNODE_RUN_FLAG_NORMAL); +// return taskNode; +// } +// +// private TaskDefinition getTaskDefinition() { +// TaskDefinition taskDefinition = new TaskDefinition(); +// taskDefinition.setCode(TASK_CODE); +// taskDefinition.setVersion(TASK_VERSION); +// taskDefinition.setTimeoutFlag(TimeoutFlag.CLOSE); +// taskDefinition.setTimeout(0); +// return taskDefinition; +// } +// +// private void setupTaskInstance(TaskNode taskNode) { +// taskInstance = new TaskInstance(); +// taskInstance.setId(1000); +// taskInstance.setTaskCode(TASK_CODE); +// taskInstance.setTaskDefinitionVersion(TASK_VERSION); +// taskInstance.setProcessInstanceId(processInstance.getId()); +// taskInstance.setState(TaskExecutionStatus.SUBMITTED_SUCCESS); +// taskInstance.setTaskType(taskNode.getType().toUpperCase()); +// taskInstance.setDependency(JSONUtils.parseObject(taskNode.getDependence(), DependentParameters.class)); +// taskInstance.setName(taskNode.getName()); +// } +// +// /** +// * DependentItem defines the condition for the dependent +// */ +// private DependentItem getDependentItemFromTaskNode(Long processDefinitionCode, long taskCode, String date, +// String cycle) { +// DependentItem dependentItem = new DependentItem(); +// dependentItem.setDefinitionCode(processDefinitionCode); +// dependentItem.setDepTaskCode(taskCode); +// dependentItem.setDateValue(date); +// dependentItem.setCycle(cycle); +// // so far, the following fields have no effect +// dependentItem.setDependResult(DependResult.SUCCESS); +// dependentItem.setStatus(TaskExecutionStatus.SUCCESS); +// return dependentItem; +// } +// +// private ProcessInstance getProcessInstanceForFindLastRunningProcess(int processInstanceId, +// WorkflowExecutionStatus state) { +// ProcessInstance processInstance = new ProcessInstance(); +// processInstance.setId(processInstanceId); +// processInstance.setState(state); +// processInstance.setTestFlag(0); +// return processInstance; +// } +// +// private TaskInstance getTaskInstanceForValidTaskList( +// int taskInstanceId, TaskExecutionStatus state, +// long taskCode, ProcessInstance processInstance) { +// TaskInstance taskInstance = new TaskInstance(); +// taskInstance.setTaskType("DEPENDENT"); +// taskInstance.setId(taskInstanceId); +// taskInstance.setTaskCode(taskCode); +// taskInstance.setProcessInstanceId(processInstance.getId()); +// taskInstance.setState(state); +// return taskInstance; +// } +// } diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/SubProcessTaskTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/SubProcessTaskTest.java index 1b03bd2cca..286c5a4c17 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/SubProcessTaskTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/SubProcessTaskTest.java @@ -17,176 +17,146 @@ package org.apache.dolphinscheduler.server.master; -import org.apache.dolphinscheduler.common.enums.TimeoutFlag; -import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; -import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager; -import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.dao.entity.TaskDefinition; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; -import org.apache.dolphinscheduler.plugin.task.api.enums.Direct; -import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; -import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy; -import org.apache.dolphinscheduler.plugin.task.api.model.Property; -import org.apache.dolphinscheduler.server.master.config.MasterConfig; -import org.apache.dolphinscheduler.server.master.runner.task.SubTaskProcessor; -import org.apache.dolphinscheduler.server.master.runner.task.TaskAction; -import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; -import org.apache.dolphinscheduler.service.model.TaskNode; -import org.apache.dolphinscheduler.service.process.ProcessService; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.List; - -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.MockedStatic; -import org.mockito.Mockito; - -public class SubProcessTaskTest { - - /** - * TaskNode.runFlag : task can be run normally - */ - public static final String FLOWNODE_RUN_FLAG_NORMAL = "NORMAL"; - - private ProcessService processService; - - private TaskInstanceDao taskInstanceDao; - - private ProcessInstance processInstance; - - private MockedStatic mockedStaticServerLifeCycleManager; - private MockedStatic mockedStaticSpringApplicationContext; - - @BeforeEach - public void before() { - MasterConfig config = new MasterConfig(); - config.setTaskCommitRetryTimes(3); - config.setTaskCommitInterval(Duration.ofSeconds(1)); - - mockedStaticSpringApplicationContext = Mockito.mockStatic(SpringApplicationContext.class); - Mockito.when(SpringApplicationContext.getBean(MasterConfig.class)).thenReturn(config); - - processService = Mockito.mock(ProcessService.class); - Mockito.when(SpringApplicationContext.getBean(ProcessService.class)).thenReturn(processService); - - taskInstanceDao = Mockito.mock(TaskInstanceDao.class); - Mockito.when(SpringApplicationContext.getBean(TaskInstanceDao.class)).thenReturn(taskInstanceDao); - - mockedStaticServerLifeCycleManager = Mockito.mockStatic(ServerLifeCycleManager.class); - Mockito.when(ServerLifeCycleManager.isStopped()).thenReturn(false); - - processInstance = getProcessInstance(); - Mockito.when(taskInstanceDao - .updateTaskInstance(Mockito.any())) - .thenReturn(true); - - TaskDefinition taskDefinition = new TaskDefinition(); - taskDefinition.setTimeoutFlag(TimeoutFlag.OPEN); - taskDefinition.setTimeoutNotifyStrategy(TaskTimeoutStrategy.WARN); - taskDefinition.setTimeout(0); - } - - @AfterEach - public void after() { - mockedStaticServerLifeCycleManager.close(); - mockedStaticSpringApplicationContext.close(); - } - - private TaskInstance testBasicInit(WorkflowExecutionStatus expectResult) { - TaskInstance taskInstance = getTaskInstance(getTaskNode(), processInstance); - - ProcessInstance subProcessInstance = getSubProcessInstance(expectResult); - subProcessInstance.setVarPool(getProperty()); - Mockito.when(processService - .findSubProcessInstance(processInstance.getId(), taskInstance.getId())) - .thenReturn(subProcessInstance); - taskInstance.setProcessInstance(processInstance); - return taskInstance; - } - - @Test - public void testBasicSuccess() { - testBasicInit(WorkflowExecutionStatus.SUCCESS); - } - - @Test - public void testFinish() { - TaskInstance taskInstance = testBasicInit(WorkflowExecutionStatus.SUCCESS); - taskInstance.setVarPool(getProperty()); - taskInstance.setTaskParams("{\"processDefinitionCode\":110," + - "\"dependence\":{},\"localParams\":[{\"prop\":\"key\"," + - "\"direct\":\"out\",\"type\":\"VARCHAR\",\"value\":\"\"}," + - "{\"prop\":\"database_name\",\"direct\":\"OUT\"," + - "\"type\":\"VARCHAR\",\"value\":\"\"}]," + - "\"conditionResult\":{\"successNode\":[],\"failedNode\":[]}," + - "\"waitStartTimeout\":{},\"switchResult\":{}}"); - SubTaskProcessor subTaskProcessor = new SubTaskProcessor(); - subTaskProcessor.init(taskInstance, processInstance); - subTaskProcessor.action(TaskAction.RUN); - TaskExecutionStatus status = taskInstance.getState(); - Assertions.assertEquals(TaskExecutionStatus.SUCCESS, status); - } - - private String getProperty() { - List varPools = new ArrayList<>(); - Property property = new Property(); - property.setProp("key"); - property.setValue("1"); - property.setDirect(Direct.OUT); - varPools.add(property); - return JSONUtils.toJsonString(varPools); - } - - @Test - public void testBasicFailure() { - testBasicInit(WorkflowExecutionStatus.FAILURE); - } - - private TaskNode getTaskNode() { - TaskNode taskNode = new TaskNode(); - taskNode.setId("tasks-10"); - taskNode.setName("S"); - taskNode.setCode(1L); - taskNode.setVersion(1); - taskNode.setType("SUB_PROCESS"); - taskNode.setRunFlag(FLOWNODE_RUN_FLAG_NORMAL); - return taskNode; - } - - private ProcessInstance getProcessInstance() { - ProcessInstance processInstance = new ProcessInstance(); - processInstance.setId(100); - processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION); - processInstance.setWarningGroupId(0); - processInstance.setName("S"); - return processInstance; - } - - private ProcessInstance getSubProcessInstance(WorkflowExecutionStatus executionStatus) { - ProcessInstance processInstance = new ProcessInstance(); - processInstance.setId(102); - processInstance.setState(executionStatus); - - return processInstance; - } - - private TaskInstance getTaskInstance(TaskNode taskNode, ProcessInstance processInstance) { - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(1000); - taskInstance.setName("S"); - taskInstance.setTaskType("SUB_PROCESS"); - taskInstance.setName(taskNode.getName()); - taskInstance.setTaskCode(taskNode.getCode()); - taskInstance.setTaskDefinitionVersion(taskNode.getVersion()); - taskInstance.setTaskType(taskNode.getType().toUpperCase()); - taskInstance.setProcessInstanceId(processInstance.getId()); - taskInstance.setState(TaskExecutionStatus.SUBMITTED_SUCCESS); - return taskInstance; - } -} +// public class SubProcessTaskTest { +// +// /** +// * TaskNode.runFlag : task can be run normally +// */ +// public static final String FLOWNODE_RUN_FLAG_NORMAL = "NORMAL"; +// +// private ProcessService processService; +// +// private TaskInstanceDao taskInstanceDao; +// +// private ProcessInstance processInstance; +// +// private MockedStatic mockedStaticServerLifeCycleManager; +// private MockedStatic mockedStaticSpringApplicationContext; +// +// @BeforeEach +// public void before() { +// MasterConfig config = new MasterConfig(); +// config.setTaskCommitRetryTimes(3); +// config.setTaskCommitInterval(Duration.ofSeconds(1)); +// +// mockedStaticSpringApplicationContext = Mockito.mockStatic(SpringApplicationContext.class); +// Mockito.when(SpringApplicationContext.getBean(MasterConfig.class)).thenReturn(config); +// +// processService = Mockito.mock(ProcessService.class); +// Mockito.when(SpringApplicationContext.getBean(ProcessService.class)).thenReturn(processService); +// +// taskInstanceDao = Mockito.mock(TaskInstanceDao.class); +// Mockito.when(SpringApplicationContext.getBean(TaskInstanceDao.class)).thenReturn(taskInstanceDao); +// +// mockedStaticServerLifeCycleManager = Mockito.mockStatic(ServerLifeCycleManager.class); +// Mockito.when(ServerLifeCycleManager.isStopped()).thenReturn(false); +// +// processInstance = getProcessInstance(); +// Mockito.when(taskInstanceDao +// .updateTaskInstance(Mockito.any())) +// .thenReturn(true); +// +// TaskDefinition taskDefinition = new TaskDefinition(); +// taskDefinition.setTimeoutFlag(TimeoutFlag.OPEN); +// taskDefinition.setTimeoutNotifyStrategy(TaskTimeoutStrategy.WARN); +// taskDefinition.setTimeout(0); +// } +// +// @AfterEach +// public void after() { +// mockedStaticServerLifeCycleManager.close(); +// mockedStaticSpringApplicationContext.close(); +// } +// +// private TaskInstance testBasicInit(WorkflowExecutionStatus expectResult) { +// TaskInstance taskInstance = getTaskInstance(getTaskNode(), processInstance); +// +// ProcessInstance subProcessInstance = getSubProcessInstance(expectResult); +// subProcessInstance.setVarPool(getProperty()); +// Mockito.when(processService +// .findSubProcessInstance(processInstance.getId(), taskInstance.getId())) +// .thenReturn(subProcessInstance); +// taskInstance.setProcessInstance(processInstance); +// return taskInstance; +// } +// +// @Test +// public void testBasicSuccess() { +// testBasicInit(WorkflowExecutionStatus.SUCCESS); +// } +// +// @Test +// public void testFinish() { +// TaskInstance taskInstance = testBasicInit(WorkflowExecutionStatus.SUCCESS); +// taskInstance.setVarPool(getProperty()); +// taskInstance.setTaskParams("{\"processDefinitionCode\":110," + +// "\"dependence\":{},\"localParams\":[{\"prop\":\"key\"," + +// "\"direct\":\"out\",\"type\":\"VARCHAR\",\"value\":\"\"}," + +// "{\"prop\":\"database_name\",\"direct\":\"OUT\"," + +// "\"type\":\"VARCHAR\",\"value\":\"\"}]," + +// "\"conditionResult\":{\"successNode\":[],\"failedNode\":[]}," + +// "\"waitStartTimeout\":{},\"switchResult\":{}}"); +// SubTaskProcessor subTaskProcessor = new SubTaskProcessor(); +// subTaskProcessor.init(taskInstance, processInstance); +// subTaskProcessor.action(TaskAction.RUN); +// TaskExecutionStatus status = taskInstance.getState(); +// Assertions.assertEquals(TaskExecutionStatus.SUCCESS, status); +// } +// +// private String getProperty() { +// List varPools = new ArrayList<>(); +// Property property = new Property(); +// property.setProp("key"); +// property.setValue("1"); +// property.setDirect(Direct.OUT); +// varPools.add(property); +// return JSONUtils.toJsonString(varPools); +// } +// +// @Test +// public void testBasicFailure() { +// testBasicInit(WorkflowExecutionStatus.FAILURE); +// } +// +// private TaskNode getTaskNode() { +// TaskNode taskNode = new TaskNode(); +// taskNode.setId("tasks-10"); +// taskNode.setName("S"); +// taskNode.setCode(1L); +// taskNode.setVersion(1); +// taskNode.setType("SUB_PROCESS"); +// taskNode.setRunFlag(FLOWNODE_RUN_FLAG_NORMAL); +// return taskNode; +// } +// +// private ProcessInstance getProcessInstance() { +// ProcessInstance processInstance = new ProcessInstance(); +// processInstance.setId(100); +// processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION); +// processInstance.setWarningGroupId(0); +// processInstance.setName("S"); +// return processInstance; +// } +// +// private ProcessInstance getSubProcessInstance(WorkflowExecutionStatus executionStatus) { +// ProcessInstance processInstance = new ProcessInstance(); +// processInstance.setId(102); +// processInstance.setState(executionStatus); +// +// return processInstance; +// } +// +// private TaskInstance getTaskInstance(TaskNode taskNode, ProcessInstance processInstance) { +// TaskInstance taskInstance = new TaskInstance(); +// taskInstance.setId(1000); +// taskInstance.setName("S"); +// taskInstance.setTaskType("SUB_PROCESS"); +// taskInstance.setName(taskNode.getName()); +// taskInstance.setTaskCode(taskNode.getCode()); +// taskInstance.setTaskDefinitionVersion(taskNode.getVersion()); +// taskInstance.setTaskType(taskNode.getType().toUpperCase()); +// taskInstance.setProcessInstanceId(processInstance.getId()); +// taskInstance.setState(TaskExecutionStatus.SUBMITTED_SUCCESS); +// return taskInstance; +// } +// } diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/SwitchTaskTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/SwitchTaskTest.java index 9bc05d283d..b1f988e58f 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/SwitchTaskTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/SwitchTaskTest.java @@ -17,145 +17,115 @@ package org.apache.dolphinscheduler.server.master; -import org.apache.dolphinscheduler.common.constants.Constants; -import org.apache.dolphinscheduler.common.enums.TimeoutFlag; -import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; -import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.dao.entity.TaskDefinition; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao; -import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; -import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy; -import org.apache.dolphinscheduler.plugin.task.api.model.SwitchResultVo; -import org.apache.dolphinscheduler.plugin.task.api.parameters.SwitchParameters; -import org.apache.dolphinscheduler.server.master.config.MasterConfig; -import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; -import org.apache.dolphinscheduler.service.process.ProcessService; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Mockito; -import org.mockito.junit.jupiter.MockitoExtension; -import org.mockito.junit.jupiter.MockitoSettings; -import org.mockito.quality.Strictness; -import org.springframework.context.ApplicationContext; - -@ExtendWith(MockitoExtension.class) -@MockitoSettings(strictness = Strictness.LENIENT) -public class SwitchTaskTest { - - private ProcessService processService; - - private ProcessInstance processInstance; - - private TaskInstanceDao taskInstanceDao; - - private TaskDefinitionDao taskDefinitionDao; - - @BeforeEach - public void before() { - ApplicationContext applicationContext = Mockito.mock(ApplicationContext.class); - SpringApplicationContext springApplicationContext = new SpringApplicationContext(); - springApplicationContext.setApplicationContext(applicationContext); - - MasterConfig config = new MasterConfig(); - Mockito.when(applicationContext.getBean(MasterConfig.class)).thenReturn(config); - config.setTaskCommitRetryTimes(3); - config.setTaskCommitInterval(Duration.ofSeconds(1)); - - processService = Mockito.mock(ProcessService.class); - Mockito.when(applicationContext.getBean(ProcessService.class)).thenReturn(processService); - - taskInstanceDao = Mockito.mock(TaskInstanceDao.class); - Mockito.when(applicationContext.getBean(TaskInstanceDao.class)).thenReturn(taskInstanceDao); - - taskDefinitionDao = Mockito.mock(TaskDefinitionDao.class); - Mockito.when(SpringApplicationContext.getBean(TaskDefinitionDao.class)).thenReturn(taskDefinitionDao); - - processInstance = getProcessInstance(); - Mockito.when(processService - .findProcessInstanceById(processInstance.getId())) - .thenReturn(processInstance); - } - - private TaskInstance testBasicInit(WorkflowExecutionStatus expectResult) { - TaskDefinition taskDefinition = new TaskDefinition(); - taskDefinition.setTimeoutFlag(TimeoutFlag.OPEN); - taskDefinition.setTimeoutNotifyStrategy(TaskTimeoutStrategy.WARN); - taskDefinition.setTimeout(0); - Mockito.when(taskDefinitionDao.findTaskDefinition(1L, 1)) - .thenReturn(taskDefinition); - TaskInstance taskInstance = getTaskInstance(getTaskNode(), processInstance); - - // for MasterBaseTaskExecThread.submit - Mockito.when(processService - .submitTask(processInstance, taskInstance)) - .thenReturn(taskInstance); - // for MasterBaseTaskExecThread.call - Mockito.when(taskInstanceDao - .findTaskInstanceById(taskInstance.getId())) - .thenReturn(taskInstance); - // for SwitchTaskExecThread.initTaskParameters - Mockito.when(taskInstanceDao.upsertTaskInstance(taskInstance)) - .thenReturn(true); - // for SwitchTaskExecThread.updateTaskState - Mockito.when(taskInstanceDao - .updateTaskInstance(taskInstance)) - .thenReturn(true); - - return taskInstance; - } - - private SwitchParameters getTaskNode() { - SwitchParameters conditionsParameters = new SwitchParameters(); - - SwitchResultVo switchResultVo1 = new SwitchResultVo(); - switchResultVo1.setCondition(" 2 == 1"); - switchResultVo1.setNextNode("t1"); - SwitchResultVo switchResultVo2 = new SwitchResultVo(); - switchResultVo2.setCondition(" 2 == 2"); - switchResultVo2.setNextNode("t2"); - SwitchResultVo switchResultVo3 = new SwitchResultVo(); - switchResultVo3.setCondition(" 3 == 2"); - switchResultVo3.setNextNode("t3"); - List list = new ArrayList<>(); - list.add(switchResultVo1); - list.add(switchResultVo2); - list.add(switchResultVo3); - conditionsParameters.setDependTaskList(list); - conditionsParameters.setNextNode("t"); - conditionsParameters.setRelation("AND"); - - return conditionsParameters; - } - - private ProcessInstance getProcessInstance() { - ProcessInstance processInstance = new ProcessInstance(); - processInstance.setId(1000); - processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION); - processInstance.setProcessDefinitionCode(1L); - return processInstance; - } - - private TaskInstance getTaskInstance(SwitchParameters conditionsParameters, ProcessInstance processInstance) { - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(1000); - Map taskParamsMap = new HashMap<>(); - taskParamsMap.put(Constants.SWITCH_RESULT, ""); - taskInstance.setTaskParams(JSONUtils.toJsonString(taskParamsMap)); - taskInstance.setSwitchDependency(conditionsParameters); - taskInstance.setName("C"); - taskInstance.setTaskType("SWITCH"); - taskInstance.setProcessInstanceId(processInstance.getId()); - taskInstance.setTaskCode(1L); - taskInstance.setTaskDefinitionVersion(1); - return taskInstance; - } -} +// @ExtendWith(MockitoExtension.class) +// @MockitoSettings(strictness = Strictness.LENIENT) +// public class SwitchTaskTest { +// +// private ProcessService processService; +// +// private ProcessInstance processInstance; +// +// private TaskInstanceDao taskInstanceDao; +// +// private TaskDefinitionDao taskDefinitionDao; +// +// @BeforeEach +// public void before() { +// ApplicationContext applicationContext = Mockito.mock(ApplicationContext.class); +// SpringApplicationContext springApplicationContext = new SpringApplicationContext(); +// springApplicationContext.setApplicationContext(applicationContext); +// +// MasterConfig config = new MasterConfig(); +// Mockito.when(applicationContext.getBean(MasterConfig.class)).thenReturn(config); +// config.setTaskCommitRetryTimes(3); +// config.setTaskCommitInterval(Duration.ofSeconds(1)); +// +// processService = Mockito.mock(ProcessService.class); +// Mockito.when(applicationContext.getBean(ProcessService.class)).thenReturn(processService); +// +// taskInstanceDao = Mockito.mock(TaskInstanceDao.class); +// Mockito.when(applicationContext.getBean(TaskInstanceDao.class)).thenReturn(taskInstanceDao); +// +// taskDefinitionDao = Mockito.mock(TaskDefinitionDao.class); +// Mockito.when(SpringApplicationContext.getBean(TaskDefinitionDao.class)).thenReturn(taskDefinitionDao); +// +// processInstance = getProcessInstance(); +// Mockito.when(processService +// .findProcessInstanceById(processInstance.getId())) +// .thenReturn(processInstance); +// } +// +// private TaskInstance testBasicInit(WorkflowExecutionStatus expectResult) { +// TaskDefinition taskDefinition = new TaskDefinition(); +// taskDefinition.setTimeoutFlag(TimeoutFlag.OPEN); +// taskDefinition.setTimeoutNotifyStrategy(TaskTimeoutStrategy.WARN); +// taskDefinition.setTimeout(0); +// Mockito.when(taskDefinitionDao.findTaskDefinition(1L, 1)) +// .thenReturn(taskDefinition); +// TaskInstance taskInstance = getTaskInstance(getTaskNode(), processInstance); +// +// // for MasterBaseTaskExecThread.submit +// Mockito.when(processService +// .submitTask(processInstance, taskInstance)) +// .thenReturn(taskInstance); +// // for MasterBaseTaskExecThread.call +// Mockito.when(taskInstanceDao +// .findTaskInstanceById(taskInstance.getId())) +// .thenReturn(taskInstance); +// // for SwitchTaskExecThread.initTaskParameters +// Mockito.when(taskInstanceDao.upsertTaskInstance(taskInstance)) +// .thenReturn(true); +// // for SwitchTaskExecThread.updateTaskState +// Mockito.when(taskInstanceDao +// .updateTaskInstance(taskInstance)) +// .thenReturn(true); +// +// return taskInstance; +// } +// +// private SwitchParameters getTaskNode() { +// SwitchParameters conditionsParameters = new SwitchParameters(); +// +// SwitchResultVo switchResultVo1 = new SwitchResultVo(); +// switchResultVo1.setCondition(" 2 == 1"); +// switchResultVo1.setNextNode("t1"); +// SwitchResultVo switchResultVo2 = new SwitchResultVo(); +// switchResultVo2.setCondition(" 2 == 2"); +// switchResultVo2.setNextNode("t2"); +// SwitchResultVo switchResultVo3 = new SwitchResultVo(); +// switchResultVo3.setCondition(" 3 == 2"); +// switchResultVo3.setNextNode("t3"); +// List list = new ArrayList<>(); +// list.add(switchResultVo1); +// list.add(switchResultVo2); +// list.add(switchResultVo3); +// conditionsParameters.setDependTaskList(list); +// conditionsParameters.setNextNode("t"); +// conditionsParameters.setRelation("AND"); +// +// return conditionsParameters; +// } +// +// private ProcessInstance getProcessInstance() { +// ProcessInstance processInstance = new ProcessInstance(); +// processInstance.setId(1000); +// processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION); +// processInstance.setProcessDefinitionCode(1L); +// return processInstance; +// } +// +// private TaskInstance getTaskInstance(SwitchParameters conditionsParameters, ProcessInstance processInstance) { +// TaskInstance taskInstance = new TaskInstance(); +// taskInstance.setId(1000); +// Map taskParamsMap = new HashMap<>(); +// taskParamsMap.put(Constants.SWITCH_RESULT, ""); +// taskInstance.setTaskParams(JSONUtils.toJsonString(taskParamsMap)); +// taskInstance.setSwitchDependency(conditionsParameters); +// taskInstance.setName("C"); +// taskInstance.setTaskType("SWITCH"); +// taskInstance.setProcessInstanceId(processInstance.getId()); +// taskInstance.setTaskCode(1L); +// taskInstance.setTaskDefinitionVersion(1); +// return taskInstance; +// } +// } diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumerTest.java deleted file mode 100644 index 7c947fbcc2..0000000000 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumerTest.java +++ /dev/null @@ -1,365 +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.consumer; - -import org.apache.dolphinscheduler.common.enums.CommandType; -import org.apache.dolphinscheduler.common.enums.Priority; -import org.apache.dolphinscheduler.common.enums.TimeoutFlag; -import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager; -import org.apache.dolphinscheduler.dao.entity.DataSource; -import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.dao.entity.TaskDefinition; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.dao.entity.Tenant; -import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; -import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; -import org.apache.dolphinscheduler.server.master.dispatch.ExecutorDispatcher; -import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException; -import org.apache.dolphinscheduler.service.process.ProcessService; -import org.apache.dolphinscheduler.service.queue.TaskPriority; -import org.apache.dolphinscheduler.service.queue.TaskPriorityQueue; -import org.apache.dolphinscheduler.spi.enums.DbType; - -import java.util.Date; -import java.util.concurrent.TimeUnit; - -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Disabled; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Mockito; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.test.context.junit.jupiter.SpringExtension; - -@ExtendWith(SpringExtension.class) -@Disabled -public class TaskPriorityQueueConsumerTest { - - @Autowired - private TaskPriorityQueue taskPriorityQueue; - - @Autowired - private TaskPriorityQueueConsumer taskPriorityQueueConsumer; - - @Autowired - private ProcessService processService; - - @Autowired - private TaskInstanceDao taskInstanceDao; - - @Autowired - private ExecutorDispatcher dispatcher; - - private static final String TENANT_CODE = "root"; - - @BeforeEach - public void init() { - - Tenant tenant = new Tenant(); - tenant.setId(1); - tenant.setTenantCode(TENANT_CODE); - tenant.setDescription(TENANT_CODE); - tenant.setQueueId(1); - tenant.setCreateTime(new Date()); - tenant.setUpdateTime(new Date()); - - Mockito.doReturn(tenant).when(processService).getTenantForProcess(TENANT_CODE, 2); - } - - @Test - public void testSHELLTask() throws Exception { - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(1); - taskInstance.setTaskType("SHELL"); - taskInstance.setProcessInstanceId(1); - taskInstance.setState(TaskExecutionStatus.KILL); - taskInstance.setProcessInstancePriority(Priority.MEDIUM); - taskInstance.setWorkerGroup("default"); - taskInstance.setExecutorId(2); - - ProcessInstance processInstance = new ProcessInstance(); - processInstance.setId(1); - processInstance.setTenantCode(TENANT_CODE); - processInstance.setCommandType(CommandType.START_PROCESS); - taskInstance.setProcessInstance(processInstance); - - ProcessDefinition processDefinition = new ProcessDefinition(); - processDefinition.setUserId(2); - taskInstance.setProcessDefine(processDefinition); - - TaskPriority taskPriority = new TaskPriority(2, 1, 2, 1, 1, "default"); - taskPriorityQueue.put(taskPriority); - - TimeUnit.SECONDS.sleep(10); - - Assertions.assertNotNull(taskInstance); - } - - @Test - public void testSQLTask() throws Exception { - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(1); - taskInstance.setTaskType("SQL"); - taskInstance.setProcessInstanceId(1); - taskInstance.setState(TaskExecutionStatus.KILL); - taskInstance.setProcessInstancePriority(Priority.MEDIUM); - taskInstance.setWorkerGroup("default"); - taskInstance.setExecutorId(2); - - ProcessInstance processInstance = new ProcessInstance(); - processInstance.setTenantCode(TENANT_CODE); - processInstance.setCommandType(CommandType.START_PROCESS); - taskInstance.setProcessInstance(processInstance); - - ProcessDefinition processDefinition = new ProcessDefinition(); - processDefinition.setUserId(2); - taskInstance.setProcessDefine(processDefinition); - TaskPriority taskPriority = new TaskPriority(2, 1, 2, 1, 1, "default"); - taskPriorityQueue.put(taskPriority); - - DataSource dataSource = new DataSource(); - dataSource.setId(1); - dataSource.setName("sqlDatasource"); - dataSource.setType(DbType.MYSQL); - dataSource.setUserId(2); - dataSource.setConnectionParams("{\"address\":\"jdbc:mysql://192.168.221.185:3306\"," - + "\"database\":\"dolphinscheduler_qiaozhanwei\"," - + "\"jdbcUrl\":\"jdbc:mysql://192.168.221.185:3306/dolphinscheduler_qiaozhanwei\"," - + "\"user\":\"root\"," - + "\"password\":\"root@123\"}"); - dataSource.setCreateTime(new Date()); - dataSource.setUpdateTime(new Date()); - - Mockito.doReturn(dataSource).when(processService).findDataSourceById(1); - - TimeUnit.SECONDS.sleep(10); - Assertions.assertNotNull(taskInstance); - } - - @Test - public void testDataxTask() throws Exception { - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(1); - taskInstance.setTaskType("DATAX"); - taskInstance.setProcessInstanceId(1); - taskInstance.setState(TaskExecutionStatus.KILL); - taskInstance.setProcessInstancePriority(Priority.MEDIUM); - taskInstance.setWorkerGroup("default"); - taskInstance.setExecutorId(2); - - ProcessInstance processInstance = new ProcessInstance(); - processInstance.setTenantCode(TENANT_CODE); - processInstance.setCommandType(CommandType.START_PROCESS); - taskInstance.setProcessInstance(processInstance); - - ProcessDefinition processDefinition = new ProcessDefinition(); - processDefinition.setUserId(2); - taskInstance.setProcessDefine(processDefinition); - TaskPriority taskPriority = new TaskPriority(2, 1, 2, 1, 1, "default"); - taskPriorityQueue.put(taskPriority); - - DataSource dataSource = new DataSource(); - dataSource.setId(80); - dataSource.setName("datax"); - dataSource.setType(DbType.MYSQL); - dataSource.setUserId(2); - dataSource.setConnectionParams("{\"address\":\"jdbc:mysql://192.168.221.185:3306\"," - + "\"database\":\"dolphinscheduler_qiaozhanwei\"," - + "\"jdbcUrl\":\"jdbc:mysql://192.168.221.185:3306/dolphinscheduler_qiaozhanwei\"," - + "\"user\":\"root\"," - + "\"password\":\"root@123\"}"); - dataSource.setCreateTime(new Date()); - dataSource.setUpdateTime(new Date()); - Mockito.doReturn(dataSource).when(processService).findDataSourceById(80); - TimeUnit.SECONDS.sleep(10); - Assertions.assertNotNull(taskInstance); - } - - @Test - public void testSqoopTask() throws Exception { - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(1); - taskInstance.setTaskType("SQOOP"); - taskInstance.setProcessInstanceId(1); - taskInstance.setState(TaskExecutionStatus.KILL); - taskInstance.setProcessInstancePriority(Priority.MEDIUM); - taskInstance.setWorkerGroup("default"); - taskInstance.setExecutorId(2); - - ProcessInstance processInstance = new ProcessInstance(); - processInstance.setTenantCode(TENANT_CODE); - processInstance.setCommandType(CommandType.START_PROCESS); - taskInstance.setProcessInstance(processInstance); - - ProcessDefinition processDefinition = new ProcessDefinition(); - processDefinition.setUserId(2); - taskInstance.setProcessDefine(processDefinition); - TaskPriority taskPriority = new TaskPriority(2, 1, 2, 1, 1, "default"); - taskPriorityQueue.put(taskPriority); - - DataSource dataSource = new DataSource(); - dataSource.setId(1); - dataSource.setName("datax"); - dataSource.setType(DbType.MYSQL); - dataSource.setUserId(2); - dataSource.setConnectionParams("{\"address\":\"jdbc:mysql://192.168.221.185:3306\"," - + "\"database\":\"dolphinscheduler_qiaozhanwei\"," - + "\"jdbcUrl\":\"jdbc:mysql://192.168.221.185:3306/dolphinscheduler_qiaozhanwei\"," - + "\"user\":\"root\"," - + "\"password\":\"root@123\"}"); - dataSource.setCreateTime(new Date()); - dataSource.setUpdateTime(new Date()); - Mockito.doReturn(dataSource).when(processService).findDataSourceById(1); - TimeUnit.SECONDS.sleep(10); - Assertions.assertNotNull(taskInstance); - } - - @Test - public void testTaskInstanceIsFinalState() { - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(1); - taskInstance.setTaskType("SHELL"); - taskInstance.setProcessInstanceId(1); - taskInstance.setState(TaskExecutionStatus.KILL); - taskInstance.setProcessInstancePriority(Priority.MEDIUM); - taskInstance.setWorkerGroup("default"); - taskInstance.setExecutorId(2); - - Mockito.doReturn(taskInstance).when(taskInstanceDao).findTaskInstanceById(1); - - Boolean state = taskPriorityQueueConsumer.taskInstanceIsFinalState(1); - Assertions.assertNotNull(state); - } - - @Test - public void testNotFoundWorkerGroup() throws Exception { - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(1); - taskInstance.setTaskType("SHELL"); - taskInstance.setProcessInstanceId(1); - taskInstance.setState(TaskExecutionStatus.KILL); - taskInstance.setProcessInstancePriority(Priority.MEDIUM); - taskInstance.setWorkerGroup("NoWorkGroup"); - taskInstance.setExecutorId(2); - - ProcessInstance processInstance = new ProcessInstance(); - processInstance.setId(1); - processInstance.setTenantCode(TENANT_CODE); - processInstance.setCommandType(CommandType.START_PROCESS); - taskInstance.setProcessInstance(processInstance); - taskInstance.setState(TaskExecutionStatus.DELAY_EXECUTION); - - ProcessDefinition processDefinition = new ProcessDefinition(); - processDefinition.setUserId(2); - taskInstance.setProcessDefine(processDefinition); - - Mockito.doReturn(taskInstance).when(taskInstanceDao).findTaskInstanceById(1); - - TaskPriority taskPriority = new TaskPriority(2, 1, 2, 1, 1, "NoWorkGroup"); - taskPriorityQueue.put(taskPriority); - - TimeUnit.SECONDS.sleep(10); - - Assertions.assertNotNull(taskInstance); - - } - - @Test - public void testDispatch() { - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(1); - taskInstance.setTaskType("SHELL"); - taskInstance.setProcessInstanceId(1); - taskInstance.setState(TaskExecutionStatus.KILL); - taskInstance.setProcessInstancePriority(Priority.MEDIUM); - taskInstance.setWorkerGroup("NoWorkGroup"); - taskInstance.setExecutorId(2); - - ProcessInstance processInstance = new ProcessInstance(); - processInstance.setId(1); - processInstance.setTenantCode(TENANT_CODE); - processInstance.setCommandType(CommandType.START_PROCESS); - taskInstance.setProcessInstance(processInstance); - taskInstance.setState(TaskExecutionStatus.DELAY_EXECUTION); - - ProcessDefinition processDefinition = new ProcessDefinition(); - processDefinition.setUserId(2); - processDefinition.setProjectCode(1L); - taskInstance.setProcessDefine(processDefinition); - - TaskDefinition taskDefinition = new TaskDefinition(); - taskDefinition.setTimeoutFlag(TimeoutFlag.OPEN); - taskInstance.setTaskDefine(taskDefinition); - - Mockito.doReturn(taskInstance).when(taskInstanceDao).findTaskInstanceById(1); - - TaskPriority taskPriority = new TaskPriority(); - taskPriority.setTaskId(1); - boolean res = false; - try { - taskPriorityQueueConsumer.dispatchTask(taskPriority); - } catch (ExecuteException e) { - throw new RuntimeException(e); - } - - Assertions.assertFalse(res); - } - - @Test - public void testRun() throws Exception { - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(1); - taskInstance.setTaskType("SHELL"); - taskInstance.setProcessInstanceId(1); - taskInstance.setState(TaskExecutionStatus.KILL); - taskInstance.setProcessInstancePriority(Priority.MEDIUM); - taskInstance.setWorkerGroup("NoWorkGroup"); - taskInstance.setExecutorId(2); - - ProcessInstance processInstance = new ProcessInstance(); - processInstance.setId(1); - processInstance.setTenantCode(TENANT_CODE); - processInstance.setCommandType(CommandType.START_PROCESS); - taskInstance.setProcessInstance(processInstance); - taskInstance.setState(TaskExecutionStatus.DELAY_EXECUTION); - - ProcessDefinition processDefinition = new ProcessDefinition(); - processDefinition.setUserId(2); - taskInstance.setProcessDefine(processDefinition); - - Mockito.doReturn(taskInstance).when(taskInstanceDao).findTaskInstanceById(1); - - TaskPriority taskPriority = new TaskPriority(2, 1, 2, 1, 1, "NoWorkGroup"); - taskPriorityQueue.put(taskPriority); - - taskPriorityQueueConsumer.run(); - - TimeUnit.SECONDS.sleep(10); - Assertions.assertNotEquals(-1, taskPriorityQueue.size()); - - } - - @AfterEach - public void close() { - ServerLifeCycleManager.toStopped(); - } - -} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/ExecutionContextTestUtils.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/ExecutionContextTestUtils.java index d2b56890b8..9588c7f914 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/ExecutionContextTestUtils.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/ExecutionContextTestUtils.java @@ -24,7 +24,7 @@ import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.remote.command.Message; -import org.apache.dolphinscheduler.remote.command.task.TaskDispatchMessage; +import org.apache.dolphinscheduler.remote.command.task.TaskDispatchRequest; import org.apache.dolphinscheduler.remote.utils.Host; import org.apache.dolphinscheduler.server.master.builder.TaskExecutionContextBuilder; import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext; @@ -52,10 +52,7 @@ public class ExecutionContextTestUtils { .buildProcessDefinitionRelatedInfo(processDefinition) .create(); - TaskDispatchMessage requestCommand = new TaskDispatchMessage(context, - "127.0.0.1:5678", - "127.0.0.1:5678", - System.currentTimeMillis()); + TaskDispatchRequest requestCommand = new TaskDispatchRequest(context); Message message = requestCommand.convert2Command(); ExecutionContext executionContext = new ExecutionContext(message, ExecutorType.WORKER, taskInstance); diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/ExecutorDispatcherTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/ExecutorDispatcherTest.java deleted file mode 100644 index ce1bef11f9..0000000000 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/ExecutorDispatcherTest.java +++ /dev/null @@ -1,77 +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.dispatch; - -import org.apache.dolphinscheduler.remote.NettyRemotingServer; -import org.apache.dolphinscheduler.remote.config.NettyServerConfig; -import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext; -import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException; -import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; -import org.apache.dolphinscheduler.server.worker.processor.TaskDispatchProcessor; -import org.apache.dolphinscheduler.server.worker.registry.WorkerRegistryClient; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Disabled; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Mockito; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.test.context.junit.jupiter.SpringExtension; - -/** - * executor dispatch test - */ -@ExtendWith(SpringExtension.class) -@Disabled -public class ExecutorDispatcherTest { - - @Autowired - private ExecutorDispatcher executorDispatcher; - - @Autowired - private WorkerRegistryClient workerRegistryClient; - - @Autowired - private WorkerConfig workerConfig; - - @Test - public void testDispatchWithException() throws ExecuteException { - ExecutionContext executionContext = ExecutionContextTestUtils.getExecutionContext(10000); - Assertions.assertThrows(ExecuteException.class, () -> { - executorDispatcher.dispatch(executionContext); - }); - } - - @Test - public void testDispatch() throws Exception { - int port = 30000; - final NettyServerConfig serverConfig = new NettyServerConfig(); - serverConfig.setListenPort(port); - NettyRemotingServer nettyRemotingServer = new NettyRemotingServer(serverConfig); - nettyRemotingServer.registerProcessor(Mockito.mock(TaskDispatchProcessor.class)); - nettyRemotingServer.start(); - // - workerConfig.setListenPort(port); - workerRegistryClient.start(); - - ExecutionContext executionContext = ExecutionContextTestUtils.getExecutionContext(port); - executorDispatcher.dispatch(executionContext); - - workerRegistryClient.close(); - } -} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/executor/NettyExecutorManagerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/executor/NettyExecutorManagerTest.java index f03ee3b4e0..de971d487e 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/executor/NettyExecutorManagerTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/executor/NettyExecutorManagerTest.java @@ -25,14 +25,14 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.remote.NettyRemotingServer; import org.apache.dolphinscheduler.remote.command.Message; -import org.apache.dolphinscheduler.remote.command.task.TaskDispatchMessage; +import org.apache.dolphinscheduler.remote.command.task.TaskDispatchRequest; import org.apache.dolphinscheduler.remote.config.NettyServerConfig; import org.apache.dolphinscheduler.remote.utils.Host; import org.apache.dolphinscheduler.server.master.builder.TaskExecutionContextBuilder; import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext; import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType; import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException; -import org.apache.dolphinscheduler.server.worker.processor.TaskDispatchProcessor; +import org.apache.dolphinscheduler.server.worker.processor.WorkerTaskDispatchProcessor; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Disabled; @@ -56,7 +56,7 @@ public class NettyExecutorManagerTest { final NettyServerConfig serverConfig = new NettyServerConfig(); serverConfig.setListenPort(30000); NettyRemotingServer nettyRemotingServer = new NettyRemotingServer(serverConfig); - nettyRemotingServer.registerProcessor(new TaskDispatchProcessor()); + nettyRemotingServer.registerProcessor(new WorkerTaskDispatchProcessor()); nettyRemotingServer.start(); TaskInstance taskInstance = Mockito.mock(TaskInstance.class); ProcessDefinition processDefinition = Mockito.mock(ProcessDefinition.class); @@ -94,10 +94,7 @@ public class NettyExecutorManagerTest { } private Message toCommand(TaskExecutionContext taskExecutionContext) { - TaskDispatchMessage requestCommand = new TaskDispatchMessage(taskExecutionContext, - "127.0.0.1:5678", - "127.0.0.1:1234", - System.currentTimeMillis()); + TaskDispatchRequest requestCommand = new TaskDispatchRequest(taskExecutionContext); return requestCommand.convert2Command(); } } diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/RoundRobinHostManagerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/RoundRobinHostManagerTest.java index c2146928be..042fda90f4 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/RoundRobinHostManagerTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/RoundRobinHostManagerTest.java @@ -26,7 +26,6 @@ import org.apache.dolphinscheduler.server.master.registry.ServerNodeManager; import java.util.Optional; -import org.assertj.core.util.Strings; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -53,8 +52,8 @@ public class RoundRobinHostManagerTest { public void testSelectWithEmptyResult() throws WorkerGroupNotFoundException { Mockito.when(serverNodeManager.getWorkerGroupNodes("default")).thenReturn(null); ExecutionContext context = ExecutionContextTestUtils.getExecutionContext(10000); - Host emptyHost = roundRobinHostManager.select(context); - Assertions.assertTrue(Strings.isNullOrEmpty(emptyHost.getAddress())); + Optional emptyHost = roundRobinHostManager.select(context.getWorkerGroup()); + Assertions.assertFalse(emptyHost.isPresent()); } @Test @@ -63,8 +62,8 @@ public class RoundRobinHostManagerTest { Mockito.when(serverNodeManager.getWorkerNodeInfo("192.168.1.1:22")) .thenReturn(Optional.of(new WorkerHeartBeat())); ExecutionContext context = ExecutionContextTestUtils.getExecutionContext(10000); - Host host = roundRobinHostManager.select(context); - Assertions.assertFalse(Strings.isNullOrEmpty(host.getAddress())); - Assertions.assertTrue(host.getAddress().equalsIgnoreCase("192.168.1.1:22")); + Optional host = roundRobinHostManager.select(context.getWorkerGroup()); + Assertions.assertTrue(host.isPresent()); + Assertions.assertTrue(host.get().getAddress().equalsIgnoreCase("192.168.1.1:22")); } } diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableTest.java index bb86c11dde..cdfa2d2755 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableTest.java @@ -37,6 +37,7 @@ import org.apache.dolphinscheduler.dao.repository.TaskDefinitionLogDao; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; +import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnableFactory; import org.apache.dolphinscheduler.service.alert.ProcessAlertManager; import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.command.CommandService; @@ -93,6 +94,8 @@ public class WorkflowExecuteRunnableTest { private CuringParamsService curingGlobalParamsService; + private DefaultTaskExecuteRunnableFactory defaultTaskExecuteRunnableFactory; + @BeforeEach public void init() throws Exception { applicationContext = Mockito.mock(ApplicationContext.class); @@ -106,6 +109,7 @@ public class WorkflowExecuteRunnableTest { processInstance = Mockito.mock(ProcessInstance.class); taskInstanceDao = Mockito.mock(TaskInstanceDao.class); taskDefinitionLogDao = Mockito.mock(TaskDefinitionLogDao.class); + defaultTaskExecuteRunnableFactory = Mockito.mock(DefaultTaskExecuteRunnableFactory.class); Map cmdParam = new HashMap<>(); cmdParam.put(CMD_PARAM_COMPLEMENT_DATA_START_DATE, "2020-01-01 00:00:00"); cmdParam.put(CMD_PARAM_COMPLEMENT_DATA_END_DATE, "2020-01-20 23:00:00"); @@ -122,7 +126,7 @@ public class WorkflowExecuteRunnableTest { new WorkflowExecuteRunnable(processInstance, commandService, processService, processInstanceDao, masterRpcClient, processAlertManager, config, stateWheelExecuteThread, curingGlobalParamsService, - taskInstanceDao, taskDefinitionLogDao)); + taskInstanceDao, taskDefinitionLogDao, defaultTaskExecuteRunnableFactory)); Field dag = WorkflowExecuteRunnable.class.getDeclaredField("dag"); dag.setAccessible(true); dag.set(workflowExecuteThread, new DAG()); 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 new file mode 100644 index 0000000000..a4f169c63b --- /dev/null +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcherTest.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.runner.dispatcher; + +import org.apache.dolphinscheduler.remote.utils.Host; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; +import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; +import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; +import org.apache.dolphinscheduler.server.master.runner.execute.TaskExecuteRunnable; + +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() { + TaskEventService taskEventService = Mockito.mock(TaskEventService.class); + MasterConfig masterConfig = Mockito.mock(MasterConfig.class); + Mockito.when(masterConfig.getMasterAddress()).thenReturn("localhost:5678"); + MasterRpcClient masterRpcClient = Mockito.mock(MasterRpcClient.class); + TaskExecuteRunnable taskExecuteRunnable = Mockito.mock(TaskExecuteRunnable.class); + + MasterTaskDispatcher masterTaskDispatcher = + new MasterTaskDispatcher(taskEventService, masterConfig, masterRpcClient); + Host taskInstanceDispatchHost = masterTaskDispatcher.getTaskInstanceDispatchHost(taskExecuteRunnable) + .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 new file mode 100644 index 0000000000..93c8465362 --- /dev/null +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatchFactoryTest.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.runner.task.blocking.BlockingLogicTask; +import org.apache.dolphinscheduler.server.master.runner.task.condition.ConditionLogicTask; +import org.apache.dolphinscheduler.server.master.runner.task.dependent.DependentLogicTask; +import org.apache.dolphinscheduler.server.master.runner.task.subworkflow.SubWorkflowLogicTask; +import org.apache.dolphinscheduler.server.master.runner.task.switchtask.SwitchLogicTask; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +@ExtendWith(MockitoExtension.class) +public class TaskDispatchFactoryTest { + + @InjectMocks + private TaskDispatchFactory taskDispatchFactory; + + @Mock + private MasterTaskDispatcher masterTaskDispatcher; + + @Mock + private WorkerTaskDispatcher workerTaskDispatcher; + + @Test + public void getTaskDispatcher() { + Assertions.assertEquals(masterTaskDispatcher, + taskDispatchFactory.getTaskDispatcher(BlockingLogicTask.TASK_TYPE)); + Assertions.assertEquals(masterTaskDispatcher, + taskDispatchFactory.getTaskDispatcher(ConditionLogicTask.TASK_TYPE)); + Assertions.assertEquals(masterTaskDispatcher, + taskDispatchFactory.getTaskDispatcher(DependentLogicTask.TASK_TYPE)); + Assertions.assertEquals(masterTaskDispatcher, + taskDispatchFactory.getTaskDispatcher(SubWorkflowLogicTask.TASK_TYPE)); + Assertions.assertEquals(masterTaskDispatcher, taskDispatchFactory.getTaskDispatcher(SwitchLogicTask.TASK_TYPE)); + + Assertions.assertEquals(workerTaskDispatcher, taskDispatchFactory.getTaskDispatcher("SHELL")); + } +} 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 new file mode 100644 index 0000000000..b1b5e308ab --- /dev/null +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcherTest.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.runner.dispatcher; + +import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; +import org.apache.dolphinscheduler.remote.utils.Host; +import org.apache.dolphinscheduler.server.master.config.MasterConfig; +import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; +import org.apache.dolphinscheduler.server.master.dispatch.host.HostManager; +import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; +import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; +import org.apache.dolphinscheduler.server.master.runner.execute.TaskExecuteRunnable; + +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() throws WorkerGroupNotFoundException { + TaskEventService taskEventService = Mockito.mock(TaskEventService.class); + MasterConfig masterConfig = Mockito.mock(MasterConfig.class); + MasterRpcClient masterRpcClient = Mockito.mock(MasterRpcClient.class); + HostManager hostManager = Mockito.mock(HostManager.class); + Mockito.when(hostManager.select(Mockito.any())).thenReturn(Optional.of(Host.of("localhost:1234"))); + WorkerTaskDispatcher workerTaskDispatcher = + new WorkerTaskDispatcher(taskEventService, masterConfig, masterRpcClient, hostManager); + + TaskExecuteRunnable taskExecuteRunnable = Mockito.mock(TaskExecuteRunnable.class); + Mockito.when(taskExecuteRunnable.getTaskExecutionContext()).thenReturn(new TaskExecutionContext()); + Optional taskInstanceDispatchHost = workerTaskDispatcher.getTaskInstanceDispatchHost(taskExecuteRunnable); + Assertions.assertEquals("localhost:1234", taskInstanceDispatchHost.get().getAddress()); + } +} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessorTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessorTest.java deleted file mode 100644 index 30c4de3cbf..0000000000 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessorTest.java +++ /dev/null @@ -1,173 +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 static org.mockito.ArgumentMatchers.any; - -import org.apache.dolphinscheduler.common.enums.CommandType; -import org.apache.dolphinscheduler.common.enums.Priority; -import org.apache.dolphinscheduler.common.enums.TimeoutFlag; -import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.dao.entity.Resource; -import org.apache.dolphinscheduler.dao.entity.TaskDefinition; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao; -import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; -import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy; -import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; -import org.apache.dolphinscheduler.service.process.ProcessService; -import org.apache.dolphinscheduler.spi.enums.ResourceType; - -import java.util.ArrayList; -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.Mockito; -import org.mockito.junit.jupiter.MockitoExtension; -import org.mockito.junit.jupiter.MockitoSettings; -import org.mockito.quality.Strictness; -import org.springframework.context.ApplicationContext; - -@ExtendWith(MockitoExtension.class) -@MockitoSettings(strictness = Strictness.LENIENT) -public class CommonTaskProcessorTest { - - private ProcessService processService; - - private TaskInstanceDao taskInstanceDao; - - private TaskDefinitionDao taskDefinitionDao; - - private CommonTaskProcessor commonTaskProcessor; - - @BeforeEach - public void setUp() { - // init spring context - ApplicationContext applicationContext = Mockito.mock(ApplicationContext.class); - SpringApplicationContext springApplicationContext = new SpringApplicationContext(); - springApplicationContext.setApplicationContext(applicationContext); - - // mock process service - processService = Mockito.mock(ProcessService.class); - Mockito.when(applicationContext.getBean(ProcessService.class)).thenReturn(processService); - - taskInstanceDao = Mockito.mock(TaskInstanceDao.class); - Mockito.when(applicationContext.getBean(TaskInstanceDao.class)).thenReturn(taskInstanceDao); - - taskDefinitionDao = Mockito.mock(TaskDefinitionDao.class); - Mockito.when(SpringApplicationContext.getBean(TaskDefinitionDao.class)).thenReturn(taskDefinitionDao); - - commonTaskProcessor = Mockito.mock(CommonTaskProcessor.class); - Mockito.when(applicationContext.getBean(CommonTaskProcessor.class)).thenReturn(commonTaskProcessor); - - TaskDefinition taskDefinition = new TaskDefinition(); - taskDefinition.setTimeoutFlag(TimeoutFlag.OPEN); - taskDefinition.setTimeoutNotifyStrategy(TaskTimeoutStrategy.WARN); - taskDefinition.setTimeout(0); - Mockito.when(taskDefinitionDao.findTaskDefinition(1L, 1)) - .thenReturn(taskDefinition); - } - - @Test - public void testGetTaskExecutionContext() throws Exception { - - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(1); - taskInstance.setTaskType("SHELL"); - taskInstance.setProcessInstanceId(1); - taskInstance.setState(TaskExecutionStatus.KILL); - taskInstance.setProcessInstancePriority(Priority.MEDIUM); - taskInstance.setWorkerGroup("NoWorkGroup"); - taskInstance.setExecutorId(2); - - ProcessInstance processInstance = new ProcessInstance(); - processInstance.setId(1); - processInstance.setTenantCode("default"); - processInstance.setCommandType(CommandType.START_PROCESS); - taskInstance.setProcessInstance(processInstance); - taskInstance.setState(TaskExecutionStatus.DELAY_EXECUTION); - - ProcessDefinition processDefinition = new ProcessDefinition(); - processDefinition.setUserId(2); - processDefinition.setProjectCode(1L); - taskInstance.setProcessDefine(processDefinition); - - TaskDefinition taskDefinition = new TaskDefinition(); - taskDefinition.setTimeoutFlag(TimeoutFlag.OPEN); - taskInstance.setTaskDefine(taskDefinition); - Mockito.doReturn(taskInstance).when(taskInstanceDao).findTaskInstanceById(1); - TaskExecutionContext taskExecutionContext = commonTaskProcessor.getTaskExecutionContext(taskInstance); - Assertions.assertNull(taskExecutionContext); - } - - @Test - public void testGetResourceFullNames() { - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setId(1); - taskInstance.setTaskType("SHELL"); - taskInstance.setProcessInstanceId(1); - taskInstance.setState(TaskExecutionStatus.KILL); - taskInstance.setProcessInstancePriority(Priority.MEDIUM); - taskInstance.setWorkerGroup("NoWorkGroup"); - taskInstance.setExecutorId(2); - // task node - commonTaskProcessor = Mockito.mock(CommonTaskProcessor.class); - Map map = commonTaskProcessor.getResourceFullNames(taskInstance); - - List resourcesList = new ArrayList<>(); - Resource resource = new Resource(); - resource.setFileName("fileName"); - resourcesList.add(resource); - Mockito.doReturn(resourcesList).when(processService).listResourceByIds(new Integer[]{123}); - Mockito.doReturn("tenantCode").when(processService).queryTenantCodeByResName(resource.getFullName(), - ResourceType.FILE); - Assertions.assertNotNull(map); - - } - - @Test - public void testReplaceTestDatSource() { - CommonTaskProcessor commonTaskProcessor1 = new CommonTaskProcessor(); - commonTaskProcessor1.processService = processService; - TaskInstance taskInstance = new TaskInstance(); - taskInstance.setTestFlag(1); - taskInstance.setTaskType("SQL"); - taskInstance.setTaskParams( - "{\"localParams\":[],\"resourceList\":[],\"type\":\"MYSQL\",\"datasource\":1,\"sql\":\"select * from 'order'\",\"sqlType\":\"0\",\"preStatements\":[],\"postStatements\":[],\"segmentSeparator\":\"\",\"displayRows\":10}"); - TaskDefinition taskDefinition = new TaskDefinition(); - taskDefinition.setTaskParams( - "{\"localParams\":[],\"resourceList\":[],\"type\":\"MYSQL\",\"datasource\":1,\"sql\":\"select * from 'order'\",\"sqlType\":\"0\",\"preStatements\":[],\"postStatements\":[],\"segmentSeparator\":\"\",\"displayRows\":10}"); - taskInstance.setTaskDefine(taskDefinition); - commonTaskProcessor1.taskInstance = taskInstance; - - // The data source instance has no bound test data source - Mockito.when(processService.queryTestDataSourceId(any(Integer.class))).thenReturn(null); - commonTaskProcessor1.convertExeEnvironmentOnlineToTest(); - - // The data source instance has bound test data source - Mockito.when(processService.queryTestDataSourceId(any(Integer.class))).thenReturn(2); - commonTaskProcessor1.convertExeEnvironmentOnlineToTest(); - } -} diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/service/FailoverServiceTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/service/FailoverServiceTest.java index d32224c7b1..2e79b44426 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/service/FailoverServiceTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/service/FailoverServiceTest.java @@ -26,6 +26,7 @@ import org.apache.dolphinscheduler.common.constants.Constants; import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.model.Server; import org.apache.dolphinscheduler.common.utils.NetUtils; +import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; @@ -136,6 +137,7 @@ public class FailoverServiceTest { processInstance.setHistoryCmd("xxx"); processInstance.setCommandType(CommandType.STOP); processInstance.setProcessDefinitionCode(123L); + processInstance.setProcessDefinition(new ProcessDefinition()); masterTaskInstance = new TaskInstance(); masterTaskInstance.setId(1); diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/utils/TaskUtilsTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/utils/TaskUtilsTest.java new file mode 100644 index 0000000000..bec04b1936 --- /dev/null +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/utils/TaskUtilsTest.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.server.master.utils; + +import org.apache.dolphinscheduler.server.master.runner.task.blocking.BlockingLogicTask; +import org.apache.dolphinscheduler.server.master.runner.task.condition.ConditionLogicTask; +import org.apache.dolphinscheduler.server.master.runner.task.dependent.DependentLogicTask; +import org.apache.dolphinscheduler.server.master.runner.task.subworkflow.SubWorkflowLogicTask; +import org.apache.dolphinscheduler.server.master.runner.task.switchtask.SwitchLogicTask; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TaskUtilsTest { + + @Test + public void isMasterTask() { + Assertions.assertTrue(TaskUtils.isMasterTask(BlockingLogicTask.TASK_TYPE)); + Assertions.assertTrue(TaskUtils.isMasterTask(ConditionLogicTask.TASK_TYPE)); + Assertions.assertTrue(TaskUtils.isMasterTask(DependentLogicTask.TASK_TYPE)); + Assertions.assertTrue(TaskUtils.isMasterTask(SubWorkflowLogicTask.TASK_TYPE)); + Assertions.assertTrue(TaskUtils.isMasterTask(SwitchLogicTask.TASK_TYPE)); + } +} diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistry.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistry.java index a2741d4d9a..20f53225ae 100644 --- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistry.java +++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistry.java @@ -229,9 +229,9 @@ public final class ZookeeperRegistry implements Registry { } catch (Exception e) { try { interProcessMutex.release(); - throw new RegistryException("zookeeper get lock error", e); + throw new RegistryException(String.format("zookeeper get lock: %s error", key), e); } catch (Exception exception) { - throw new RegistryException("zookeeper release lock error", e); + throw new RegistryException(String.format("zookeeper get lock: %s error", key), e); } } } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java index e92ffd4961..bbc5f960de 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java @@ -74,7 +74,7 @@ public class NettyRemotingClient implements AutoCloseable { private final NettyClientConfig clientConfig; - private final Semaphore asyncSemaphore = new Semaphore(200, true); + private final Semaphore asyncSemaphore = new Semaphore(1024, true); private final ExecutorService callbackExecutor; @@ -128,8 +128,7 @@ public class NettyRemotingClient implements AutoCloseable { .addLast(new NettyDecoder(), clientHandler, encoder); } }); - this.responseFutureExecutor.scheduleAtFixedRate(ResponseFuture::scanFutureTable, 5000, 1000, - TimeUnit.MILLISECONDS); + this.responseFutureExecutor.scheduleWithFixedDelay(ResponseFuture::scanFutureTable, 0, 1, TimeUnit.SECONDS); isStarted.compareAndSet(false, true); } @@ -199,8 +198,8 @@ public class NettyRemotingClient implements AutoCloseable { /** * sync send * - * @param host host - * @param message command + * @param host host + * @param message command * @param timeoutMillis timeoutMillis * @return command */ @@ -221,7 +220,7 @@ public class NettyRemotingClient implements AutoCloseable { } responseFuture.setCause(future.cause()); responseFuture.putResponse(null); - log.error("send command {} to host {} failed", message, host); + log.error("send command {} to host {} failed: {}", message, host, responseFuture.getCause()); }); /* * sync wait for result diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/MessageType.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/MessageType.java index 636daa0913..407e92296b 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/MessageType.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/MessageType.java @@ -36,10 +36,8 @@ public enum MessageType { */ TASK_EXECUTE_START, - /** - * dispatch task request - */ - TASK_DISPATCH_MESSAGE, + TASK_DISPATCH_REQUEST, + TASK_DISPATCH_RESPONSE, /** * task execute running, from worker to master @@ -63,10 +61,6 @@ public enum MessageType { TASK_KILL_REQUEST, - TASK_REJECT, - - TASK_REJECT_MESSAGE_ACK, - /** * task savepoint, for stream task */ @@ -107,12 +101,16 @@ public enum MessageType { /** * update taskInstance's PID request */ - TASK_UPDATE_PID_MESSAGE, + TASK_UPDATE_RUNTIME_MESSAGE, /** * update taskInstance's PID response ack, from master to worker */ - TASK_UPDATE_PID__MESSAGE_ACK, + TASK_UPDATE_RUNTIME_MESSAGE_ACK, + + WORKFLOW_METRICS_CLEANUP, + + PAUSE_TASK_INSTANCE, + ; - WORKFLOW_METRICS_CLEANUP; } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/RequestMessageBuilder.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/RequestMessageBuilder.java index dc816ec951..32933804a4 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/RequestMessageBuilder.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/RequestMessageBuilder.java @@ -31,5 +31,13 @@ public interface RequestMessageBuilder extends Serializable { return message; } + default Message convert2Command(int opaque) { + Message message = new Message(opaque); + message.setType(getCommandType()); + byte[] body = JsonSerializer.serialize(this); + message.setBody(body); + return message; + } + MessageType getCommandType(); } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskDispatchMessage.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskDispatchRequest.java similarity index 65% rename from dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskDispatchMessage.java rename to dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskDispatchRequest.java index ac95891051..000cce9fc3 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskDispatchMessage.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskDispatchRequest.java @@ -18,38 +18,28 @@ package org.apache.dolphinscheduler.remote.command.task; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.remote.command.BaseMessage; import org.apache.dolphinscheduler.remote.command.MessageType; +import org.apache.dolphinscheduler.remote.command.RequestMessageBuilder; +import lombok.AllArgsConstructor; import lombok.Data; -import lombok.EqualsAndHashCode; import lombok.NoArgsConstructor; -import lombok.ToString; /** * The task dispatch message, means dispatch a task to worker. */ @Data @NoArgsConstructor -@ToString(callSuper = true) -@EqualsAndHashCode(callSuper = true) -public class TaskDispatchMessage extends BaseMessage { +@AllArgsConstructor +public class TaskDispatchRequest implements RequestMessageBuilder { private static final long serialVersionUID = -1L; private TaskExecutionContext taskExecutionContext; - public TaskDispatchMessage(TaskExecutionContext taskExecutionContext, - String messageSenderAddress, - String messageReceiverAddress, - long messageSendTime) { - super(messageSenderAddress, messageReceiverAddress, messageSendTime); - this.taskExecutionContext = taskExecutionContext; - } - @Override public MessageType getCommandType() { - return MessageType.TASK_DISPATCH_MESSAGE; + return MessageType.TASK_DISPATCH_REQUEST; } } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskRejectMessage.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskDispatchResponse.java similarity index 58% rename from dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskRejectMessage.java rename to dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskDispatchResponse.java index 58ae5b0380..f9c9bfb319 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskRejectMessage.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskDispatchResponse.java @@ -17,44 +17,34 @@ package org.apache.dolphinscheduler.remote.command.task; -import org.apache.dolphinscheduler.remote.command.BaseMessage; import org.apache.dolphinscheduler.remote.command.MessageType; +import org.apache.dolphinscheduler.remote.command.ResponseMessageBuilder; +import lombok.AllArgsConstructor; import lombok.Data; -import lombok.EqualsAndHashCode; import lombok.NoArgsConstructor; -import lombok.ToString; -/** - * Task reject message, means the task has been rejected by the worker. - */ @Data @NoArgsConstructor -@ToString(callSuper = true) -@EqualsAndHashCode(callSuper = true) -public class TaskRejectMessage extends BaseMessage { - - /** - * taskInstanceId - */ - private int taskInstanceId; - - /** - * host - */ - private String host; - - /** - * process instance id - */ - private int processInstanceId; - - public TaskRejectMessage(String messageSenderAddress, String messageReceiverAddress, long messageSendTime) { - super(messageSenderAddress, messageReceiverAddress, messageSendTime); +@AllArgsConstructor +public class TaskDispatchResponse implements ResponseMessageBuilder { + + private Integer taskInstanceId; + + private boolean dispatchSuccess; + + private String message; + + public static TaskDispatchResponse success(Integer taskInstanceId) { + return new TaskDispatchResponse(taskInstanceId, true, "dispatch success"); + } + + public static TaskDispatchResponse failed(Integer taskInstanceId, String message) { + return new TaskDispatchResponse(taskInstanceId, false, message); } @Override public MessageType getCommandType() { - return MessageType.TASK_REJECT; + return MessageType.TASK_DISPATCH_RESPONSE; } } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskRejectMessageAck.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskPauseRequest.java similarity index 62% rename from dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskRejectMessageAck.java rename to dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskPauseRequest.java index 98bab90953..210a5155bb 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskRejectMessageAck.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskPauseRequest.java @@ -17,36 +17,29 @@ package org.apache.dolphinscheduler.remote.command.task; -import org.apache.dolphinscheduler.remote.command.BaseMessage; import org.apache.dolphinscheduler.remote.command.MessageType; +import org.apache.dolphinscheduler.remote.command.RequestMessageBuilder; import lombok.Data; -import lombok.EqualsAndHashCode; import lombok.NoArgsConstructor; import lombok.ToString; @Data @NoArgsConstructor @ToString(callSuper = true) -@EqualsAndHashCode(callSuper = true) -public class TaskRejectMessageAck extends BaseMessage { - - private int taskInstanceId; - private boolean success; - - public TaskRejectMessageAck(boolean success, - int taskInstanceId, - String messageSenderAddress, - String messageReceiverAddress, - long messageSendTime) { - super(messageSenderAddress, messageReceiverAddress, messageSendTime); - this.success = success; +public class TaskPauseRequest implements RequestMessageBuilder { + + private static final long serialVersionUID = -1L; + + private Integer taskInstanceId; + + public TaskPauseRequest(Integer taskInstanceId) { this.taskInstanceId = taskInstanceId; } @Override public MessageType getCommandType() { - return MessageType.TASK_REJECT_MESSAGE_ACK; + return MessageType.PAUSE_TASK_INSTANCE; } } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskUpdatePidAckMessage.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskUpdateRuntimeAckMessage.java similarity index 90% rename from dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskUpdatePidAckMessage.java rename to dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskUpdateRuntimeAckMessage.java index 317bcee188..c6e2a47479 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskUpdatePidAckMessage.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskUpdateRuntimeAckMessage.java @@ -33,14 +33,14 @@ import lombok.NoArgsConstructor; @Builder @AllArgsConstructor @NoArgsConstructor -public class TaskUpdatePidAckMessage implements RequestMessageBuilder { +public class TaskUpdateRuntimeAckMessage implements RequestMessageBuilder { private boolean success; private int taskInstanceId; @Override public MessageType getCommandType() { - return MessageType.TASK_UPDATE_PID__MESSAGE_ACK; + return MessageType.TASK_UPDATE_RUNTIME_MESSAGE_ACK; } } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskUpdatePidMessage.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskUpdateRuntimeMessage.java similarity index 88% rename from dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskUpdatePidMessage.java rename to dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskUpdateRuntimeMessage.java index 056ad891d6..e90727d46b 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskUpdatePidMessage.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/task/TaskUpdateRuntimeMessage.java @@ -32,7 +32,7 @@ import lombok.ToString; @NoArgsConstructor @ToString(callSuper = true) @EqualsAndHashCode(callSuper = true) -public class TaskUpdatePidMessage extends BaseMessage { +public class TaskUpdateRuntimeMessage extends BaseMessage { /** * taskInstanceId @@ -64,13 +64,13 @@ public class TaskUpdatePidMessage extends BaseMessage { */ private int processId; - public TaskUpdatePidMessage(String messageSenderAddress, String messageReceiverAddress, long messageSendTime) { + public TaskUpdateRuntimeMessage(String messageSenderAddress, String messageReceiverAddress, long messageSendTime) { super(messageSenderAddress, messageReceiverAddress, messageSendTime); } @Override public MessageType getCommandType() { - return MessageType.TASK_UPDATE_PID_MESSAGE; + return MessageType.TASK_UPDATE_RUNTIME_MESSAGE; } } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyServerConfig.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyServerConfig.java index 4ec8a0f7a7..798ea732c5 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyServerConfig.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyServerConfig.java @@ -18,9 +18,9 @@ package org.apache.dolphinscheduler.remote.config; import org.apache.dolphinscheduler.remote.utils.Constants; -/** - * netty server config - */ +import lombok.Data; + +@Data public class NettyServerConfig { /** @@ -57,60 +57,4 @@ public class NettyServerConfig { * listen port */ private int listenPort = 12346; - - public int getListenPort() { - return listenPort; - } - - public void setListenPort(int listenPort) { - this.listenPort = listenPort; - } - - public int getSoBacklog() { - return soBacklog; - } - - public void setSoBacklog(int soBacklog) { - this.soBacklog = soBacklog; - } - - public boolean isTcpNoDelay() { - return tcpNoDelay; - } - - public void setTcpNoDelay(boolean tcpNoDelay) { - this.tcpNoDelay = tcpNoDelay; - } - - public boolean isSoKeepalive() { - return soKeepalive; - } - - public void setSoKeepalive(boolean soKeepalive) { - this.soKeepalive = soKeepalive; - } - - public int getSendBufferSize() { - return sendBufferSize; - } - - public void setSendBufferSize(int sendBufferSize) { - this.sendBufferSize = sendBufferSize; - } - - public int getReceiveBufferSize() { - return receiveBufferSize; - } - - public void setReceiveBufferSize(int receiveBufferSize) { - this.receiveBufferSize = receiveBufferSize; - } - - public int getWorkerThread() { - return workerThread; - } - - public void setWorkerThread(int workerThread) { - this.workerThread = workerThread; - } } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java index ea56190d10..40b223d6e4 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java @@ -20,8 +20,6 @@ package org.apache.dolphinscheduler.remote.future; import org.apache.dolphinscheduler.remote.command.Message; import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; @@ -85,7 +83,9 @@ public class ResponseFuture { * @return command */ public Message waitResponse() throws InterruptedException { - this.latch.await(timeoutMillis, TimeUnit.MILLISECONDS); + if (!latch.await(timeoutMillis, TimeUnit.MILLISECONDS)) { + log.warn("Wait response timeout, request id {}", opaque); + } return this.responseMessage; } @@ -180,24 +180,22 @@ public class ResponseFuture { * scan future table */ public static void scanFutureTable() { - final List futureList = new LinkedList<>(); Iterator> it = FUTURE_TABLE.entrySet().iterator(); while (it.hasNext()) { Map.Entry next = it.next(); ResponseFuture future = next.getValue(); - if ((future.getBeginTimestamp() + future.getTimeoutMillis() + 1000) <= System.currentTimeMillis()) { - futureList.add(future); - it.remove(); - log.warn("remove timeout request : {}", future); + if ((future.getBeginTimestamp() + future.getTimeoutMillis() + 1000) > System.currentTimeMillis()) { + continue; } - } - for (ResponseFuture future : futureList) { try { + // todo: use thread pool to execute the async callback, otherwise will block the scan thread future.release(); future.executeInvokeCallback(); } catch (Exception ex) { - log.warn("scanFutureTable, execute callback error", ex); + log.error("ScanFutureTable, execute callback error, requestId: {}", future.getOpaque(), ex); } + it.remove(); + log.debug("Remove timeout request: {}", future); } } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/BaseLogProcessor.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/BaseLogProcessor.java index 52f6c9dd70..c651c01939 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/BaseLogProcessor.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/BaseLogProcessor.java @@ -20,6 +20,6 @@ package org.apache.dolphinscheduler.remote.processor; import lombok.extern.slf4j.Slf4j; @Slf4j -public abstract class BaseLogProcessor { +public abstract class BaseLogProcessor implements MasterRpcProcessor, WorkerRpcProcessor { } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/MasterRpcProcessor.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/MasterRpcProcessor.java new file mode 100644 index 0000000000..2bac00ab84 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/MasterRpcProcessor.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.remote.processor; + +public interface MasterRpcProcessor extends NettyRequestProcessor { +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/WorkerRpcProcessor.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/WorkerRpcProcessor.java new file mode 100644 index 0000000000..898d56475c --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/WorkerRpcProcessor.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.remote.processor; + +public interface WorkerRpcProcessor extends NettyRequestProcessor { +} diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java index 97c9e9f28e..a7e01101a7 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java @@ -88,11 +88,11 @@ public interface ProcessService { void setSubProcessParam(ProcessInstance subProcessInstance); - TaskInstance submitTaskWithRetry(ProcessInstance processInstance, TaskInstance taskInstance, int commitRetryTimes, - long commitInterval); + boolean submitTaskWithRetry(ProcessInstance processInstance, TaskInstance taskInstance, int commitRetryTimes, + long commitInterval); @Transactional - TaskInstance submitTask(ProcessInstance processInstance, TaskInstance taskInstance); + boolean submitTask(ProcessInstance processInstance, TaskInstance taskInstance); void createSubWorkProcess(ProcessInstance parentProcessInstance, TaskInstance task); diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java index be997c934a..de19938dd9 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java @@ -1119,17 +1119,15 @@ public class ProcessServiceImpl implements ProcessService { * retry submit task to db */ @Override - public TaskInstance submitTaskWithRetry(ProcessInstance processInstance, TaskInstance taskInstance, - int commitRetryTimes, long commitInterval) { + public boolean submitTaskWithRetry(ProcessInstance processInstance, TaskInstance taskInstance, + int commitRetryTimes, long commitInterval) { int retryTimes = 1; - TaskInstance task = null; while (retryTimes <= commitRetryTimes) { try { // submit task to db // Only want to use transaction here - task = submitTask(processInstance, taskInstance); - if (task != null && task.getId() != null) { - break; + if (submitTask(processInstance, taskInstance)) { + return true; } log.error( "task commit to db failed , taskCode: {} has already retry {} times, please check the database", @@ -1142,7 +1140,7 @@ public class ProcessServiceImpl implements ProcessService { retryTimes += 1; } } - return task; + return false; } /** @@ -1156,33 +1154,32 @@ public class ProcessServiceImpl implements ProcessService { */ @Override @Transactional - public TaskInstance submitTask(ProcessInstance processInstance, TaskInstance taskInstance) { + public boolean submitTask(ProcessInstance processInstance, TaskInstance taskInstance) { log.info("Start save taskInstance to database : {}, processInstance id:{}, state: {}", taskInstance.getName(), taskInstance.getProcessInstanceId(), processInstance.getState()); // submit to db - TaskInstance task = taskInstanceDao.submitTaskInstanceToDB(taskInstance, processInstance); - if (task == null) { + if (!taskInstanceDao.submitTaskInstanceToDB(taskInstance, processInstance)) { log.error("Save taskInstance to db error, task name:{}, process id:{} state: {} ", taskInstance.getName(), taskInstance.getProcessInstance().getId(), processInstance.getState()); - return null; + return false; } - if (!task.getState().isFinished()) { - createSubWorkProcess(processInstance, task); + if (!taskInstance.getState().isFinished()) { + createSubWorkProcess(processInstance, taskInstance); } log.info( "End save taskInstance to db successfully:{}, taskInstanceName: {}, taskInstance state:{}, processInstanceId:{}, processInstanceState: {}", - task.getId(), - task.getName(), - task.getState(), + taskInstance.getId(), + taskInstance.getName(), + taskInstance.getState(), processInstance.getId(), processInstance.getState()); - return task; + return true; } /** diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/utils/DagHelper.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/utils/DagHelper.java index 0f47466153..f8f858b18e 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/utils/DagHelper.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/utils/DagHelper.java @@ -423,7 +423,7 @@ public class DagHelper { if (CollectionUtils.isEmpty(switchTaskList)) { switchTaskList = new ArrayList<>(); } - conditionResultVoList.remove(resultConditionLocation); + // conditionResultVoList.remove(resultConditionLocation); for (SwitchResultVo info : conditionResultVoList) { if (CollectionUtils.isEmpty(info.getNextNode())) { continue; diff --git a/dolphinscheduler-standalone-server/src/main/java/org/apache/dolphinscheduler/StandaloneServer.java b/dolphinscheduler-standalone-server/src/main/java/org/apache/dolphinscheduler/StandaloneServer.java index e6f0bc5ee5..14808ab629 100644 --- a/dolphinscheduler-standalone-server/src/main/java/org/apache/dolphinscheduler/StandaloneServer.java +++ b/dolphinscheduler-standalone-server/src/main/java/org/apache/dolphinscheduler/StandaloneServer.java @@ -19,39 +19,26 @@ package org.apache.dolphinscheduler; import org.apache.curator.test.TestingServer; -import java.io.IOException; - -import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.springframework.boot.SpringApplication; import org.springframework.boot.autoconfigure.SpringBootApplication; -import org.springframework.boot.context.event.ApplicationFailedEvent; -import org.springframework.context.ApplicationEvent; -import org.springframework.context.ApplicationListener; -import org.springframework.context.event.ContextClosedEvent; @SpringBootApplication @Slf4j -public class StandaloneServer implements ApplicationListener { - - private static TestingServer zookeeperServer; +public class StandaloneServer { public static void main(String[] args) throws Exception { - zookeeperServer = new TestingServer(true); - System.setProperty("registry.zookeeper.connect-string", zookeeperServer.getConnectString()); - SpringApplication.run(StandaloneServer.class, args); - } - - @Override - public void onApplicationEvent(@NonNull ApplicationEvent event) { - if (event instanceof ApplicationFailedEvent || event instanceof ContextClosedEvent) { - try (TestingServer closedServer = zookeeperServer) { - // close the zookeeper server - log.info("Receive spring context close event: {}, will closed zookeeper server", event); - } catch (IOException e) { - log.error("Close zookeeper server error", e); - } + try { + // We cannot use try-with-resources to close "TestingServer", since SpringApplication.run() will not block + // the main thread. + TestingServer zookeeperServer = new TestingServer(true); + System.setProperty("registry.zookeeper.connect-string", zookeeperServer.getConnectString()); + SpringApplication.run(StandaloneServer.class, args); + } catch (Exception ex) { + log.error("StandaloneServer start failed", ex); + System.exit(1); } } + } diff --git a/dolphinscheduler-standalone-server/src/main/resources/application.yaml b/dolphinscheduler-standalone-server/src/main/resources/application.yaml index 02666512b5..9dd63d68f6 100644 --- a/dolphinscheduler-standalone-server/src/main/resources/application.yaml +++ b/dolphinscheduler-standalone-server/src/main/resources/application.yaml @@ -142,8 +142,8 @@ master: state-wheel-interval: 5s # master max cpuload avg percentage, only higher than the system cpu load average, master server can schedule. default value 1: will use 100% cpu max-cpu-load-avg: 1 - # master reserved memory, only lower than system available memory, master server can schedule. default value 0.3, only the available memory is higher than 30%, master server can schedule. - reserved-memory: 0.3 + # master reserved memory, only lower than system available memory, master server can schedule. default value 0.1, only the available memory is higher than 10%, master server can schedule. + reserved-memory: 0.1 # failover interval failover-interval: 10m # kill yarn/k8s application when failover taskInstance, default true @@ -165,8 +165,8 @@ worker: tenant-distributed-user: false # worker max cpuload avg, only higher than the system cpu load average, worker server can be dispatched tasks. default value 1: will use 100% cpu. max-cpu-load-avg: 1 - # worker reserved memory, only lower than system available memory, worker server can be dispatched tasks. default value 0.3, only the available memory is higher than 30%, worker server can receive task. - reserved-memory: 0.3 + # worker reserved memory, only lower than system available memory, worker server can be dispatched tasks. default value 0.1, only the available memory is higher than 10%, worker server can receive task. + reserved-memory: 0.1 task-execute-threads-full-policy: REJECT alert: diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractTask.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractTask.java index 1986957c98..30f5b7d30f 100644 --- a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractTask.java +++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractTask.java @@ -155,19 +155,14 @@ public abstract class AbstractTask { * @return exit status */ public TaskExecutionStatus getExitStatus() { - TaskExecutionStatus status; switch (getExitStatusCode()) { case TaskConstants.EXIT_CODE_SUCCESS: - status = TaskExecutionStatus.SUCCESS; - break; + return TaskExecutionStatus.SUCCESS; case TaskConstants.EXIT_CODE_KILL: - status = TaskExecutionStatus.KILL; - break; + return TaskExecutionStatus.KILL; default: - status = TaskExecutionStatus.FAILURE; - break; + return TaskExecutionStatus.FAILURE; } - return status; } /** diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/log/TaskInstanceLogHeader.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/log/TaskInstanceLogHeader.java similarity index 98% rename from dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/log/TaskInstanceLogHeader.java rename to dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/log/TaskInstanceLogHeader.java index 4bfb995bb5..02ee2e1c99 100644 --- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/log/TaskInstanceLogHeader.java +++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/log/TaskInstanceLogHeader.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.dolphinscheduler.server.worker.log; +package org.apache.dolphinscheduler.plugin.task.api.log; import java.util.List; diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/model/SwitchResultVo.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/model/SwitchResultVo.java index 65577890cc..411f6e1014 100644 --- a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/model/SwitchResultVo.java +++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/model/SwitchResultVo.java @@ -20,23 +20,18 @@ package org.apache.dolphinscheduler.plugin.task.api.model; import java.util.ArrayList; import java.util.List; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@NoArgsConstructor +@AllArgsConstructor public class SwitchResultVo { private String condition; private List nextNode; - public String getCondition() { - return condition; - } - - public void setCondition(String condition) { - this.condition = condition; - } - - public List getNextNode() { - return nextNode; - } - public void setNextNode(Object nextNode) { if (nextNode instanceof String) { List nextNodeList = new ArrayList<>(); diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskRejectMessageSender.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskRejectMessageSender.java deleted file mode 100644 index 25b2cfb66e..0000000000 --- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskRejectMessageSender.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.worker.message; - -import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; -import org.apache.dolphinscheduler.remote.command.MessageType; -import org.apache.dolphinscheduler.remote.command.task.TaskRejectMessage; -import org.apache.dolphinscheduler.remote.exceptions.RemotingException; -import org.apache.dolphinscheduler.remote.utils.Host; -import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; -import org.apache.dolphinscheduler.server.worker.rpc.WorkerRpcClient; - -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.stereotype.Component; - -@Component -public class TaskRejectMessageSender implements MessageSender { - - @Autowired - private WorkerRpcClient workerRpcClient; - - @Autowired - private WorkerConfig workerConfig; - - @Override - public void sendMessage(TaskRejectMessage message) throws RemotingException { - workerRpcClient.send(Host.of(message.getMessageReceiverAddress()), message.convert2Command()); - } - - @Override - public TaskRejectMessage buildMessage(TaskExecutionContext taskExecutionContext) { - TaskRejectMessage taskRejectMessage = new TaskRejectMessage(workerConfig.getWorkerAddress(), - taskExecutionContext.getWorkflowInstanceHost(), - System.currentTimeMillis()); - taskRejectMessage.setTaskInstanceId(taskExecutionContext.getTaskInstanceId()); - taskRejectMessage.setProcessInstanceId(taskExecutionContext.getProcessInstanceId()); - taskRejectMessage.setHost(taskExecutionContext.getHost()); - return taskRejectMessage; - } - - @Override - public MessageType getMessageType() { - return MessageType.TASK_REJECT; - } -} diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskUpdatePidMessageSender.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/WorkerTaskUpdateRuntimeMessageSender.java similarity index 82% rename from dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskUpdatePidMessageSender.java rename to dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/WorkerTaskUpdateRuntimeMessageSender.java index 9a5d434ede..6ab3564649 100644 --- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/TaskUpdatePidMessageSender.java +++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/message/WorkerTaskUpdateRuntimeMessageSender.java @@ -19,7 +19,7 @@ package org.apache.dolphinscheduler.server.worker.message; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.remote.command.MessageType; -import org.apache.dolphinscheduler.remote.command.task.TaskUpdatePidMessage; +import org.apache.dolphinscheduler.remote.command.task.TaskUpdateRuntimeMessage; import org.apache.dolphinscheduler.remote.exceptions.RemotingException; import org.apache.dolphinscheduler.remote.utils.Host; import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; @@ -31,7 +31,7 @@ import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; @Component -public class TaskUpdatePidMessageSender implements MessageSender { +public class WorkerTaskUpdateRuntimeMessageSender implements MessageSender { @Autowired private WorkerRpcClient workerRpcClient; @@ -40,14 +40,14 @@ public class TaskUpdatePidMessageSender implements MessageSender nettyRequestProcessors; + private List workerRpcProcessors; private NettyRemotingClient nettyRemotingClient; @@ -50,8 +50,8 @@ public class WorkerRpcClient implements AutoCloseable { NettyClientConfig nettyClientConfig = new NettyClientConfig(); this.nettyRemotingClient = new NettyRemotingClient(nettyClientConfig); // we only use the client to handle the ack message, we can optimize this, send ack to the nettyServer. - for (NettyRequestProcessor nettyRequestProcessor : nettyRequestProcessors) { - this.nettyRemotingClient.registerProcessor(nettyRequestProcessor); + for (WorkerRpcProcessor workerRpcProcessor : workerRpcProcessors) { + this.nettyRemotingClient.registerProcessor(workerRpcProcessor); } log.info("Worker rpc client started"); diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/WorkerRpcServer.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/WorkerRpcServer.java index eac13d273e..94b509498a 100644 --- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/WorkerRpcServer.java +++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/WorkerRpcServer.java @@ -19,7 +19,7 @@ package org.apache.dolphinscheduler.server.worker.rpc; import org.apache.dolphinscheduler.remote.NettyRemotingServer; import org.apache.dolphinscheduler.remote.config.NettyServerConfig; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.processor.WorkerRpcProcessor; import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; import java.io.Closeable; @@ -35,7 +35,7 @@ import org.springframework.stereotype.Service; public class WorkerRpcServer implements Closeable { @Autowired - private List nettyRequestProcessors; + private List workerRpcProcessors; @Autowired private WorkerConfig workerConfig; @@ -43,16 +43,16 @@ public class WorkerRpcServer implements Closeable { private NettyRemotingServer nettyRemotingServer; public void start() { - log.info("Worker rpc server starting"); + log.info("Worker rpc server starting..."); NettyServerConfig serverConfig = new NettyServerConfig(); - nettyRemotingServer = new NettyRemotingServer(serverConfig); serverConfig.setListenPort(workerConfig.getListenPort()); - for (NettyRequestProcessor nettyRequestProcessor : nettyRequestProcessors) { - nettyRemotingServer.registerProcessor(nettyRequestProcessor); - log.info("Success register netty processor: {}", nettyRequestProcessor.getClass().getName()); + nettyRemotingServer = new NettyRemotingServer(serverConfig); + for (WorkerRpcProcessor workerRpcProcessor : workerRpcProcessors) { + nettyRemotingServer.registerProcessor(workerRpcProcessor); + log.info("Success register WorkerRpcProcessor: {}", workerRpcProcessor.getClass().getName()); } this.nettyRemotingServer.start(); - log.info("Worker rpc server started"); + log.info("Worker rpc server started..."); } @Override 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 index 9dba2bf8cd..b6a8f0fe72 100644 --- 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 @@ -64,7 +64,7 @@ public class TaskCallbackImpl implements TaskCallBack { return; } - workerMessageSender.sendMessageWithRetry(taskExecutionContext, MessageType.TASK_UPDATE_PID_MESSAGE); + workerMessageSender.sendMessageWithRetry(taskExecutionContext, MessageType.TASK_UPDATE_RUNTIME_MESSAGE); } } diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecuteRunnable.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecuteRunnable.java index 8189e78144..598fc6cfc9 100644 --- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecuteRunnable.java +++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecuteRunnable.java @@ -28,13 +28,14 @@ import org.apache.dolphinscheduler.plugin.datasource.api.utils.CommonUtils; import org.apache.dolphinscheduler.plugin.storage.api.StorageOperate; 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.TaskExecutionContextCacheManager; 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.utils.LogUtils; import org.apache.dolphinscheduler.plugin.task.api.utils.ProcessUtils; @@ -43,7 +44,6 @@ import org.apache.dolphinscheduler.remote.command.alert.AlertSendRequest; import org.apache.dolphinscheduler.remote.exceptions.RemotingException; import org.apache.dolphinscheduler.remote.utils.Host; import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; -import org.apache.dolphinscheduler.server.worker.log.TaskInstanceLogHeader; import org.apache.dolphinscheduler.server.worker.registry.WorkerRegistryClient; import org.apache.dolphinscheduler.server.worker.rpc.WorkerMessageSender; import org.apache.dolphinscheduler.server.worker.rpc.WorkerRpcClient; @@ -112,28 +112,31 @@ public abstract class WorkerTaskExecuteRunnable implements Runnable { } protected void afterThrowing(Throwable throwable) throws TaskException { - cancelTask(); + if (cancelTask()) { + log.info("Cancel the task successfully"); + } TaskExecutionContextCacheManager.removeByTaskInstanceId(taskExecutionContext.getTaskInstanceId()); taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.FAILURE); taskExecutionContext.setEndTime(System.currentTimeMillis()); workerMessageSender.sendMessageWithRetry(taskExecutionContext, MessageType.TASK_EXECUTE_RESULT_MESSAGE); - log.info( - "Get a exception when execute the task, will send the task execute result to master, the current task execute result is {}", - TaskExecutionStatus.FAILURE); + log.info("Get a exception when execute the task, will send the task status: {} to master: {}", + TaskExecutionStatus.FAILURE.name(), taskExecutionContext.getHost()); } - public void cancelTask() { + public boolean cancelTask() { // cancel the task - if (task != null) { - try { - task.cancel(); - ProcessUtils.cancelApplication(taskExecutionContext); - } catch (Exception e) { - log.error( - "Task execute failed and cancel the application failed, this will not affect the taskInstance status, but you need to check manual", - e); - } + 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; } } @@ -195,36 +198,34 @@ public abstract class WorkerTaskExecuteRunnable implements Runnable { protected void beforeExecute() { taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.RUNNING_EXECUTION); workerMessageSender.sendMessageWithRetry(taskExecutionContext, MessageType.TASK_EXECUTE_RUNNING_MESSAGE); - log.info("Set task status to {}", TaskExecutionStatus.RUNNING_EXECUTION); + log.info("Send task status {} master: {}", TaskExecutionStatus.RUNNING_EXECUTION.name(), + taskExecutionContext.getHost()); TaskExecutionCheckerUtils.checkTenantExist(workerConfig, taskExecutionContext); - log.info("TenantCode: {} check success", taskExecutionContext.getTenantCode()); + log.info("TenantCode: {} check successfully", taskExecutionContext.getTenantCode()); TaskExecutionCheckerUtils.createProcessLocalPathIfAbsent(taskExecutionContext); - log.info("ProcessExecDir:{} check success", taskExecutionContext.getExecutePath()); + log.info("WorkflowInstanceExecDir: {} check successfully", taskExecutionContext.getExecutePath()); - TaskExecutionCheckerUtils.downloadResourcesIfNeeded(storageOperate, taskExecutionContext, log); - log.info("Resources:{} check success", taskExecutionContext.getResources()); + TaskExecutionCheckerUtils.downloadResourcesIfNeeded(storageOperate, taskExecutionContext); + log.info("Download resources: {} successfully", taskExecutionContext.getResources()); TaskFilesTransferUtils.downloadUpstreamFiles(taskExecutionContext, storageOperate); + log.info("Download upstream files: {} successfully", + TaskFilesTransferUtils.getFileLocalParams(taskExecutionContext, Direct.IN)); - TaskChannel taskChannel = taskPluginManager.getTaskChannelMap().get(taskExecutionContext.getTaskType()); - if (null == taskChannel) { - throw new TaskPluginException(String.format("%s task plugin not found, please check config file.", - taskExecutionContext.getTaskType())); - } - task = taskChannel.createTask(taskExecutionContext); - if (task == null) { - throw new TaskPluginException(String.format("%s task is null, please check the task plugin is correct", - taskExecutionContext.getTaskType())); - } - log.info("Task plugin: {} create success", taskExecutionContext.getTaskType()); + task = Optional.ofNullable(taskPluginManager.getTaskChannelMap().get(taskExecutionContext.getTaskType())) + .map(taskChannel -> taskChannel.createTask(taskExecutionContext)) + .orElseThrow(() -> new TaskPluginException(taskExecutionContext.getTaskType() + + " task plugin not found, please check the task type is correct.")); + 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 success"); + log.info("Success initialized task plugin instance successfully"); task.getParameters().setVarPool(taskExecutionContext.getVarPool()); - log.info("Success set taskVarPool: {}", taskExecutionContext.getVarPool()); + log.info("Set taskVarPool: {} successfully", taskExecutionContext.getVarPool()); } @@ -241,7 +242,6 @@ public abstract class WorkerTaskExecuteRunnable implements Runnable { } Host alertServerAddress = alertServerAddressOptional.get(); - log.info("The current task need to send alert, begin to send alert"); TaskExecutionStatus status = task.getExitStatus(); TaskAlertInfo taskAlertInfo = task.getTaskAlertInfo(); int strategy = @@ -253,7 +253,7 @@ public abstract class WorkerTaskExecuteRunnable implements Runnable { strategy); try { workerRpcClient.send(alertServerAddress, alertCommand.convert2Command()); - log.info("Success send alert to : {}", alertServerAddress); + log.info("Send alert to: {} successfully", alertServerAddress); } catch (RemotingException e) { log.error("Send alert to: {} failed, alertCommand: {}", alertServerAddress, alertCommand, e); } @@ -261,16 +261,19 @@ public abstract class WorkerTaskExecuteRunnable implements Runnable { protected void sendTaskResult() { taskExecutionContext.setCurrentExecutionStatus(task.getExitStatus()); - taskExecutionContext.setEndTime(System.currentTimeMillis()); 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, storageOperate); - workerMessageSender.sendMessageWithRetry(taskExecutionContext, MessageType.TASK_EXECUTE_RESULT_MESSAGE); + log.info("Upload output files: {} successfully", + TaskFilesTransferUtils.getFileLocalParams(taskExecutionContext, Direct.OUT)); - log.info("Send task execute result to master, the current task status: {}", - taskExecutionContext.getCurrentExecutionStatus()); + workerMessageSender.sendMessageWithRetry(taskExecutionContext, MessageType.TASK_EXECUTE_RESULT_MESSAGE); + log.info("Send task execute status: {} to master : {}", taskExecutionContext.getCurrentExecutionStatus().name(), + taskExecutionContext.getHost()); } protected void clearTaskExecPathIfNeeded() { diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/utils/TaskExecutionCheckerUtils.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/utils/TaskExecutionCheckerUtils.java index 8b66a7e893..eb77a4182b 100644 --- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/utils/TaskExecutionCheckerUtils.java +++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/utils/TaskExecutionCheckerUtils.java @@ -47,8 +47,6 @@ import java.util.Map; import lombok.extern.slf4j.Slf4j; -import org.slf4j.Logger; - @Slf4j public class TaskExecutionCheckerUtils { @@ -109,7 +107,7 @@ public class TaskExecutionCheckerUtils { } public static void downloadResourcesIfNeeded(StorageOperate storageOperate, - TaskExecutionContext taskExecutionContext, Logger logger) { + TaskExecutionContext taskExecutionContext) { String execLocalPath = taskExecutionContext.getExecutePath(); Map projectRes = taskExecutionContext.getResources(); if (MapUtils.isEmpty(projectRes)) { @@ -122,7 +120,7 @@ public class TaskExecutionCheckerUtils { if (notExist) { downloadFiles.add(Pair.of(key, value)); } else { - logger.info("file : {} exists ", resFile.getName()); + log.info("file : {} exists ", resFile.getName()); } }); if (!downloadFiles.isEmpty() && !PropertyUtils.getResUploadStartupState()) { @@ -133,8 +131,11 @@ public class TaskExecutionCheckerUtils { for (Pair fileDownload : downloadFiles) { try { String fullName = fileDownload.getLeft(); - String fileName = fileDownload.getRight(); - logger.info("get resource file from path:{}", fullName); + // we do not actually get & need tenantCode with this implementation right now. + String tenantCode = fileDownload.getRight(); + // TODO: Need a better way to get fileName because this implementation is tricky. + String fileName = storageOperate.getResourceFileName(fullName); + log.info("get resource file from path:{}", fullName); long resourceDownloadStartTime = System.currentTimeMillis(); storageOperate.download(taskExecutionContext.getTenantCode(), fullName, diff --git a/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/processor/TaskDispatchProcessorTest.java b/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/processor/WorkerTaskDispatchProcessorTest.java similarity index 85% rename from dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/processor/TaskDispatchProcessorTest.java rename to dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/processor/WorkerTaskDispatchProcessorTest.java index b4fcc535de..c65d66cb66 100644 --- a/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/processor/TaskDispatchProcessorTest.java +++ b/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/processor/WorkerTaskDispatchProcessorTest.java @@ -21,8 +21,7 @@ import org.apache.dolphinscheduler.plugin.storage.api.StorageOperate; import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; import org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager; import org.apache.dolphinscheduler.remote.command.Message; -import org.apache.dolphinscheduler.remote.command.MessageType; -import org.apache.dolphinscheduler.remote.command.task.TaskDispatchMessage; +import org.apache.dolphinscheduler.remote.command.task.TaskDispatchRequest; import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; import org.apache.dolphinscheduler.server.worker.registry.WorkerRegistryClient; import org.apache.dolphinscheduler.server.worker.rpc.WorkerMessageSender; @@ -42,10 +41,10 @@ import io.netty.channel.Channel; * test task execute processor */ @ExtendWith(MockitoExtension.class) -public class TaskDispatchProcessorTest { +public class WorkerTaskDispatchProcessorTest { @InjectMocks - private TaskDispatchProcessor taskDispatchProcessor; + private WorkerTaskDispatchProcessor workerTaskDispatchProcessor; @Mock private WorkerConfig workerConfig; @@ -73,19 +72,13 @@ public class TaskDispatchProcessorTest { Channel channel = Mockito.mock(Channel.class); TaskExecutionContext taskExecutionContext = getTaskExecutionContext(); Message dispatchMessage = createDispatchCommand(taskExecutionContext); - taskDispatchProcessor.process(channel, dispatchMessage); + workerTaskDispatchProcessor.process(channel, dispatchMessage); Mockito.verify(workerManagerThread, Mockito.atMostOnce()).offer(Mockito.any()); - Mockito.verify(workerMessageSender, Mockito.never()).sendMessageWithRetry(taskExecutionContext, - MessageType.TASK_REJECT); } public Message createDispatchCommand(TaskExecutionContext taskExecutionContext) { - return new TaskDispatchMessage( - taskExecutionContext, - "localhost:5678", - "localhost:1234", - System.currentTimeMillis()).convert2Command(); + return new TaskDispatchRequest(taskExecutionContext).convert2Command(); } public TaskExecutionContext getTaskExecutionContext() {