mirror of
https://gitee.com/dolphinscheduler/DolphinScheduler.git
synced 2024-11-29 18:58:05 +08:00
[DSIP-46] Unify the logic task params data structure between front-end and back-end (#16193)
This commit is contained in:
parent
f69e06408f
commit
ec6a3e0727
@ -19,22 +19,14 @@ package org.apache.dolphinscheduler.api;
|
||||
|
||||
import org.apache.dolphinscheduler.api.metrics.ApiServerMetrics;
|
||||
import org.apache.dolphinscheduler.common.CommonConfiguration;
|
||||
import org.apache.dolphinscheduler.common.enums.PluginType;
|
||||
import org.apache.dolphinscheduler.common.thread.DefaultUncaughtExceptionHandler;
|
||||
import org.apache.dolphinscheduler.dao.DaoConfiguration;
|
||||
import org.apache.dolphinscheduler.dao.PluginDao;
|
||||
import org.apache.dolphinscheduler.dao.entity.PluginDefine;
|
||||
import org.apache.dolphinscheduler.plugin.datasource.api.plugin.DataSourceProcessorProvider;
|
||||
import org.apache.dolphinscheduler.plugin.storage.api.StorageConfiguration;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager;
|
||||
import org.apache.dolphinscheduler.registry.api.RegistryConfiguration;
|
||||
import org.apache.dolphinscheduler.service.ServiceConfiguration;
|
||||
import org.apache.dolphinscheduler.spi.params.PluginParamsTransfer;
|
||||
import org.apache.dolphinscheduler.spi.params.base.PluginParams;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
|
||||
@ -68,17 +60,7 @@ public class ApiApplicationServer {
|
||||
@EventListener
|
||||
public void run(ApplicationReadyEvent readyEvent) {
|
||||
log.info("Received spring application context ready event will load taskPlugin and write to DB");
|
||||
// install task plugin
|
||||
TaskPluginManager.loadPlugin();
|
||||
DataSourceProcessorProvider.initialize();
|
||||
for (Map.Entry<String, TaskChannelFactory> entry : TaskPluginManager.getTaskChannelFactoryMap().entrySet()) {
|
||||
String taskPluginName = entry.getKey();
|
||||
TaskChannelFactory taskChannelFactory = entry.getValue();
|
||||
List<PluginParams> params = taskChannelFactory.getParams();
|
||||
String paramsJson = PluginParamsTransfer.transferParamsToJson(params);
|
||||
|
||||
PluginDefine pluginDefine = new PluginDefine(taskPluginName, PluginType.TASK.getDesc(), paramsJson);
|
||||
pluginDao.addOrUpdatePluginDefine(pluginDefine);
|
||||
}
|
||||
TaskPluginManager.loadTaskPlugin();
|
||||
}
|
||||
}
|
||||
|
@ -89,8 +89,8 @@ import org.apache.dolphinscheduler.extract.master.dto.WorkflowExecuteDto;
|
||||
import org.apache.dolphinscheduler.extract.master.transportor.StreamingTaskTriggerRequest;
|
||||
import org.apache.dolphinscheduler.extract.master.transportor.StreamingTaskTriggerResponse;
|
||||
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStateChangeEvent;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
|
||||
import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType;
|
||||
import org.apache.dolphinscheduler.service.command.CommandService;
|
||||
import org.apache.dolphinscheduler.service.cron.CronUtils;
|
||||
@ -361,7 +361,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
|
||||
// find out the process definition code
|
||||
Set<Long> processDefinitionCodeSet = new HashSet<>();
|
||||
taskDefinitions.stream()
|
||||
.filter(task -> TaskConstants.TASK_TYPE_SUB_PROCESS.equalsIgnoreCase(task.getTaskType())).forEach(
|
||||
.filter(task -> TaskTypeUtils.isSubWorkflowTask(task.getTaskType())).forEach(
|
||||
taskDefinition -> processDefinitionCodeSet.add(Long.valueOf(
|
||||
JSONUtils.getNodeString(taskDefinition.getTaskParams(),
|
||||
CMD_PARAM_SUB_PROCESS_DEFINE_CODE))));
|
||||
|
@ -39,7 +39,7 @@ import static org.apache.dolphinscheduler.common.constants.Constants.IMPORT_SUFF
|
||||
import static org.apache.dolphinscheduler.common.constants.Constants.LOCAL_PARAMS;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.LOCAL_PARAMS_LIST;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_SQL;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager.checkTaskParameters;
|
||||
|
||||
import org.apache.dolphinscheduler.api.dto.DagDataSchedule;
|
||||
import org.apache.dolphinscheduler.api.dto.treeview.Instance;
|
||||
@ -109,12 +109,12 @@ import org.apache.dolphinscheduler.dao.repository.ProcessDefinitionDao;
|
||||
import org.apache.dolphinscheduler.dao.repository.ProcessDefinitionLogDao;
|
||||
import org.apache.dolphinscheduler.dao.repository.TaskDefinitionLogDao;
|
||||
import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.SqlType;
|
||||
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.parameters.ParametersNode;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.SqlParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.sql.SqlTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.service.alert.ListenerEventAlertManager;
|
||||
import org.apache.dolphinscheduler.service.model.TaskNode;
|
||||
import org.apache.dolphinscheduler.service.process.ProcessService;
|
||||
@ -421,11 +421,7 @@ public class ProcessDefinitionServiceImpl extends BaseServiceImpl implements Pro
|
||||
throw new ServiceException(Status.DATA_IS_NOT_VALID, taskDefinitionJson);
|
||||
}
|
||||
for (TaskDefinitionLog taskDefinitionLog : taskDefinitionLogs) {
|
||||
if (!TaskPluginManager.checkTaskParameters(ParametersNode.builder()
|
||||
.taskType(taskDefinitionLog.getTaskType())
|
||||
.taskParams(taskDefinitionLog.getTaskParams())
|
||||
.dependence(taskDefinitionLog.getDependence())
|
||||
.build())) {
|
||||
if (!checkTaskParameters(taskDefinitionLog.getTaskType(), taskDefinitionLog.getTaskParams())) {
|
||||
log.error(
|
||||
"Generate task definition list failed, the given task definition parameter is invalided, taskName: {}, taskDefinition: {}",
|
||||
taskDefinitionLog.getName(), taskDefinitionLog);
|
||||
@ -1386,7 +1382,7 @@ public class ProcessDefinitionServiceImpl extends BaseServiceImpl implements Pro
|
||||
sqlParameters.setLocalParams(Collections.emptyList());
|
||||
taskDefinition.setTaskParams(JSONUtils.toJsonString(sqlParameters));
|
||||
taskDefinition.setCode(CodeGenerateUtils.genCode());
|
||||
taskDefinition.setTaskType(TASK_TYPE_SQL);
|
||||
taskDefinition.setTaskType(SqlTaskChannelFactory.NAME);
|
||||
taskDefinition.setFailRetryTimes(0);
|
||||
taskDefinition.setFailRetryInterval(0);
|
||||
taskDefinition.setTimeoutFlag(TimeoutFlag.CLOSE);
|
||||
@ -1615,13 +1611,7 @@ public class ProcessDefinitionServiceImpl extends BaseServiceImpl implements Pro
|
||||
|
||||
// check whether the process definition json is normal
|
||||
for (TaskNode taskNode : taskNodes) {
|
||||
if (!TaskPluginManager.checkTaskParameters(ParametersNode.builder()
|
||||
.taskType(taskNode.getType())
|
||||
.taskParams(taskNode.getTaskParams())
|
||||
.dependence(taskNode.getDependence())
|
||||
.switchResult(taskNode.getSwitchResult())
|
||||
.build())) {
|
||||
log.error("Task node {} parameter invalid.", taskNode.getName());
|
||||
if (!checkTaskParameters(taskNode.getType(), taskNode.getParams())) {
|
||||
putMsg(result, Status.PROCESS_NODE_S_PARAMETER_INVALID, taskNode.getName());
|
||||
return result;
|
||||
}
|
||||
@ -1891,7 +1881,7 @@ public class ProcessDefinitionServiceImpl extends BaseServiceImpl implements Pro
|
||||
|
||||
long subProcessCode = 0L;
|
||||
// if process is sub process, the return sub id, or sub id=0
|
||||
if (taskInstance.isSubProcess()) {
|
||||
if (TaskTypeUtils.isSubWorkflowTask(taskInstance.getTaskType())) {
|
||||
TaskDefinition taskDefinition = taskDefinitionMap.get(taskInstance.getTaskCode());
|
||||
subProcessCode = Long.parseLong(JSONUtils.parseObject(
|
||||
taskDefinition.getTaskParams()).path(CMD_PARAM_SUB_PROCESS_DEFINE_CODE).asText());
|
||||
|
@ -26,8 +26,7 @@ import static org.apache.dolphinscheduler.common.constants.Constants.GLOBAL_PARA
|
||||
import static org.apache.dolphinscheduler.common.constants.Constants.LOCAL_PARAMS;
|
||||
import static org.apache.dolphinscheduler.common.constants.Constants.PROCESS_INSTANCE_STATE;
|
||||
import static org.apache.dolphinscheduler.common.constants.Constants.TASK_LIST;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_DEPENDENT;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_SUB_PROCESS;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager.checkTaskParameters;
|
||||
|
||||
import org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant;
|
||||
import org.apache.dolphinscheduler.api.dto.DynamicSubWorkflowDto;
|
||||
@ -77,11 +76,10 @@ import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
|
||||
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceMapDao;
|
||||
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
|
||||
import org.apache.dolphinscheduler.dao.utils.WorkflowUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.DependResult;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.ParameterUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
|
||||
import org.apache.dolphinscheduler.service.expand.CuringParamsService;
|
||||
import org.apache.dolphinscheduler.service.model.TaskNode;
|
||||
import org.apache.dolphinscheduler.service.process.ProcessService;
|
||||
@ -498,7 +496,7 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
|
||||
throw new ServiceException(Status.TASK_INSTANCE_NOT_EXISTS, taskId);
|
||||
}
|
||||
|
||||
if (!taskInstance.isDynamic()) {
|
||||
if (!TaskTypeUtils.isDynamicTask(taskInstance.getTaskType())) {
|
||||
putMsg(result, Status.TASK_INSTANCE_NOT_DYNAMIC_TASK, taskInstance.getName());
|
||||
throw new ServiceException(Status.TASK_INSTANCE_NOT_EXISTS, taskId);
|
||||
}
|
||||
@ -548,7 +546,7 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
|
||||
*/
|
||||
private void addDependResultForTaskList(User loginUser, List<TaskInstance> taskInstanceList) throws IOException {
|
||||
for (TaskInstance taskInstance : taskInstanceList) {
|
||||
if (TASK_TYPE_DEPENDENT.equalsIgnoreCase(taskInstance.getTaskType())) {
|
||||
if (TaskTypeUtils.isDependentTask(taskInstance.getTaskType())) {
|
||||
log.info("DEPENDENT type task instance need to set dependent result, taskCode:{}, taskInstanceId:{}",
|
||||
taskInstance.getTaskCode(), taskInstance.getId());
|
||||
// TODO The result of dependent item should not be obtained from the log, waiting for optimization.
|
||||
@ -628,9 +626,7 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
|
||||
return result;
|
||||
}
|
||||
|
||||
if (!taskInstance.isSubProcess()) {
|
||||
log.warn("Task instance is not {} type instance, projectCode:{}, taskInstanceId:{}.",
|
||||
TASK_TYPE_SUB_PROCESS, projectCode, taskId);
|
||||
if (!TaskTypeUtils.isSubWorkflowTask(taskInstance.getTaskType())) {
|
||||
putMsg(result, Status.TASK_INSTANCE_NOT_SUB_WORKFLOW_INSTANCE, taskInstance.getName());
|
||||
return result;
|
||||
}
|
||||
@ -714,11 +710,7 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
|
||||
return result;
|
||||
}
|
||||
for (TaskDefinitionLog taskDefinitionLog : taskDefinitionLogs) {
|
||||
if (!TaskPluginManager.checkTaskParameters(ParametersNode.builder()
|
||||
.taskType(taskDefinitionLog.getTaskType())
|
||||
.taskParams(taskDefinitionLog.getTaskParams())
|
||||
.dependence(taskDefinitionLog.getDependence())
|
||||
.build())) {
|
||||
if (!checkTaskParameters(taskDefinitionLog.getTaskType(), taskDefinitionLog.getTaskParams())) {
|
||||
log.error("Task parameters are invalid, taskDefinitionName:{}.", taskDefinitionLog.getName());
|
||||
putMsg(result, Status.PROCESS_NODE_S_PARAMETER_INVALID, taskDefinitionLog.getName());
|
||||
return result;
|
||||
|
@ -18,9 +18,6 @@
|
||||
package org.apache.dolphinscheduler.api.service.impl;
|
||||
|
||||
import static java.util.stream.Collectors.toSet;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_CONDITIONS;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_DEPENDENT;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_SUB_PROCESS;
|
||||
|
||||
import org.apache.dolphinscheduler.api.dto.taskRelation.TaskRelationCreateRequest;
|
||||
import org.apache.dolphinscheduler.api.dto.taskRelation.TaskRelationFilterRequest;
|
||||
@ -46,6 +43,7 @@ import org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationMapper;
|
||||
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
|
||||
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionLogMapper;
|
||||
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
|
||||
import org.apache.dolphinscheduler.service.process.ProcessService;
|
||||
|
||||
import org.apache.commons.collections4.CollectionUtils;
|
||||
@ -354,9 +352,9 @@ public class ProcessTaskRelationServiceImpl extends BaseServiceImpl implements P
|
||||
}
|
||||
updateProcessDefiniteVersion(loginUser, result, processDefinition);
|
||||
updateRelation(loginUser, result, processDefinition, processTaskRelationList);
|
||||
if (TASK_TYPE_CONDITIONS.equals(taskDefinition.getTaskType())
|
||||
|| TASK_TYPE_DEPENDENT.equals(taskDefinition.getTaskType())
|
||||
|| TASK_TYPE_SUB_PROCESS.equals(taskDefinition.getTaskType())) {
|
||||
if (TaskTypeUtils.isConditionTask(taskDefinition.getTaskType())
|
||||
|| TaskTypeUtils.isSubWorkflowTask(taskDefinition.getTaskType())
|
||||
|| TaskTypeUtils.isDependentTask(taskDefinition.getTaskType())) {
|
||||
int deleteTaskDefinition = taskDefinitionMapper.deleteByCode(taskCode);
|
||||
if (0 == deleteTaskDefinition) {
|
||||
log.error("Delete task definition error, taskDefinitionCode:{}.", taskCode);
|
||||
|
@ -24,6 +24,7 @@ import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationCon
|
||||
import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.TASK_VERSION_VIEW;
|
||||
import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_DEFINITION;
|
||||
import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_SWITCH_TO_THIS_VERSION;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager.checkTaskParameters;
|
||||
|
||||
import org.apache.dolphinscheduler.api.dto.task.TaskCreateRequest;
|
||||
import org.apache.dolphinscheduler.api.dto.task.TaskFilterRequest;
|
||||
@ -67,8 +68,6 @@ import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionLogMapper;
|
||||
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
|
||||
import org.apache.dolphinscheduler.dao.repository.ProcessTaskRelationLogDao;
|
||||
import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
|
||||
import org.apache.dolphinscheduler.service.process.ProcessService;
|
||||
|
||||
import org.apache.commons.collections4.CollectionUtils;
|
||||
@ -167,11 +166,7 @@ public class TaskDefinitionServiceImpl extends BaseServiceImpl implements TaskDe
|
||||
return result;
|
||||
}
|
||||
for (TaskDefinitionLog taskDefinitionLog : taskDefinitionLogs) {
|
||||
if (!TaskPluginManager.checkTaskParameters(ParametersNode.builder()
|
||||
.taskType(taskDefinitionLog.getTaskType())
|
||||
.taskParams(taskDefinitionLog.getTaskParams())
|
||||
.dependence(taskDefinitionLog.getDependence())
|
||||
.build())) {
|
||||
if (!checkTaskParameters(taskDefinitionLog.getTaskType(), taskDefinitionLog.getTaskParams())) {
|
||||
log.warn("Task definition {} parameters are invalid.", taskDefinitionLog.getName());
|
||||
putMsg(result, Status.PROCESS_NODE_S_PARAMETER_INVALID, taskDefinitionLog.getName());
|
||||
return result;
|
||||
@ -208,11 +203,7 @@ public class TaskDefinitionServiceImpl extends BaseServiceImpl implements TaskDe
|
||||
Project project = projectMapper.queryByCode(taskDefinition.getProjectCode());
|
||||
projectService.checkProjectAndAuthThrowException(user, project, permissions);
|
||||
|
||||
if (!TaskPluginManager.checkTaskParameters(ParametersNode.builder()
|
||||
.taskType(taskDefinition.getTaskType())
|
||||
.taskParams(taskDefinition.getTaskParams())
|
||||
.dependence(taskDefinition.getDependence())
|
||||
.build())) {
|
||||
if (!checkTaskParameters(taskDefinition.getTaskType(), taskDefinition.getTaskParams())) {
|
||||
throw new ServiceException(Status.PROCESS_NODE_S_PARAMETER_INVALID, taskDefinition.getName());
|
||||
}
|
||||
}
|
||||
@ -321,12 +312,7 @@ public class TaskDefinitionServiceImpl extends BaseServiceImpl implements TaskDe
|
||||
putMsg(result, Status.DATA_IS_NOT_VALID, taskDefinitionJsonObj);
|
||||
return result;
|
||||
}
|
||||
if (!TaskPluginManager.checkTaskParameters(ParametersNode.builder()
|
||||
.taskType(taskDefinition.getTaskType())
|
||||
.taskParams(taskDefinition.getTaskParams())
|
||||
.dependence(taskDefinition.getDependence())
|
||||
.build())) {
|
||||
log.error("Task definition {} parameters are invalid", taskDefinition.getName());
|
||||
if (!checkTaskParameters(taskDefinition.getTaskType(), taskDefinition.getTaskParams())) {
|
||||
putMsg(result, Status.PROCESS_NODE_S_PARAMETER_INVALID, taskDefinition.getName());
|
||||
return result;
|
||||
}
|
||||
@ -732,13 +718,7 @@ public class TaskDefinitionServiceImpl extends BaseServiceImpl implements TaskDe
|
||||
putMsg(result, Status.DATA_IS_NOT_VALID, taskDefinitionJsonObj);
|
||||
return null;
|
||||
}
|
||||
if (!TaskPluginManager.checkTaskParameters(ParametersNode.builder()
|
||||
.taskType(taskDefinitionToUpdate.getTaskType())
|
||||
.taskParams(taskDefinitionToUpdate.getTaskParams())
|
||||
.dependence(taskDefinitionToUpdate.getDependence())
|
||||
.build())) {
|
||||
log.warn("Task definition parameters are invalid, taskDefinitionName:{}.",
|
||||
taskDefinitionToUpdate.getName());
|
||||
if (!checkTaskParameters(taskDefinitionToUpdate.getTaskType(), taskDefinitionToUpdate.getTaskParams())) {
|
||||
putMsg(result, Status.PROCESS_NODE_S_PARAMETER_INVALID, taskDefinitionToUpdate.getName());
|
||||
return null;
|
||||
}
|
||||
|
@ -17,8 +17,6 @@
|
||||
|
||||
package org.apache.dolphinscheduler.api.service.impl;
|
||||
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_DEPENDENT;
|
||||
|
||||
import org.apache.dolphinscheduler.api.enums.Status;
|
||||
import org.apache.dolphinscheduler.api.exceptions.ServiceException;
|
||||
import org.apache.dolphinscheduler.api.service.WorkFlowLineageService;
|
||||
@ -39,6 +37,7 @@ import org.apache.dolphinscheduler.dao.mapper.WorkFlowLineageMapper;
|
||||
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.TaskTypeUtils;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
|
||||
@ -147,7 +146,7 @@ public class WorkFlowLineageServiceImpl extends BaseServiceImpl implements WorkF
|
||||
List<Long> processDefinitionCodes) {
|
||||
for (DependentProcessDefinition dependentProcessDefinition : dependentDefinitionList) {
|
||||
for (DependentTaskModel dependentTaskModel : dependentProcessDefinition.getDependentParameters()
|
||||
.getDependTaskList()) {
|
||||
.getDependence().getDependTaskList()) {
|
||||
for (DependentItem dependentItem : dependentTaskModel.getDependItemList()) {
|
||||
if (!processDefinitionCodes.contains(dependentItem.getDefinitionCode())) {
|
||||
processDefinitionCodes.add(dependentItem.getDefinitionCode());
|
||||
@ -220,12 +219,12 @@ public class WorkFlowLineageServiceImpl extends BaseServiceImpl implements WorkF
|
||||
List<TaskDefinitionLog> taskDefinitionLogs = taskDefinitionLogMapper.queryByTaskDefinitions(taskDefinitionList);
|
||||
for (TaskDefinitionLog taskDefinitionLog : taskDefinitionLogs) {
|
||||
if (taskDefinitionLog.getProjectCode() == projectCode) {
|
||||
if (taskDefinitionLog.getTaskType().equals(TASK_TYPE_DEPENDENT)) {
|
||||
if (TaskTypeUtils.isDependentTask(taskDefinitionLog.getTaskType())) {
|
||||
DependentParameters dependentParameters =
|
||||
JSONUtils.parseObject(taskDefinitionLog.getDependence(), DependentParameters.class);
|
||||
if (dependentParameters != null) {
|
||||
List<DependentTaskModel> dependTaskList =
|
||||
dependentParameters.getDependTaskList();
|
||||
dependentParameters.getDependence().getDependTaskList();
|
||||
if (!CollectionUtils.isEmpty(dependTaskList)) {
|
||||
for (DependentTaskModel taskModel : dependTaskList) {
|
||||
List<DependentItem> dependItemList = taskModel.getDependItemList();
|
||||
@ -247,9 +246,9 @@ public class WorkFlowLineageServiceImpl extends BaseServiceImpl implements WorkF
|
||||
/**
|
||||
* Query and return tasks dependence with string format, is a wrapper of queryTaskDepOnTask and task query method.
|
||||
*
|
||||
* @param projectCode Project code want to query tasks dependence
|
||||
* @param projectCode Project code want to query tasks dependence
|
||||
* @param processDefinitionCode Process definition code want to query tasks dependence
|
||||
* @param taskCode Task code want to query tasks dependence
|
||||
* @param taskCode Task code want to query tasks dependence
|
||||
* @return Optional of formatter message
|
||||
*/
|
||||
@Override
|
||||
@ -271,7 +270,7 @@ public class WorkFlowLineageServiceImpl extends BaseServiceImpl implements WorkF
|
||||
/**
|
||||
* Query tasks depend on process definition, include upstream or downstream
|
||||
*
|
||||
* @param projectCode Project code want to query tasks dependence
|
||||
* @param projectCode Project code want to query tasks dependence
|
||||
* @param processDefinitionCode Process definition code want to query tasks dependence
|
||||
* @return Set of TaskMainInfo
|
||||
*/
|
||||
@ -291,7 +290,7 @@ public class WorkFlowLineageServiceImpl extends BaseServiceImpl implements WorkF
|
||||
* Query downstream tasks depend on a process definition or a task
|
||||
*
|
||||
* @param processDefinitionCode Process definition code want to query tasks dependence
|
||||
* @param taskCode Task code want to query tasks dependence
|
||||
* @param taskCode Task code want to query tasks dependence
|
||||
* @return downstream dependent tasks
|
||||
*/
|
||||
@Override
|
||||
|
@ -627,7 +627,8 @@ public class ProcessInstanceServiceTest {
|
||||
try (
|
||||
MockedStatic<TaskPluginManager> taskPluginManagerMockedStatic =
|
||||
Mockito.mockStatic(TaskPluginManager.class)) {
|
||||
taskPluginManagerMockedStatic.when(() -> TaskPluginManager.checkTaskParameters(Mockito.any()))
|
||||
taskPluginManagerMockedStatic
|
||||
.when(() -> TaskPluginManager.checkTaskParameters(Mockito.any(), Mockito.any()))
|
||||
.thenReturn(true);
|
||||
Map<String, Object> processInstanceFinishRes =
|
||||
processInstanceService.updateProcessInstance(loginUser, projectCode, 1,
|
||||
|
@ -64,6 +64,7 @@ import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionLogMapper;
|
||||
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
|
||||
import org.apache.dolphinscheduler.dao.repository.ProcessTaskRelationLogDao;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager;
|
||||
import org.apache.dolphinscheduler.plugin.task.shell.ShellTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.service.process.ProcessService;
|
||||
import org.apache.dolphinscheduler.service.process.ProcessServiceImpl;
|
||||
|
||||
@ -165,7 +166,8 @@ public class TaskDefinitionServiceImplTest {
|
||||
try (
|
||||
MockedStatic<TaskPluginManager> taskPluginManagerMockedStatic =
|
||||
Mockito.mockStatic(TaskPluginManager.class)) {
|
||||
taskPluginManagerMockedStatic.when(() -> TaskPluginManager.checkTaskParameters(Mockito.any()))
|
||||
taskPluginManagerMockedStatic
|
||||
.when(() -> TaskPluginManager.checkTaskParameters(Mockito.any(), Mockito.any()))
|
||||
.thenReturn(true);
|
||||
Project project = getProject();
|
||||
when(projectMapper.queryByCode(PROJECT_CODE)).thenReturn(project);
|
||||
@ -194,7 +196,8 @@ public class TaskDefinitionServiceImplTest {
|
||||
try (
|
||||
MockedStatic<TaskPluginManager> taskPluginManagerMockedStatic =
|
||||
Mockito.mockStatic(TaskPluginManager.class)) {
|
||||
taskPluginManagerMockedStatic.when(() -> TaskPluginManager.checkTaskParameters(Mockito.any()))
|
||||
taskPluginManagerMockedStatic
|
||||
.when(() -> TaskPluginManager.checkTaskParameters(Mockito.any(), Mockito.any()))
|
||||
.thenReturn(true);
|
||||
String taskDefinitionJson = getTaskDefinitionJson();
|
||||
|
||||
@ -441,7 +444,8 @@ public class TaskDefinitionServiceImplTest {
|
||||
() -> taskDefinitionService.createTaskDefinitionV2(user, taskCreateRequest));
|
||||
|
||||
// error task definition
|
||||
taskCreateRequest.setTaskParams(TASK_PARAMETER);
|
||||
taskCreateRequest.setTaskType(ShellTaskChannelFactory.NAME);
|
||||
taskCreateRequest.setTaskParams(JSONUtils.toJsonString(new HashMap<>()));
|
||||
doNothing().when(projectService).checkProjectAndAuthThrowException(user, getProject(), TASK_DEFINITION_CREATE);
|
||||
assertThrowsServiceException(Status.PROCESS_NODE_S_PARAMETER_INVALID,
|
||||
() -> taskDefinitionService.createTaskDefinitionV2(user, taskCreateRequest));
|
||||
@ -449,7 +453,8 @@ public class TaskDefinitionServiceImplTest {
|
||||
try (
|
||||
MockedStatic<TaskPluginManager> taskPluginManagerMockedStatic =
|
||||
Mockito.mockStatic(TaskPluginManager.class)) {
|
||||
taskPluginManagerMockedStatic.when(() -> TaskPluginManager.checkTaskParameters(Mockito.any()))
|
||||
taskPluginManagerMockedStatic
|
||||
.when(() -> TaskPluginManager.checkTaskParameters(Mockito.any(), Mockito.any()))
|
||||
.thenReturn(true);
|
||||
|
||||
// error create task definition object
|
||||
@ -502,7 +507,8 @@ public class TaskDefinitionServiceImplTest {
|
||||
try (
|
||||
MockedStatic<TaskPluginManager> taskPluginManagerMockedStatic =
|
||||
Mockito.mockStatic(TaskPluginManager.class)) {
|
||||
taskPluginManagerMockedStatic.when(() -> TaskPluginManager.checkTaskParameters(Mockito.any()))
|
||||
taskPluginManagerMockedStatic
|
||||
.when(() -> TaskPluginManager.checkTaskParameters(Mockito.any(), Mockito.any()))
|
||||
.thenReturn(false);
|
||||
assertThrowsServiceException(Status.PROCESS_NODE_S_PARAMETER_INVALID,
|
||||
() -> taskDefinitionService.updateTaskDefinitionV2(user, TASK_CODE, taskUpdateRequest));
|
||||
@ -511,7 +517,8 @@ public class TaskDefinitionServiceImplTest {
|
||||
try (
|
||||
MockedStatic<TaskPluginManager> taskPluginManagerMockedStatic =
|
||||
Mockito.mockStatic(TaskPluginManager.class)) {
|
||||
taskPluginManagerMockedStatic.when(() -> TaskPluginManager.checkTaskParameters(Mockito.any()))
|
||||
taskPluginManagerMockedStatic
|
||||
.when(() -> TaskPluginManager.checkTaskParameters(Mockito.any(), Mockito.any()))
|
||||
.thenReturn(true);
|
||||
// error task definition nothing update
|
||||
when(processService.isTaskOnline(TASK_CODE)).thenReturn(false);
|
||||
@ -616,7 +623,8 @@ public class TaskDefinitionServiceImplTest {
|
||||
try (
|
||||
MockedStatic<TaskPluginManager> taskPluginManagerMockedStatic =
|
||||
Mockito.mockStatic(TaskPluginManager.class)) {
|
||||
taskPluginManagerMockedStatic.when(() -> TaskPluginManager.checkTaskParameters(Mockito.any()))
|
||||
taskPluginManagerMockedStatic
|
||||
.when(() -> TaskPluginManager.checkTaskParameters(Mockito.any(), Mockito.any()))
|
||||
.thenReturn(true);
|
||||
String taskDefinitionJson = getTaskDefinitionJson();
|
||||
TaskDefinition taskDefinition = getTaskDefinition();
|
||||
|
@ -272,8 +272,6 @@ public final class Constants {
|
||||
*/
|
||||
public static final String FLOWNODE_RUN_FLAG_NORMAL = "NORMAL";
|
||||
|
||||
public static final String COMMON_TASK_TYPE = "common";
|
||||
|
||||
public static final String DEFAULT = "default";
|
||||
public static final String PASSWORD = "password";
|
||||
public static final String XXXXXX = "******";
|
||||
|
@ -27,7 +27,6 @@ public enum StateEventType {
|
||||
TASK_TIMEOUT(3, "task timeout"),
|
||||
WAKE_UP_TASK_GROUP(4, "wait task group"),
|
||||
TASK_RETRY(5, "task retry"),
|
||||
PROCESS_BLOCKED(6, "process blocked"),
|
||||
PROCESS_SUBMIT_FAILED(7, "process submit failed");
|
||||
|
||||
StateEventType(int code, String descp) {
|
||||
|
@ -38,8 +38,6 @@ public enum WorkflowExecutionStatus {
|
||||
SUCCESS(7, "success"),
|
||||
DELAY_EXECUTION(12, "delay execution"),
|
||||
SERIAL_WAIT(14, "serial wait"),
|
||||
READY_BLOCK(15, "ready block"),
|
||||
BLOCK(16, "block"),
|
||||
WAIT_TO_RUN(17, "wait to run"),
|
||||
;
|
||||
|
||||
@ -59,7 +57,6 @@ public enum WorkflowExecutionStatus {
|
||||
READY_PAUSE.getCode(),
|
||||
READY_STOP.getCode(),
|
||||
SERIAL_WAIT.getCode(),
|
||||
READY_BLOCK.getCode(),
|
||||
WAIT_TO_RUN.getCode()
|
||||
};
|
||||
|
||||
@ -91,7 +88,7 @@ public enum WorkflowExecutionStatus {
|
||||
|
||||
public boolean isFinished() {
|
||||
// todo: do we need to remove pause/block in finished judge?
|
||||
return isSuccess() || isFailure() || isStop() || isPause() || isBlock();
|
||||
return isSuccess() || isFailure() || isStop() || isPause();
|
||||
}
|
||||
|
||||
/**
|
||||
@ -119,10 +116,6 @@ public enum WorkflowExecutionStatus {
|
||||
return this == STOP;
|
||||
}
|
||||
|
||||
public boolean isBlock() {
|
||||
return this == BLOCK;
|
||||
}
|
||||
|
||||
public static int[] getNeedFailoverWorkflowInstanceState() {
|
||||
return NEED_FAILOVER_STATES;
|
||||
}
|
||||
|
@ -17,7 +17,6 @@
|
||||
|
||||
package org.apache.dolphinscheduler.dao.entity;
|
||||
|
||||
import org.apache.dolphinscheduler.common.constants.Constants;
|
||||
import org.apache.dolphinscheduler.common.enums.CycleEnum;
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.model.DependentItem;
|
||||
@ -67,7 +66,7 @@ public class DependentProcessDefinition {
|
||||
*/
|
||||
public CycleEnum getDependentCycle(long upstreamProcessDefinitionCode) {
|
||||
DependentParameters dependentParameters = this.getDependentParameters();
|
||||
List<DependentTaskModel> dependentTaskModelList = dependentParameters.getDependTaskList();
|
||||
List<DependentTaskModel> dependentTaskModelList = dependentParameters.getDependence().getDependTaskList();
|
||||
|
||||
for (DependentTaskModel dependentTaskModel : dependentTaskModelList) {
|
||||
List<DependentItem> dependentItemList = dependentTaskModel.getDependItemList();
|
||||
@ -104,11 +103,7 @@ public class DependentProcessDefinition {
|
||||
}
|
||||
|
||||
public DependentParameters getDependentParameters() {
|
||||
return JSONUtils.parseObject(getDependence(), DependentParameters.class);
|
||||
}
|
||||
|
||||
public String getDependence() {
|
||||
return JSONUtils.getNodeString(this.taskParams, Constants.DEPENDENCE);
|
||||
return JSONUtils.parseObject(taskParams, DependentParameters.class);
|
||||
}
|
||||
|
||||
public String getProcessDefinitionName() {
|
||||
|
@ -201,12 +201,6 @@ public class ProcessInstance {
|
||||
*/
|
||||
private Date restartTime;
|
||||
|
||||
/**
|
||||
* workflow block flag
|
||||
*/
|
||||
@TableField(exist = false)
|
||||
private boolean isBlocked;
|
||||
|
||||
/**
|
||||
* test flag
|
||||
*/
|
||||
|
@ -17,30 +17,15 @@
|
||||
|
||||
package org.apache.dolphinscheduler.dao.entity;
|
||||
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_BLOCKING;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_CONDITIONS;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_DEPENDENT;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_DYNAMIC;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_SUB_PROCESS;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_SWITCH;
|
||||
|
||||
import org.apache.dolphinscheduler.common.constants.Constants;
|
||||
import org.apache.dolphinscheduler.common.enums.Flag;
|
||||
import org.apache.dolphinscheduler.common.enums.Priority;
|
||||
import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
|
||||
import org.apache.dolphinscheduler.common.utils.DateUtils;
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ConditionsParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.SwitchParameters;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import lombok.Data;
|
||||
@ -50,7 +35,6 @@ import com.baomidou.mybatisplus.annotation.IdType;
|
||||
import com.baomidou.mybatisplus.annotation.TableField;
|
||||
import com.baomidou.mybatisplus.annotation.TableId;
|
||||
import com.baomidou.mybatisplus.annotation.TableName;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
|
||||
/**
|
||||
* task instance
|
||||
@ -190,21 +174,6 @@ public class TaskInstance implements Serializable {
|
||||
@TableField(updateStrategy = FieldStrategy.IGNORED)
|
||||
private String cacheKey;
|
||||
|
||||
/**
|
||||
* dependency
|
||||
*/
|
||||
@TableField(exist = false)
|
||||
private DependentParameters dependency;
|
||||
|
||||
@TableField(exist = false)
|
||||
private ConditionsParameters conditionsParameters;
|
||||
|
||||
/**
|
||||
* switch dependency
|
||||
*/
|
||||
@TableField(exist = false)
|
||||
private SwitchParameters switchDependency;
|
||||
|
||||
/**
|
||||
* duration
|
||||
*/
|
||||
@ -310,80 +279,6 @@ public class TaskInstance implements Serializable {
|
||||
this.executePath = executePath;
|
||||
}
|
||||
|
||||
public DependentParameters getDependency() {
|
||||
if (this.dependency == null) {
|
||||
Map<String, Object> taskParamsMap =
|
||||
JSONUtils.parseObject(this.getTaskParams(), new TypeReference<Map<String, Object>>() {
|
||||
});
|
||||
this.dependency =
|
||||
JSONUtils.parseObject((String) taskParamsMap.get(Constants.DEPENDENCE), DependentParameters.class);
|
||||
}
|
||||
return this.dependency;
|
||||
}
|
||||
|
||||
public void setDependency(DependentParameters dependency) {
|
||||
this.dependency = dependency;
|
||||
}
|
||||
|
||||
public ConditionsParameters getConditionsParameters() {
|
||||
if (this.conditionsParameters == null) {
|
||||
Map<String, Object> taskParamsMap =
|
||||
JSONUtils.parseObject(this.getTaskParams(), new TypeReference<Map<String, Object>>() {
|
||||
});
|
||||
this.conditionsParameters =
|
||||
JSONUtils.parseObject((String) taskParamsMap.get(Constants.DEPENDENCE), ConditionsParameters.class);
|
||||
}
|
||||
return conditionsParameters;
|
||||
}
|
||||
|
||||
public ConditionsParameters.ConditionResult getConditionResult() {
|
||||
Map<String, Object> taskParamsMap =
|
||||
JSONUtils.parseObject(this.getTaskParams(), new TypeReference<Map<String, Object>>() {
|
||||
});
|
||||
String conditionResult = (String) taskParamsMap.getOrDefault(Constants.CONDITION_RESULT, "");
|
||||
if (StringUtils.isNotEmpty(conditionResult)) {
|
||||
return JSONUtils.parseObject(conditionResult, new TypeReference<ConditionsParameters.ConditionResult>() {
|
||||
});
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public void setConditionResult(ConditionsParameters conditionsParameters) {
|
||||
if (conditionsParameters == null) {
|
||||
return;
|
||||
}
|
||||
Map<String, Object> taskParamsMap =
|
||||
JSONUtils.parseObject(this.getTaskParams(), new TypeReference<Map<String, Object>>() {
|
||||
});
|
||||
if (taskParamsMap == null) {
|
||||
taskParamsMap = new HashMap<>();
|
||||
}
|
||||
taskParamsMap.put(Constants.CONDITION_RESULT, JSONUtils.toJsonString(conditionsParameters));
|
||||
this.setTaskParams(JSONUtils.toJsonString(taskParamsMap));
|
||||
}
|
||||
|
||||
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<String, Object> taskParamsMap =
|
||||
JSONUtils.parseObject(this.getTaskParams(), new TypeReference<Map<String, Object>>() {
|
||||
});
|
||||
this.switchDependency =
|
||||
JSONUtils.parseObject((String) taskParamsMap.get(Constants.SWITCH_RESULT), SwitchParameters.class);
|
||||
}
|
||||
return this.switchDependency;
|
||||
}
|
||||
|
||||
public void setSwitchDependency(SwitchParameters switchDependency) {
|
||||
Map<String, Object> taskParamsMap =
|
||||
JSONUtils.parseObject(this.getTaskParams(), new TypeReference<Map<String, Object>>() {
|
||||
});
|
||||
taskParamsMap.put(Constants.SWITCH_RESULT, JSONUtils.toJsonString(switchDependency));
|
||||
this.switchDependency = switchDependency;
|
||||
this.setTaskParams(JSONUtils.toJsonString(taskParamsMap));
|
||||
}
|
||||
|
||||
public boolean isTaskComplete() {
|
||||
|
||||
return this.getState().isSuccess()
|
||||
@ -392,30 +287,6 @@ public class TaskInstance implements Serializable {
|
||||
|| this.getState().isForceSuccess();
|
||||
}
|
||||
|
||||
public boolean isSubProcess() {
|
||||
return TASK_TYPE_SUB_PROCESS.equalsIgnoreCase(this.taskType);
|
||||
}
|
||||
|
||||
public boolean isDependTask() {
|
||||
return TASK_TYPE_DEPENDENT.equalsIgnoreCase(this.taskType);
|
||||
}
|
||||
|
||||
public boolean isDynamic() {
|
||||
return TASK_TYPE_DYNAMIC.equalsIgnoreCase(this.taskType);
|
||||
}
|
||||
|
||||
public boolean isConditionsTask() {
|
||||
return TASK_TYPE_CONDITIONS.equalsIgnoreCase(this.taskType);
|
||||
}
|
||||
|
||||
public boolean isSwitchTask() {
|
||||
return TASK_TYPE_SWITCH.equalsIgnoreCase(this.taskType);
|
||||
}
|
||||
|
||||
public boolean isBlockingTask() {
|
||||
return TASK_TYPE_BLOCKING.equalsIgnoreCase(this.taskType);
|
||||
}
|
||||
|
||||
public boolean isFirstRun() {
|
||||
return endTime == null;
|
||||
}
|
||||
@ -427,7 +298,7 @@ public class TaskInstance implements Serializable {
|
||||
* @return can try result
|
||||
*/
|
||||
public boolean taskCanRetry() {
|
||||
if (this.isSubProcess()) {
|
||||
if (TaskTypeUtils.isSubWorkflowTask(getTaskType())) {
|
||||
return false;
|
||||
}
|
||||
if (this.getState() == TaskExecutionStatus.NEED_FAULT_TOLERANCE) {
|
||||
|
@ -55,11 +55,9 @@ public class TaskInstanceUtils {
|
||||
target.setPid(source.getPid());
|
||||
target.setAppLink(source.getAppLink());
|
||||
target.setFlag(source.getFlag());
|
||||
target.setDependency(source.getDependency());
|
||||
// todo: we need to cpoy the task params and then copy switchDependency, since the setSwitchDependency rely on
|
||||
// task params, this is really a very bad practice.
|
||||
target.setTaskParams(source.getTaskParams());
|
||||
target.setSwitchDependency(source.getSwitchDependency());
|
||||
target.setDuration(source.getDuration());
|
||||
target.setMaxRetryTimes(source.getMaxRetryTimes());
|
||||
target.setRetryInterval(source.getRetryInterval());
|
||||
|
@ -17,12 +17,7 @@
|
||||
|
||||
package org.apache.dolphinscheduler.dao.entity;
|
||||
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_CONDITIONS;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_DEPENDENT;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_SUB_PROCESS;
|
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.DependentRelation;
|
||||
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;
|
||||
@ -30,35 +25,10 @@ import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameter
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
public class TaskInstanceTest {
|
||||
|
||||
/**
|
||||
* task instance sub process
|
||||
*/
|
||||
@Test
|
||||
public void testTaskInstanceIsSubProcess() {
|
||||
TaskInstance taskInstance = new TaskInstance();
|
||||
|
||||
// sub process
|
||||
taskInstance.setTaskType(TASK_TYPE_SUB_PROCESS);
|
||||
Assertions.assertTrue(taskInstance.isSubProcess());
|
||||
|
||||
// not sub process
|
||||
taskInstance.setTaskType("HTTP");
|
||||
Assertions.assertFalse(taskInstance.isSubProcess());
|
||||
|
||||
// sub process
|
||||
taskInstance.setTaskType(TASK_TYPE_CONDITIONS);
|
||||
Assertions.assertTrue(taskInstance.isConditionsTask());
|
||||
|
||||
// sub process
|
||||
taskInstance.setTaskType(TASK_TYPE_DEPENDENT);
|
||||
Assertions.assertTrue(taskInstance.isDependTask());
|
||||
}
|
||||
|
||||
/**
|
||||
* test for TaskInstance.getDependence
|
||||
*/
|
||||
@ -66,7 +36,6 @@ public class TaskInstanceTest {
|
||||
public void testTaskInstanceGetDependence() {
|
||||
TaskInstance taskInstance = new TaskInstance();
|
||||
taskInstance.setTaskParams(JSONUtils.toJsonString(getDependentParameters()));
|
||||
taskInstance.getDependency();
|
||||
}
|
||||
|
||||
/**
|
||||
@ -82,8 +51,6 @@ public class TaskInstanceTest {
|
||||
dependentItem.setDefinitionCode(222L);
|
||||
dependentItem.setCycle("today");
|
||||
dependentItems.add(dependentItem);
|
||||
dependentParameters.setDependTaskList(dependTaskList);
|
||||
dependentParameters.setRelation(DependentRelation.AND);
|
||||
return dependentParameters;
|
||||
}
|
||||
}
|
||||
|
@ -65,11 +65,9 @@ class ProcessInstanceDaoImplTest extends BaseDaoTest {
|
||||
WorkflowExecutionStatus.READY_STOP));
|
||||
processInstanceDao.insert(createWorkflowInstance(workflowDefinitionCode, workflowDefinitionVersion,
|
||||
WorkflowExecutionStatus.SERIAL_WAIT));
|
||||
processInstanceDao.insert(createWorkflowInstance(workflowDefinitionCode, workflowDefinitionVersion,
|
||||
WorkflowExecutionStatus.READY_BLOCK));
|
||||
processInstanceDao.insert(createWorkflowInstance(workflowDefinitionCode, workflowDefinitionVersion,
|
||||
WorkflowExecutionStatus.WAIT_TO_RUN));
|
||||
assertEquals(8, processInstanceDao
|
||||
assertEquals(7, processInstanceDao
|
||||
.queryByWorkflowCodeVersionStatus(workflowDefinitionCode, workflowDefinitionVersion, status).size());
|
||||
}
|
||||
|
||||
|
21
dolphinscheduler-dao/src/test/resources/logback.xml
Normal file
21
dolphinscheduler-dao/src/test/resources/logback.xml
Normal file
@ -0,0 +1,21 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
~ Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
~ contributor license agreements. See the NOTICE file distributed with
|
||||
~ this work for additional information regarding copyright ownership.
|
||||
~ The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
~ (the "License"); you may not use this file except in compliance with
|
||||
~ the License. You may obtain a copy of the License at
|
||||
~
|
||||
~ http://www.apache.org/licenses/LICENSE-2.0
|
||||
~
|
||||
~ Unless required by applicable law or agreed to in writing, software
|
||||
~ distributed under the License is distributed on an "AS IS" BASIS,
|
||||
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
~ See the License for the specific language governing permissions and
|
||||
~ limitations under the License.
|
||||
-->
|
||||
|
||||
<configuration scan="true" scanPeriod="120 seconds">
|
||||
<logger name="*" level="ERROR"/>
|
||||
</configuration>
|
@ -108,7 +108,7 @@ public class MasterServer implements IStoppable {
|
||||
this.masterRPCServer.start();
|
||||
|
||||
// install task plugin
|
||||
TaskPluginManager.loadPlugin();
|
||||
TaskPluginManager.loadTaskPlugin();
|
||||
DataSourceProcessorProvider.initialize();
|
||||
|
||||
this.masterSlotManager.start();
|
||||
|
@ -1,59 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.dolphinscheduler.server.master.event;
|
||||
|
||||
import org.apache.dolphinscheduler.common.enums.StateEventType;
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.BlockingParameters;
|
||||
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
|
||||
|
||||
import java.util.Optional;
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
|
||||
import com.google.auto.service.AutoService;
|
||||
|
||||
@AutoService(StateEventHandler.class)
|
||||
@Slf4j
|
||||
public class WorkflowBlockStateEventHandler implements StateEventHandler {
|
||||
|
||||
@Override
|
||||
public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable,
|
||||
StateEvent stateEvent) throws StateEventHandleError {
|
||||
log.info("Handle workflow instance state block event");
|
||||
Optional<TaskInstance> taskInstanceOptional =
|
||||
workflowExecuteRunnable.getTaskInstance(stateEvent.getTaskInstanceId());
|
||||
if (!taskInstanceOptional.isPresent()) {
|
||||
throw new StateEventHandleError("Cannot find taskInstance from taskMap by taskInstanceId: "
|
||||
+ stateEvent.getTaskInstanceId());
|
||||
}
|
||||
TaskInstance task = taskInstanceOptional.get();
|
||||
|
||||
BlockingParameters parameters = JSONUtils.parseObject(task.getTaskParams(), BlockingParameters.class);
|
||||
if (parameters != null && parameters.isAlertWhenBlocking()) {
|
||||
workflowExecuteRunnable.processBlock();
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StateEventType getEventType() {
|
||||
return StateEventType.PROCESS_BLOCKED;
|
||||
}
|
||||
}
|
@ -43,7 +43,6 @@ import org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager;
|
||||
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.AbstractParameters;
|
||||
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.utils.LogUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.ParameterUtils;
|
||||
@ -89,8 +88,6 @@ public class StreamTaskExecuteRunnable implements Runnable {
|
||||
|
||||
protected ProcessTaskRelationMapper processTaskRelationMapper;
|
||||
|
||||
protected TaskPluginManager taskPluginManager;
|
||||
|
||||
private StreamTaskInstanceExecCacheManager streamTaskInstanceExecCacheManager;
|
||||
|
||||
protected TaskDefinition taskDefinition;
|
||||
@ -115,7 +112,6 @@ public class StreamTaskExecuteRunnable implements Runnable {
|
||||
this.processService = SpringApplicationContext.getBean(ProcessService.class);
|
||||
this.masterConfig = SpringApplicationContext.getBean(MasterConfig.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);
|
||||
this.streamTaskInstanceExecCacheManager =
|
||||
@ -311,14 +307,11 @@ public class StreamTaskExecuteRunnable implements Runnable {
|
||||
return null;
|
||||
}
|
||||
|
||||
TaskChannel taskChannel = taskPluginManager.getTaskChannel(taskInstance.getTaskType());
|
||||
ResourceParametersHelper resources = taskChannel.getResources(taskInstance.getTaskParams());
|
||||
TaskChannel taskChannel = TaskPluginManager.getTaskChannel(taskInstance.getTaskType());
|
||||
ResourceParametersHelper resources = taskChannel.parseParameters(taskInstance.getTaskParams()).getResources();
|
||||
|
||||
AbstractParameters baseParam = taskPluginManager.getParameters(
|
||||
ParametersNode.builder()
|
||||
.taskType(taskInstance.getTaskType())
|
||||
.taskParams(taskInstance.getTaskParams())
|
||||
.build());
|
||||
AbstractParameters baseParam =
|
||||
TaskPluginManager.parseTaskParameters(taskInstance.getTaskType(), taskInstance.getTaskParams());
|
||||
Map<String, Property> propertyMap = paramParsingPreparation(taskInstance, baseParam);
|
||||
TaskExecutionContext taskExecutionContext = TaskExecutionContextBuilder.get()
|
||||
.buildWorkflowInstanceHost(masterConfig.getMasterAddress())
|
||||
|
@ -49,7 +49,6 @@ import org.apache.dolphinscheduler.plugin.task.api.model.JdbcInfo;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.K8sTaskParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.dataquality.DataQualityParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.AbstractResourceParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.DataSourceParameters;
|
||||
@ -74,7 +73,6 @@ import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
|
||||
@ -102,16 +100,15 @@ public class TaskExecutionContextFactory {
|
||||
public TaskExecutionContext createTaskExecutionContext(TaskInstance taskInstance) throws TaskExecutionContextCreateException {
|
||||
ProcessInstance workflowInstance = taskInstance.getProcessInstance();
|
||||
|
||||
ResourceParametersHelper resources =
|
||||
Optional.ofNullable(TaskPluginManager.getTaskChannel(taskInstance.getTaskType()))
|
||||
.map(taskChannel -> taskChannel.getResources(taskInstance.getTaskParams()))
|
||||
.orElse(null);
|
||||
ResourceParametersHelper resources = TaskPluginManager.getTaskChannel(taskInstance.getTaskType())
|
||||
.parseParameters(taskInstance.getTaskParams())
|
||||
.getResources();
|
||||
setTaskResourceInfo(resources);
|
||||
|
||||
Map<String, Property> businessParamsMap = curingParamsService.preBuildBusinessParams(workflowInstance);
|
||||
|
||||
AbstractParameters baseParam = TaskPluginManager.getParameters(ParametersNode.builder()
|
||||
.taskType(taskInstance.getTaskType()).taskParams(taskInstance.getTaskParams()).build());
|
||||
AbstractParameters baseParam =
|
||||
TaskPluginManager.parseTaskParameters(taskInstance.getTaskType(), taskInstance.getTaskParams());
|
||||
Map<String, Property> propertyMap =
|
||||
curingParamsService.paramParsingPreparation(taskInstance, baseParam, workflowInstance);
|
||||
TaskExecutionContext taskExecutionContext = TaskExecutionContextBuilder.get()
|
||||
|
@ -28,7 +28,6 @@ import static org.apache.dolphinscheduler.common.constants.Constants.COMMA;
|
||||
import static org.apache.dolphinscheduler.common.constants.DateConstants.YYYY_MM_DD_HH_MM_SS;
|
||||
import static org.apache.dolphinscheduler.dao.utils.EnvironmentUtils.getEnvironmentCodeOrDefault;
|
||||
import static org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils.getWorkerGroupOrDefault;
|
||||
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.CommandType;
|
||||
@ -61,8 +60,8 @@ import org.apache.dolphinscheduler.extract.worker.transportor.UpdateWorkflowHost
|
||||
import org.apache.dolphinscheduler.extract.worker.transportor.UpdateWorkflowHostResponse;
|
||||
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.parameters.SwitchParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.VarPoolUtils;
|
||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
|
||||
import org.apache.dolphinscheduler.server.master.event.StateEvent;
|
||||
@ -76,7 +75,6 @@ import org.apache.dolphinscheduler.server.master.graph.IWorkflowGraph;
|
||||
import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics;
|
||||
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnableFactory;
|
||||
import org.apache.dolphinscheduler.server.master.runner.taskgroup.TaskGroupCoordinator;
|
||||
import org.apache.dolphinscheduler.server.master.utils.TaskUtils;
|
||||
import org.apache.dolphinscheduler.server.master.utils.WorkflowInstanceUtils;
|
||||
import org.apache.dolphinscheduler.service.alert.ListenerEventAlertManager;
|
||||
import org.apache.dolphinscheduler.service.alert.ProcessAlertManager;
|
||||
@ -300,7 +298,7 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable {
|
||||
stateEvent,
|
||||
stateEventHandleException);
|
||||
ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS);
|
||||
} catch (Exception e) {
|
||||
} catch (Throwable e) {
|
||||
// we catch the exception here, since if the state event handle failed, the state event will still
|
||||
// keep
|
||||
// in the stateEvents queue.
|
||||
@ -381,9 +379,7 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable {
|
||||
if (taskInstance.getIsCache().equals(Flag.YES)) {
|
||||
saveCacheTaskInstance(taskInstance);
|
||||
}
|
||||
if (!workflowInstance.isBlocked()) {
|
||||
submitPostNode(taskInstance.getTaskCode());
|
||||
}
|
||||
submitPostNode(taskInstance.getTaskCode());
|
||||
} else if (taskInstance.taskCanRetry() && !workflowInstance.getState().isReadyStop()) {
|
||||
// retry task
|
||||
log.info("Retry taskInstance taskInstance state: {}", taskInstance.getState());
|
||||
@ -950,16 +946,6 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable {
|
||||
if (!taskInstance.getState().isFinished()) {
|
||||
taskExecuteRunnable.dispatch();
|
||||
} else {
|
||||
if (workflowExecuteContext.getWorkflowInstance().isBlocked()) {
|
||||
TaskStateEvent processBlockEvent = TaskStateEvent.builder()
|
||||
.processInstanceId(workflowExecuteContext.getWorkflowInstance().getId())
|
||||
.taskInstanceId(taskInstance.getId())
|
||||
.status(taskInstance.getState())
|
||||
.type(StateEventType.PROCESS_BLOCKED)
|
||||
.build();
|
||||
this.stateEvents.add(processBlockEvent);
|
||||
}
|
||||
|
||||
TaskStateEvent taskStateChangeEvent = TaskStateEvent.builder()
|
||||
.processInstanceId(workflowExecuteContext.getWorkflowInstance().getId())
|
||||
.taskInstanceId(taskInstance.getId())
|
||||
@ -1096,7 +1082,7 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable {
|
||||
taskInstance.setRetryInterval(taskNode.getRetryInterval());
|
||||
|
||||
// set task param
|
||||
taskInstance.setTaskParams(taskNode.getTaskParams());
|
||||
taskInstance.setTaskParams(taskNode.getParams());
|
||||
|
||||
// set task group and priority
|
||||
taskInstance.setTaskGroupId(taskNode.getTaskGroupId());
|
||||
@ -1275,7 +1261,7 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable {
|
||||
|
||||
private boolean tryToTakeOverTaskInstance(TaskInstance taskInstance) {
|
||||
ProcessInstance workflowInstance = workflowExecuteContext.getWorkflowInstance();
|
||||
if (TaskUtils.isMasterTask(taskInstance.getTaskType())) {
|
||||
if (TaskTypeUtils.isLogicTask(taskInstance.getTaskType())) {
|
||||
return false;
|
||||
}
|
||||
try {
|
||||
@ -1341,13 +1327,14 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable {
|
||||
if (depTaskState.isKill()) {
|
||||
return DependResult.NON_EXEC;
|
||||
}
|
||||
// ignore task state if current task is block
|
||||
if (taskNode.isBlockingTask()) {
|
||||
|
||||
// always return success if current task is condition
|
||||
if (TaskTypeUtils.isConditionTask(taskNode.getType())) {
|
||||
continue;
|
||||
}
|
||||
|
||||
// always return success if current task is condition
|
||||
if (taskNode.isConditionsTask()) {
|
||||
// always return success if current task is switch
|
||||
if (TaskTypeUtils.isSwitchTask(taskNode.getType())) {
|
||||
continue;
|
||||
}
|
||||
|
||||
@ -1392,7 +1379,7 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable {
|
||||
private boolean dependTaskSuccess(Long dependNodeCode, Long nextNodeCode) {
|
||||
DAG<Long, TaskNode, TaskNodeRelation> dag = workflowExecuteContext.getWorkflowGraph().getDag();
|
||||
TaskNode dependentNode = dag.getNode(dependNodeCode);
|
||||
if (dependentNode.isConditionsTask()) {
|
||||
if (TaskTypeUtils.isConditionTask(dependentNode.getType())) {
|
||||
// condition task need check the branch to run
|
||||
List<Long> nextTaskList =
|
||||
DagHelper.parseConditionTask(dependNodeCode, skipTaskNodeMap, dag, getCompleteTaskInstanceMap());
|
||||
@ -1405,12 +1392,6 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable {
|
||||
}
|
||||
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);
|
||||
}
|
||||
Optional<TaskInstance> existTaskInstanceOptional = getTaskInstance(dependNodeCode);
|
||||
if (!existTaskInstanceOptional.isPresent()) {
|
||||
return false;
|
||||
@ -1450,8 +1431,7 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable {
|
||||
*/
|
||||
private WorkflowExecutionStatus runningState(WorkflowExecutionStatus state) {
|
||||
if (state == WorkflowExecutionStatus.READY_STOP || state == WorkflowExecutionStatus.READY_PAUSE
|
||||
|| state == WorkflowExecutionStatus.READY_BLOCK ||
|
||||
state == WorkflowExecutionStatus.DELAY_EXECUTION) {
|
||||
|| state == WorkflowExecutionStatus.DELAY_EXECUTION) {
|
||||
// if the running task is not completed, the state remains unchanged
|
||||
return state;
|
||||
} else {
|
||||
@ -1510,7 +1490,7 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable {
|
||||
}
|
||||
|
||||
List<TaskInstance> pauseList = getCompleteTaskByState(TaskExecutionStatus.PAUSE);
|
||||
if (CollectionUtils.isNotEmpty(pauseList) || workflowInstance.isBlocked() || !isComplementEnd()
|
||||
if (CollectionUtils.isNotEmpty(pauseList) || !isComplementEnd()
|
||||
|| standByTaskInstancePriorityQueue.size() > 0) {
|
||||
return WorkflowExecutionStatus.PAUSE;
|
||||
} else {
|
||||
@ -1518,30 +1498,6 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* prepare for block
|
||||
* if process has tasks still running, pause them
|
||||
* if readyToSubmitTaskQueue is not empty, kill them
|
||||
* else return block status directly
|
||||
*
|
||||
* @return ExecutionStatus
|
||||
*/
|
||||
private WorkflowExecutionStatus processReadyBlock() {
|
||||
if (taskExecuteRunnableMap.size() > 0) {
|
||||
for (DefaultTaskExecuteRunnable taskExecuteRunnable : taskExecuteRunnableMap.values()) {
|
||||
if (!TASK_TYPE_BLOCKING.equals(taskExecuteRunnable.getTaskInstance().getTaskType())) {
|
||||
taskExecuteRunnable.pause();
|
||||
}
|
||||
}
|
||||
}
|
||||
if (standByTaskInstancePriorityQueue.size() > 0) {
|
||||
for (Iterator<TaskInstance> iter = standByTaskInstancePriorityQueue.iterator(); iter.hasNext();) {
|
||||
iter.next().setState(TaskExecutionStatus.PAUSE);
|
||||
}
|
||||
}
|
||||
return WorkflowExecutionStatus.BLOCK;
|
||||
}
|
||||
|
||||
/**
|
||||
* generate the latest process instance status by the tasks state
|
||||
*
|
||||
@ -1557,13 +1513,6 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable {
|
||||
return executionStatus;
|
||||
}
|
||||
|
||||
// block
|
||||
if (state == WorkflowExecutionStatus.READY_BLOCK) {
|
||||
WorkflowExecutionStatus executionStatus = processReadyBlock();
|
||||
log.info("The workflowInstance is ready to block, the workflowInstance status is {}", executionStatus);
|
||||
return executionStatus;
|
||||
}
|
||||
|
||||
// pause
|
||||
if (state == WorkflowExecutionStatus.READY_PAUSE) {
|
||||
WorkflowExecutionStatus executionStatus = processReadyPause();
|
||||
@ -2038,7 +1987,7 @@ public class WorkflowExecuteRunnable implements IWorkflowExecuteRunnable {
|
||||
}
|
||||
|
||||
private void sendTaskLogOnMasterToRemoteIfNeeded(TaskInstance taskInstance) {
|
||||
if (RemoteLogUtils.isRemoteLoggingEnable() && TaskUtils.isMasterTask(taskInstance.getTaskType())) {
|
||||
if (RemoteLogUtils.isRemoteLoggingEnable() && TaskTypeUtils.isLogicTask(taskInstance.getTaskType())) {
|
||||
RemoteLogUtils.sendRemoteLog(taskInstance.getLogPath());
|
||||
log.info("Master sends task log {} to remote storage asynchronously.", taskInstance.getLogPath());
|
||||
}
|
||||
|
@ -18,7 +18,7 @@
|
||||
package org.apache.dolphinscheduler.server.master.runner.dispatcher;
|
||||
|
||||
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
|
||||
import org.apache.dolphinscheduler.server.master.utils.TaskUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
|
||||
@ -36,7 +36,10 @@ public class TaskDispatchFactory {
|
||||
private WorkerTaskDispatcher workerTaskDispatcher;
|
||||
|
||||
public TaskDispatcher getTaskDispatcher(String taskType) {
|
||||
return TaskUtils.isMasterTask(taskType) ? masterTaskDispatcher : workerTaskDispatcher;
|
||||
if (TaskTypeUtils.isLogicTask(taskType)) {
|
||||
return masterTaskDispatcher;
|
||||
}
|
||||
return workerTaskDispatcher;
|
||||
}
|
||||
|
||||
public TaskDispatcher getTaskDispatcher(TaskInstance taskInstance) {
|
||||
|
@ -17,8 +17,8 @@
|
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.operator;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
|
||||
import org.apache.dolphinscheduler.server.master.runner.DefaultTaskExecuteRunnable;
|
||||
import org.apache.dolphinscheduler.server.master.utils.TaskUtils;
|
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired;
|
||||
import org.springframework.stereotype.Component;
|
||||
@ -51,28 +51,29 @@ public class TaskExecuteRunnableOperatorManager {
|
||||
private LogicTaskExecuteRunnableTimeoutOperator logicTaskTimeoutOperator;
|
||||
|
||||
public TaskExecuteRunnableOperator getTaskKillOperator(DefaultTaskExecuteRunnable defaultTaskExecuteRunnable) {
|
||||
if (TaskUtils.isMasterTask(defaultTaskExecuteRunnable.getTaskInstance().getTaskType())) {
|
||||
if (TaskTypeUtils.isLogicTask(defaultTaskExecuteRunnable.getTaskInstance().getTaskType())) {
|
||||
return logicTaskKillOperator;
|
||||
}
|
||||
return taskKillOperator;
|
||||
}
|
||||
|
||||
public TaskExecuteRunnableOperator getTaskPauseOperator(DefaultTaskExecuteRunnable defaultTaskExecuteRunnable) {
|
||||
if (TaskUtils.isMasterTask(defaultTaskExecuteRunnable.getTaskInstance().getTaskType())) {
|
||||
if (TaskTypeUtils.isLogicTask(defaultTaskExecuteRunnable.getTaskInstance().getTaskType())) {
|
||||
|
||||
return logicTaskPauseOperator;
|
||||
}
|
||||
return taskPauseOperator;
|
||||
}
|
||||
|
||||
public TaskExecuteRunnableOperator getTaskDispatchOperator(DefaultTaskExecuteRunnable defaultTaskExecuteRunnable) {
|
||||
if (TaskUtils.isMasterTask(defaultTaskExecuteRunnable.getTaskInstance().getTaskType())) {
|
||||
if (TaskTypeUtils.isLogicTask(defaultTaskExecuteRunnable.getTaskInstance().getTaskType())) {
|
||||
return logicTaskDispatchOperator;
|
||||
}
|
||||
return taskDispatchOperator;
|
||||
}
|
||||
|
||||
public TaskExecuteRunnableOperator getTaskTimeoutOperator(DefaultTaskExecuteRunnable defaultTaskExecuteRunnable) {
|
||||
if (TaskUtils.isMasterTask(defaultTaskExecuteRunnable.getTaskInstance().getTaskType())) {
|
||||
if (TaskTypeUtils.isLogicTask(defaultTaskExecuteRunnable.getTaskInstance().getTaskType())) {
|
||||
return logicTaskTimeoutOperator;
|
||||
}
|
||||
return taskTimeoutOperator;
|
||||
|
@ -18,9 +18,12 @@
|
||||
package org.apache.dolphinscheduler.server.master.runner.task;
|
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException;
|
||||
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
|
||||
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
|
||||
@ -28,10 +31,20 @@ import lombok.extern.slf4j.Slf4j;
|
||||
public abstract class BaseSyncLogicTask<T extends AbstractParameters> implements ISyncLogicTask {
|
||||
|
||||
protected final TaskExecutionContext taskExecutionContext;
|
||||
|
||||
protected final WorkflowExecuteRunnable workflowExecuteRunnable;
|
||||
protected final TaskInstance taskInstance;
|
||||
protected final T taskParameters;
|
||||
|
||||
protected BaseSyncLogicTask(TaskExecutionContext taskExecutionContext, T taskParameters) {
|
||||
protected BaseSyncLogicTask(WorkflowExecuteRunnable workflowExecuteRunnable,
|
||||
TaskExecutionContext taskExecutionContext,
|
||||
T taskParameters) throws LogicTaskInitializeException {
|
||||
this.taskExecutionContext = taskExecutionContext;
|
||||
this.workflowExecuteRunnable = workflowExecuteRunnable;
|
||||
this.taskInstance =
|
||||
workflowExecuteRunnable.getTaskInstance(taskExecutionContext.getTaskInstanceId()).orElseThrow(
|
||||
() -> new LogicTaskInitializeException(
|
||||
"Cannot find the task instance in workflow execute runnable"));
|
||||
this.taskParameters = taskParameters;
|
||||
log.info("Success initialize task parameters: \n{}", JSONUtils.toPrettyJsonString(taskParameters));
|
||||
}
|
||||
|
@ -1,139 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.dolphinscheduler.server.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<BlockingParameters> {
|
||||
|
||||
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<BlockingParameters>() {
|
||||
}));
|
||||
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()).getWorkflowExecuteContext()
|
||||
.getWorkflowInstance();
|
||||
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<Long, TaskInstance> completeTaskList = taskInstanceDao
|
||||
.queryValidTaskListByWorkflowInstanceId(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<DependResult> tempResultList = new ArrayList<>();
|
||||
for (DependentTaskModel dependentTaskModel : dependentParameters.getDependTaskList()) {
|
||||
List<DependResult> 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<Long, TaskInstance> 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;
|
||||
}
|
||||
|
||||
}
|
@ -1,51 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.dolphinscheduler.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<BlockingLogicTask> {
|
||||
|
||||
@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;
|
||||
}
|
||||
}
|
@ -17,9 +17,8 @@
|
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.task.condition;
|
||||
|
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
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;
|
||||
@ -27,6 +26,8 @@ import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.model.DependentItem;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ConditionsParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.DependentUtils;
|
||||
import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException;
|
||||
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
|
||||
import org.apache.dolphinscheduler.server.master.runner.task.BaseSyncLogicTask;
|
||||
|
||||
import java.util.List;
|
||||
@ -37,47 +38,42 @@ import java.util.stream.Collectors;
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
|
||||
@Slf4j
|
||||
public class ConditionLogicTask extends BaseSyncLogicTask<ConditionsParameters> {
|
||||
|
||||
public static final String TASK_TYPE = "CONDITIONS";
|
||||
|
||||
private final TaskInstanceDao taskInstanceDao;
|
||||
private final ProcessInstanceDao workflowInstanceDao;
|
||||
|
||||
private final TaskInstance taskInstance;
|
||||
|
||||
public ConditionLogicTask(TaskExecutionContext taskExecutionContext,
|
||||
TaskInstance taskInstance,
|
||||
TaskInstanceDao taskInstanceDao,
|
||||
ProcessInstanceDao workflowInstanceDao) {
|
||||
// todo: we need to change the parameter in front-end, so that we can directly use json to parse
|
||||
super(taskExecutionContext, taskInstance.getConditionsParameters());
|
||||
public ConditionLogicTask(WorkflowExecuteRunnable workflowExecuteRunnable,
|
||||
TaskExecutionContext taskExecutionContext,
|
||||
TaskInstanceDao taskInstanceDao) throws LogicTaskInitializeException {
|
||||
super(workflowExecuteRunnable, taskExecutionContext,
|
||||
JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference<ConditionsParameters>() {
|
||||
}));
|
||||
this.taskInstanceDao = taskInstanceDao;
|
||||
this.workflowInstanceDao = workflowInstanceDao;
|
||||
this.taskInstance = taskInstance;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handle() {
|
||||
// calculate the conditionResult
|
||||
DependResult conditionResult = calculateConditionResult();
|
||||
log.info("The condition result is {}", conditionResult);
|
||||
taskParameters.setConditionSuccess(conditionResult == DependResult.SUCCESS);
|
||||
taskInstance.setConditionsParameters(taskParameters);
|
||||
taskParameters.getConditionResult().setConditionSuccess(conditionResult == DependResult.SUCCESS);
|
||||
taskInstance.setTaskParams(JSONUtils.toJsonString(taskParameters));
|
||||
taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.SUCCESS);
|
||||
}
|
||||
|
||||
private DependResult calculateConditionResult() {
|
||||
final ProcessInstance processInstance =
|
||||
workflowInstanceDao.queryById(taskExecutionContext.getProcessInstanceId());
|
||||
final List<TaskInstance> taskInstances =
|
||||
taskInstanceDao.queryValidTaskListByWorkflowInstanceId(processInstance.getId(),
|
||||
processInstance.getTestFlag());
|
||||
final Map<Long, TaskInstance> taskInstanceMap =
|
||||
taskInstances.stream().collect(Collectors.toMap(TaskInstance::getTaskCode, Function.identity()));
|
||||
final List<TaskInstance> taskInstances = taskInstanceDao.queryValidTaskListByWorkflowInstanceId(
|
||||
taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTestFlag());
|
||||
final Map<Long, TaskInstance> taskInstanceMap = taskInstances.stream()
|
||||
.collect(Collectors.toMap(TaskInstance::getTaskCode, Function.identity()));
|
||||
|
||||
List<DependResult> dependResults = taskParameters.getDependTaskList().stream()
|
||||
ConditionsParameters.ConditionDependency dependence = taskParameters.getDependence();
|
||||
List<DependResult> dependResults = dependence.getDependTaskList()
|
||||
.stream()
|
||||
.map(dependentTaskModel -> DependentUtils.getDependResultForRelation(
|
||||
dependentTaskModel.getRelation(),
|
||||
dependentTaskModel.getDependItemList()
|
||||
@ -85,7 +81,7 @@ public class ConditionLogicTask extends BaseSyncLogicTask<ConditionsParameters>
|
||||
.map(dependentItem -> getDependResultForItem(dependentItem, taskInstanceMap))
|
||||
.collect(Collectors.toList())))
|
||||
.collect(Collectors.toList());
|
||||
return DependentUtils.getDependResultForRelation(taskParameters.getRelation(), dependResults);
|
||||
return DependentUtils.getDependResultForRelation(dependence.getRelation(), dependResults);
|
||||
}
|
||||
|
||||
private DependResult getDependResultForItem(DependentItem item, Map<Long, TaskInstance> taskInstanceMap) {
|
||||
@ -96,8 +92,9 @@ public class ConditionLogicTask extends BaseSyncLogicTask<ConditionsParameters>
|
||||
return DependResult.FAILED;
|
||||
}
|
||||
|
||||
DependResult dependResult =
|
||||
Objects.equals(item.getStatus(), taskInstance.getState()) ? DependResult.SUCCESS : 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());
|
||||
|
@ -17,12 +17,12 @@
|
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.task.condition;
|
||||
|
||||
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.task.ConditionsLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
|
||||
import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException;
|
||||
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
|
||||
import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory;
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
@ -36,28 +36,19 @@ 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 {
|
||||
TaskInstance taskInstance =
|
||||
processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId())
|
||||
.getTaskInstance(taskExecutionContext.getTaskInstanceId())
|
||||
.orElseThrow(() -> new LogicTaskInitializeException(
|
||||
"Cannot find the task instance in workflow execute runnable"));
|
||||
return new ConditionLogicTask(
|
||||
taskExecutionContext,
|
||||
taskInstance,
|
||||
taskInstanceDao,
|
||||
processInstanceDao);
|
||||
WorkflowExecuteRunnable workflowExecuteRunnable = processInstanceExecCacheManager.getByProcessInstanceId(
|
||||
taskExecutionContext.getProcessInstanceId());
|
||||
return new ConditionLogicTask(workflowExecuteRunnable, taskExecutionContext, taskInstanceDao);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTaskType() {
|
||||
return ConditionLogicTask.TASK_TYPE;
|
||||
return ConditionsLogicTaskChannelFactory.NAME;
|
||||
}
|
||||
}
|
||||
|
@ -122,10 +122,12 @@ public class DependentAsyncTaskExecuteFunction implements AsyncTaskExecuteFuncti
|
||||
|
||||
private List<DependentExecute> initializeDependentTaskList() {
|
||||
log.info("Begin to initialize dependent task list");
|
||||
List<DependentTaskModel> dependTaskList = dependentParameters.getDependence().getDependTaskList();
|
||||
|
||||
final Set<Long> projectCodes = new HashSet<>();
|
||||
final Set<Long> processDefinitionCodes = new HashSet<>();
|
||||
final Set<Long> taskDefinitionCodes = new HashSet<>();
|
||||
for (DependentTaskModel taskModel : dependentParameters.getDependTaskList()) {
|
||||
for (DependentTaskModel taskModel : dependTaskList) {
|
||||
for (DependentItem dependentItem : taskModel.getDependItemList()) {
|
||||
projectCodes.add(dependentItem.getProjectCode());
|
||||
processDefinitionCodes.add(dependentItem.getDefinitionCode());
|
||||
@ -142,7 +144,7 @@ public class DependentAsyncTaskExecuteFunction implements AsyncTaskExecuteFuncti
|
||||
.collect(Collectors.toMap(TaskDefinition::getCode, Function.identity()));
|
||||
final TaskInstance taskInstance =
|
||||
taskInstanceDao.queryById(taskExecutionContext.getTaskInstanceId());
|
||||
List<DependentExecute> dependentExecutes = dependentParameters.getDependTaskList()
|
||||
List<DependentExecute> dependentExecutes = dependTaskList
|
||||
.stream()
|
||||
.map(dependentTaskModel -> {
|
||||
for (DependentItem dependentItem : dependentTaskModel.getDependItemList()) {
|
||||
@ -210,7 +212,7 @@ public class DependentAsyncTaskExecuteFunction implements AsyncTaskExecuteFuncti
|
||||
}
|
||||
dependResultList.add(dependResult);
|
||||
}
|
||||
return DependentUtils.getDependResultForRelation(this.dependentParameters.getRelation(),
|
||||
return DependentUtils.getDependResultForRelation(dependentParameters.getDependence().getRelation(),
|
||||
dependResultList);
|
||||
}
|
||||
|
||||
@ -218,7 +220,8 @@ public class DependentAsyncTaskExecuteFunction implements AsyncTaskExecuteFuncti
|
||||
boolean isAllDependentTaskFinished = true;
|
||||
for (DependentExecute dependentExecute : dependentTaskList) {
|
||||
if (!dependentExecute.finish(dependentDate, processInstance.getTestFlag(),
|
||||
dependentParameters.getFailurePolicy(), dependentParameters.getFailureWaitingTime())) {
|
||||
dependentParameters.getDependence().getFailurePolicy(),
|
||||
dependentParameters.getDependence().getFailureWaitingTime())) {
|
||||
isAllDependentTaskFinished = false;
|
||||
}
|
||||
dependentExecute.getDependResultMap().forEach((dependentKey, dependResult) -> {
|
||||
@ -238,7 +241,7 @@ public class DependentAsyncTaskExecuteFunction implements AsyncTaskExecuteFuncti
|
||||
|
||||
@Override
|
||||
public @NonNull Duration getAsyncTaskStateCheckInterval() {
|
||||
return dependentParameters.getCheckInterval() == null ? DEFAULT_STATE_CHECK_INTERVAL
|
||||
: Duration.ofSeconds(dependentParameters.getCheckInterval());
|
||||
return dependentParameters.getDependence().getCheckInterval() == null ? DEFAULT_STATE_CHECK_INTERVAL
|
||||
: Duration.ofSeconds(dependentParameters.getDependence().getCheckInterval());
|
||||
}
|
||||
}
|
||||
|
@ -17,6 +17,7 @@
|
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.task.dependent;
|
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
|
||||
import org.apache.dolphinscheduler.dao.repository.ProcessDefinitionDao;
|
||||
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
|
||||
@ -26,8 +27,6 @@ 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;
|
||||
@ -37,6 +36,8 @@ import java.util.Date;
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
|
||||
@Slf4j
|
||||
public class DependentLogicTask extends BaseAsyncLogicTask<DependentParameters> {
|
||||
|
||||
@ -48,7 +49,7 @@ public class DependentLogicTask extends BaseAsyncLogicTask<DependentParameters>
|
||||
private final TaskInstanceDao taskInstanceDao;
|
||||
private final ProcessInstanceDao processInstanceDao;
|
||||
|
||||
private final ProcessInstanceExecCacheManager processInstanceExecCacheManager;
|
||||
private final WorkflowExecuteRunnable workflowExecuteRunnable;
|
||||
|
||||
public DependentLogicTask(TaskExecutionContext taskExecutionContext,
|
||||
ProjectDao projectDao,
|
||||
@ -56,19 +57,16 @@ public class DependentLogicTask extends BaseAsyncLogicTask<DependentParameters>
|
||||
TaskDefinitionDao taskDefinitionDao,
|
||||
TaskInstanceDao taskInstanceDao,
|
||||
ProcessInstanceDao processInstanceDao,
|
||||
ProcessInstanceExecCacheManager processInstanceExecCacheManager) throws LogicTaskInitializeException {
|
||||
WorkflowExecuteRunnable workflowExecuteRunnable) {
|
||||
super(taskExecutionContext,
|
||||
processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId())
|
||||
.getTaskInstance(taskExecutionContext.getTaskInstanceId())
|
||||
.orElseThrow(() -> new LogicTaskInitializeException(
|
||||
"Cannot find the task instance in workflow execute runnable"))
|
||||
.getDependency());
|
||||
JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference<DependentParameters>() {
|
||||
}));
|
||||
this.projectDao = projectDao;
|
||||
this.processDefinitionDao = processDefinitionDao;
|
||||
this.taskDefinitionDao = taskDefinitionDao;
|
||||
this.taskInstanceDao = taskInstanceDao;
|
||||
this.processInstanceDao = processInstanceDao;
|
||||
this.processInstanceExecCacheManager = processInstanceExecCacheManager;
|
||||
this.workflowExecuteRunnable = workflowExecuteRunnable;
|
||||
|
||||
}
|
||||
|
||||
@ -85,8 +83,6 @@ public class DependentLogicTask extends BaseAsyncLogicTask<DependentParameters>
|
||||
|
||||
@Override
|
||||
public void pause() throws MasterTaskExecuteException {
|
||||
WorkflowExecuteRunnable workflowExecuteRunnable =
|
||||
processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId());
|
||||
if (workflowExecuteRunnable == null) {
|
||||
log.error("Cannot find the WorkflowExecuteRunnable");
|
||||
return;
|
||||
|
@ -25,6 +25,7 @@ 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.WorkflowExecuteRunnable;
|
||||
import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory;
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
@ -52,6 +53,12 @@ public class DependentLogicTaskPluginFactory implements ILogicTaskPluginFactory<
|
||||
|
||||
@Override
|
||||
public DependentLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException {
|
||||
int workflowInstanceId = taskExecutionContext.getProcessInstanceId();
|
||||
WorkflowExecuteRunnable workflowExecuteRunnable =
|
||||
processInstanceExecCacheManager.getByProcessInstanceId(workflowInstanceId);
|
||||
if (workflowExecuteRunnable == null) {
|
||||
throw new LogicTaskInitializeException("Cannot find the WorkflowExecuteRunnable: " + workflowInstanceId);
|
||||
}
|
||||
return new DependentLogicTask(
|
||||
taskExecutionContext,
|
||||
projectDao,
|
||||
@ -59,7 +66,7 @@ public class DependentLogicTaskPluginFactory implements ILogicTaskPluginFactory<
|
||||
taskDefinitionDao,
|
||||
taskInstanceDao,
|
||||
processInstanceDao,
|
||||
processInstanceExecCacheManager);
|
||||
workflowExecuteRunnable);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -27,7 +27,6 @@ import org.apache.dolphinscheduler.extract.master.ITaskInstanceExecutionEventLis
|
||||
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStateChangeEvent;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.SubProcessParameters;
|
||||
import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
|
||||
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;
|
||||
@ -41,29 +40,27 @@ import com.fasterxml.jackson.core.type.TypeReference;
|
||||
public class SubWorkflowLogicTask extends BaseAsyncLogicTask<SubProcessParameters> {
|
||||
|
||||
public static final String TASK_TYPE = "SUB_PROCESS";
|
||||
private final ProcessInstanceExecCacheManager processInstanceExecCacheManager;
|
||||
private final WorkflowExecuteRunnable workflowExecuteRunnable;
|
||||
private final ProcessInstanceDao processInstanceDao;
|
||||
|
||||
public SubWorkflowLogicTask(TaskExecutionContext taskExecutionContext,
|
||||
ProcessInstanceExecCacheManager processInstanceExecCacheManager,
|
||||
WorkflowExecuteRunnable workflowExecuteRunnable,
|
||||
ProcessInstanceDao processInstanceDao) {
|
||||
super(taskExecutionContext,
|
||||
JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference<SubProcessParameters>() {
|
||||
}));
|
||||
this.processInstanceExecCacheManager = processInstanceExecCacheManager;
|
||||
this.workflowExecuteRunnable = workflowExecuteRunnable;
|
||||
this.processInstanceDao = processInstanceDao;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() throws MasterTaskExecuteException {
|
||||
public AsyncTaskExecuteFunction getAsyncTaskExecuteFunction() {
|
||||
// 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;
|
||||
@ -99,8 +96,6 @@ public class SubWorkflowLogicTask extends BaseAsyncLogicTask<SubProcessParameter
|
||||
|
||||
@Override
|
||||
public void kill() {
|
||||
WorkflowExecuteRunnable workflowExecuteRunnable =
|
||||
processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId());
|
||||
if (workflowExecuteRunnable == null) {
|
||||
log.warn("Cannot find WorkflowExecuteRunnable");
|
||||
return;
|
||||
|
@ -20,6 +20,8 @@ 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.exception.LogicTaskInitializeException;
|
||||
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
|
||||
import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory;
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
@ -37,8 +39,15 @@ public class SubWorkflowLogicTaskPluginFactory implements ILogicTaskPluginFactor
|
||||
private ProcessInstanceExecCacheManager processInstanceExecCacheManager;
|
||||
|
||||
@Override
|
||||
public SubWorkflowLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) {
|
||||
return new SubWorkflowLogicTask(taskExecutionContext, processInstanceExecCacheManager, processInstanceDao);
|
||||
public SubWorkflowLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException {
|
||||
int workflowInstanceId = taskExecutionContext.getProcessInstanceId();
|
||||
WorkflowExecuteRunnable workflowExecuteRunnable =
|
||||
processInstanceExecCacheManager.getByProcessInstanceId(workflowInstanceId);
|
||||
if (workflowExecuteRunnable == null) {
|
||||
throw new LogicTaskInitializeException(
|
||||
"Cannot find the WorkflowExecuteRunnable by : " + workflowInstanceId);
|
||||
}
|
||||
return new SubWorkflowLogicTask(taskExecutionContext, workflowExecuteRunnable, processInstanceDao);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -24,21 +24,26 @@ import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.model.SwitchResultVo;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.SwitchParameters;
|
||||
import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
|
||||
import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException;
|
||||
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
|
||||
import org.apache.dolphinscheduler.server.master.graph.IWorkflowGraph;
|
||||
import org.apache.dolphinscheduler.server.master.runner.IWorkflowExecuteContext;
|
||||
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.dolphinscheduler.service.model.TaskNode;
|
||||
|
||||
import org.apache.commons.collections4.CollectionUtils;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
|
||||
@Slf4j
|
||||
public class SwitchLogicTask extends BaseSyncLogicTask<SwitchParameters> {
|
||||
|
||||
@ -47,121 +52,96 @@ public class SwitchLogicTask extends BaseSyncLogicTask<SwitchParameters> {
|
||||
private final WorkflowExecuteRunnable workflowExecuteRunnable;
|
||||
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());
|
||||
this.workflowExecuteRunnable =
|
||||
processInstanceExecCacheManager.getByProcessInstanceId(taskExecutionContext.getProcessInstanceId());
|
||||
this.taskInstance = workflowExecuteRunnable.getTaskInstance(taskExecutionContext.getTaskInstanceId())
|
||||
.orElseThrow(() -> new LogicTaskInitializeException(
|
||||
"Cannot find the task instance in workflow execute runnable"));
|
||||
// Since the default branch is not in the dependTaskList, we need to add it to the end
|
||||
// otherwise the default branch will never be skipped in DAGHelper
|
||||
addDefaultBranchToEnd();
|
||||
public SwitchLogicTask(WorkflowExecuteRunnable workflowExecuteRunnable,
|
||||
TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException {
|
||||
super(workflowExecuteRunnable,
|
||||
taskExecutionContext,
|
||||
JSONUtils.parseObject(taskExecutionContext.getTaskParams(), new TypeReference<SwitchParameters>() {
|
||||
}));
|
||||
this.workflowExecuteRunnable = workflowExecuteRunnable;
|
||||
this.taskInstance =
|
||||
workflowExecuteRunnable.getTaskInstance(taskExecutionContext.getTaskInstanceId()).orElseThrow(
|
||||
() -> new LogicTaskInitializeException(
|
||||
"Cannot find the task instance in workflow execute runnable"));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handle() throws MasterTaskExecuteException {
|
||||
if (CollectionUtils.isEmpty(taskParameters.getDependTaskList())) {
|
||||
if (CollectionUtils.isEmpty(taskParameters.getSwitchResult().getDependTaskList())) {
|
||||
// If the branch is empty then will go into the default branch
|
||||
// This case shouldn't happen, we can directly throw exception and forbid the user to set branch
|
||||
log.info("The switch items is empty");
|
||||
moveToDefaultBranch();
|
||||
} else {
|
||||
calculateSwitchBranch();
|
||||
}
|
||||
taskInstance.setSwitchDependency(taskParameters);
|
||||
checkIfBranchExist(taskParameters.getNextBranch());
|
||||
taskInstance.setTaskParams(JSONUtils.toJsonString(taskParameters));
|
||||
taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.SUCCESS);
|
||||
log.info("Switch task execute finished: {}", taskExecutionContext.getCurrentExecutionStatus().name());
|
||||
}
|
||||
|
||||
private void moveToDefaultBranch() {
|
||||
List<SwitchResultVo> switchResultVos = taskParameters.getDependTaskList();
|
||||
SwitchResultVo defaultSwitchResultVo = getDefaultSwitchResultVo();
|
||||
checkIfBranchExist(defaultSwitchResultVo.getNextNode());
|
||||
|
||||
taskParameters.setResultConditionLocation(switchResultVos.size() - 1);
|
||||
|
||||
log.info("Begin to move to the default branch");
|
||||
if (taskParameters.getSwitchResult().getNextNode() == null) {
|
||||
throw new IllegalArgumentException(
|
||||
"The default branch is empty, please check the switch task configuration");
|
||||
}
|
||||
taskParameters.setNextBranch(taskParameters.getSwitchResult().getNextNode());
|
||||
log.info("The condition is not satisfied, move to the default branch: {}",
|
||||
taskParameters.getNextNode().stream().map(node -> workflowExecuteRunnable.getWorkflowExecuteContext()
|
||||
.getWorkflowGraph().getDag().getNode(node).getName()).collect(Collectors.toList()));
|
||||
getTaskName(taskParameters.getNextBranch()));
|
||||
}
|
||||
|
||||
private void calculateSwitchBranch() {
|
||||
List<SwitchResultVo> switchResultVos = taskParameters.getDependTaskList();
|
||||
List<SwitchResultVo> switchResultVos = taskParameters.getSwitchResult().getDependTaskList();
|
||||
Map<String, Property> globalParams = taskExecutionContext.getPrepareParamsMap();
|
||||
Map<String, Property> varParams = JSONUtils
|
||||
.toList(taskInstance.getVarPool(), Property.class)
|
||||
.stream()
|
||||
.collect(Collectors.toMap(Property::getProp, Property -> Property));
|
||||
|
||||
int finalConditionLocation = -1;
|
||||
// The last one is the default branch, no need to calculate
|
||||
for (int i = 0; i < switchResultVos.size() - 1; i++) {
|
||||
SwitchResultVo switchResultVo = switchResultVos.get(i);
|
||||
log.info("Begin to execute {} condition: {} ", i, switchResultVo.getCondition());
|
||||
String content = SwitchTaskUtils.generateContentWithTaskParams(switchResultVo.getCondition(), globalParams,
|
||||
varParams);
|
||||
log.info("Format condition sentence::{} successfully", content);
|
||||
boolean result;
|
||||
Long nextBranch = null;
|
||||
for (SwitchResultVo switchResultVo : switchResultVos) {
|
||||
log.info("Begin to execute switch item: {} ", switchResultVo);
|
||||
try {
|
||||
result = SwitchTaskUtils.evaluate(content);
|
||||
log.info("Execute condition sentence: {} successfully: {}", content, result);
|
||||
if (result) {
|
||||
String content = SwitchTaskUtils.generateContentWithTaskParams(switchResultVo.getCondition(),
|
||||
globalParams, varParams);
|
||||
log.info("Format condition sentence::{} successfully", content);
|
||||
boolean conditionResult = SwitchTaskUtils.evaluate(content);
|
||||
log.info("Execute condition sentence: {} successfully: {}", content, conditionResult);
|
||||
if (conditionResult) {
|
||||
// If matched, break the loop
|
||||
finalConditionLocation = i;
|
||||
nextBranch = switchResultVo.getNextNode();
|
||||
break;
|
||||
}
|
||||
} catch (Exception e) {
|
||||
log.info("Execute condition sentence: {} failed", content, e);
|
||||
log.info("Execute switch item: {} failed", switchResultVo, e);
|
||||
}
|
||||
}
|
||||
// If the finalConditionLocation is -1, then the default branch will be executed
|
||||
if (finalConditionLocation >= 0) {
|
||||
List<Long> nextNodes = switchResultVos.get(finalConditionLocation).getNextNode();
|
||||
checkIfBranchExist(nextNodes);
|
||||
log.info("The condition is satisfied, move to the branch: {}",
|
||||
switchResultVos.get(finalConditionLocation).getNextNode().stream()
|
||||
.map(node -> workflowExecuteRunnable.getWorkflowExecuteContext().getWorkflowGraph().getDag()
|
||||
.getNode(node).getName())
|
||||
.collect(Collectors.toList()));
|
||||
taskParameters.setResultConditionLocation(finalConditionLocation);
|
||||
} else {
|
||||
|
||||
if (nextBranch == null) {
|
||||
log.info("All switch item is not satisfied");
|
||||
moveToDefaultBranch();
|
||||
}
|
||||
}
|
||||
|
||||
private void checkIfBranchExist(List<Long> branchNode) {
|
||||
if (CollectionUtils.isEmpty(branchNode)) {
|
||||
throw new IllegalArgumentException("The branchNode is empty, please check the switch task configuration");
|
||||
private void checkIfBranchExist(Long branchNode) {
|
||||
if (branchNode == null) {
|
||||
throw new IllegalArgumentException("The branch is empty, please check the switch task configuration");
|
||||
}
|
||||
for (Long branch : branchNode) {
|
||||
if (branch == null) {
|
||||
throw new IllegalArgumentException("The branch is empty, please check the switch task configuration");
|
||||
}
|
||||
if (!workflowExecuteRunnable.getWorkflowExecuteContext().getWorkflowGraph().getDag().containsNode(branch)) {
|
||||
throw new IllegalArgumentException(
|
||||
"The branch(code= " + branchNode
|
||||
+ ") is not in the dag, please check the switch task configuration");
|
||||
}
|
||||
if (!workflowExecuteRunnable.getWorkflowExecuteContext().getWorkflowGraph().getDag().containsNode(branchNode)) {
|
||||
throw new IllegalArgumentException(
|
||||
"The branch(code= " + branchNode
|
||||
+ ") is not in the dag, please check the switch task configuration");
|
||||
}
|
||||
}
|
||||
|
||||
private void addDefaultBranchToEnd() {
|
||||
SwitchResultVo switchResultVo = new SwitchResultVo(null, taskParameters.getNextNode());
|
||||
List<SwitchResultVo> dependTaskList = taskParameters.getDependTaskList();
|
||||
if (!dependTaskList.contains(switchResultVo)) {
|
||||
dependTaskList.add(switchResultVo);
|
||||
}
|
||||
}
|
||||
|
||||
private SwitchResultVo getDefaultSwitchResultVo() {
|
||||
return taskParameters.getDependTaskList().get(taskParameters.getDependTaskList().size() - 1);
|
||||
private String getTaskName(Long taskCode) {
|
||||
Optional<TaskNode> taskNode = Optional.ofNullable(workflowExecuteRunnable.getWorkflowExecuteContext())
|
||||
.map(IWorkflowExecuteContext::getWorkflowGraph)
|
||||
.map(IWorkflowGraph::getDag)
|
||||
.map(dag -> dag.getNode(taskCode));
|
||||
return taskNode.map(TaskNode::getName).orElse(null);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -20,6 +20,7 @@ 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.WorkflowExecuteRunnable;
|
||||
import org.apache.dolphinscheduler.server.master.runner.task.ILogicTaskPluginFactory;
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
@ -36,7 +37,14 @@ public class SwitchLogicTaskPluginFactory implements ILogicTaskPluginFactory<Swi
|
||||
|
||||
@Override
|
||||
public SwitchLogicTask createLogicTask(TaskExecutionContext taskExecutionContext) throws LogicTaskInitializeException {
|
||||
return new SwitchLogicTask(taskExecutionContext, processInstanceExecCacheManager);
|
||||
int workflowInstanceId = taskExecutionContext.getProcessInstanceId();
|
||||
WorkflowExecuteRunnable workflowExecuteRunnable =
|
||||
processInstanceExecCacheManager.getByProcessInstanceId(workflowInstanceId);
|
||||
if (workflowExecuteRunnable == null) {
|
||||
throw new LogicTaskInitializeException(
|
||||
"Cannot find the WorkflowExecuteRunnable by : " + workflowInstanceId);
|
||||
}
|
||||
return new SwitchLogicTask(workflowExecuteRunnable, taskExecutionContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -31,6 +31,7 @@ import org.apache.dolphinscheduler.extract.common.transportor.GetAppIdResponse;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
|
||||
import org.apache.dolphinscheduler.registry.api.RegistryClient;
|
||||
import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType;
|
||||
import org.apache.dolphinscheduler.server.master.builder.TaskExecutionContextBuilder;
|
||||
@ -40,7 +41,6 @@ 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.utils.TaskUtils;
|
||||
import org.apache.dolphinscheduler.service.process.ProcessService;
|
||||
import org.apache.dolphinscheduler.service.utils.ProcessUtils;
|
||||
|
||||
@ -167,7 +167,7 @@ public class WorkerFailoverService {
|
||||
|
||||
taskInstance.setProcessInstance(processInstance);
|
||||
|
||||
if (!TaskUtils.isMasterTask(taskInstance.getTaskType())) {
|
||||
if (!TaskTypeUtils.isLogicTask(taskInstance.getTaskType())) {
|
||||
killYarnTask(taskInstance, processInstance);
|
||||
} else {
|
||||
log.info("The failover taskInstance is a master task, no need to failover in worker failover");
|
||||
|
@ -1,48 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.dolphinscheduler.server.master.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.dynamic.DynamicLogicTask;
|
||||
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<String> MASTER_TASK_TYPES = Sets.newHashSet(
|
||||
BlockingLogicTask.TASK_TYPE,
|
||||
ConditionLogicTask.TASK_TYPE,
|
||||
DependentLogicTask.TASK_TYPE,
|
||||
SubWorkflowLogicTask.TASK_TYPE,
|
||||
SwitchLogicTask.TASK_TYPE,
|
||||
DynamicLogicTask.TASK_TYPE);
|
||||
|
||||
public boolean isMasterTask(String taskType) {
|
||||
return MASTER_TASK_TYPES.contains(taskType);
|
||||
}
|
||||
}
|
@ -392,7 +392,6 @@ public class WorkflowExecuteRunnableTest {
|
||||
// task instance already finished, not dispatch
|
||||
TaskInstance taskInstance = new TaskInstance();
|
||||
taskInstance.setState(TaskExecutionStatus.PAUSE);
|
||||
Mockito.when(processInstance.isBlocked()).thenReturn(true);
|
||||
TaskExecuteRunnable taskExecuteRunnable = Mockito.mock(TaskExecuteRunnable.class);
|
||||
workflowExecuteThread.tryToDispatchTaskInstance(taskInstance, taskExecuteRunnable);
|
||||
Mockito.verify(taskExecuteRunnable, Mockito.never()).dispatch();
|
||||
|
@ -17,15 +17,18 @@
|
||||
|
||||
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 static com.google.common.truth.Truth.assertThat;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.ConditionsLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.DependentLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.DynamicLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.SubWorkflowLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.SwitchLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.shell.ShellTaskChannelFactory;
|
||||
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
import org.mockito.InjectMocks;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
@ -42,18 +45,20 @@ public class TaskDispatchFactoryTest {
|
||||
@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));
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {
|
||||
ConditionsLogicTaskChannelFactory.NAME,
|
||||
DependentLogicTaskChannelFactory.NAME,
|
||||
DynamicLogicTaskChannelFactory.NAME,
|
||||
SubWorkflowLogicTaskChannelFactory.NAME,
|
||||
SwitchLogicTaskChannelFactory.NAME})
|
||||
public void getTaskDispatcher_withLogicTask(String taskType) {
|
||||
assertThat(taskDispatchFactory.getTaskDispatcher(taskType)).isSameInstanceAs(masterTaskDispatcher);
|
||||
}
|
||||
|
||||
Assertions.assertEquals(workerTaskDispatcher, taskDispatchFactory.getTaskDispatcher("SHELL"));
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {ShellTaskChannelFactory.NAME})
|
||||
public void getTaskDispatcher_withWorkerTask(String taskType) {
|
||||
assertThat(taskDispatchFactory.getTaskDispatcher(taskType)).isSameInstanceAs(workerTaskDispatcher);
|
||||
}
|
||||
}
|
||||
|
@ -19,6 +19,7 @@ package org.apache.dolphinscheduler.server.master.runner.task.dynamic;
|
||||
|
||||
import org.apache.dolphinscheduler.common.enums.CommandType;
|
||||
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
|
||||
import org.apache.dolphinscheduler.dao.mapper.CommandMapper;
|
||||
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
|
||||
@ -43,8 +44,6 @@ import org.mockito.Mock;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
class DynamicLogicTaskTest {
|
||||
|
||||
@ -74,14 +73,11 @@ class DynamicLogicTaskTest {
|
||||
|
||||
private DynamicLogicTask dynamicLogicTask;
|
||||
|
||||
private ObjectMapper objectMapper;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() {
|
||||
// Set up your test environment before each test.
|
||||
dynamicParameters = new DynamicParameters();
|
||||
taskExecutionContext = Mockito.mock(TaskExecutionContext.class);
|
||||
objectMapper = new ObjectMapper();
|
||||
processInstance = new ProcessInstance();
|
||||
Mockito.when(processInstanceDao.queryById(Mockito.any())).thenReturn(processInstance);
|
||||
dynamicLogicTask = new DynamicLogicTask(
|
||||
@ -95,7 +91,7 @@ class DynamicLogicTaskTest {
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGenerateParameterGroup() throws Exception {
|
||||
void testGenerateParameterGroup() {
|
||||
DynamicInputParameter dynamicInputParameter1 = new DynamicInputParameter();
|
||||
dynamicInputParameter1.setName("param1");
|
||||
dynamicInputParameter1.setValue("a,b,c");
|
||||
@ -113,7 +109,7 @@ class DynamicLogicTaskTest {
|
||||
|
||||
Mockito.when(taskExecutionContext.getPrepareParamsMap()).thenReturn(new HashMap<>());
|
||||
Mockito.when(taskExecutionContext.getTaskParams())
|
||||
.thenReturn(objectMapper.writeValueAsString(dynamicParameters));
|
||||
.thenReturn(JSONUtils.toJsonString(dynamicParameters));
|
||||
|
||||
dynamicLogicTask = new DynamicLogicTask(
|
||||
taskExecutionContext,
|
||||
|
@ -17,8 +17,6 @@
|
||||
|
||||
package org.apache.dolphinscheduler.server.master.service;
|
||||
|
||||
import static org.apache.dolphinscheduler.common.constants.Constants.COMMON_TASK_TYPE;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_SWITCH;
|
||||
import static org.mockito.BDDMockito.given;
|
||||
import static org.mockito.Mockito.doNothing;
|
||||
|
||||
@ -31,6 +29,8 @@ import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
|
||||
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.task.SwitchLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.shell.ShellTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.registry.api.RegistryClient;
|
||||
import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType;
|
||||
import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
|
||||
@ -139,13 +139,13 @@ public class FailoverServiceTest {
|
||||
masterTaskInstance.setId(1);
|
||||
masterTaskInstance.setStartTime(new Date());
|
||||
masterTaskInstance.setHost(testMasterHost);
|
||||
masterTaskInstance.setTaskType(TASK_TYPE_SWITCH);
|
||||
masterTaskInstance.setTaskType(SwitchLogicTaskChannelFactory.NAME);
|
||||
|
||||
workerTaskInstance = new TaskInstance();
|
||||
workerTaskInstance.setId(2);
|
||||
workerTaskInstance.setStartTime(new Date());
|
||||
workerTaskInstance.setHost(testWorkerHost);
|
||||
workerTaskInstance.setTaskType(COMMON_TASK_TYPE);
|
||||
workerTaskInstance.setTaskType(ShellTaskChannelFactory.NAME);
|
||||
|
||||
given(processService.queryNeedFailoverProcessInstances(Mockito.anyString()))
|
||||
.willReturn(Arrays.asList(processInstance));
|
||||
|
@ -1,39 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.dolphinscheduler.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));
|
||||
}
|
||||
}
|
@ -17,27 +17,17 @@
|
||||
|
||||
package org.apache.dolphinscheduler.service.model;
|
||||
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_BLOCKING;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_CONDITIONS;
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_SWITCH;
|
||||
|
||||
import org.apache.dolphinscheduler.common.constants.Constants;
|
||||
import org.apache.dolphinscheduler.common.enums.Priority;
|
||||
import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
|
||||
import org.apache.dolphinscheduler.common.model.PreviousTaskNode;
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.TaskTimeoutParameter;
|
||||
|
||||
import org.apache.commons.collections4.CollectionUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
|
||||
import com.fasterxml.jackson.databind.annotation.JsonSerialize;
|
||||
|
||||
@ -118,11 +108,6 @@ public class TaskNode {
|
||||
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
|
||||
private String preTasks;
|
||||
|
||||
/**
|
||||
* node dependency list
|
||||
*/
|
||||
private List<PreviousTaskNode> preTaskNodeList;
|
||||
|
||||
/**
|
||||
* users store additional information
|
||||
*/
|
||||
@ -135,25 +120,6 @@ public class TaskNode {
|
||||
*/
|
||||
private List<Long> depList;
|
||||
|
||||
/**
|
||||
* outer dependency information
|
||||
*/
|
||||
@JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class)
|
||||
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
|
||||
private String dependence;
|
||||
|
||||
@JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class)
|
||||
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
|
||||
private String conditionResult;
|
||||
|
||||
@JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class)
|
||||
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
|
||||
private String switchResult;
|
||||
|
||||
@JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class)
|
||||
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
|
||||
private String waitStartTimeout;
|
||||
|
||||
/**
|
||||
* task instance priority
|
||||
*/
|
||||
@ -312,10 +278,8 @@ public class TaskNode {
|
||||
&& Objects.equals(preTasks, taskNode.preTasks)
|
||||
&& Objects.equals(extras, taskNode.extras)
|
||||
&& Objects.equals(runFlag, taskNode.runFlag)
|
||||
&& Objects.equals(dependence, taskNode.dependence)
|
||||
&& Objects.equals(workerGroup, taskNode.workerGroup)
|
||||
&& Objects.equals(environmentCode, taskNode.environmentCode)
|
||||
&& Objects.equals(conditionResult, taskNode.conditionResult)
|
||||
&& CollectionUtils.isEqualCollection(depList, taskNode.depList)
|
||||
&& Objects.equals(taskExecuteType, taskNode.taskExecuteType);
|
||||
}
|
||||
@ -325,14 +289,6 @@ public class TaskNode {
|
||||
return Objects.hash(name, desc, type, params, preTasks, extras, depList, runFlag);
|
||||
}
|
||||
|
||||
public String getDependence() {
|
||||
return dependence;
|
||||
}
|
||||
|
||||
public void setDependence(String dependence) {
|
||||
this.dependence = dependence;
|
||||
}
|
||||
|
||||
public int getMaxRetryTimes() {
|
||||
return maxRetryTimes;
|
||||
}
|
||||
@ -373,14 +329,6 @@ public class TaskNode {
|
||||
this.workerGroup = workerGroup;
|
||||
}
|
||||
|
||||
public String getConditionResult() {
|
||||
return conditionResult;
|
||||
}
|
||||
|
||||
public void setConditionResult(String conditionResult) {
|
||||
this.conditionResult = conditionResult;
|
||||
}
|
||||
|
||||
public int getDelayTime() {
|
||||
return delayTime;
|
||||
}
|
||||
@ -405,64 +353,6 @@ public class TaskNode {
|
||||
this.version = version;
|
||||
}
|
||||
|
||||
/**
|
||||
* get task time out parameter
|
||||
*
|
||||
* @return task time out parameter
|
||||
*/
|
||||
public TaskTimeoutParameter getTaskTimeoutParameter() {
|
||||
if (!StringUtils.isEmpty(this.getTimeout())) {
|
||||
String formatStr =
|
||||
String.format("%s,%s", TaskTimeoutStrategy.WARN.name(), TaskTimeoutStrategy.FAILED.name());
|
||||
String taskTimeout = this.getTimeout().replace(formatStr, TaskTimeoutStrategy.WARNFAILED.name());
|
||||
return JSONUtils.parseObject(taskTimeout, TaskTimeoutParameter.class);
|
||||
}
|
||||
return new TaskTimeoutParameter(false);
|
||||
}
|
||||
|
||||
public boolean isConditionsTask() {
|
||||
return TASK_TYPE_CONDITIONS.equalsIgnoreCase(this.getType());
|
||||
}
|
||||
|
||||
public boolean isSwitchTask() {
|
||||
return TASK_TYPE_SWITCH.equalsIgnoreCase(this.getType());
|
||||
}
|
||||
|
||||
public List<PreviousTaskNode> getPreTaskNodeList() {
|
||||
return preTaskNodeList;
|
||||
}
|
||||
|
||||
public boolean isBlockingTask() {
|
||||
return TASK_TYPE_BLOCKING.equalsIgnoreCase(this.getType());
|
||||
}
|
||||
|
||||
public void setPreTaskNodeList(List<PreviousTaskNode> preTaskNodeList) {
|
||||
this.preTaskNodeList = preTaskNodeList;
|
||||
}
|
||||
|
||||
public String getTaskParams() {
|
||||
Map<String, Object> taskParams = JSONUtils.parseObject(this.params, new TypeReference<Map<String, Object>>() {
|
||||
});
|
||||
|
||||
if (taskParams == null) {
|
||||
taskParams = new HashMap<>();
|
||||
}
|
||||
taskParams.put(Constants.CONDITION_RESULT, this.conditionResult);
|
||||
taskParams.put(Constants.DEPENDENCE, this.dependence);
|
||||
taskParams.put(Constants.SWITCH_RESULT, this.switchResult);
|
||||
taskParams.put(Constants.WAIT_START_TIMEOUT, this.waitStartTimeout);
|
||||
return JSONUtils.toJsonString(taskParams);
|
||||
}
|
||||
|
||||
public Map<String, Object> taskParamsToJsonObj(String taskParams) {
|
||||
Map<String, Object> taskParamsMap = JSONUtils.parseObject(taskParams, new TypeReference<Map<String, Object>>() {
|
||||
});
|
||||
if (taskParamsMap == null) {
|
||||
taskParamsMap = new HashMap<>();
|
||||
}
|
||||
return taskParamsMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "TaskNode{"
|
||||
@ -478,11 +368,8 @@ public class TaskNode {
|
||||
+ ", retryInterval=" + retryInterval
|
||||
+ ", params='" + params + '\''
|
||||
+ ", preTasks='" + preTasks + '\''
|
||||
+ ", preTaskNodeList=" + preTaskNodeList
|
||||
+ ", extras='" + extras + '\''
|
||||
+ ", depList=" + depList
|
||||
+ ", dependence='" + dependence + '\''
|
||||
+ ", conditionResult='" + conditionResult + '\''
|
||||
+ ", taskInstancePriority=" + taskInstancePriority
|
||||
+ ", workerGroup='" + workerGroup + '\''
|
||||
+ ", environmentCode=" + environmentCode
|
||||
@ -500,22 +387,6 @@ public class TaskNode {
|
||||
return this.environmentCode;
|
||||
}
|
||||
|
||||
public String getSwitchResult() {
|
||||
return switchResult;
|
||||
}
|
||||
|
||||
public void setSwitchResult(String switchResult) {
|
||||
this.switchResult = switchResult;
|
||||
}
|
||||
|
||||
public String getWaitStartTimeout() {
|
||||
return this.waitStartTimeout;
|
||||
}
|
||||
|
||||
public void setWaitStartTimeout(String waitStartTimeout) {
|
||||
this.waitStartTimeout = waitStartTimeout;
|
||||
}
|
||||
|
||||
public int getTaskGroupId() {
|
||||
return taskGroupId;
|
||||
}
|
||||
|
@ -176,8 +176,6 @@ public interface ProcessService {
|
||||
|
||||
DqComparisonType getComparisonTypeById(int id);
|
||||
|
||||
void changeTaskGroupQueueStatus(int taskId, TaskGroupQueueStatus status);
|
||||
|
||||
TaskGroupQueue insertIntoTaskGroupQueue(Integer taskId,
|
||||
String taskName,
|
||||
Integer groupId,
|
||||
|
@ -120,6 +120,7 @@ import org.apache.dolphinscheduler.plugin.task.api.model.Property;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.SubProcessParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.TaskTimeoutParameter;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
|
||||
import org.apache.dolphinscheduler.service.command.CommandService;
|
||||
import org.apache.dolphinscheduler.service.cron.CronUtils;
|
||||
import org.apache.dolphinscheduler.service.exceptions.CronParseException;
|
||||
@ -1051,7 +1052,7 @@ public class ProcessServiceImpl implements ProcessService {
|
||||
*/
|
||||
private void initTaskInstance(TaskInstance taskInstance) {
|
||||
|
||||
if (!taskInstance.isSubProcess()
|
||||
if (!TaskTypeUtils.isSubWorkflowTask(taskInstance.getTaskType())
|
||||
&& (taskInstance.getState().isKill() || taskInstance.getState().isFailure())) {
|
||||
taskInstance.setFlag(Flag.NO);
|
||||
taskInstanceDao.updateById(taskInstance);
|
||||
@ -1169,7 +1170,7 @@ public class ProcessServiceImpl implements ProcessService {
|
||||
*/
|
||||
@Override
|
||||
public void createSubWorkProcess(ProcessInstance parentProcessInstance, TaskInstance task) {
|
||||
if (!task.isSubProcess()) {
|
||||
if (!TaskTypeUtils.isSubWorkflowTask(task.getTaskType())) {
|
||||
return;
|
||||
}
|
||||
// check create sub work flow firstly
|
||||
@ -1940,13 +1941,7 @@ public class ProcessServiceImpl implements ProcessService {
|
||||
: Constants.FLOWNODE_RUN_FLAG_FORBIDDEN);
|
||||
taskNode.setMaxRetryTimes(taskDefinitionLog.getFailRetryTimes());
|
||||
taskNode.setRetryInterval(taskDefinitionLog.getFailRetryInterval());
|
||||
Map<String, Object> taskParamsMap = taskNode.taskParamsToJsonObj(taskDefinitionLog.getTaskParams());
|
||||
taskNode.setConditionResult(JSONUtils.toJsonString(taskParamsMap.get(Constants.CONDITION_RESULT)));
|
||||
taskNode.setSwitchResult(JSONUtils.toJsonString(taskParamsMap.get(Constants.SWITCH_RESULT)));
|
||||
taskNode.setDependence(JSONUtils.toJsonString(taskParamsMap.get(Constants.DEPENDENCE)));
|
||||
taskParamsMap.remove(Constants.CONDITION_RESULT);
|
||||
taskParamsMap.remove(Constants.DEPENDENCE);
|
||||
taskNode.setParams(JSONUtils.toJsonString(taskParamsMap));
|
||||
taskNode.setParams(taskDefinitionLog.getTaskParams());
|
||||
taskNode.setTaskInstancePriority(taskDefinitionLog.getTaskPriority());
|
||||
taskNode.setWorkerGroup(taskDefinitionLog.getWorkerGroup());
|
||||
taskNode.setEnvironmentCode(taskDefinitionLog.getEnvironmentCode());
|
||||
@ -2039,22 +2034,6 @@ public class ProcessServiceImpl implements ProcessService {
|
||||
return dqComparisonTypeMapper.selectById(id);
|
||||
}
|
||||
|
||||
/**
|
||||
* release the TGQ resource when the corresponding task is finished.
|
||||
*
|
||||
* @param taskId task id
|
||||
* @return the result code and msg
|
||||
*/
|
||||
|
||||
@Override
|
||||
public void changeTaskGroupQueueStatus(int taskId, TaskGroupQueueStatus status) {
|
||||
TaskGroupQueue taskGroupQueue = taskGroupQueueMapper.queryByTaskId(taskId);
|
||||
taskGroupQueue.setInQueue(Flag.NO.getCode());
|
||||
taskGroupQueue.setStatus(status);
|
||||
taskGroupQueue.setUpdateTime(new Date(System.currentTimeMillis()));
|
||||
taskGroupQueueMapper.updateById(taskGroupQueue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskGroupQueue insertIntoTaskGroupQueue(Integer taskInstanceId,
|
||||
String taskName,
|
||||
|
@ -23,10 +23,12 @@ import org.apache.dolphinscheduler.common.model.TaskNodeRelation;
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelation;
|
||||
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.model.SwitchResultVo;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ConditionsParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.SwitchParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.ConditionsLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.SwitchLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
|
||||
import org.apache.dolphinscheduler.service.model.TaskNode;
|
||||
import org.apache.dolphinscheduler.service.process.ProcessDag;
|
||||
|
||||
@ -35,6 +37,7 @@ import org.apache.commons.lang3.StringUtils;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
@ -44,6 +47,9 @@ import java.util.Set;
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* dag tools
|
||||
*/
|
||||
@ -228,22 +234,6 @@ public class DagHelper {
|
||||
return processDag;
|
||||
}
|
||||
|
||||
/**
|
||||
* find node by node name
|
||||
*
|
||||
* @param nodeDetails nodeDetails
|
||||
* @param nodeName nodeName
|
||||
* @return task node
|
||||
*/
|
||||
public static TaskNode findNodeByName(List<TaskNode> nodeDetails, String nodeName) {
|
||||
for (TaskNode taskNode : nodeDetails) {
|
||||
if (taskNode.getName().equals(nodeName)) {
|
||||
return taskNode;
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* find node by node code
|
||||
*
|
||||
@ -305,10 +295,10 @@ public class DagHelper {
|
||||
|
||||
if (preNodeCode == null) {
|
||||
startVertexes = dag.getBeginNode();
|
||||
} else if (dag.getNode(preNodeCode).isConditionsTask()) {
|
||||
} else if (TaskTypeUtils.isConditionTask(dag.getNode(preNodeCode).getType())) {
|
||||
List<Long> conditionTaskList = parseConditionTask(preNodeCode, skipTaskNodeList, dag, completeTaskList);
|
||||
startVertexes.addAll(conditionTaskList);
|
||||
} else if (dag.getNode(preNodeCode).isSwitchTask()) {
|
||||
} else if (TaskTypeUtils.isSwitchTask(dag.getNode(preNodeCode).getType())) {
|
||||
List<Long> conditionTaskList = parseSwitchTask(preNodeCode, skipTaskNodeList, dag, completeTaskList);
|
||||
startVertexes.addAll(conditionTaskList);
|
||||
} else {
|
||||
@ -321,7 +311,7 @@ public class DagHelper {
|
||||
continue;
|
||||
}
|
||||
if (isTaskNodeNeedSkip(taskNode, skipTaskNodeList)) {
|
||||
setTaskNodeSkip(subsequent, dag, completeTaskList, skipTaskNodeList);
|
||||
setTaskNodeSkip(subsequent, dag, skipTaskNodeList);
|
||||
continue;
|
||||
}
|
||||
if (!DagHelper.allDependsForbiddenOrEnd(taskNode, dag, skipTaskNodeList, completeTaskList)) {
|
||||
@ -362,18 +352,20 @@ public class DagHelper {
|
||||
Map<Long, TaskInstance> completeTaskList) {
|
||||
List<Long> conditionTaskList = new ArrayList<>();
|
||||
TaskNode taskNode = dag.getNode(nodeCode);
|
||||
if (!taskNode.isConditionsTask()) {
|
||||
if (!TaskTypeUtils.isConditionTask(taskNode.getType())) {
|
||||
return conditionTaskList;
|
||||
}
|
||||
if (!completeTaskList.containsKey(nodeCode)) {
|
||||
return conditionTaskList;
|
||||
}
|
||||
TaskInstance taskInstance = completeTaskList.get(nodeCode);
|
||||
ConditionsParameters conditionsParameters = taskInstance.getConditionsParameters();
|
||||
ConditionsParameters.ConditionResult conditionResult = taskInstance.getConditionResult();
|
||||
ConditionsParameters conditionsParameters =
|
||||
JSONUtils.parseObject(taskInstance.getTaskParams(), new TypeReference<ConditionsParameters>() {
|
||||
});
|
||||
ConditionsParameters.ConditionResult conditionResult = conditionsParameters.getConditionResult();
|
||||
|
||||
List<Long> skipNodeList = new ArrayList<>();
|
||||
if (conditionsParameters.isConditionSuccess()) {
|
||||
if (conditionResult.isConditionSuccess()) {
|
||||
conditionTaskList = conditionResult.getSuccessNode();
|
||||
skipNodeList = conditionResult.getFailedNode();
|
||||
} else {
|
||||
@ -382,7 +374,7 @@ public class DagHelper {
|
||||
}
|
||||
|
||||
if (CollectionUtils.isNotEmpty(skipNodeList)) {
|
||||
skipNodeList.forEach(skipNode -> setTaskNodeSkip(skipNode, dag, completeTaskList, skipTaskNodeList));
|
||||
skipNodeList.forEach(skipNode -> setTaskNodeSkip(skipNode, dag, skipTaskNodeList));
|
||||
}
|
||||
// the conditionTaskList maybe null if no next task
|
||||
conditionTaskList = Optional.ofNullable(conditionTaskList).orElse(new ArrayList<>());
|
||||
@ -402,82 +394,50 @@ public class DagHelper {
|
||||
Map<Long, TaskInstance> completeTaskList) {
|
||||
List<Long> conditionTaskList = new ArrayList<>();
|
||||
TaskNode taskNode = dag.getNode(nodeCode);
|
||||
if (!taskNode.isSwitchTask()) {
|
||||
if (!SwitchLogicTaskChannelFactory.NAME.equals(taskNode.getType())) {
|
||||
return conditionTaskList;
|
||||
}
|
||||
if (!completeTaskList.containsKey(nodeCode)) {
|
||||
return conditionTaskList;
|
||||
}
|
||||
conditionTaskList = skipTaskNode4Switch(taskNode, skipTaskNodeList, completeTaskList, dag);
|
||||
conditionTaskList = skipTaskNode4Switch(skipTaskNodeList, completeTaskList.get(nodeCode), dag);
|
||||
return conditionTaskList;
|
||||
}
|
||||
|
||||
public static List<Long> skipTaskNode4Switch(TaskNode taskNode,
|
||||
Map<Long, TaskNode> skipTaskNodeList,
|
||||
Map<Long, TaskInstance> completeTaskList,
|
||||
public static List<Long> skipTaskNode4Switch(Map<Long, TaskNode> skipTaskNodeList,
|
||||
TaskInstance taskInstance,
|
||||
DAG<Long, TaskNode, TaskNodeRelation> dag) {
|
||||
|
||||
SwitchParameters switchParameters =
|
||||
completeTaskList.get(taskNode.getCode()).getSwitchDependency();
|
||||
int resultConditionLocation = switchParameters.getResultConditionLocation();
|
||||
List<SwitchResultVo> conditionResultVoList = switchParameters.getDependTaskList();
|
||||
List<Long> switchTaskList = conditionResultVoList.get(resultConditionLocation).getNextNode();
|
||||
Set<Long> switchNeedWorkCodes = new HashSet<>();
|
||||
if (CollectionUtils.isEmpty(switchTaskList)) {
|
||||
return new ArrayList<>();
|
||||
}
|
||||
// get all downstream nodes of the branch that the switch node needs to execute
|
||||
for (Long switchTaskCode : switchTaskList) {
|
||||
getSwitchNeedWorkCodes(switchTaskCode, dag, switchNeedWorkCodes);
|
||||
}
|
||||
// conditionResultVoList.remove(resultConditionLocation);
|
||||
for (SwitchResultVo info : conditionResultVoList) {
|
||||
if (CollectionUtils.isEmpty(info.getNextNode())) {
|
||||
continue;
|
||||
}
|
||||
for (Long nextNode : info.getNextNode()) {
|
||||
setSwitchTaskNodeSkip(nextNode, dag, completeTaskList, skipTaskNodeList,
|
||||
switchNeedWorkCodes);
|
||||
}
|
||||
}
|
||||
return switchTaskList;
|
||||
}
|
||||
JSONUtils.parseObject(taskInstance.getTaskParams(), new TypeReference<SwitchParameters>() {
|
||||
});
|
||||
|
||||
/**
|
||||
* get all downstream nodes of the branch that the switch node needs to execute
|
||||
*
|
||||
* @param taskCode
|
||||
* @param dag
|
||||
* @param switchNeedWorkCodes
|
||||
*/
|
||||
public static void getSwitchNeedWorkCodes(Long taskCode, DAG<Long, TaskNode, TaskNodeRelation> dag,
|
||||
Set<Long> switchNeedWorkCodes) {
|
||||
switchNeedWorkCodes.add(taskCode);
|
||||
Set<Long> subsequentNodes = dag.getSubsequentNodes(taskCode);
|
||||
if (org.apache.commons.collections.CollectionUtils.isNotEmpty(subsequentNodes)) {
|
||||
for (Long subCode : subsequentNodes) {
|
||||
getSwitchNeedWorkCodes(subCode, dag, switchNeedWorkCodes);
|
||||
}
|
||||
SwitchParameters.SwitchResult switchResult = switchParameters.getSwitchResult();
|
||||
Long nextBranch = switchParameters.getNextBranch();
|
||||
if (switchResult == null) {
|
||||
log.error("switchResult is null, please check the switch task configuration");
|
||||
return Collections.emptyList();
|
||||
}
|
||||
if (nextBranch == null) {
|
||||
log.error("switchParameters.getNextBranch() is null, please check the switch task configuration");
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
|
||||
private static void setSwitchTaskNodeSkip(Long skipNodeCode,
|
||||
DAG<Long, TaskNode, TaskNodeRelation> dag,
|
||||
Map<Long, TaskInstance> completeTaskList,
|
||||
Map<Long, TaskNode> skipTaskNodeList,
|
||||
Set<Long> switchNeedWorkCodes) {
|
||||
// ignore when the node that needs to be skipped exists on the branch that the switch type node needs to execute
|
||||
if (!dag.containsNode(skipNodeCode) || switchNeedWorkCodes.contains(skipNodeCode)) {
|
||||
return;
|
||||
Set<Long> allNextBranches = new HashSet<>();
|
||||
if (switchResult.getNextNode() != null) {
|
||||
allNextBranches.add(switchResult.getNextNode());
|
||||
}
|
||||
skipTaskNodeList.putIfAbsent(skipNodeCode, dag.getNode(skipNodeCode));
|
||||
Collection<Long> postNodeList = dag.getSubsequentNodes(skipNodeCode);
|
||||
for (Long post : postNodeList) {
|
||||
TaskNode postNode = dag.getNode(post);
|
||||
if (isTaskNodeNeedSkip(postNode, skipTaskNodeList)) {
|
||||
setTaskNodeSkip(post, dag, completeTaskList, skipTaskNodeList);
|
||||
if (CollectionUtils.isNotEmpty(switchResult.getDependTaskList())) {
|
||||
for (SwitchResultVo switchResultVo : switchResult.getDependTaskList()) {
|
||||
allNextBranches.add(switchResultVo.getNextNode());
|
||||
}
|
||||
}
|
||||
|
||||
allNextBranches.remove(nextBranch);
|
||||
|
||||
for (Long branch : allNextBranches) {
|
||||
setTaskNodeSkip(branch, dag, skipTaskNodeList);
|
||||
}
|
||||
return Lists.newArrayList(nextBranch);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -485,7 +445,6 @@ public class DagHelper {
|
||||
*/
|
||||
private static void setTaskNodeSkip(Long skipNodeCode,
|
||||
DAG<Long, TaskNode, TaskNodeRelation> dag,
|
||||
Map<Long, TaskInstance> completeTaskList,
|
||||
Map<Long, TaskNode> skipTaskNodeList) {
|
||||
if (!dag.containsNode(skipNodeCode)) {
|
||||
return;
|
||||
@ -495,7 +454,7 @@ public class DagHelper {
|
||||
for (Long post : postNodeList) {
|
||||
TaskNode postNode = dag.getNode(post);
|
||||
if (isTaskNodeNeedSkip(postNode, skipTaskNodeList)) {
|
||||
setTaskNodeSkip(post, dag, completeTaskList, skipTaskNodeList);
|
||||
setTaskNodeSkip(post, dag, skipTaskNodeList);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -591,31 +550,7 @@ public class DagHelper {
|
||||
*/
|
||||
public static boolean haveConditionsAfterNode(Long parentNodeCode,
|
||||
DAG<Long, TaskNode, TaskNodeRelation> dag) {
|
||||
return haveSubAfterNode(parentNodeCode, dag, TaskConstants.TASK_TYPE_CONDITIONS);
|
||||
}
|
||||
|
||||
/**
|
||||
* is there have conditions after the parent node
|
||||
*/
|
||||
public static boolean haveConditionsAfterNode(Long parentNodeCode, List<TaskNode> taskNodes) {
|
||||
if (CollectionUtils.isEmpty(taskNodes)) {
|
||||
return false;
|
||||
}
|
||||
for (TaskNode taskNode : taskNodes) {
|
||||
List<Long> preTasksList = JSONUtils.toList(taskNode.getPreTasks(), Long.class);
|
||||
if (preTasksList.contains(parentNodeCode) && taskNode.isConditionsTask()) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* is there have blocking node after the parent node
|
||||
*/
|
||||
public static boolean haveBlockingAfterNode(Long parentNodeCode,
|
||||
DAG<Long, TaskNode, TaskNodeRelation> dag) {
|
||||
return haveSubAfterNode(parentNodeCode, dag, TaskConstants.TASK_TYPE_BLOCKING);
|
||||
return haveSubAfterNode(parentNodeCode, dag, ConditionsLogicTaskChannelFactory.NAME);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -17,19 +17,19 @@
|
||||
|
||||
package org.apache.dolphinscheduler.service.utils;
|
||||
|
||||
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_CONDITIONS;
|
||||
|
||||
import org.apache.dolphinscheduler.common.constants.Constants;
|
||||
import org.apache.dolphinscheduler.common.enums.TaskDependType;
|
||||
import org.apache.dolphinscheduler.common.graph.DAG;
|
||||
import org.apache.dolphinscheduler.common.model.TaskNodeRelation;
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.model.SwitchResultVo;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ConditionsParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.SwitchParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.ConditionsLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.DependentLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.SwitchLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.service.model.TaskNode;
|
||||
import org.apache.dolphinscheduler.service.process.ProcessDag;
|
||||
|
||||
@ -44,10 +44,9 @@ import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.truth.Truth;
|
||||
|
||||
/**
|
||||
* dag helper test
|
||||
*/
|
||||
public class DagHelperTest {
|
||||
|
||||
@Test
|
||||
@ -90,26 +89,105 @@ public class DagHelperTest {
|
||||
boolean canSubmit = DagHelper.haveAllNodeAfterNode(parentNodeCode, dag);
|
||||
Assertions.assertTrue(canSubmit);
|
||||
|
||||
boolean haveBlocking = DagHelper.haveBlockingAfterNode(parentNodeCode, dag);
|
||||
Assertions.assertTrue(haveBlocking);
|
||||
|
||||
boolean haveConditions = DagHelper.haveConditionsAfterNode(parentNodeCode, dag);
|
||||
Assertions.assertTrue(haveConditions);
|
||||
|
||||
boolean dependent = DagHelper.haveSubAfterNode(parentNodeCode, dag, TaskConstants.TASK_TYPE_DEPENDENT);
|
||||
boolean dependent = DagHelper.haveSubAfterNode(parentNodeCode, dag, DependentLogicTaskChannelFactory.NAME);
|
||||
Assertions.assertFalse(dependent);
|
||||
}
|
||||
|
||||
/**
|
||||
* test task node can submit
|
||||
*
|
||||
* @throws JsonProcessingException if error throws JsonProcessingException
|
||||
*/
|
||||
@Test
|
||||
public void testTaskNodeCanSubmit() throws IOException {
|
||||
public void testTaskNodeCanSubmit() {
|
||||
List<TaskNode> taskNodeList = new ArrayList<>();
|
||||
TaskNode node1 = new TaskNode();
|
||||
node1.setId("1");
|
||||
node1.setName("1");
|
||||
node1.setCode(1);
|
||||
node1.setType("SHELL");
|
||||
taskNodeList.add(node1);
|
||||
|
||||
TaskNode node2 = new TaskNode();
|
||||
node2.setId("2");
|
||||
node2.setName("2");
|
||||
node2.setCode(2);
|
||||
node2.setType("SHELL");
|
||||
List<String> dep2 = new ArrayList<>();
|
||||
dep2.add("1");
|
||||
node2.setPreTasks(JSONUtils.toJsonString(dep2));
|
||||
taskNodeList.add(node2);
|
||||
|
||||
TaskNode node4 = new TaskNode();
|
||||
node4.setId("4");
|
||||
node4.setName("4");
|
||||
node4.setCode(4);
|
||||
node4.setType("SHELL");
|
||||
taskNodeList.add(node4);
|
||||
|
||||
TaskNode node3 = new TaskNode();
|
||||
node3.setId("3");
|
||||
node3.setName("3");
|
||||
node3.setCode(3);
|
||||
node3.setType("SHELL");
|
||||
List<String> dep3 = new ArrayList<>();
|
||||
dep3.add("2");
|
||||
dep3.add("4");
|
||||
node3.setPreTasks(JSONUtils.toJsonString(dep3));
|
||||
taskNodeList.add(node3);
|
||||
|
||||
TaskNode node5 = new TaskNode();
|
||||
node5.setId("5");
|
||||
node5.setName("5");
|
||||
node5.setCode(5);
|
||||
node5.setType("SHELL");
|
||||
List<String> dep5 = new ArrayList<>();
|
||||
dep5.add("3");
|
||||
dep5.add("8");
|
||||
node5.setPreTasks(JSONUtils.toJsonString(dep5));
|
||||
taskNodeList.add(node5);
|
||||
|
||||
TaskNode node6 = new TaskNode();
|
||||
node6.setId("6");
|
||||
node6.setName("6");
|
||||
node6.setCode(6);
|
||||
node6.setType("SHELL");
|
||||
List<String> dep6 = new ArrayList<>();
|
||||
dep6.add("3");
|
||||
node6.setPreTasks(JSONUtils.toJsonString(dep6));
|
||||
taskNodeList.add(node6);
|
||||
|
||||
TaskNode node7 = new TaskNode();
|
||||
node7.setId("7");
|
||||
node7.setName("7");
|
||||
node7.setCode(7);
|
||||
node7.setType("SHELL");
|
||||
List<String> dep7 = new ArrayList<>();
|
||||
dep7.add("5");
|
||||
node7.setPreTasks(JSONUtils.toJsonString(dep7));
|
||||
taskNodeList.add(node7);
|
||||
|
||||
TaskNode node8 = new TaskNode();
|
||||
node8.setId("8");
|
||||
node8.setName("8");
|
||||
node8.setCode(8);
|
||||
node8.setType("SHELL");
|
||||
List<String> dep8 = new ArrayList<>();
|
||||
dep8.add("2");
|
||||
node8.setPreTasks(JSONUtils.toJsonString(dep8));
|
||||
taskNodeList.add(node8);
|
||||
|
||||
List<Long> startNodes = new ArrayList<>();
|
||||
List<Long> recoveryNodes = new ArrayList<>();
|
||||
List<TaskNode> destTaskNodeList = DagHelper.generateFlowNodeListByStartNode(taskNodeList,
|
||||
startNodes, recoveryNodes, TaskDependType.TASK_POST);
|
||||
List<TaskNodeRelation> taskNodeRelations = DagHelper.generateRelationListByFlowNodes(destTaskNodeList);
|
||||
ProcessDag processDag = new ProcessDag();
|
||||
processDag.setEdges(taskNodeRelations);
|
||||
processDag.setNodes(destTaskNodeList);
|
||||
|
||||
// 1->2->3->5->7
|
||||
// 4->3->6
|
||||
DAG<Long, TaskNode, TaskNodeRelation> dag = generateDag();
|
||||
// 1->2->8->5->7
|
||||
DAG<Long, TaskNode, TaskNodeRelation> dag = DagHelper.buildDagGraph(processDag);
|
||||
TaskNode taskNode3 = dag.getNode(3L);
|
||||
Map<Long, TaskInstance> completeTaskList = new HashMap<>();
|
||||
Map<Long, TaskNode> skipNodeList = new HashMap<>();
|
||||
@ -117,7 +195,7 @@ public class DagHelperTest {
|
||||
Boolean canSubmit = false;
|
||||
|
||||
// 2/4 are forbidden submit 3
|
||||
TaskNode node2 = dag.getNode(2L);
|
||||
node2 = dag.getNode(2L);
|
||||
node2.setRunFlag(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN);
|
||||
TaskNode nodex = dag.getNode(4L);
|
||||
nodex.setRunFlag(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN);
|
||||
@ -132,21 +210,107 @@ public class DagHelperTest {
|
||||
Assertions.assertEquals(canSubmit, false);
|
||||
|
||||
// 2/3 forbidden submit 5
|
||||
TaskNode node3 = dag.getNode(3L);
|
||||
node3 = dag.getNode(3L);
|
||||
node3.setRunFlag(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN);
|
||||
TaskNode node8 = dag.getNode(8L);
|
||||
node8 = dag.getNode(8L);
|
||||
node8.setRunFlag(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN);
|
||||
TaskNode node5 = dag.getNode(5L);
|
||||
node5 = dag.getNode(5L);
|
||||
canSubmit = DagHelper.allDependsForbiddenOrEnd(node5, dag, skipNodeList, completeTaskList);
|
||||
Assertions.assertEquals(canSubmit, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* test parse post node list
|
||||
*/
|
||||
@Test
|
||||
public void testParsePostNodeList() throws IOException {
|
||||
DAG<Long, TaskNode, TaskNodeRelation> dag = generateDag();
|
||||
public void testParsePostNodeList() {
|
||||
List<TaskNode> taskNodeList = new ArrayList<>();
|
||||
TaskNode node1 = new TaskNode();
|
||||
node1.setId("1");
|
||||
node1.setName("1");
|
||||
node1.setCode(1);
|
||||
node1.setType("SHELL");
|
||||
taskNodeList.add(node1);
|
||||
|
||||
TaskNode node2 = new TaskNode();
|
||||
node2.setId("2");
|
||||
node2.setName("2");
|
||||
node2.setCode(2);
|
||||
node2.setType("SHELL");
|
||||
List<String> dep2 = new ArrayList<>();
|
||||
dep2.add("1");
|
||||
node2.setPreTasks(JSONUtils.toJsonString(dep2));
|
||||
taskNodeList.add(node2);
|
||||
|
||||
TaskNode node4 = new TaskNode();
|
||||
node4.setId("4");
|
||||
node4.setName("4");
|
||||
node4.setCode(4);
|
||||
node4.setType("SHELL");
|
||||
taskNodeList.add(node4);
|
||||
|
||||
TaskNode node3 = new TaskNode();
|
||||
node3.setId("3");
|
||||
node3.setName("3");
|
||||
node3.setCode(3);
|
||||
node3.setType("SHELL");
|
||||
List<String> dep3 = new ArrayList<>();
|
||||
dep3.add("2");
|
||||
dep3.add("4");
|
||||
node3.setPreTasks(JSONUtils.toJsonString(dep3));
|
||||
taskNodeList.add(node3);
|
||||
|
||||
TaskNode node5 = new TaskNode();
|
||||
node5.setId("5");
|
||||
node5.setName("5");
|
||||
node5.setCode(5);
|
||||
node5.setType("SHELL");
|
||||
List<String> dep5 = new ArrayList<>();
|
||||
dep5.add("3");
|
||||
dep5.add("8");
|
||||
node5.setPreTasks(JSONUtils.toJsonString(dep5));
|
||||
taskNodeList.add(node5);
|
||||
|
||||
TaskNode node6 = new TaskNode();
|
||||
node6.setId("6");
|
||||
node6.setName("6");
|
||||
node6.setCode(6);
|
||||
node6.setType("SHELL");
|
||||
List<String> dep6 = new ArrayList<>();
|
||||
dep6.add("3");
|
||||
node6.setPreTasks(JSONUtils.toJsonString(dep6));
|
||||
taskNodeList.add(node6);
|
||||
|
||||
TaskNode node7 = new TaskNode();
|
||||
node7.setId("7");
|
||||
node7.setName("7");
|
||||
node7.setCode(7);
|
||||
node7.setType("SHELL");
|
||||
List<String> dep7 = new ArrayList<>();
|
||||
dep7.add("5");
|
||||
node7.setPreTasks(JSONUtils.toJsonString(dep7));
|
||||
taskNodeList.add(node7);
|
||||
|
||||
TaskNode node8 = new TaskNode();
|
||||
node8.setId("8");
|
||||
node8.setName("8");
|
||||
node8.setCode(8);
|
||||
node8.setType("SHELL");
|
||||
List<String> dep8 = new ArrayList<>();
|
||||
dep8.add("2");
|
||||
node8.setPreTasks(JSONUtils.toJsonString(dep8));
|
||||
taskNodeList.add(node8);
|
||||
|
||||
List<Long> startNodes = new ArrayList<>();
|
||||
List<Long> recoveryNodes = new ArrayList<>();
|
||||
List<TaskNode> destTaskNodeList = DagHelper.generateFlowNodeListByStartNode(taskNodeList,
|
||||
startNodes, recoveryNodes, TaskDependType.TASK_POST);
|
||||
List<TaskNodeRelation> taskNodeRelations = DagHelper.generateRelationListByFlowNodes(destTaskNodeList);
|
||||
ProcessDag processDag = new ProcessDag();
|
||||
processDag.setEdges(taskNodeRelations);
|
||||
processDag.setNodes(destTaskNodeList);
|
||||
|
||||
// 1->2->3->5->7
|
||||
// 4->3->6
|
||||
// 1->2->8->5->7
|
||||
DAG<Long, TaskNode, TaskNodeRelation> dag = DagHelper.buildDagGraph(processDag);
|
||||
Map<Long, TaskInstance> completeTaskList = new HashMap<>();
|
||||
Map<Long, TaskNode> skipNodeList = new HashMap<>();
|
||||
|
||||
@ -206,11 +370,6 @@ public class DagHelperTest {
|
||||
Assertions.assertTrue(postNodes.contains(7L));
|
||||
}
|
||||
|
||||
/**
|
||||
* test forbidden post node
|
||||
*
|
||||
* @throws JsonProcessingException
|
||||
*/
|
||||
@Test
|
||||
public void testForbiddenPostNode() throws IOException {
|
||||
DAG<Long, TaskNode, TaskNodeRelation> dag = generateDag();
|
||||
@ -244,11 +403,6 @@ public class DagHelperTest {
|
||||
Assertions.assertTrue(postNodes.contains(3L));
|
||||
}
|
||||
|
||||
/**
|
||||
* test condition post node
|
||||
*
|
||||
* @throws JsonProcessingException
|
||||
*/
|
||||
@Test
|
||||
public void testConditionPostNode() throws IOException {
|
||||
DAG<Long, TaskNode, TaskNodeRelation> dag = generateDag();
|
||||
@ -262,26 +416,18 @@ public class DagHelperTest {
|
||||
completeTaskList.put(4L, new TaskInstance());
|
||||
|
||||
TaskInstance taskInstance3 = new TaskInstance();
|
||||
taskInstance3.setTaskType(TASK_TYPE_CONDITIONS);
|
||||
Map<String, Object> params = new HashMap<>();
|
||||
taskInstance3.setTaskType(ConditionsLogicTaskChannelFactory.NAME);
|
||||
ConditionsParameters.ConditionResult conditionResult = ConditionsParameters.ConditionResult.builder()
|
||||
.conditionSuccess(true)
|
||||
.successNode(Lists.newArrayList(5L))
|
||||
.failedNode(Lists.newArrayList(6L))
|
||||
.build();
|
||||
ConditionsParameters conditionsParameters = new ConditionsParameters();
|
||||
conditionsParameters.setConditionSuccess(true);
|
||||
params.put(Constants.DEPENDENCE, "{\"conditionSuccess\": true}");
|
||||
params.put(Constants.CONDITION_RESULT, "{\n"
|
||||
+
|
||||
" \"successNode\": [5\n"
|
||||
+
|
||||
" ],\n"
|
||||
+
|
||||
" \"failedNode\": [6\n"
|
||||
+
|
||||
" ]\n"
|
||||
+
|
||||
" }");
|
||||
taskInstance3.setTaskParams(JSONUtils.toJsonString(params));
|
||||
conditionsParameters.setConditionResult(conditionResult);
|
||||
taskInstance3.setTaskParams(JSONUtils.toJsonString(conditionsParameters));
|
||||
taskInstance3.setState(TaskExecutionStatus.SUCCESS);
|
||||
TaskNode node3 = dag.getNode(3L);
|
||||
node3.setType(TASK_TYPE_CONDITIONS);
|
||||
node3.setType(ConditionsLogicTaskChannelFactory.NAME);
|
||||
// complete 1/2/3/4 expect:8
|
||||
completeTaskList.put(3L, taskInstance3);
|
||||
postNodes = DagHelper.parsePostNodes(null, skipNodeList, dag, completeTaskList);
|
||||
@ -308,34 +454,117 @@ public class DagHelperTest {
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSwitchPostNode() throws IOException {
|
||||
DAG<Long, TaskNode, TaskNodeRelation> dag = generateDag2();
|
||||
public void testSwitchPostNode() {
|
||||
List<TaskNode> taskNodeList = new ArrayList<>();
|
||||
|
||||
TaskNode node = new TaskNode();
|
||||
node.setId("0");
|
||||
node.setName("0");
|
||||
node.setCode(0);
|
||||
node.setType("SHELL");
|
||||
taskNodeList.add(node);
|
||||
|
||||
TaskNode node1 = new TaskNode();
|
||||
node1.setId("1");
|
||||
node1.setName("1");
|
||||
node1.setCode(1);
|
||||
node1.setType(SwitchLogicTaskChannelFactory.NAME);
|
||||
SwitchParameters switchParameters = new SwitchParameters();
|
||||
node1.setParams(JSONUtils.toJsonString(switchParameters));
|
||||
taskNodeList.add(node1);
|
||||
|
||||
TaskNode node2 = new TaskNode();
|
||||
node2.setId("2");
|
||||
node2.setName("2");
|
||||
node2.setCode(2);
|
||||
node2.setType("SHELL");
|
||||
List<String> dep2 = new ArrayList<>();
|
||||
dep2.add("1");
|
||||
node2.setPreTasks(JSONUtils.toJsonString(dep2));
|
||||
taskNodeList.add(node2);
|
||||
|
||||
TaskNode node4 = new TaskNode();
|
||||
node4.setId("4");
|
||||
node4.setName("4");
|
||||
node4.setCode(4);
|
||||
node4.setType("SHELL");
|
||||
List<String> dep4 = new ArrayList<>();
|
||||
dep4.add("1");
|
||||
node4.setPreTasks(JSONUtils.toJsonString(dep4));
|
||||
taskNodeList.add(node4);
|
||||
|
||||
TaskNode node5 = new TaskNode();
|
||||
node5.setId("5");
|
||||
node5.setName("5");
|
||||
node5.setCode(5);
|
||||
node5.setType("SHELL");
|
||||
List<Long> dep5 = new ArrayList<>();
|
||||
dep5.add(1L);
|
||||
node5.setPreTasks(JSONUtils.toJsonString(dep5));
|
||||
taskNodeList.add(node5);
|
||||
|
||||
TaskNode node6 = new TaskNode();
|
||||
node5.setId("6");
|
||||
node5.setName("6");
|
||||
node5.setCode(6);
|
||||
node5.setType("SHELL");
|
||||
List<Long> dep6 = new ArrayList<>();
|
||||
dep5.add(2L);
|
||||
dep5.add(4L);
|
||||
node5.setPreTasks(JSONUtils.toJsonString(dep6));
|
||||
taskNodeList.add(node6);
|
||||
|
||||
List<Long> startNodes = new ArrayList<>();
|
||||
List<Long> recoveryNodes = new ArrayList<>();
|
||||
|
||||
// 0
|
||||
// 1->2->6
|
||||
// 1->4->6
|
||||
// 1->5
|
||||
List<TaskNode> destTaskNodeList = DagHelper.generateFlowNodeListByStartNode(taskNodeList,
|
||||
startNodes, recoveryNodes, TaskDependType.TASK_POST);
|
||||
List<TaskNodeRelation> taskNodeRelations = DagHelper.generateRelationListByFlowNodes(destTaskNodeList);
|
||||
ProcessDag processDag = new ProcessDag();
|
||||
processDag.setEdges(taskNodeRelations);
|
||||
processDag.setNodes(destTaskNodeList);
|
||||
|
||||
DAG<Long, TaskNode, TaskNodeRelation> dag = DagHelper.buildDagGraph(processDag);
|
||||
Map<Long, TaskNode> skipTaskNodeList = new HashMap<>();
|
||||
Map<Long, TaskInstance> completeTaskList = new HashMap<>();
|
||||
completeTaskList.put(0l, new TaskInstance());
|
||||
completeTaskList.put(0L, new TaskInstance());
|
||||
TaskInstance taskInstance = new TaskInstance();
|
||||
taskInstance.setState(TaskExecutionStatus.SUCCESS);
|
||||
taskInstance.setTaskCode(1l);
|
||||
Map<String, Object> taskParamsMap = new HashMap<>();
|
||||
taskParamsMap.put(Constants.SWITCH_RESULT, "");
|
||||
taskInstance.setTaskParams(JSONUtils.toJsonString(taskParamsMap));
|
||||
taskInstance.setSwitchDependency(getSwitchNode());
|
||||
taskInstance.setTaskCode(1L);
|
||||
taskInstance.setTaskType(SwitchLogicTaskChannelFactory.NAME);
|
||||
switchParameters = SwitchParameters.builder()
|
||||
.nextBranch(5L)
|
||||
.switchResult(SwitchParameters.SwitchResult.builder()
|
||||
.dependTaskList(Lists.newArrayList(
|
||||
new SwitchResultVo("", 2L),
|
||||
new SwitchResultVo("", 4L)))
|
||||
.nextNode(5L)
|
||||
.build())
|
||||
.build();
|
||||
taskInstance.setTaskParams(JSONUtils.toJsonString(switchParameters));
|
||||
completeTaskList.put(1l, taskInstance);
|
||||
DagHelper.skipTaskNode4Switch(dag.getNode(1l), skipTaskNodeList, completeTaskList, dag);
|
||||
List<Long> nextBranch = DagHelper.skipTaskNode4Switch(skipTaskNodeList, taskInstance, dag);
|
||||
Assertions.assertNotNull(skipTaskNodeList.get(2L));
|
||||
Assertions.assertEquals(1, skipTaskNodeList.size());
|
||||
Assertions.assertNotNull(skipTaskNodeList.get(4L));
|
||||
Assertions.assertEquals(2, skipTaskNodeList.size());
|
||||
Truth.assertThat(nextBranch).containsExactly(5L);
|
||||
}
|
||||
|
||||
/**
|
||||
* process:
|
||||
* 1->2->3->5->7
|
||||
* 4->3->6
|
||||
* 1->2->8->5->7
|
||||
* DAG graph:
|
||||
* 4 -> -> 6
|
||||
* \ /
|
||||
* 4 -> -> 6
|
||||
* \ /
|
||||
* 1 -> 2 -> 3 -> 5 -> 7
|
||||
* \ /
|
||||
* -> 8 ->
|
||||
* \ /
|
||||
* -> 8 ->
|
||||
*
|
||||
* @return dag
|
||||
* @throws JsonProcessingException if error throws JsonProcessingException
|
||||
@ -429,108 +658,6 @@ public class DagHelperTest {
|
||||
return DagHelper.buildDagGraph(processDag);
|
||||
}
|
||||
|
||||
/**
|
||||
* DAG graph:
|
||||
* -> 2->
|
||||
* / \
|
||||
* / \
|
||||
* 0->1(switch)->5 6
|
||||
* \ /
|
||||
* \ /
|
||||
* -> 4->
|
||||
*
|
||||
* @return dag
|
||||
* @throws JsonProcessingException if error throws JsonProcessingException
|
||||
*/
|
||||
private DAG<Long, TaskNode, TaskNodeRelation> generateDag2() throws IOException {
|
||||
List<TaskNode> taskNodeList = new ArrayList<>();
|
||||
|
||||
TaskNode node = new TaskNode();
|
||||
node.setId("0");
|
||||
node.setName("0");
|
||||
node.setCode(0);
|
||||
node.setType("SHELL");
|
||||
taskNodeList.add(node);
|
||||
|
||||
TaskNode node1 = new TaskNode();
|
||||
node1.setId("1");
|
||||
node1.setName("1");
|
||||
node1.setCode(1);
|
||||
node1.setType("switch");
|
||||
node1.setDependence(JSONUtils.toJsonString(getSwitchNode()));
|
||||
taskNodeList.add(node1);
|
||||
|
||||
TaskNode node2 = new TaskNode();
|
||||
node2.setId("2");
|
||||
node2.setName("2");
|
||||
node2.setCode(2);
|
||||
node2.setType("SHELL");
|
||||
List<String> dep2 = new ArrayList<>();
|
||||
dep2.add("1");
|
||||
node2.setPreTasks(JSONUtils.toJsonString(dep2));
|
||||
taskNodeList.add(node2);
|
||||
|
||||
TaskNode node4 = new TaskNode();
|
||||
node4.setId("4");
|
||||
node4.setName("4");
|
||||
node4.setCode(4);
|
||||
node4.setType("SHELL");
|
||||
List<String> dep4 = new ArrayList<>();
|
||||
dep4.add("1");
|
||||
node4.setPreTasks(JSONUtils.toJsonString(dep4));
|
||||
taskNodeList.add(node4);
|
||||
|
||||
TaskNode node5 = new TaskNode();
|
||||
node5.setId("5");
|
||||
node5.setName("5");
|
||||
node5.setCode(5);
|
||||
node5.setType("SHELL");
|
||||
List<Long> dep5 = new ArrayList<>();
|
||||
dep5.add(1L);
|
||||
node5.setPreTasks(JSONUtils.toJsonString(dep5));
|
||||
taskNodeList.add(node5);
|
||||
|
||||
TaskNode node6 = new TaskNode();
|
||||
node5.setId("6");
|
||||
node5.setName("6");
|
||||
node5.setCode(6);
|
||||
node5.setType("SHELL");
|
||||
List<Long> dep6 = new ArrayList<>();
|
||||
dep5.add(2L);
|
||||
dep5.add(4L);
|
||||
node5.setPreTasks(JSONUtils.toJsonString(dep6));
|
||||
taskNodeList.add(node6);
|
||||
|
||||
List<Long> startNodes = new ArrayList<>();
|
||||
List<Long> recoveryNodes = new ArrayList<>();
|
||||
List<TaskNode> destTaskNodeList = DagHelper.generateFlowNodeListByStartNode(taskNodeList,
|
||||
startNodes, recoveryNodes, TaskDependType.TASK_POST);
|
||||
List<TaskNodeRelation> taskNodeRelations = DagHelper.generateRelationListByFlowNodes(destTaskNodeList);
|
||||
ProcessDag processDag = new ProcessDag();
|
||||
processDag.setEdges(taskNodeRelations);
|
||||
processDag.setNodes(destTaskNodeList);
|
||||
return DagHelper.buildDagGraph(processDag);
|
||||
}
|
||||
|
||||
private SwitchParameters getSwitchNode() {
|
||||
SwitchParameters conditionsParameters = new SwitchParameters();
|
||||
SwitchResultVo switchResultVo1 = new SwitchResultVo();
|
||||
switchResultVo1.setCondition(" 2 == 1");
|
||||
switchResultVo1.setNextNode(2L);
|
||||
SwitchResultVo switchResultVo2 = new SwitchResultVo();
|
||||
switchResultVo2.setCondition(" 2 == 2");
|
||||
switchResultVo2.setNextNode(4L);
|
||||
List<SwitchResultVo> list = new ArrayList<>();
|
||||
list.add(switchResultVo1);
|
||||
list.add(switchResultVo2);
|
||||
conditionsParameters.setDependTaskList(list);
|
||||
conditionsParameters.setNextNode(5L);
|
||||
conditionsParameters.setRelation("AND");
|
||||
conditionsParameters.setResultConditionLocation(1);
|
||||
// in: AND(AND(1 is SUCCESS))
|
||||
return conditionsParameters;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBuildDagGraph() {
|
||||
String shellJson =
|
||||
|
@ -78,8 +78,11 @@ public class HdfsStorageOperator extends AbstractStorageOperator implements Clos
|
||||
private void initHdfsPath() {
|
||||
Path path = new Path(resourceBaseAbsolutePath);
|
||||
if (!fs.exists(path)) {
|
||||
fs.mkdirs(path);
|
||||
log.info("Create hdfs path: {}", path);
|
||||
if (!fs.mkdirs(path)) {
|
||||
log.info("Create hdfs path: {} failed", path);
|
||||
} else {
|
||||
log.error("Create hdfs path: {} success", path);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -297,5 +297,10 @@
|
||||
<groupId>org.projectlombok</groupId>
|
||||
<artifactId>lombok</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.junit.jupiter</groupId>
|
||||
<artifactId>junit-jupiter</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
|
@ -15,21 +15,10 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.dolphinscheduler.common.enums;
|
||||
|
||||
public enum BlockingOpportunity {
|
||||
|
||||
BLOCKING_ON_SUCCESS("BlockingOnSuccess"),
|
||||
BLOCKING_ON_FAILED("BlockingOnFailed");
|
||||
|
||||
private final String desc;
|
||||
|
||||
BlockingOpportunity(String desc) {
|
||||
this.desc = desc;
|
||||
}
|
||||
|
||||
public String getDesc() {
|
||||
return desc;
|
||||
}
|
||||
package org.apache.dolphinscheduler.plugin.task.api;
|
||||
|
||||
/**
|
||||
* Used to mark a task channel as a logic task channel, the logic task channel is a special task channel that will be executed at master.
|
||||
*/
|
||||
public interface ILogicTaskChannel extends TaskChannel {
|
||||
}
|
@ -18,19 +18,11 @@
|
||||
package org.apache.dolphinscheduler.plugin.task.api;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
|
||||
|
||||
public interface TaskChannel {
|
||||
|
||||
void cancelApplication(boolean status);
|
||||
|
||||
// todo: return ITask
|
||||
AbstractTask createTask(TaskExecutionContext taskRequest);
|
||||
|
||||
// todo: return IParameters
|
||||
AbstractParameters parseParameters(ParametersNode parametersNode);
|
||||
|
||||
ResourceParametersHelper getResources(String parameters);
|
||||
AbstractParameters parseParameters(String taskParams);
|
||||
|
||||
}
|
||||
|
@ -17,16 +17,19 @@
|
||||
|
||||
package org.apache.dolphinscheduler.plugin.task.api;
|
||||
|
||||
import org.apache.dolphinscheduler.spi.common.UiChannelFactory;
|
||||
import org.apache.dolphinscheduler.spi.plugin.PrioritySPI;
|
||||
import org.apache.dolphinscheduler.spi.plugin.SPIIdentify;
|
||||
|
||||
public interface TaskChannelFactory extends UiChannelFactory, PrioritySPI {
|
||||
public interface TaskChannelFactory extends PrioritySPI {
|
||||
|
||||
default SPIIdentify getIdentify() {
|
||||
return SPIIdentify.builder()
|
||||
.name(getName())
|
||||
.build();
|
||||
}
|
||||
|
||||
String getName();
|
||||
|
||||
TaskChannel create();
|
||||
|
||||
default SPIIdentify getIdentify() {
|
||||
return SPIIdentify.builder().name(getName()).build();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -362,24 +362,10 @@ public class TaskConstants {
|
||||
*/
|
||||
public static final String DATA_QUALITY_JAR_DIR = "data-quality.jar.dir";
|
||||
|
||||
public static final String TASK_TYPE_CONDITIONS = "CONDITIONS";
|
||||
|
||||
public static final String TASK_TYPE_SWITCH = "SWITCH";
|
||||
|
||||
public static final String TASK_TYPE_SUB_PROCESS = "SUB_PROCESS";
|
||||
|
||||
public static final String TASK_TYPE_DYNAMIC = "DYNAMIC";
|
||||
|
||||
public static final String TASK_TYPE_DEPENDENT = "DEPENDENT";
|
||||
|
||||
public static final String TASK_TYPE_SQL = "SQL";
|
||||
|
||||
public static final String TASK_TYPE_DATA_QUALITY = "DATA_QUALITY";
|
||||
|
||||
public static final Set<String> TASK_TYPE_SET_K8S = Sets.newHashSet("K8S", "KUBEFLOW");
|
||||
|
||||
public static final String TASK_TYPE_BLOCKING = "BLOCKING";
|
||||
|
||||
/**
|
||||
* azure config
|
||||
*/
|
||||
@ -389,13 +375,6 @@ public class TaskConstants {
|
||||
public static final String AZURE_SECRET_TENANT_ID = "resource.azure.tenant.id";
|
||||
public static final String QUERY_INTERVAL = "resource.query.interval";
|
||||
|
||||
/**
|
||||
* aws config
|
||||
*/
|
||||
public static final String AWS_ACCESS_KEY_ID = "resource.aws.access.key.id";
|
||||
public static final String AWS_SECRET_ACCESS_KEY = "resource.aws.secret.access.key";
|
||||
public static final String AWS_REGION = "resource.aws.region";
|
||||
|
||||
/**
|
||||
* alibaba cloud config
|
||||
*/
|
||||
|
@ -17,21 +17,13 @@
|
||||
|
||||
package org.apache.dolphinscheduler.plugin.task.api;
|
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import static com.google.common.base.Preconditions.checkNotNull;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.BlockingParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ConditionsParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.DynamicParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.SubProcessParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.SwitchParameters;
|
||||
import org.apache.dolphinscheduler.spi.plugin.PrioritySPIFactory;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import lombok.extern.slf4j.Slf4j;
|
||||
@ -39,15 +31,15 @@ import lombok.extern.slf4j.Slf4j;
|
||||
@Slf4j
|
||||
public class TaskPluginManager {
|
||||
|
||||
private static final Map<String, TaskChannelFactory> taskChannelFactoryMap = new HashMap<>();
|
||||
private static final Map<String, TaskChannel> taskChannelMap = new HashMap<>();
|
||||
|
||||
private static final AtomicBoolean loadedFlag = new AtomicBoolean(false);
|
||||
|
||||
/**
|
||||
* Load task plugins from classpath.
|
||||
*/
|
||||
public static void loadPlugin() {
|
||||
static {
|
||||
loadTaskPlugin();
|
||||
}
|
||||
|
||||
public static void loadTaskPlugin() {
|
||||
if (!loadedFlag.compareAndSet(false, true)) {
|
||||
log.warn("The task plugin has already been loaded");
|
||||
return;
|
||||
@ -59,7 +51,6 @@ public class TaskPluginManager {
|
||||
|
||||
log.info("Registering task plugin: {} - {}", factoryName, factory.getClass().getSimpleName());
|
||||
|
||||
taskChannelFactoryMap.put(factoryName, factory);
|
||||
taskChannelMap.put(factoryName, factory.create());
|
||||
|
||||
log.info("Registered task plugin: {} - {}", factoryName, factory.getClass().getSimpleName());
|
||||
@ -67,48 +58,49 @@ public class TaskPluginManager {
|
||||
|
||||
}
|
||||
|
||||
public static Map<String, TaskChannel> getTaskChannelMap() {
|
||||
return Collections.unmodifiableMap(taskChannelMap);
|
||||
}
|
||||
|
||||
public static Map<String, TaskChannelFactory> getTaskChannelFactoryMap() {
|
||||
return Collections.unmodifiableMap(taskChannelFactoryMap);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the TaskChannel by type, if the TaskChannel is not found, will throw
|
||||
* @param type task type, cannot be null
|
||||
* @throws IllegalArgumentException if the TaskChannel is not found
|
||||
*/
|
||||
public static TaskChannel getTaskChannel(String type) {
|
||||
return getTaskChannelMap().get(type);
|
||||
checkNotNull(type, "type cannot be null");
|
||||
TaskChannel taskChannel = taskChannelMap.get(type);
|
||||
if (taskChannel == null) {
|
||||
throw new IllegalArgumentException("Cannot find TaskChannel for : " + type);
|
||||
}
|
||||
return taskChannel;
|
||||
}
|
||||
|
||||
public static boolean checkTaskParameters(ParametersNode parametersNode) {
|
||||
AbstractParameters abstractParameters = getParameters(parametersNode);
|
||||
return abstractParameters != null && abstractParameters.checkParameters();
|
||||
/**
|
||||
* Check if the task parameters is validated
|
||||
* @param taskType task type, cannot be null
|
||||
* @param taskParams task parameters
|
||||
* @return true if the task parameters is validated, otherwise false
|
||||
* @throws IllegalArgumentException if the TaskChannel is not found
|
||||
* @throws IllegalArgumentException if cannot deserialize the task parameters
|
||||
*/
|
||||
public static boolean checkTaskParameters(String taskType, String taskParams) {
|
||||
AbstractParameters abstractParameters = parseTaskParameters(taskType, taskParams);
|
||||
return abstractParameters.checkParameters();
|
||||
}
|
||||
|
||||
public static AbstractParameters getParameters(ParametersNode parametersNode) {
|
||||
String taskType = parametersNode.getTaskType();
|
||||
if (Objects.isNull(taskType)) {
|
||||
return null;
|
||||
}
|
||||
switch (taskType) {
|
||||
case TaskConstants.TASK_TYPE_CONDITIONS:
|
||||
return JSONUtils.parseObject(parametersNode.getTaskParams(), ConditionsParameters.class);
|
||||
case TaskConstants.TASK_TYPE_SWITCH:
|
||||
return JSONUtils.parseObject(parametersNode.getTaskParams(), SwitchParameters.class);
|
||||
case TaskConstants.TASK_TYPE_SUB_PROCESS:
|
||||
return JSONUtils.parseObject(parametersNode.getTaskParams(), SubProcessParameters.class);
|
||||
case TaskConstants.TASK_TYPE_DEPENDENT:
|
||||
return JSONUtils.parseObject(parametersNode.getTaskParams(), DependentParameters.class);
|
||||
case TaskConstants.TASK_TYPE_BLOCKING:
|
||||
return JSONUtils.parseObject(parametersNode.getTaskParams(), BlockingParameters.class);
|
||||
case TaskConstants.TASK_TYPE_DYNAMIC:
|
||||
return JSONUtils.parseObject(parametersNode.getTaskParams(), DynamicParameters.class);
|
||||
default:
|
||||
TaskChannel taskChannel = getTaskChannelMap().get(taskType);
|
||||
if (Objects.isNull(taskChannel)) {
|
||||
return null;
|
||||
}
|
||||
return taskChannel.parseParameters(parametersNode);
|
||||
/**
|
||||
* Parse the task parameters
|
||||
* @param taskType task type, cannot be null
|
||||
* @param taskParams task parameters
|
||||
* @return AbstractParameters
|
||||
* @throws IllegalArgumentException if the TaskChannel is not found
|
||||
* @throws IllegalArgumentException if cannot deserialize the task parameters
|
||||
*/
|
||||
public static AbstractParameters parseTaskParameters(String taskType, String taskParams) {
|
||||
checkNotNull(taskType, "taskType cannot be null");
|
||||
TaskChannel taskChannel = getTaskChannel(taskType);
|
||||
AbstractParameters abstractParameters = taskChannel.parseParameters(taskParams);
|
||||
if (abstractParameters == null) {
|
||||
throw new IllegalArgumentException("Cannot parse task parameters: " + taskParams + " for : " + taskType);
|
||||
}
|
||||
return abstractParameters;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -17,12 +17,16 @@
|
||||
|
||||
package org.apache.dolphinscheduler.plugin.task.api.model;
|
||||
|
||||
import lombok.AllArgsConstructor;
|
||||
import lombok.Builder;
|
||||
import lombok.Data;
|
||||
import lombok.NoArgsConstructor;
|
||||
import lombok.NonNull;
|
||||
|
||||
@Data
|
||||
@Builder
|
||||
@NoArgsConstructor
|
||||
@AllArgsConstructor
|
||||
public class DynamicInputParameter {
|
||||
|
||||
@NonNull
|
||||
|
@ -17,9 +17,6 @@
|
||||
|
||||
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;
|
||||
@ -30,15 +27,5 @@ import lombok.NoArgsConstructor;
|
||||
public class SwitchResultVo {
|
||||
|
||||
private String condition;
|
||||
private List<Long> nextNode;
|
||||
|
||||
public void setNextNode(Object nextNode) {
|
||||
if (nextNode instanceof Long) {
|
||||
List<Long> nextNodeList = new ArrayList<>();
|
||||
nextNodeList.add((Long) nextNode);
|
||||
this.nextNode = nextNodeList;
|
||||
} else {
|
||||
this.nextNode = (ArrayList) nextNode;
|
||||
}
|
||||
}
|
||||
private Long nextNode;
|
||||
}
|
||||
|
@ -1,51 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.dolphinscheduler.plugin.task.api.parameters;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
|
||||
public class BlockingParameters extends AbstractParameters {
|
||||
|
||||
// condition of blocking: BlockingOnFailed or BlockingOnSuccess
|
||||
private String blockingOpportunity;
|
||||
|
||||
// if true, alert when blocking, otherwise do nothing
|
||||
|
||||
private boolean isAlertWhenBlocking;
|
||||
|
||||
@Override
|
||||
public boolean checkParameters() {
|
||||
return !StringUtils.isEmpty(blockingOpportunity);
|
||||
}
|
||||
|
||||
public String getBlockingOpportunity() {
|
||||
return blockingOpportunity;
|
||||
}
|
||||
|
||||
public void setBlockingCondition(String blockingOpportunity) {
|
||||
this.blockingOpportunity = blockingOpportunity;
|
||||
}
|
||||
|
||||
public boolean isAlertWhenBlocking() {
|
||||
return isAlertWhenBlocking;
|
||||
}
|
||||
|
||||
public void setAlertWhenBlocking(boolean alertWhenBlocking) {
|
||||
isAlertWhenBlocking = alertWhenBlocking;
|
||||
}
|
||||
}
|
@ -19,36 +19,59 @@ package org.apache.dolphinscheduler.plugin.task.api.parameters;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.DependentRelation;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.model.DependentTaskModel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import org.apache.commons.collections4.CollectionUtils;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import lombok.AllArgsConstructor;
|
||||
import lombok.Builder;
|
||||
import lombok.Data;
|
||||
import lombok.NoArgsConstructor;
|
||||
|
||||
@Data
|
||||
@Builder
|
||||
@NoArgsConstructor
|
||||
@AllArgsConstructor
|
||||
public class ConditionsParameters extends AbstractParameters {
|
||||
|
||||
// depend node list and state, only need task name
|
||||
private List<DependentTaskModel> dependTaskList;
|
||||
private DependentRelation relation;
|
||||
private ConditionDependency dependence;
|
||||
|
||||
private boolean conditionSuccess;
|
||||
private ConditionResult conditionResult;
|
||||
|
||||
@Override
|
||||
public boolean checkParameters() {
|
||||
if (dependence == null || CollectionUtils.isEmpty(dependence.getDependTaskList())) {
|
||||
return false;
|
||||
}
|
||||
if (conditionResult == null || CollectionUtils.isEmpty(conditionResult.getSuccessNode())
|
||||
|| CollectionUtils.isEmpty(conditionResult.getFailedNode())) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ResourceInfo> getResourceFilesList() {
|
||||
return new ArrayList<>();
|
||||
@Data
|
||||
@Builder
|
||||
@NoArgsConstructor
|
||||
@AllArgsConstructor
|
||||
public static class ConditionDependency {
|
||||
|
||||
private List<DependentTaskModel> dependTaskList;
|
||||
|
||||
private DependentRelation relation;
|
||||
}
|
||||
|
||||
@Data
|
||||
@Builder
|
||||
@NoArgsConstructor
|
||||
@AllArgsConstructor
|
||||
public static class ConditionResult {
|
||||
|
||||
private boolean conditionSuccess;
|
||||
|
||||
private List<Long> successNode;
|
||||
|
||||
private List<Long> failedNode;
|
||||
}
|
||||
|
||||
|
@ -20,28 +20,55 @@ package org.apache.dolphinscheduler.plugin.task.api.parameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.DependentRelation;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.model.DependentTaskModel;
|
||||
|
||||
import org.apache.commons.collections4.CollectionUtils;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import lombok.AllArgsConstructor;
|
||||
import lombok.Builder;
|
||||
import lombok.Data;
|
||||
import lombok.EqualsAndHashCode;
|
||||
import lombok.NoArgsConstructor;
|
||||
|
||||
@Data
|
||||
@EqualsAndHashCode(callSuper = true)
|
||||
public class DependentParameters extends AbstractParameters {
|
||||
|
||||
private List<DependentTaskModel> dependTaskList;
|
||||
private DependentRelation relation;
|
||||
/** Time unit is second */
|
||||
private Integer checkInterval;
|
||||
private DependentFailurePolicyEnum failurePolicy;
|
||||
/** Time unit is minutes */
|
||||
private Integer failureWaitingTime;
|
||||
private Dependence dependence;
|
||||
|
||||
@Override
|
||||
public boolean checkParameters() {
|
||||
if (dependence == null) {
|
||||
return false;
|
||||
}
|
||||
if (CollectionUtils.isEmpty(dependence.getDependTaskList())) {
|
||||
return false;
|
||||
}
|
||||
if (dependence.getCheckInterval() != null && dependence.getCheckInterval() <= 0) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Data
|
||||
@Builder
|
||||
@NoArgsConstructor
|
||||
@AllArgsConstructor
|
||||
public static class Dependence {
|
||||
|
||||
private List<DependentTaskModel> dependTaskList;
|
||||
private DependentRelation relation;
|
||||
/**
|
||||
* Time unit is second
|
||||
*/
|
||||
private Integer checkInterval;
|
||||
private DependentFailurePolicyEnum failurePolicy;
|
||||
/**
|
||||
* Time unit is minutes
|
||||
*/
|
||||
private Integer failureWaitingTime;
|
||||
}
|
||||
|
||||
/**
|
||||
* the dependent task failure policy.
|
||||
*/
|
||||
|
@ -21,9 +21,15 @@ import org.apache.dolphinscheduler.plugin.task.api.model.DynamicInputParameter;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import lombok.AllArgsConstructor;
|
||||
import lombok.Builder;
|
||||
import lombok.Data;
|
||||
import lombok.NoArgsConstructor;
|
||||
|
||||
@Data
|
||||
@Builder
|
||||
@NoArgsConstructor
|
||||
@AllArgsConstructor
|
||||
public class DynamicParameters extends AbstractParameters {
|
||||
|
||||
/**
|
||||
|
@ -1,116 +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.plugin.task.api.parameters;
|
||||
|
||||
/**
|
||||
* TODO <p>Need to optimize, why there are multiple task parameter variables:taskParams,dependence,switchResult</p>
|
||||
*
|
||||
*/
|
||||
public class ParametersNode {
|
||||
|
||||
private String taskType;
|
||||
|
||||
private String taskParams;
|
||||
|
||||
private String dependence;
|
||||
|
||||
private String switchResult;
|
||||
|
||||
public static ParametersNode.ParametersNodeBuilder builder() {
|
||||
return new ParametersNode.ParametersNodeBuilder();
|
||||
}
|
||||
|
||||
public static class ParametersNodeBuilder {
|
||||
|
||||
private String taskType;
|
||||
|
||||
private String taskParams;
|
||||
|
||||
private String dependence;
|
||||
|
||||
private String switchResult;
|
||||
|
||||
public ParametersNodeBuilder taskType(String taskType) {
|
||||
this.taskType = taskType;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ParametersNodeBuilder taskParams(String taskParams) {
|
||||
this.taskParams = taskParams;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ParametersNodeBuilder dependence(String dependence) {
|
||||
this.dependence = dependence;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ParametersNodeBuilder switchResult(String switchResult) {
|
||||
this.switchResult = switchResult;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ParametersNode build() {
|
||||
return new ParametersNode(this.taskType, this.taskParams, this.dependence, this.switchResult);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public ParametersNode() {
|
||||
|
||||
}
|
||||
|
||||
public ParametersNode(String taskType, String taskParams, String dependence, String switchResult) {
|
||||
this.taskType = taskType;
|
||||
this.taskParams = taskParams;
|
||||
this.dependence = dependence;
|
||||
this.switchResult = switchResult;
|
||||
}
|
||||
|
||||
public String getTaskType() {
|
||||
return taskType;
|
||||
}
|
||||
|
||||
public void setTaskType(String taskType) {
|
||||
this.taskType = taskType;
|
||||
}
|
||||
|
||||
public String getTaskParams() {
|
||||
return taskParams;
|
||||
}
|
||||
|
||||
public void setTaskParams(String taskParams) {
|
||||
this.taskParams = taskParams;
|
||||
}
|
||||
|
||||
public String getDependence() {
|
||||
return dependence;
|
||||
}
|
||||
|
||||
public void setDependence(String dependence) {
|
||||
this.dependence = dependence;
|
||||
}
|
||||
|
||||
public String getSwitchResult() {
|
||||
return switchResult;
|
||||
}
|
||||
|
||||
public void setSwitchResult(String switchResult) {
|
||||
this.switchResult = switchResult;
|
||||
}
|
||||
}
|
@ -19,9 +19,6 @@ package org.apache.dolphinscheduler.plugin.task.api.parameters;
|
||||
|
||||
public class SubProcessParameters extends AbstractParameters {
|
||||
|
||||
/**
|
||||
* process definition id
|
||||
*/
|
||||
private long processDefinitionCode;
|
||||
|
||||
public void setProcessDefinitionCode(long processDefinitionCode) {
|
||||
@ -34,7 +31,7 @@ public class SubProcessParameters extends AbstractParameters {
|
||||
|
||||
@Override
|
||||
public boolean checkParameters() {
|
||||
return this.processDefinitionCode != 0;
|
||||
return this.processDefinitionCode > 0;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -17,69 +17,56 @@
|
||||
|
||||
package org.apache.dolphinscheduler.plugin.task.api.parameters;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.DependentRelation;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.model.SwitchResultVo;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import org.apache.commons.collections4.CollectionUtils;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import lombok.AllArgsConstructor;
|
||||
import lombok.Builder;
|
||||
import lombok.Data;
|
||||
import lombok.NoArgsConstructor;
|
||||
|
||||
@Data
|
||||
@Builder
|
||||
@NoArgsConstructor
|
||||
@AllArgsConstructor
|
||||
public class SwitchParameters extends AbstractParameters {
|
||||
|
||||
private DependentRelation dependRelation;
|
||||
private String relation;
|
||||
private List<Long> nextNode;
|
||||
// due to history reasons, the field name is switchResult
|
||||
private SwitchResult switchResult;
|
||||
|
||||
// The next branch which should be executed after the switch logic task executed.
|
||||
private Long nextBranch;
|
||||
|
||||
@Override
|
||||
public boolean checkParameters() {
|
||||
if (switchResult == null) {
|
||||
return false;
|
||||
}
|
||||
if (CollectionUtils.isEmpty(switchResult.getDependTaskList()) && switchResult.getNextNode() == null) {
|
||||
return false;
|
||||
}
|
||||
for (SwitchResultVo switchResultVo : switchResult.getDependTaskList()) {
|
||||
if (switchResultVo == null || switchResultVo.getNextNode() == null) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private int resultConditionLocation;
|
||||
private List<SwitchResultVo> dependTaskList;
|
||||
@Data
|
||||
@Builder
|
||||
@AllArgsConstructor
|
||||
@NoArgsConstructor
|
||||
public static class SwitchResult {
|
||||
|
||||
public DependentRelation getDependRelation() {
|
||||
return dependRelation;
|
||||
// switch condition
|
||||
private List<SwitchResultVo> dependTaskList;
|
||||
|
||||
// default branch node code in switch task
|
||||
private Long nextNode;
|
||||
}
|
||||
|
||||
public void setDependRelation(DependentRelation dependRelation) {
|
||||
this.dependRelation = dependRelation;
|
||||
}
|
||||
|
||||
public int getResultConditionLocation() {
|
||||
return resultConditionLocation;
|
||||
}
|
||||
|
||||
public void setResultConditionLocation(int resultConditionLocation) {
|
||||
this.resultConditionLocation = resultConditionLocation;
|
||||
}
|
||||
|
||||
public String getRelation() {
|
||||
return relation;
|
||||
}
|
||||
|
||||
public void setRelation(String relation) {
|
||||
this.relation = relation;
|
||||
}
|
||||
|
||||
public List<SwitchResultVo> getDependTaskList() {
|
||||
return dependTaskList;
|
||||
}
|
||||
|
||||
public void setDependTaskList(List<SwitchResultVo> dependTaskList) {
|
||||
this.dependTaskList = dependTaskList;
|
||||
}
|
||||
|
||||
public List<Long> getNextNode() {
|
||||
return nextNode;
|
||||
}
|
||||
|
||||
public void setNextNode(Object nextNode) {
|
||||
if (nextNode instanceof Long) {
|
||||
List<Long> nextNodeList = new ArrayList<>();
|
||||
nextNodeList.add((Long) nextNode);
|
||||
this.nextNode = nextNodeList;
|
||||
} else {
|
||||
this.nextNode = (ArrayList) nextNode;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -17,13 +17,8 @@
|
||||
|
||||
package org.apache.dolphinscheduler.plugin.task.api.stream;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
|
||||
|
||||
public interface StreamTaskChannel extends TaskChannel {
|
||||
|
||||
AbstractTask pauseTask(TaskExecutionContext taskExecutionContext);
|
||||
|
||||
AbstractTask recoverTask(TaskExecutionContext taskExecutionContext);
|
||||
}
|
||||
|
@ -15,47 +15,17 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.dolphinscheduler.dao.entity;
|
||||
package org.apache.dolphinscheduler.plugin.task.api.task;
|
||||
|
||||
import java.util.Date;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.ILogicTaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
|
||||
|
||||
import lombok.Data;
|
||||
public abstract class AbstractLogicTaskChannel implements ILogicTaskChannel {
|
||||
|
||||
@Data
|
||||
public class WorkerServer {
|
||||
@Override
|
||||
public AbstractTask createTask(TaskExecutionContext taskRequest) {
|
||||
throw new UnsupportedOperationException("createTask is not supported");
|
||||
}
|
||||
|
||||
/**
|
||||
* id
|
||||
*/
|
||||
private int id;
|
||||
|
||||
/**
|
||||
* host
|
||||
*/
|
||||
private String host;
|
||||
|
||||
/**
|
||||
* port
|
||||
*/
|
||||
private int port;
|
||||
|
||||
/**
|
||||
* zookeeper directory
|
||||
*/
|
||||
private String zkDirectory;
|
||||
|
||||
/**
|
||||
* resource info
|
||||
*/
|
||||
private String resInfo;
|
||||
|
||||
/**
|
||||
* create time
|
||||
*/
|
||||
private Date createTime;
|
||||
|
||||
/**
|
||||
* last heart beat time
|
||||
*/
|
||||
private Date lastHeartbeatTime;
|
||||
}
|
@ -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.plugin.task.api.task;
|
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ConditionsParameters;
|
||||
|
||||
public class ConditionsLogicTaskChannel extends AbstractLogicTaskChannel {
|
||||
|
||||
@Override
|
||||
public AbstractParameters parseParameters(String taskParams) {
|
||||
return JSONUtils.parseObject(taskParams, ConditionsParameters.class);
|
||||
}
|
||||
|
||||
}
|
@ -15,29 +15,25 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.dolphinscheduler.spi.common;
|
||||
package org.apache.dolphinscheduler.plugin.task.api.task;
|
||||
|
||||
import org.apache.dolphinscheduler.spi.params.base.PluginParams;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
|
||||
|
||||
import java.util.List;
|
||||
import com.google.auto.service.AutoService;
|
||||
|
||||
public interface UiChannelFactory {
|
||||
@AutoService(TaskChannelFactory.class)
|
||||
public class ConditionsLogicTaskChannelFactory implements TaskChannelFactory {
|
||||
|
||||
/**
|
||||
* plugin name
|
||||
* Must be UNIQUE .
|
||||
* This alert plugin name eg: email , message ...
|
||||
* Name can often be displayed on the page ui eg : email , message , MR , spark , hive ...
|
||||
*
|
||||
* @return this alert plugin name
|
||||
*/
|
||||
String getName();
|
||||
public static final String NAME = "CONDITIONS";
|
||||
|
||||
/**
|
||||
* Returns the configurable parameters that this plugin needs to display on the web ui
|
||||
*
|
||||
* @return this alert plugin params
|
||||
*/
|
||||
List<PluginParams> getParams();
|
||||
@Override
|
||||
public String getName() {
|
||||
return NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskChannel create() {
|
||||
return new ConditionsLogicTaskChannel();
|
||||
}
|
||||
}
|
@ -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.plugin.task.api.task;
|
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameters;
|
||||
|
||||
public class DependentLogicTaskChannel extends AbstractLogicTaskChannel {
|
||||
|
||||
@Override
|
||||
public AbstractParameters parseParameters(String taskParams) {
|
||||
return JSONUtils.parseObject(taskParams, DependentParameters.class);
|
||||
}
|
||||
}
|
@ -0,0 +1,38 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.dolphinscheduler.plugin.task.api.task;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
|
||||
|
||||
import com.google.auto.service.AutoService;
|
||||
|
||||
@AutoService(TaskChannelFactory.class)
|
||||
public class DependentLogicTaskChannelFactory implements TaskChannelFactory {
|
||||
|
||||
public static final String NAME = "DEPENDENT";
|
||||
@Override
|
||||
public String getName() {
|
||||
return NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskChannel create() {
|
||||
return new DependentLogicTaskChannel();
|
||||
}
|
||||
}
|
@ -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.plugin.task.api.task;
|
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.DynamicParameters;
|
||||
|
||||
public class DynamicLogicTaskChannel extends AbstractLogicTaskChannel {
|
||||
|
||||
@Override
|
||||
public AbstractParameters parseParameters(String taskParams) {
|
||||
return JSONUtils.parseObject(taskParams, DynamicParameters.class);
|
||||
}
|
||||
}
|
@ -0,0 +1,38 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.dolphinscheduler.plugin.task.api.task;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
|
||||
|
||||
import com.google.auto.service.AutoService;
|
||||
|
||||
@AutoService(TaskChannelFactory.class)
|
||||
public class DynamicLogicTaskChannelFactory implements TaskChannelFactory {
|
||||
|
||||
public static final String NAME = "DYNAMIC";
|
||||
@Override
|
||||
public String getName() {
|
||||
return NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskChannel create() {
|
||||
return new DynamicLogicTaskChannel();
|
||||
}
|
||||
}
|
@ -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.plugin.task.api.task;
|
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.SubProcessParameters;
|
||||
|
||||
public class SubWorkflowLogicTaskChannel extends AbstractLogicTaskChannel {
|
||||
|
||||
@Override
|
||||
public AbstractParameters parseParameters(String taskParams) {
|
||||
return JSONUtils.parseObject(taskParams, SubProcessParameters.class);
|
||||
}
|
||||
}
|
@ -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.plugin.task.api.task;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
|
||||
|
||||
import com.google.auto.service.AutoService;
|
||||
|
||||
@AutoService(TaskChannelFactory.class)
|
||||
public class SubWorkflowLogicTaskChannelFactory implements TaskChannelFactory {
|
||||
|
||||
public static final String NAME = "SUB_PROCESS";
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskChannel create() {
|
||||
return new SubWorkflowLogicTaskChannel();
|
||||
}
|
||||
}
|
@ -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.plugin.task.api.task;
|
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.SwitchParameters;
|
||||
|
||||
public class SwitchLogicTaskChannel extends AbstractLogicTaskChannel {
|
||||
|
||||
@Override
|
||||
public AbstractParameters parseParameters(String taskParams) {
|
||||
return JSONUtils.parseObject(taskParams, SwitchParameters.class);
|
||||
}
|
||||
}
|
@ -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.plugin.task.api.task;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
|
||||
|
||||
import com.google.auto.service.AutoService;
|
||||
|
||||
@AutoService(TaskChannelFactory.class)
|
||||
public class SwitchLogicTaskChannelFactory implements TaskChannelFactory {
|
||||
|
||||
public static final String NAME = "SWITCH";
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskChannel create() {
|
||||
return new SwitchLogicTaskChannel();
|
||||
}
|
||||
}
|
@ -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.plugin.task.api.utils;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.ILogicTaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.ConditionsLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.DynamicLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.SubWorkflowLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.SwitchLogicTaskChannelFactory;
|
||||
|
||||
import lombok.experimental.UtilityClass;
|
||||
|
||||
@UtilityClass
|
||||
public class TaskTypeUtils {
|
||||
|
||||
public boolean isSwitchTask(String taskType) {
|
||||
return SwitchLogicTaskChannelFactory.NAME.equals(taskType);
|
||||
}
|
||||
|
||||
public boolean isConditionTask(String taskType) {
|
||||
return ConditionsLogicTaskChannelFactory.NAME.equals(taskType);
|
||||
}
|
||||
|
||||
public boolean isSubWorkflowTask(String taskType) {
|
||||
return SubWorkflowLogicTaskChannelFactory.NAME.equals(taskType);
|
||||
}
|
||||
|
||||
public boolean isDependentTask(String taskType) {
|
||||
return SubWorkflowLogicTaskChannelFactory.NAME.equals(taskType);
|
||||
}
|
||||
|
||||
public boolean isDynamicTask(String taskType) {
|
||||
return DynamicLogicTaskChannelFactory.NAME.equals(taskType);
|
||||
}
|
||||
|
||||
public boolean isLogicTask(String taskType) {
|
||||
return TaskPluginManager.getTaskChannel(taskType) instanceof ILogicTaskChannel;
|
||||
}
|
||||
|
||||
}
|
@ -1,50 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.dolphinscheduler.plugin.task.api.utils;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
import lombok.experimental.UtilityClass;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
@UtilityClass
|
||||
public class TaskUtils {
|
||||
|
||||
private final String blockingLogicTask = "BLOCKING";
|
||||
private final String conditionLogicTask = "CONDITIONS";
|
||||
|
||||
private final String dependentLogicTask = "DEPENDENT";
|
||||
private final String subWorkflowLogicTask = "SUB_PROCESS";
|
||||
private final String switchLogicTask = "SWITCH";
|
||||
private final String dynamicLogicTask = "DYNAMIC";
|
||||
|
||||
// todo: Add to SPI
|
||||
private final Set<String> MASTER_TASK_TYPES = Sets.newHashSet(
|
||||
blockingLogicTask,
|
||||
conditionLogicTask,
|
||||
dependentLogicTask,
|
||||
subWorkflowLogicTask,
|
||||
switchLogicTask,
|
||||
dynamicLogicTask);
|
||||
|
||||
// todo: add to task plugin spi
|
||||
public boolean isLogicTask(String taskType) {
|
||||
return MASTER_TASK_TYPES.contains(taskType);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,44 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.dolphinscheduler.plugin.task.api;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.ConditionsLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.DependentLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.DynamicLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.SubWorkflowLogicTaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.task.SwitchLogicTaskChannelFactory;
|
||||
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
class TaskPluginManagerTest {
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {
|
||||
ConditionsLogicTaskChannelFactory.NAME,
|
||||
DependentLogicTaskChannelFactory.NAME,
|
||||
DynamicLogicTaskChannelFactory.NAME,
|
||||
SubWorkflowLogicTaskChannelFactory.NAME,
|
||||
SwitchLogicTaskChannelFactory.NAME})
|
||||
void testGetTaskChannel_logicTaskChannel(String type) {
|
||||
assertThat(TaskPluginManager.getTaskChannel(type)).isNotNull();
|
||||
}
|
||||
|
||||
}
|
@ -22,31 +22,17 @@ import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
|
||||
|
||||
/**
|
||||
* chunjun task channel
|
||||
*/
|
||||
public class ChunJunTaskChannel implements TaskChannel {
|
||||
|
||||
@Override
|
||||
public void cancelApplication(boolean status) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractTask createTask(TaskExecutionContext taskExecutionContext) {
|
||||
return new ChunJunTask(taskExecutionContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractParameters parseParameters(ParametersNode parametersNode) {
|
||||
return JSONUtils.parseObject(parametersNode.getTaskParams(), ChunJunParameters.class);
|
||||
public AbstractParameters parseParameters(String taskParams) {
|
||||
return JSONUtils.parseObject(taskParams, ChunJunParameters.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResourceParametersHelper getResources(String parameters) {
|
||||
return JSONUtils.parseObject(parameters, ChunJunParameters.class).getResources();
|
||||
}
|
||||
}
|
||||
|
@ -19,9 +19,6 @@ package org.apache.dolphinscheduler.plugin.task.chunjun;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.spi.params.base.PluginParams;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import com.google.auto.service.AutoService;
|
||||
|
||||
@ -47,13 +44,4 @@ public class ChunJunTaskChannelFactory implements TaskChannelFactory {
|
||||
return "CHUNJUN";
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the configurable parameters that this plugin needs to display on the web ui
|
||||
*
|
||||
* @return this plugin params
|
||||
*/
|
||||
@Override
|
||||
public List<PluginParams> getParams() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
@ -21,27 +21,17 @@ import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
|
||||
|
||||
public class DatafactoryTaskChannel implements TaskChannel {
|
||||
|
||||
@Override
|
||||
public void cancelApplication(boolean status) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public DatafactoryTask createTask(TaskExecutionContext taskRequest) {
|
||||
return new DatafactoryTask(taskRequest);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractParameters parseParameters(ParametersNode parametersNode) {
|
||||
return JSONUtils.parseObject(parametersNode.getTaskParams(), DatafactoryParameters.class);
|
||||
public AbstractParameters parseParameters(String taskParams) {
|
||||
return JSONUtils.parseObject(taskParams, DatafactoryParameters.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResourceParametersHelper getResources(String parameters) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
@ -19,10 +19,6 @@ package org.apache.dolphinscheduler.plugin.task.datafactory;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.spi.params.base.PluginParams;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import com.google.auto.service.AutoService;
|
||||
|
||||
@ -39,8 +35,4 @@ public class DatafactoryTaskChannelFactory implements TaskChannelFactory {
|
||||
return "DATA_FACTORY";
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<PluginParams> getParams() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
|
@ -22,29 +22,18 @@ import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.dataquality.DataQualityParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
|
||||
|
||||
public class DataQualityTaskChannel implements TaskChannel {
|
||||
|
||||
@Override
|
||||
public void cancelApplication(boolean status) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractTask createTask(TaskExecutionContext taskRequest) {
|
||||
return new DataQualityTask(taskRequest);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractParameters parseParameters(ParametersNode parametersNode) {
|
||||
return JSONUtils.parseObject(parametersNode.getTaskParams(), DataQualityParameters.class);
|
||||
public AbstractParameters parseParameters(String taskParams) {
|
||||
return JSONUtils.parseObject(taskParams, DataQualityParameters.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResourceParametersHelper getResources(String parameters) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
@ -19,9 +19,6 @@ package org.apache.dolphinscheduler.plugin.task.dq;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.spi.params.base.PluginParams;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import com.google.auto.service.AutoService;
|
||||
|
||||
@ -33,11 +30,6 @@ public class DataQualityTaskChannelFactory implements TaskChannelFactory {
|
||||
return "DATA_QUALITY";
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<PluginParams> getParams() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskChannel create() {
|
||||
return new DataQualityTaskChannel();
|
||||
|
@ -21,28 +21,17 @@ import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
|
||||
|
||||
public class DatasyncTaskChannel implements TaskChannel {
|
||||
|
||||
@Override
|
||||
public void cancelApplication(boolean status) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public DatasyncTask createTask(TaskExecutionContext taskRequest) {
|
||||
return new DatasyncTask(taskRequest);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractParameters parseParameters(ParametersNode parametersNode) {
|
||||
return JSONUtils.parseObject(parametersNode.getTaskParams(), DatasyncParameters.class);
|
||||
public AbstractParameters parseParameters(String taskParams) {
|
||||
return JSONUtils.parseObject(taskParams, DatasyncParameters.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResourceParametersHelper getResources(String parameters) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
@ -19,10 +19,6 @@ package org.apache.dolphinscheduler.plugin.task.datasync;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.spi.params.base.PluginParams;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import com.google.auto.service.AutoService;
|
||||
|
||||
@ -39,8 +35,4 @@ public class DatasyncTaskChannelFactory implements TaskChannelFactory {
|
||||
return "DATASYNC";
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<PluginParams> getParams() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
|
@ -22,28 +22,17 @@ import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
|
||||
|
||||
public class DataxTaskChannel implements TaskChannel {
|
||||
|
||||
@Override
|
||||
public void cancelApplication(boolean status) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractTask createTask(TaskExecutionContext taskRequest) {
|
||||
return new DataxTask(taskRequest);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractParameters parseParameters(ParametersNode parametersNode) {
|
||||
return JSONUtils.parseObject(parametersNode.getTaskParams(), DataxParameters.class);
|
||||
public AbstractParameters parseParameters(String taskParams) {
|
||||
return JSONUtils.parseObject(taskParams, DataxParameters.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResourceParametersHelper getResources(String parameters) {
|
||||
return JSONUtils.parseObject(parameters, DataxParameters.class).getResources();
|
||||
}
|
||||
}
|
||||
|
@ -19,9 +19,6 @@ package org.apache.dolphinscheduler.plugin.task.datax;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.spi.params.base.PluginParams;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import com.google.auto.service.AutoService;
|
||||
|
||||
@ -33,11 +30,6 @@ public class DataxTaskChannelFactory implements TaskChannelFactory {
|
||||
return "DATAX";
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<PluginParams> getParams() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskChannel create() {
|
||||
return new DataxTaskChannel();
|
||||
|
@ -22,28 +22,17 @@ import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
|
||||
|
||||
public class DinkyTaskChannel implements TaskChannel {
|
||||
|
||||
@Override
|
||||
public void cancelApplication(boolean status) {
|
||||
// nothing to do
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractTask createTask(TaskExecutionContext taskRequest) {
|
||||
return new DinkyTask(taskRequest);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractParameters parseParameters(ParametersNode parametersNode) {
|
||||
return JSONUtils.parseObject(parametersNode.getTaskParams(), DinkyParameters.class);
|
||||
public AbstractParameters parseParameters(String taskParams) {
|
||||
return JSONUtils.parseObject(taskParams, DinkyParameters.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResourceParametersHelper getResources(String parameters) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
@ -19,10 +19,6 @@ package org.apache.dolphinscheduler.plugin.task.dinky;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.spi.params.base.PluginParams;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import com.google.auto.service.AutoService;
|
||||
|
||||
@ -34,11 +30,6 @@ public class DinkyTaskChannelFactory implements TaskChannelFactory {
|
||||
return "DINKY";
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<PluginParams> getParams() {
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskChannel create() {
|
||||
return new DinkyTaskChannel();
|
||||
|
@ -21,29 +21,17 @@ import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
|
||||
|
||||
public class DmsTaskChannel implements TaskChannel {
|
||||
|
||||
@Override
|
||||
public void cancelApplication(boolean status) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public DmsTask createTask(TaskExecutionContext taskRequest) {
|
||||
return new DmsTask(taskRequest);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractParameters parseParameters(ParametersNode parametersNode) {
|
||||
return JSONUtils.parseObject(parametersNode.getTaskParams(), DmsParameters.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResourceParametersHelper getResources(String parameters) {
|
||||
return null;
|
||||
public AbstractParameters parseParameters(String taskParams) {
|
||||
return JSONUtils.parseObject(taskParams, DmsParameters.class);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,10 +19,6 @@ package org.apache.dolphinscheduler.plugin.task.dms;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.spi.params.base.PluginParams;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import com.google.auto.service.AutoService;
|
||||
|
||||
@ -39,8 +35,4 @@ public class DmsTaskChannelFactory implements TaskChannelFactory {
|
||||
return "DMS";
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<PluginParams> getParams() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
|
@ -21,29 +21,17 @@ import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
|
||||
|
||||
public class DvcTaskChannel implements TaskChannel {
|
||||
|
||||
@Override
|
||||
public void cancelApplication(boolean status) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public DvcTask createTask(TaskExecutionContext taskRequest) {
|
||||
return new DvcTask(taskRequest);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractParameters parseParameters(ParametersNode parametersNode) {
|
||||
return JSONUtils.parseObject(parametersNode.getTaskParams(), DvcParameters.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResourceParametersHelper getResources(String parameters) {
|
||||
return null;
|
||||
public AbstractParameters parseParameters(String taskParams) {
|
||||
return JSONUtils.parseObject(taskParams, DvcParameters.class);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,14 +19,6 @@ package org.apache.dolphinscheduler.plugin.task.dvc;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.spi.params.base.ParamsOptions;
|
||||
import org.apache.dolphinscheduler.spi.params.base.PluginParams;
|
||||
import org.apache.dolphinscheduler.spi.params.base.Validate;
|
||||
import org.apache.dolphinscheduler.spi.params.input.InputParam;
|
||||
import org.apache.dolphinscheduler.spi.params.radio.RadioParam;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import com.google.auto.service.AutoService;
|
||||
|
||||
@ -43,19 +35,4 @@ public class DvcTaskChannelFactory implements TaskChannelFactory {
|
||||
return "DVC";
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<PluginParams> getParams() {
|
||||
List<PluginParams> paramsList = new ArrayList<>();
|
||||
|
||||
InputParam nodeName = InputParam.newBuilder("name", "$t('Node name')")
|
||||
.addValidate(Validate.newBuilder().setRequired(true).build()).build();
|
||||
|
||||
RadioParam runFlag = RadioParam.newBuilder("runFlag", "RUN_FLAG")
|
||||
.addParamsOptions(new ParamsOptions("NORMAL", "NORMAL", false))
|
||||
.addParamsOptions(new ParamsOptions("FORBIDDEN", "FORBIDDEN", false)).build();
|
||||
|
||||
paramsList.add(nodeName);
|
||||
paramsList.add(runFlag);
|
||||
return paramsList;
|
||||
}
|
||||
}
|
||||
|
@ -22,16 +22,9 @@ import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
|
||||
|
||||
public class EmrTaskChannel implements TaskChannel {
|
||||
|
||||
@Override
|
||||
public void cancelApplication(boolean status) {
|
||||
// no need
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractTask createTask(TaskExecutionContext taskRequest) {
|
||||
EmrParameters emrParameters = JSONUtils.parseObject(taskRequest.getTaskParams(), EmrParameters.class);
|
||||
@ -46,12 +39,8 @@ public class EmrTaskChannel implements TaskChannel {
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractParameters parseParameters(ParametersNode parametersNode) {
|
||||
return JSONUtils.parseObject(parametersNode.getTaskParams(), EmrParameters.class);
|
||||
public AbstractParameters parseParameters(String taskParams) {
|
||||
return JSONUtils.parseObject(taskParams, EmrParameters.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResourceParametersHelper getResources(String parameters) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
@ -19,10 +19,6 @@ package org.apache.dolphinscheduler.plugin.task.emr;
|
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
|
||||
import org.apache.dolphinscheduler.spi.params.base.PluginParams;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import com.google.auto.service.AutoService;
|
||||
|
||||
@ -34,11 +30,6 @@ public class EmrTaskChannelFactory implements TaskChannelFactory {
|
||||
return "EMR";
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<PluginParams> getParams() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskChannel create() {
|
||||
return new EmrTaskChannel();
|
||||
|
@ -18,42 +18,20 @@
|
||||
package org.apache.dolphinscheduler.plugin.task.flink;
|
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
|
||||
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
|
||||
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.stream.StreamTaskChannel;
|
||||
|
||||
public class FlinkStreamTaskChannel implements StreamTaskChannel {
|
||||
|
||||
@Override
|
||||
public void cancelApplication(boolean status) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public FlinkStreamTask createTask(TaskExecutionContext taskRequest) {
|
||||
return new FlinkStreamTask(taskRequest);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractParameters parseParameters(ParametersNode parametersNode) {
|
||||
return JSONUtils.parseObject(parametersNode.getTaskParams(), FlinkStreamParameters.class);
|
||||
public AbstractParameters parseParameters(String taskParams) {
|
||||
return JSONUtils.parseObject(taskParams, FlinkStreamParameters.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResourceParametersHelper getResources(String parameters) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractTask pauseTask(TaskExecutionContext taskExecutionContext) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractTask recoverTask(TaskExecutionContext taskExecutionContext) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user