From d86ceeecbea923e4f083a2c357beeb3b43003c43 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Thu, 23 Jul 2020 21:54:48 +0800 Subject: [PATCH 01/49] [implement] change task state --- .../controller/TaskInstanceController.java | 25 ++++++++++ .../dolphinscheduler/api/enums/Status.java | 2 + .../api/service/TaskInstanceService.java | 47 +++++++++++++++++++ .../TaskInstanceControllerTest.java | 5 ++ .../api/service/TaskInstanceServiceTest.java | 5 ++ .../common/enums/ExecutionStatus.java | 6 ++- 6 files changed, 88 insertions(+), 2 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java index c0ad88f48107..a4a28a555d3b 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java @@ -34,6 +34,7 @@ import java.util.Map; +import static org.apache.dolphinscheduler.api.enums.Status.FORCE_TASK_SUCCESS_ERROR; import static org.apache.dolphinscheduler.api.enums.Status.QUERY_TASK_LIST_PAGING_ERROR; /** @@ -103,4 +104,28 @@ public Result queryTaskListPaging(@ApiIgnore @RequestAttribute(value = Constants return returnDataListPaging(result); } + /** + * change one single task instance's state from failure to forced success + * + * @param loginUser login user + * @param projectName project name + * @param taskInstanceId task instance id + * @return the result code and msg + */ + @ApiOperation(value = "force-success", notes = "") + @ApiImplicitParams({ + @ApiImplicitParam(name = "taskInstanceId", value = "TASK_INTSTANCE_ID", required = true, dataType = "Int", example = "2") + }) + @PostMapping(value = "/force-success") + @ResponseStatus(HttpStatus.OK) + @ApiException(FORCE_TASK_SUCCESS_ERROR) + public Result forceSingleTaskSuccess(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser, + @ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName, + @RequestParam(value = "taskInstanceId") Integer taskInstanceId) { + logger.info("force task success, login user: {}, project:{}, task instance id:{}", + loginUser.getUserName(), projectName, taskInstanceId); + Map result = taskInstanceService.forceSingleTaskSuccess(loginUser, projectName, taskInstanceId); + return returnDataList(result); + } + } diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java index ec0b8f0a7694..aabefa81e3d6 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java @@ -176,6 +176,8 @@ public enum Status { DELETE_WORKER_GROUP_FAIL(10147,"delete worker group fail ", "删除worker分组失败"), COPY_PROCESS_DEFINITION_ERROR(10148,"copy process definition error", "复制工作流错误"), USER_DISABLED(10149,"The current user is disabled", "当前用户已停用"), + FORCE_TASK_SUCCESS_ERROR(10150, "force task success error", "强制成功任务实例错误"), + TASK_INSTANCE_STATE_OPETATION_ERROR(10151, "the status of task instance {0} is {1},Cannot perform force success operation", "任务实例[{0}]的状态是[{1}],无法执行强制成功操作"), UDF_FUNCTION_NOT_EXIST(20001, "UDF function not found", "UDF函数不存在"), UDF_FUNCTION_EXISTS(20002, "UDF function already exists", "UDF函数已存在"), diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java index 170278e02f62..12fd60977c92 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java @@ -138,4 +138,51 @@ public Map queryTaskListPaging(User loginUser, String projectName return result; } + + /** + * change one single task instance's state from failure to forced success + * + * @param loginUser login user + * @param projectName project name + * @param taskInstanceId task instance id + * @return the result code and msg + */ + public Map forceSingleTaskSuccess(User loginUser, String projectName, Integer taskInstanceId) { + Map result = new HashMap<>(5); + Project project = projectMapper.queryByName(projectName); + + // check user auth + Map checkResult = projectService.checkProjectAndAuth(loginUser, project, projectName); + Status status = (Status) checkResult.get(Constants.STATUS); + if (status != Status.SUCCESS) { + return checkResult; + } + + // check whether the task instance can be found + TaskInstance task = taskInstanceMapper.selectById(taskInstanceId); + if (task == null) { + putMsg(result, Status.TASK_INSTANCE_NOT_FOUND); + return result; + } + + // check whether the task instance state type is failure + if (!task.getState().typeIsFailure()) { + // FIXME: 这个status的msg中是包含参数的,需要处理一下(已完善,待测试) + putMsg(result, Status.TASK_INSTANCE_STATE_OPETATION_ERROR, taskInstanceId, task.getState().toString()); + return result; + } + + // change the state of the task instance + task.setState(ExecutionStatus.FORCED_SUCCESS); + int changedNum = taskInstanceMapper.updateById(task); + if (changedNum > 0) { + putMsg(result, Status.SUCCESS); + } + else { + // FIXME: 或许应该再加一个状态码 + putMsg(result, Status.FORCE_TASK_SUCCESS_ERROR); + } + + return result; + } } diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java index 368981c0d370..60da75b55494 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java @@ -62,4 +62,9 @@ public void testQueryTaskListPaging() throws Exception { Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue()); logger.info(mvcResult.getResponse().getContentAsString()); } + + @Test + public void forceSingleTaskSuccess() { + // TODO: 加入测试 + } } diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java index ebb61395776a..0d2ffb3bb653 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java @@ -189,4 +189,9 @@ private void putMsg(Map result, Status status, Object... statusP result.put(Constants.MSG, status.getMsg()); } } + + @Test + public void forceSingleTaskSuccess() { + // TODO: 加入测试 + } } \ No newline at end of file diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/ExecutionStatus.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/ExecutionStatus.java index 7cc073e09e09..714fde4862b1 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/ExecutionStatus.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/ExecutionStatus.java @@ -41,6 +41,7 @@ public enum ExecutionStatus { * 9 kill * 10 waiting thread * 11 waiting depend node complete + * 12 forced success */ SUBMITTED_SUCCESS(0, "submit success"), RUNNING_EXEUTION(1, "running"), @@ -53,7 +54,8 @@ public enum ExecutionStatus { NEED_FAULT_TOLERANCE(8, "need fault tolerance"), KILL(9, "kill"), WAITTING_THREAD(10, "waiting thread"), - WAITTING_DEPEND(11, "waiting depend node complete"); + WAITTING_DEPEND(11, "waiting depend node complete"), + FORCED_SUCCESS(12, "forced success"); ExecutionStatus(int code, String descp){ this.code = code; @@ -77,7 +79,7 @@ public enum ExecutionStatus { * @return status */ public boolean typeIsSuccess(){ - return this == SUCCESS; + return this == SUCCESS || this == FORCED_SUCCESS; } /** From 3fceee95703a37256c54ba9131da30ea61b444eb Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Thu, 23 Jul 2020 22:50:32 +0800 Subject: [PATCH 02/49] =?UTF-8?q?=E4=BB=8E=E5=BC=BA=E5=88=B6=E6=88=90?= =?UTF-8?q?=E5=8A=9F=E7=9A=84=E8=8A=82=E7=82=B9=E5=A4=84=E5=90=AF=E5=8A=A8?= =?UTF-8?q?-apiserver=E9=83=A8=E5=88=86?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 1. 新添加executeType 2. 新添加对应的commandType 3. 修改controller对应的service部分 4. 补充processService中的verifyIsNeedCreateCommand的判断逻辑 --- .../org/apache/dolphinscheduler/api/enums/ExecuteType.java | 3 ++- .../apache/dolphinscheduler/api/service/ExecutorService.java | 5 +++++ .../apache/dolphinscheduler/common/enums/CommandType.java | 4 +++- .../dolphinscheduler/service/process/ProcessService.java | 1 + 4 files changed, 11 insertions(+), 2 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/ExecuteType.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/ExecuteType.java index cc1797295a37..d5dfa3148e9b 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/ExecuteType.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/ExecuteType.java @@ -29,8 +29,9 @@ public enum ExecuteType { * 3 resume failure * 4 stop * 5 pause + * 6 resume from forced success */ - NONE,REPEAT_RUNNING, RECOVER_SUSPENDED_PROCESS, START_FAILURE_TASK_PROCESS, STOP, PAUSE; + NONE,REPEAT_RUNNING, RECOVER_SUSPENDED_PROCESS, START_FAILURE_TASK_PROCESS, STOP, PAUSE, RESUME_FROM_FORCED_SUCCESS; public static ExecuteType getEnum(int value){ diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java index 6a8dad4f2a8b..0d67617904ce 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java @@ -17,6 +17,7 @@ package org.apache.dolphinscheduler.api.service; +import com.sun.org.apache.bcel.internal.generic.BREAKPOINT; import org.apache.dolphinscheduler.api.enums.ExecuteType; import org.apache.dolphinscheduler.api.enums.Status; import org.apache.dolphinscheduler.common.Constants; @@ -264,6 +265,9 @@ public Map execute(User loginUser, String projectName, Integer p result = updateProcessInstancePrepare(processInstance, CommandType.PAUSE, ExecutionStatus.READY_PAUSE); } break; + case RESUME_FROM_FORCED_SUCCESS: + result = insertCommand(loginUser, processInstanceId, processDefinition.getId(), CommandType.RESUME_FROM_FORCED_SUCCESS); + break; default: logger.error("unknown execute type : {}", executeType); putMsg(result, Status.REQUEST_PARAMS_NOT_VALID_ERROR, "unknown execute type"); @@ -310,6 +314,7 @@ private Map checkExecuteType(ProcessInstance processInstance, Ex } break; case START_FAILURE_TASK_PROCESS: + case RESUME_FROM_FORCED_SUCCESS: if (executionStatus.typeIsFailure()) { checkResult = true; } diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/CommandType.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/CommandType.java index 9682016d6f38..3b663bc74d29 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/CommandType.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/CommandType.java @@ -39,6 +39,7 @@ public enum CommandType { * 8 pause a process * 9 stop a process * 10 recover waiting thread + * 11 resume process from forced-success task nodes */ START_PROCESS(0, "start a new process"), START_CURRENT_TASK_PROCESS(1, "start a new process from current nodes"), @@ -50,7 +51,8 @@ public enum CommandType { REPEAT_RUNNING(7, "repeat running a process"), PAUSE(8, "pause a process"), STOP(9, "stop a process"), - RECOVER_WAITTING_THREAD(10, "recover waiting thread"); + RECOVER_WAITTING_THREAD(10, "recover waiting thread"), + RESUME_FROM_FORCED_SUCCESS(11, "resume process from forced-success task nodes"); CommandType(int code, String descp){ this.code = code; diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java index 769fe0928a7f..21de9f04d77c 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java @@ -200,6 +200,7 @@ public Boolean verifyIsNeedCreateCommand(Command command){ cmdTypeMap.put(CommandType.REPEAT_RUNNING,1); cmdTypeMap.put(CommandType.RECOVER_SUSPENDED_PROCESS,1); cmdTypeMap.put(CommandType.START_FAILURE_TASK_PROCESS,1); + cmdTypeMap.put(CommandType.RESUME_FROM_FORCED_SUCCESS,1); CommandType commandType = command.getCommandType(); if(cmdTypeMap.containsKey(commandType)){ From 5e9d664b8081407c9dd0dae68a21bd7238f02713 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Sun, 26 Jul 2020 00:19:17 +0800 Subject: [PATCH 03/49] =?UTF-8?q?=E4=BB=8E=E5=BC=BA=E5=88=B6=E6=88=90?= =?UTF-8?q?=E5=8A=9F=E5=A4=84=E5=90=AF=E5=8A=A8-masterServer=E9=83=A8?= =?UTF-8?q?=E5=88=86?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 1. 添加两个查询(mapper.java和mapper.xml) 2. 添加ProcessService中constructProcessInstance中对应的部分 --- .../dao/mapper/TaskInstanceMapper.java | 7 +++ .../dao/mapper/TaskInstanceMapper.xml | 14 ++++++ .../service/process/ProcessService.java | 47 +++++++++++++++++++ 3 files changed, 68 insertions(+) diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java index ac23b25c9c91..9b3268e1b85f 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java @@ -69,4 +69,11 @@ IPage queryTaskInstanceListPaging(IPage page, @Param("startTime") Date startTime, @Param("endTime") Date endTime ); + + List queryTaskByPIdAndStateAndType(@Param("processInstanceId") Integer processInstanceId, + @Param("state") int state, + @Param("taskType") String taskType); + + List queryTasksBySubProcessTaskIdAndState(@Param("subProcessTaskId") Integer subProcessTaskId, + @Param("state") Integer state); } diff --git a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml index 143761bf8c93..ebf446a0d877 100644 --- a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml +++ b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml @@ -129,4 +129,18 @@ order by instance.start_time desc + + diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java index 21de9f04d77c..8a379ee44617 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java @@ -690,6 +690,38 @@ private ProcessInstance constructProcessInstance(Command command, String host){ initComplementDataParam(processDefinition, processInstance, cmdParam); break; case SCHEDULER: + break; + case RESUME_FROM_FORCED_SUCCESS: + // TODO: 在这里初始化processInstance的时候可能需要加入startnodelist参数 + + // find forced-success tasks here + List forcedSuccessList = this.findTaskIdByInstanceState(processInstance.getId(), ExecutionStatus.FORCED_SUCCESS); + // deal with sub_process nodes + List failedSubList = this.findTaskIdByInstanceStateAndType(processInstance.getId(), ExecutionStatus.FAILURE, TaskType.SUB_PROCESS); + List toleranceSubList = this.findTaskIdByInstanceStateAndType(processInstance.getId(), ExecutionStatus.NEED_FAULT_TOLERANCE, TaskType.SUB_PROCESS); + List killedSubList = this.findTaskIdByInstanceStateAndType(processInstance.getId(), ExecutionStatus.KILL, TaskType.SUB_PROCESS); + + failedSubList.addAll(toleranceSubList); + failedSubList.addAll(killedSubList); + for (int i = 0; i < failedSubList.size(); i++) { + List tmpResultList = this.findTaskIdBySubProcessTaskIdAndState(failedSubList.get(i), ExecutionStatus.FORCED_SUCCESS); + // if there is forced success in the sub_process + if (tmpResultList != null && tmpResultList.size() > 0) { + forcedSuccessList.add(failedSubList.get(i)); + // change sub_process task's state into submitted_success + TaskInstance taskInstance = this.findTaskInstanceById(failedSubList.get(i)); + taskInstance.setState(ExecutionStatus.SUBMITTED_SUCCESS); + updateTaskInstance(taskInstance); + } + } + + // set resume node list + cmdParam.remove(CMDPARAM_RECOVERY_START_NODE_STRING); + cmdParam.put(Constants.CMDPARAM_RECOVERY_START_NODE_STRING, + String.join(Constants.COMMA, convertIntListToString(forcedSuccessList))); + processInstance.setCommandParam(JSONUtils.toJsonString(cmdParam)); + processInstance.setRunTimes(runTime + 1); + break; default: break; @@ -1247,6 +1279,21 @@ public List findTaskIdByInstanceState(int instanceId, ExecutionStatus s return taskInstanceMapper.queryTaskByProcessIdAndState(instanceId, state.ordinal()); } + /** + * get id list by task state and type + * @param instanceId process instance id + * @param state task instance state + * @param taskType task type + * @return task instance id list + */ + public List findTaskIdByInstanceStateAndType(int instanceId, ExecutionStatus state, TaskType taskType){ + return taskInstanceMapper.queryTaskByPIdAndStateAndType(instanceId, state.ordinal(), taskType.toString()); + } + + public List findTaskIdBySubProcessTaskIdAndState(int taskId, ExecutionStatus state){ + return taskInstanceMapper.queryTasksBySubProcessTaskIdAndState(taskId, state.ordinal()); + } + /** * find valid task list by process definition id * @param processInstanceId processInstanceId From e912a8094c7226532e4ea9260a4ad1cfbd152f51 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Sun, 26 Jul 2020 11:49:03 +0800 Subject: [PATCH 04/49] fix bug in mapper --- .../dolphinscheduler/dao/mapper/TaskInstanceMapper.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml index ebf446a0d877..8b654a2ec828 100644 --- a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml +++ b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml @@ -129,17 +129,17 @@ order by instance.start_time desc - select id from t_ds_task_instance where process_instance_id = #{processInstanceId} and task_type = #{taskType} and state = #{state} and flag = 1 - select id from t_ds_task_instance where process_instance_id = - (select process_instance_id in t_ds_relation_process_instance where parent_task_instance_id = #{subProcessTaskId}) + (select process_instance_id from t_ds_relation_process_instance where parent_task_instance_id = #{subProcessTaskId}) and state = #{state} and flag = 1 From fa0bb5e546cfc09e59e096efe0d2ac33bed68a4d Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Tue, 28 Jul 2020 12:58:51 +0800 Subject: [PATCH 05/49] [bugfix] issue #3324 --- .../master/runner/MasterExecThread.java | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java index 4b5c3f7d0000..df1e1397601a 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java @@ -670,6 +670,11 @@ private DependResult isTaskDepsComplete(String taskName) { } TaskNode taskNode = dag.getNode(taskName); + // condition node directly return success + if (taskNode.isConditionsTask()) { + return DependResult.SUCCESS; + } + List depNameList = taskNode.getDepList(); for(String depsNode : depNameList ){ if(!dag.containsNode(depsNode) @@ -682,10 +687,16 @@ private DependResult isTaskDepsComplete(String taskName) { return DependResult.WAITING; } ExecutionStatus depTaskState = completeTaskList.get(depsNode).getState(); - // conditions task would not return failed. - if(depTaskState.typeIsFailure() - && !DagHelper.haveConditionsAfterNode(depsNode, dag ) - && !dag.getNode(depsNode).isConditionsTask()){ + // conditions task should be handled separately + if (dag.getNode(depsNode).isConditionsTask()) { + List tmpTaskList = parseConditionTask(depsNode); + if (tmpTaskList.contains(taskName)){ + return DependResult.SUCCESS; + } + return DependResult.FAILED; + } + + if(depTaskState.typeIsFailure()){ return DependResult.FAILED; } From 23277bf160838d7ee4720cdc649bb1f6031740c9 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Tue, 4 Aug 2020 16:51:35 +0800 Subject: [PATCH 06/49] Revert "[bugfix] issue #3324" This reverts commit fa0bb5e546cfc09e59e096efe0d2ac33bed68a4d. --- .../master/runner/MasterExecThread.java | 19 ++++--------------- 1 file changed, 4 insertions(+), 15 deletions(-) diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java index df1e1397601a..4b5c3f7d0000 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java @@ -670,11 +670,6 @@ private DependResult isTaskDepsComplete(String taskName) { } TaskNode taskNode = dag.getNode(taskName); - // condition node directly return success - if (taskNode.isConditionsTask()) { - return DependResult.SUCCESS; - } - List depNameList = taskNode.getDepList(); for(String depsNode : depNameList ){ if(!dag.containsNode(depsNode) @@ -687,16 +682,10 @@ private DependResult isTaskDepsComplete(String taskName) { return DependResult.WAITING; } ExecutionStatus depTaskState = completeTaskList.get(depsNode).getState(); - // conditions task should be handled separately - if (dag.getNode(depsNode).isConditionsTask()) { - List tmpTaskList = parseConditionTask(depsNode); - if (tmpTaskList.contains(taskName)){ - return DependResult.SUCCESS; - } - return DependResult.FAILED; - } - - if(depTaskState.typeIsFailure()){ + // conditions task would not return failed. + if(depTaskState.typeIsFailure() + && !DagHelper.haveConditionsAfterNode(depsNode, dag ) + && !dag.getNode(depsNode).isConditionsTask()){ return DependResult.FAILED; } From 93653768c813640da7cb3de6cbe15cd8c8aa72a5 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Tue, 4 Aug 2020 20:14:36 +0800 Subject: [PATCH 07/49] =?UTF-8?q?=E6=94=AF=E6=8C=81sub-process=E5=92=8Ccon?= =?UTF-8?q?dition=E7=9A=84TFS?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../api/service/TaskInstanceService.java | 2 +- .../dolphinscheduler/dao/utils/DagHelper.java | 24 +++++-- .../dao/utils/DagHelperTest.java | 6 +- .../master/runner/MasterExecThread.java | 66 ++++++++++++++++--- .../server/master/MasterCommandTest.java | 2 +- .../service/process/ProcessService.java | 6 ++ 6 files changed, 87 insertions(+), 19 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java index 12fd60977c92..2140af497f06 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java @@ -179,7 +179,7 @@ public Map forceSingleTaskSuccess(User loginUser, String project putMsg(result, Status.SUCCESS); } else { - // FIXME: 或许应该再加一个状态码 + // FIXME: 或许应该再加一个状态码,如果按照process的执行逻辑,那么是不需要的 putMsg(result, Status.FORCE_TASK_SUCCESS_ERROR); } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java index d3b829cb4fb2..1901af318a77 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java @@ -17,6 +17,7 @@ package org.apache.dolphinscheduler.dao.utils; +import com.amazonaws.services.simpleworkflow.model.TaskList; import org.apache.dolphinscheduler.common.enums.TaskDependType; import org.apache.dolphinscheduler.common.graph.DAG; import org.apache.dolphinscheduler.common.model.TaskNode; @@ -262,7 +263,7 @@ public static TaskNode findNodeByName(List nodeDetails, String nodeNam * @return start Vertex list */ public static Collection getStartVertex(String parentNodeName, DAG dag, - Map completeTaskList){ + Map completeTaskList, List allNodes){ if(completeTaskList == null){ completeTaskList = new HashMap<>(); @@ -286,10 +287,10 @@ public static Collection getStartVertex(String parentNodeName, DAG postNodes = getStartVertex(start, dag, completeTaskList); + Collection postNodes = getStartVertex(start, dag, completeTaskList, allNodes); for(String post : postNodes){ TaskNode postNode = dag.getNode(post); - if(taskNodeCanSubmit(postNode, dag, completeTaskList)){ + if(taskNodeCanSubmit(postNode, dag, completeTaskList, allNodes)){ tmpStartVertexs.add(post); } } @@ -307,7 +308,8 @@ public static Collection getStartVertex(String parentNodeName, DAG dag, - Map completeTaskList) { + Map completeTaskList, + List allNodes) { List dependList = taskNode.getDepList(); if(dependList == null){ @@ -316,6 +318,10 @@ public static boolean taskNodeCanSubmit(TaskNode taskNode, for(String dependNodeName : dependList){ TaskNode dependNode = dag.getNode(dependNodeName); + // when executing resume_from_forced_success, depend node may be not in dag + if (dependNode == null) { + dependNode = findNodeByName(allNodes, dependNodeName); + } if(!dependNode.isForbidden() && !completeTaskList.containsKey(dependNodeName)){ return false; } @@ -324,6 +330,16 @@ public static boolean taskNodeCanSubmit(TaskNode taskNode, } + /** + * generate process data and return all task nodes + * @param processDefinitionJson process definition json + * @return task nodes list + */ + public static List getAllTaskNodesFromFlowJson(String processDefinitionJson) { + ProcessData processData = JSONUtils.parseObject(processDefinitionJson, ProcessData.class); + return processData.getTasks(); + } + /*** * build dag graph * @param processDag processDag diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/utils/DagHelperTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/utils/DagHelperTest.java index 14dfe0b75005..0f1fe9234f97 100644 --- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/utils/DagHelperTest.java +++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/utils/DagHelperTest.java @@ -58,21 +58,21 @@ public void testTaskNodeCanSubmit() throws JsonProcessingException { node2.setRunFlag(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN); TaskNode nodex = dag.getNode("4"); nodex.setRunFlag(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN); - canSubmit = DagHelper.taskNodeCanSubmit(taskNode3, dag, completeTaskList); + canSubmit = DagHelper.taskNodeCanSubmit(taskNode3, dag, completeTaskList, null); Assert.assertEquals(canSubmit, true); // 2forbidden, 3 cannot be submit completeTaskList.putIfAbsent("2", new TaskInstance()); TaskNode nodey = dag.getNode("4"); nodey.setRunFlag(""); - canSubmit = DagHelper.taskNodeCanSubmit(taskNode3, dag, completeTaskList); + canSubmit = DagHelper.taskNodeCanSubmit(taskNode3, dag, completeTaskList, null); Assert.assertEquals(canSubmit, false); // 2/3 forbidden submit 5 TaskNode node3 = dag.getNode("3"); node3.setRunFlag(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN); TaskNode node5 = dag.getNode("5"); - canSubmit = DagHelper.taskNodeCanSubmit(node5, dag, completeTaskList); + canSubmit = DagHelper.taskNodeCanSubmit(node5, dag, completeTaskList, null); Assert.assertEquals(canSubmit, true); } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java index 4b5c3f7d0000..b37f6f66c27a 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java @@ -130,6 +130,11 @@ public class MasterExecThread implements Runnable { */ private DAG dag; + /** + * all task nodes generated by process instance json + */ + private List allNodes; + /** * process service */ @@ -364,6 +369,8 @@ private void buildFlowDag() throws Exception { } // generate process dag dag = DagHelper.buildDagGraph(processDag); + + allNodes = DagHelper.getAllTaskNodesFromFlowJson(processInstance.getProcessInstanceJson()); } /** @@ -533,7 +540,7 @@ private boolean isTaskNodeNeedSkip(TaskNode taskNode){ private void setTaskNodeSkip(List taskNodesSkipList){ for(String skipNode : taskNodesSkipList){ skipTaskNodeList.putIfAbsent(skipNode, dag.getNode(skipNode)); - Collection postNodeList = DagHelper.getStartVertex(skipNode, dag, completeTaskList); + Collection postNodeList = DagHelper.getStartVertex(skipNode, dag, completeTaskList, allNodes); List postSkipList = new ArrayList<>(); for(String post : postNodeList){ TaskNode postNode = dag.getNode(post); @@ -555,6 +562,10 @@ private void setTaskNodeSkip(List taskNodesSkipList){ private List parseConditionTask(String nodeName){ List conditionTaskList = new ArrayList<>(); TaskNode taskNode = dag.getNode(nodeName); + if (taskNode == null) { + taskNode = DagHelper.findNodeByName(allNodes, nodeName); + } + if(!taskNode.isConditionsTask()){ return conditionTaskList; } @@ -593,7 +604,7 @@ private List parsePostNodeList(String previousNodeName){ if(taskNode != null && taskNode.isConditionsTask()){ return parseConditionTask(previousNodeName); } - Collection postNodeCollection = DagHelper.getStartVertex(previousNodeName, dag, completeTaskList); + Collection postNodeCollection = DagHelper.getStartVertex(previousNodeName, dag, completeTaskList, allNodes); List postSkipList = new ArrayList<>(); // delete success node, parse the past nodes // if conditions node, @@ -672,26 +683,33 @@ private DependResult isTaskDepsComplete(String taskName) { TaskNode taskNode = dag.getNode(taskName); List depNameList = taskNode.getDepList(); for(String depsNode : depNameList ){ - if(!dag.containsNode(depsNode) + // in this condition, it needs to check the command type + if((!dag.containsNode(depsNode) && !processInstance.getCommandType().equals(CommandType.RESUME_FROM_FORCED_SUCCESS)) || forbiddenTaskList.containsKey(depsNode) || skipTaskNodeList.containsKey(depsNode)){ continue; } - // dependencies must be fully completed + // all the dependencies must be fully completed if(!completeTaskList.containsKey(depsNode)){ + // if dag doesn't contain it either, the depend will never be executed, so return failed + if (!dag.containsNode(depsNode)) { + return DependResult.FAILED; + } return DependResult.WAITING; } ExecutionStatus depTaskState = completeTaskList.get(depsNode).getState(); - // conditions task would not return failed. - if(depTaskState.typeIsFailure() - && !DagHelper.haveConditionsAfterNode(depsNode, dag ) - && !dag.getNode(depsNode).isConditionsTask()){ - return DependResult.FAILED; - } if(depTaskState.typeIsPause() || depTaskState.typeIsCancel()){ return DependResult.WAITING; } + + // ignore task state if current task is condition + if(taskNode.isConditionsTask()){ + continue; + } + if(!dependTaskSuccess(depsNode, taskName)){ + return DependResult.FAILED; + } } logger.info("taskName: {} completeDependTaskList: {}", taskName, Arrays.toString(completeTaskList.keySet().toArray())); @@ -699,6 +717,34 @@ private DependResult isTaskDepsComplete(String taskName) { return DependResult.SUCCESS; } + /** + * depend node is completed, but here need check the condition task branch is the next node + * @param dependNodeName + * @param nextNodeName + * @return + */ + private boolean dependTaskSuccess(String dependNodeName, String nextNodeName){ + TaskNode tmpNode = dag.getNode(dependNodeName); + // if dag doesn't contain depend node + if (tmpNode == null) { + tmpNode = DagHelper.findNodeByName(allNodes, dependNodeName); + } + + if(tmpNode.isConditionsTask()){ + //condition task need check the branch to run + List nextTaskList = parseConditionTask(dependNodeName); + if(!nextTaskList.contains(nextNodeName)){ + return false; + } + }else { + ExecutionStatus depTaskState = completeTaskList.get(dependNodeName).getState(); + if(depTaskState.typeIsFailure()){ + return false; + } + } + return true; + } + /** * query task instance by complete state diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterCommandTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterCommandTest.java index d541f43a3b6c..c36521b8554a 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterCommandTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterCommandTest.java @@ -121,7 +121,7 @@ public void testDagHelper(){ new ArrayList<>(), new ArrayList<>(), TaskDependType.TASK_POST); DAG dag = DagHelper.buildDagGraph(processDag); - Collection start = DagHelper.getStartVertex("1", dag, null); + Collection start = DagHelper.getStartVertex("1", dag, null, null); System.out.println(start.toString()); diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java index 8a379ee44617..7ccc937a022d 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java @@ -1290,6 +1290,12 @@ public List findTaskIdByInstanceStateAndType(int instanceId, ExecutionS return taskInstanceMapper.queryTaskByPIdAndStateAndType(instanceId, state.ordinal(), taskType.toString()); } + /** + * get tasks in sub_process by sub_process task id and state + * @param taskId task id + * @param state task instance state + * @return task instance id list + */ public List findTaskIdBySubProcessTaskIdAndState(int taskId, ExecutionStatus state){ return taskInstanceMapper.queryTasksBySubProcessTaskIdAndState(taskId, state.ordinal()); } From 5d02e307728b210c865f0e492812cec2035e93db Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Wed, 5 Aug 2020 19:35:11 +0800 Subject: [PATCH 08/49] =?UTF-8?q?=E8=B0=83=E6=95=B4=E6=95=B0=E6=8D=AE?= =?UTF-8?q?=E5=BA=93=E8=AE=BF=E9=97=AE=E5=87=BD=E6=95=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dao/mapper/TaskInstanceMapper.java | 9 ++-- .../dao/mapper/TaskInstanceMapper.xml | 17 ++++-- .../service/process/ProcessService.java | 52 ++++++++++++------- 3 files changed, 52 insertions(+), 26 deletions(-) diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java index 9b3268e1b85f..5d22184b8d4e 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java @@ -70,10 +70,11 @@ IPage queryTaskInstanceListPaging(IPage page, @Param("endTime") Date endTime ); - List queryTaskByPIdAndStateAndType(@Param("processInstanceId") Integer processInstanceId, - @Param("state") int state, + List queryTaskByPIdAndStatusAndType(@Param("processInstanceId") Integer processInstanceId, + @Param("states") int[] stateArray, @Param("taskType") String taskType); - List queryTasksBySubProcessTaskIdAndState(@Param("subProcessTaskId") Integer subProcessTaskId, - @Param("state") Integer state); + List queryTaskBySubProcessTaskIdAndStatusAndType(@Param("subProcessTaskId") Integer subProcessTaskId, + @Param("states") int[] stateArray, + @Param("taskType") String taskType); } diff --git a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml index 8b654a2ec828..a25b46eecb41 100644 --- a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml +++ b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml @@ -129,18 +129,27 @@ order by instance.start_time desc - select id from t_ds_task_instance where process_instance_id = #{processInstanceId} and task_type = #{taskType} - and state = #{state} + and state in + + #{i} + and flag = 1 - select id from t_ds_task_instance where process_instance_id = (select process_instance_id from t_ds_relation_process_instance where parent_task_instance_id = #{subProcessTaskId}) - and state = #{state} + and state in + + #{i} + + + and task_type = #{taskType} + and flag = 1 diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java index 7ccc937a022d..48f13133cc60 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java @@ -38,7 +38,6 @@ import org.springframework.stereotype.Component; import org.springframework.transaction.annotation.Transactional; -import java.io.File; import java.util.*; import java.util.stream.Collectors; @@ -692,19 +691,23 @@ private ProcessInstance constructProcessInstance(Command command, String host){ case SCHEDULER: break; case RESUME_FROM_FORCED_SUCCESS: - // TODO: 在这里初始化processInstance的时候可能需要加入startnodelist参数 - // find forced-success tasks here List forcedSuccessList = this.findTaskIdByInstanceState(processInstance.getId(), ExecutionStatus.FORCED_SUCCESS); // deal with sub_process nodes - List failedSubList = this.findTaskIdByInstanceStateAndType(processInstance.getId(), ExecutionStatus.FAILURE, TaskType.SUB_PROCESS); - List toleranceSubList = this.findTaskIdByInstanceStateAndType(processInstance.getId(), ExecutionStatus.NEED_FAULT_TOLERANCE, TaskType.SUB_PROCESS); - List killedSubList = this.findTaskIdByInstanceStateAndType(processInstance.getId(), ExecutionStatus.KILL, TaskType.SUB_PROCESS); +// List failedSubList = this.findTaskIdByInstanceStatusAndType(processInstance.getId(), ExecutionStatus.FAILURE, TaskType.SUB_PROCESS); +// List toleranceSubList = this.findTaskIdByInstanceStatusAndType(processInstance.getId(), ExecutionStatus.NEED_FAULT_TOLERANCE, TaskType.SUB_PROCESS); +// List killedSubList = this.findTaskIdByInstanceStatusAndType(processInstance.getId(), ExecutionStatus.KILL, TaskType.SUB_PROCESS); +// +// failedSubList.addAll(toleranceSubList); +// failedSubList.addAll(killedSubList); + List failedSubList = this.findTaskIdByInstanceStatusAndType(processInstance.getId(), + new ExecutionStatus[]{ExecutionStatus.FAILURE, ExecutionStatus.KILL, ExecutionStatus.NEED_FAULT_TOLERANCE}, + TaskType.SUB_PROCESS); - failedSubList.addAll(toleranceSubList); - failedSubList.addAll(killedSubList); for (int i = 0; i < failedSubList.size(); i++) { - List tmpResultList = this.findTaskIdBySubProcessTaskIdAndState(failedSubList.get(i), ExecutionStatus.FORCED_SUCCESS); + List tmpResultList = this.findTaskIdInSubProcessByStatusAndType(failedSubList.get(i), + new ExecutionStatus[]{ExecutionStatus.FORCED_SUCCESS}, + null); // if there is forced success in the sub_process if (tmpResultList != null && tmpResultList.size() > 0) { forcedSuccessList.add(failedSubList.get(i)); @@ -1281,23 +1284,36 @@ public List findTaskIdByInstanceState(int instanceId, ExecutionStatus s /** * get id list by task state and type - * @param instanceId process instance id - * @param state task instance state + * @param processInstanceId process instance id + * @param states task instance state array * @param taskType task type * @return task instance id list */ - public List findTaskIdByInstanceStateAndType(int instanceId, ExecutionStatus state, TaskType taskType){ - return taskInstanceMapper.queryTaskByPIdAndStateAndType(instanceId, state.ordinal(), taskType.toString()); + public List findTaskIdByInstanceStatusAndType(int processInstanceId, ExecutionStatus[] states, TaskType taskType){ + int[] stateArray = new int[states.length]; + for (int i = 0; i < states.length; i++) { + stateArray[i] = states[i].ordinal(); + } + return taskInstanceMapper.queryTaskByPIdAndStatusAndType(processInstanceId, stateArray, taskType.toString()); } /** - * get tasks in sub_process by sub_process task id and state - * @param taskId task id - * @param state task instance state + * get tasks in sub_process by sub_process task id and state and type + * if param type is null, it queries all types + * @param taskId task instance id + * @param states task instance state array + * @param taskType task type * @return task instance id list */ - public List findTaskIdBySubProcessTaskIdAndState(int taskId, ExecutionStatus state){ - return taskInstanceMapper.queryTasksBySubProcessTaskIdAndState(taskId, state.ordinal()); + public List findTaskIdInSubProcessByStatusAndType(int taskId, ExecutionStatus[] states, TaskType taskType){ + int[] stateArray = new int[states.length]; + for (int i = 0; i < states.length; i++) { + stateArray[i] = states[i].ordinal(); + } + if (taskType == null) { + return taskInstanceMapper.queryTaskBySubProcessTaskIdAndStatusAndType(taskId, stateArray, null); + } + return taskInstanceMapper.queryTaskBySubProcessTaskIdAndStatusAndType(taskId, stateArray, taskType.toString()); } /** From 18ae56bb6a3516db69ecb880ca6ec1e9c88305e8 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Wed, 5 Aug 2020 21:42:09 +0800 Subject: [PATCH 09/49] =?UTF-8?q?=E6=94=B9=E5=8F=98=E5=8E=9F=E6=9C=89?= =?UTF-8?q?=E7=9A=84dag=E6=9E=84=E5=BB=BA=E5=8F=82=E6=95=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 1. 这样的话可以完整的构建整个dag,恢复之前的上下文;也没有任务可能重复执行的担忧 2. 不需要额外去处理process执行完之后的状态(主要是部分与整体的原因) 3. RecoverNodeIdList也不会重复 --- .../service/process/ProcessService.java | 51 ++++++++++++------- 1 file changed, 32 insertions(+), 19 deletions(-) diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java index 48f13133cc60..dc2e468fcd56 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java @@ -691,40 +691,29 @@ private ProcessInstance constructProcessInstance(Command command, String host){ case SCHEDULER: break; case RESUME_FROM_FORCED_SUCCESS: - // find forced-success tasks here - List forcedSuccessList = this.findTaskIdByInstanceState(processInstance.getId(), ExecutionStatus.FORCED_SUCCESS); - // deal with sub_process nodes -// List failedSubList = this.findTaskIdByInstanceStatusAndType(processInstance.getId(), ExecutionStatus.FAILURE, TaskType.SUB_PROCESS); -// List toleranceSubList = this.findTaskIdByInstanceStatusAndType(processInstance.getId(), ExecutionStatus.NEED_FAULT_TOLERANCE, TaskType.SUB_PROCESS); -// List killedSubList = this.findTaskIdByInstanceStatusAndType(processInstance.getId(), ExecutionStatus.KILL, TaskType.SUB_PROCESS); -// -// failedSubList.addAll(toleranceSubList); -// failedSubList.addAll(killedSubList); List failedSubList = this.findTaskIdByInstanceStatusAndType(processInstance.getId(), new ExecutionStatus[]{ExecutionStatus.FAILURE, ExecutionStatus.KILL, ExecutionStatus.NEED_FAULT_TOLERANCE}, TaskType.SUB_PROCESS); for (int i = 0; i < failedSubList.size(); i++) { - List tmpResultList = this.findTaskIdInSubProcessByStatusAndType(failedSubList.get(i), - new ExecutionStatus[]{ExecutionStatus.FORCED_SUCCESS}, - null); - // if there is forced success in the sub_process - if (tmpResultList != null && tmpResultList.size() > 0) { - forcedSuccessList.add(failedSubList.get(i)); + // if there exists forced success in the sub_process + if (haveForcedSuccessInSubProcess(failedSubList.get(i))) { // change sub_process task's state into submitted_success TaskInstance taskInstance = this.findTaskInstanceById(failedSubList.get(i)); taskInstance.setState(ExecutionStatus.SUBMITTED_SUCCESS); updateTaskInstance(taskInstance); } } - - // set resume node list + /** + * set resume node list to null + * 1. we can have a complete dag in the ExecThread so that it can restore the previous context + * 2. each time the operation is done the state of process will be reasonable as usual + */ cmdParam.remove(CMDPARAM_RECOVERY_START_NODE_STRING); cmdParam.put(Constants.CMDPARAM_RECOVERY_START_NODE_STRING, - String.join(Constants.COMMA, convertIntListToString(forcedSuccessList))); + String.join(Constants.COMMA, convertIntListToString(null))); processInstance.setCommandParam(JSONUtils.toJsonString(cmdParam)); processInstance.setRunTimes(runTime + 1); - break; default: break; @@ -733,6 +722,30 @@ private ProcessInstance constructProcessInstance(Command command, String host){ return processInstance; } + /** + * recursively check if a sub process node contains forced success node + * @param taskInstanceId task instance id + * @return true or false + */ + public boolean haveForcedSuccessInSubProcess(int taskInstanceId) { + List forcedSuccessList = this.findTaskIdInSubProcessByStatusAndType(taskInstanceId, + new ExecutionStatus[]{ExecutionStatus.FORCED_SUCCESS}, + null); + if (forcedSuccessList != null && forcedSuccessList.size() > 0) { + return true; + } + + List childSubList = this.findTaskIdInSubProcessByStatusAndType(taskInstanceId, + new ExecutionStatus[]{ExecutionStatus.FAILURE, ExecutionStatus.KILL, ExecutionStatus.NEED_FAULT_TOLERANCE}, + TaskType.SUB_PROCESS); + for (Integer child : childSubList) { + if (haveForcedSuccessInSubProcess(child)) { + return true; + } + } + return false; + } + /** * return complement data if the process start with complement data * @param processInstance processInstance From d0e6b09dd167bb09def5ac381b22b69878b894c8 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Wed, 5 Aug 2020 23:30:09 +0800 Subject: [PATCH 10/49] =?UTF-8?q?=E5=8A=A0=E5=85=A5valid=E6=A3=80=E9=AA=8C?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dolphinscheduler/api/enums/Status.java | 1 + .../api/service/ExecutorService.java | 28 ++++++++++++++++++- 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java index aabefa81e3d6..cbd18bd9c65e 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java @@ -231,6 +231,7 @@ public enum Status { EXPORT_PROCESS_DEFINE_BY_ID_ERROR(50028,"export process definition by id error", "导出工作流定义错误"), BATCH_EXPORT_PROCESS_DEFINE_BY_IDS_ERROR(50028,"batch export process definition by ids error", "批量导出工作流定义错误"), IMPORT_PROCESS_DEFINE_ERROR(50029,"import process definition error", "导入工作流定义错误"), + NO_VALID_FORCED_SUCCESS_TASK(50030, "there is no valid forced success node in process instance {0}", "工作流实例[{0}]中不包含有效的强制成功的任务实例"), HDFS_NOT_STARTUP(60001,"hdfs not startup", "hdfs未启用"), diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java index 0d67617904ce..0e01b912bec5 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java @@ -266,7 +266,11 @@ public Map execute(User loginUser, String projectName, Integer p } break; case RESUME_FROM_FORCED_SUCCESS: - result = insertCommand(loginUser, processInstanceId, processDefinition.getId(), CommandType.RESUME_FROM_FORCED_SUCCESS); + if (!this.checkValidForcedSuccessTask(processInstanceId)) { + putMsg(result, Status.NO_VALID_FORCED_SUCCESS_TASK, processInstance.getName()); + } else { + result = insertCommand(loginUser, processInstanceId, processDefinition.getId(), CommandType.RESUME_FROM_FORCED_SUCCESS); + } break; default: logger.error("unknown execute type : {}", executeType); @@ -600,4 +604,26 @@ private Map checkResultAndAuth(User loginUser, String projectNam return null; } + /** + * check if the process instance contains valid forced success task + * + * @param processInstanceId + * @return + */ + private boolean checkValidForcedSuccessTask(int processInstanceId) { + List forcedSuccessList = processService.findTaskIdByInstanceState(processInstanceId, ExecutionStatus.FORCED_SUCCESS); + if (forcedSuccessList != null && forcedSuccessList.size() > 0) { + return true; + } + + List failedSubList = processService.findTaskIdByInstanceStatusAndType(processInstanceId, + new ExecutionStatus[]{ExecutionStatus.FAILURE, ExecutionStatus.KILL, ExecutionStatus.NEED_FAULT_TOLERANCE}, + TaskType.SUB_PROCESS); + for (int i = 0; i < failedSubList.size(); i++) { + if (processService.haveForcedSuccessInSubProcess(failedSubList.get(i))) { + return true; + } + } + return false; + } } From 48afb2e2f871d04bd47b579822eca53d433313f3 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Thu, 6 Aug 2020 21:04:11 +0800 Subject: [PATCH 11/49] =?UTF-8?q?=E5=A2=9E=E5=8A=A0=E5=BC=BA=E5=88=B6?= =?UTF-8?q?=E6=88=90=E5=8A=9F=E5=AF=B9=E8=BF=90=E8=A1=8C=E4=B8=ADprocess?= =?UTF-8?q?=E7=9A=84=E5=BD=B1=E5=93=8D?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 1. 失败任务强制成功后会继续submit后续结点 2. 失败重试的节点强制成功后会继续submit后面结点,并且停止retry 3. 对于失败的sub-process如果其中的结点强制成功了,在parent-process中不会有任何影响,只能等到结束后“从强制成功处继续执行”。 --- .../master/runner/MasterExecThread.java | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java index b37f6f66c27a..4b039f1a939a 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java @@ -1075,6 +1075,7 @@ private void runProcess(){ // updateProcessInstance completed task status // failure priority is higher than pause // if a task fails, other suspended tasks need to be reset kill + // check if there exists forced success nodes in errorTaskList if(errorTaskList.size() > 0){ for(Map.Entry entry: completeTaskList.entrySet()) { TaskInstance completeTask = entry.getValue(); @@ -1084,6 +1085,23 @@ private void runProcess(){ processService.updateTaskInstance(completeTask); } } + + for(Map.Entry entry: errorTaskList.entrySet()) { + TaskInstance errorTask = entry.getValue(); + TaskInstance currentTask = processService.findTaskInstanceById(errorTask.getId()); + if (currentTask == null) { + continue; + } + // for nodes that have been forced success + if (errorTask.getState().typeIsFailure() && currentTask.getState().equals(ExecutionStatus.FORCED_SUCCESS)) { + // update state in this thread and remove from errorTaskList + errorTask.setState(currentTask.getState()); + logger.info("task: {} has been forced success, remove it from error task list", errorTask.getName()); + errorTaskList.remove(errorTask.getName()); + // submit post nodes + submitPostNode(errorTask.getName()); + } + } } if(canSubmitTaskToQueue()){ submitStandByTask(); @@ -1177,6 +1195,20 @@ private boolean retryTaskIntervalOverTime(TaskInstance taskInstance){ private void submitStandByTask(){ for(Map.Entry entry: readyToSubmitTaskList.entrySet()) { TaskInstance task = entry.getValue(); + // stop tasks which is retrying if forced success happens + if (task.taskCanRetry()) { + TaskInstance tmpTask = processService.findTaskInstanceById(task.getId()); + if (tmpTask != null) { + if (tmpTask.getState().equals(ExecutionStatus.FORCED_SUCCESS)) { + task.setState(tmpTask.getState()); + logger.info("task: {} has been forced success, put it into complete task list and stop retrying", task.getName()); + removeTaskFromStandbyList(task); + completeTaskList.put(task.getName(), task); + submitPostNode(task.getName()); + continue; + } + } + } DependResult dependResult = getDependResultForTask(task); if(DependResult.SUCCESS == dependResult){ if(retryTaskIntervalOverTime(task)){ From 90a060128fb7a30faac5d05b65e3744ab1655e51 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Fri, 7 Aug 2020 15:39:48 +0800 Subject: [PATCH 12/49] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E4=B8=8E=E6=96=B0?= =?UTF-8?q?=E5=A2=9EcommandType=E7=9B=B8=E5=85=B3=E7=9A=84=E4=BB=A3?= =?UTF-8?q?=E7=A0=81?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 1. dataAnalysis 2. alertManager 3. businessTimeUtils的schedule选择 --- .../dolphinscheduler/api/service/DataAnalysisService.java | 4 +++- .../common/utils/placeholder/BusinessTimeUtils.java | 1 + .../apache/dolphinscheduler/server/utils/AlertManager.java | 2 ++ 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataAnalysisService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataAnalysisService.java index 39bec56357bf..0375e9b69967 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataAnalysisService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataAnalysisService.java @@ -243,7 +243,8 @@ public Map countCommandState(User loginUser, int projectId, Stri // init data map /** * START_PROCESS, START_CURRENT_TASK_PROCESS, RECOVER_TOLERANCE_FAULT_PROCESS, RECOVER_SUSPENDED_PROCESS, - START_FAILURE_TASK_PROCESS,COMPLEMENT_DATA,SCHEDULER, REPEAT_RUNNING,PAUSE,STOP,RECOVER_WAITTING_THREAD; + START_FAILURE_TASK_PROCESS,COMPLEMENT_DATA,SCHEDULER, REPEAT_RUNNING,PAUSE,STOP,RECOVER_WAITTING_THREAD, + RESUME_FROM_FORCED_SUCCESS; */ dataMap.put(CommandType.START_PROCESS,commonCommand); dataMap.put(CommandType.START_CURRENT_TASK_PROCESS,commonCommand); @@ -256,6 +257,7 @@ public Map countCommandState(User loginUser, int projectId, Stri dataMap.put(CommandType.PAUSE,commonCommand); dataMap.put(CommandType.STOP,commonCommand); dataMap.put(CommandType.RECOVER_WAITTING_THREAD,commonCommand); + dataMap.put(CommandType.RESUME_FROM_FORCED_SUCCESS, commonCommand); // put command state for (CommandCount executeStatusCount : commandStateCounts){ diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/BusinessTimeUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/BusinessTimeUtils.java index 23db4b626b10..9350612f60f9 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/BusinessTimeUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/BusinessTimeUtils.java @@ -54,6 +54,7 @@ public static Map getBusinessTime(CommandType commandType, Date case RECOVER_SUSPENDED_PROCESS: case START_FAILURE_TASK_PROCESS: case REPEAT_RUNNING: + case RESUME_FROM_FORCED_SUCCESS: case SCHEDULER: default: businessDate = addDays(new Date(), -1); diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/AlertManager.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/AlertManager.java index 49ec9d3fdd21..015b1d0d3111 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/AlertManager.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/AlertManager.java @@ -81,6 +81,8 @@ private String getCommandCnName(CommandType commandType) { return "pause"; case STOP: return "stop"; + case RESUME_FROM_FORCED_SUCCESS: + return "resume from forced success"; default: return "unknown type"; } From c8fb89794255c9895133672de0b5aafd793e6e25 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Fri, 7 Aug 2020 16:35:57 +0800 Subject: [PATCH 13/49] =?UTF-8?q?=E6=B7=BB=E5=8A=A0dataAnalysis=E7=9A=84?= =?UTF-8?q?=E7=8A=B6=E6=80=81=E7=BB=9F=E8=AE=A1=E7=9B=B8=E5=85=B3=E7=9A=84?= =?UTF-8?q?=E9=83=A8=E5=88=86?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 1. taskStateCount中添加了强制成功的状态 2. processStateCount中剔除强制成功的状态 --- .../dolphinscheduler/api/dto/TaskCountDto.java | 14 ++++++++++++++ .../api/service/DataAnalysisService.java | 3 +++ 2 files changed, 17 insertions(+) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/dto/TaskCountDto.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/dto/TaskCountDto.java index 6b0391f111ef..a17896580a31 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/dto/TaskCountDto.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/dto/TaskCountDto.java @@ -54,6 +54,7 @@ private void countTaskDtos(List taskInstanceStateCounts){ int needFaultTolerance = 0; int kill = 0; int waittingThread = 0; + int forcedSuccess = 0; for(ExecuteStatusCount taskInstanceStateCount : taskInstanceStateCounts){ ExecutionStatus status = taskInstanceStateCount.getExecutionStatus(); @@ -92,6 +93,9 @@ private void countTaskDtos(List taskInstanceStateCounts){ case WAITTING_THREAD: waittingThread += taskInstanceStateCount.getCount(); break; + case FORCED_SUCCESS: + forcedSuccess += taskInstanceStateCount.getCount(); + break; default: break; @@ -109,8 +113,18 @@ private void countTaskDtos(List taskInstanceStateCounts){ this.taskCountDtos.add(new TaskStateCount(ExecutionStatus.NEED_FAULT_TOLERANCE, needFaultTolerance)); this.taskCountDtos.add(new TaskStateCount(ExecutionStatus.KILL, kill)); this.taskCountDtos.add(new TaskStateCount(ExecutionStatus.WAITTING_THREAD, waittingThread)); + this.taskCountDtos.add(new TaskStateCount(ExecutionStatus.FORCED_SUCCESS, forcedSuccess)); } + // remove the specified state + public void removeStateFromCountList(ExecutionStatus status) { + for(TaskStateCount count : this.taskCountDtos) { + if (count.getTaskStateType().equals(status)) { + this.taskCountDtos.remove(count); + break; + } + } + } public List getTaskCountDtos(){ return taskCountDtos; diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataAnalysisService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataAnalysisService.java index 0375e9b69967..691e7eec3f45 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataAnalysisService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataAnalysisService.java @@ -23,6 +23,7 @@ import org.apache.dolphinscheduler.api.enums.Status; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.CommandType; +import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.UserType; import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.StringUtils; @@ -153,6 +154,8 @@ public Map countProcessInstanceStateByProject(User loginUser, int if (processInstanceStateCounts != null) { TaskCountDto taskCountResult = new TaskCountDto(processInstanceStateCounts); + // process state count needs to remove state of forced success + taskCountResult.removeStateFromCountList(ExecutionStatus.FORCED_SUCCESS); result.put(Constants.DATA_LIST, taskCountResult); putMsg(result, Status.SUCCESS); } From 079768fdaaadd8b601c80afcbb2606afb15aba3b Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Fri, 7 Aug 2020 18:05:04 +0800 Subject: [PATCH 14/49] revert --- .../dolphinscheduler/dao/utils/DagHelper.java | 23 +++------------- .../dao/utils/DagHelperTest.java | 6 ++--- .../master/runner/MasterExecThread.java | 26 +++---------------- .../server/master/MasterCommandTest.java | 2 +- 4 files changed, 11 insertions(+), 46 deletions(-) diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java index 1901af318a77..a16c14ebc18d 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java @@ -263,7 +263,7 @@ public static TaskNode findNodeByName(List nodeDetails, String nodeNam * @return start Vertex list */ public static Collection getStartVertex(String parentNodeName, DAG dag, - Map completeTaskList, List allNodes){ + Map completeTaskList){ if(completeTaskList == null){ completeTaskList = new HashMap<>(); @@ -287,10 +287,10 @@ public static Collection getStartVertex(String parentNodeName, DAG postNodes = getStartVertex(start, dag, completeTaskList, allNodes); + Collection postNodes = getStartVertex(start, dag, completeTaskList); for(String post : postNodes){ TaskNode postNode = dag.getNode(post); - if(taskNodeCanSubmit(postNode, dag, completeTaskList, allNodes)){ + if(taskNodeCanSubmit(postNode, dag, completeTaskList)){ tmpStartVertexs.add(post); } } @@ -308,8 +308,7 @@ public static Collection getStartVertex(String parentNodeName, DAG dag, - Map completeTaskList, - List allNodes) { + Map completeTaskList) { List dependList = taskNode.getDepList(); if(dependList == null){ @@ -318,10 +317,6 @@ public static boolean taskNodeCanSubmit(TaskNode taskNode, for(String dependNodeName : dependList){ TaskNode dependNode = dag.getNode(dependNodeName); - // when executing resume_from_forced_success, depend node may be not in dag - if (dependNode == null) { - dependNode = findNodeByName(allNodes, dependNodeName); - } if(!dependNode.isForbidden() && !completeTaskList.containsKey(dependNodeName)){ return false; } @@ -330,16 +325,6 @@ public static boolean taskNodeCanSubmit(TaskNode taskNode, } - /** - * generate process data and return all task nodes - * @param processDefinitionJson process definition json - * @return task nodes list - */ - public static List getAllTaskNodesFromFlowJson(String processDefinitionJson) { - ProcessData processData = JSONUtils.parseObject(processDefinitionJson, ProcessData.class); - return processData.getTasks(); - } - /*** * build dag graph * @param processDag processDag diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/utils/DagHelperTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/utils/DagHelperTest.java index 0f1fe9234f97..14dfe0b75005 100644 --- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/utils/DagHelperTest.java +++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/utils/DagHelperTest.java @@ -58,21 +58,21 @@ public void testTaskNodeCanSubmit() throws JsonProcessingException { node2.setRunFlag(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN); TaskNode nodex = dag.getNode("4"); nodex.setRunFlag(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN); - canSubmit = DagHelper.taskNodeCanSubmit(taskNode3, dag, completeTaskList, null); + canSubmit = DagHelper.taskNodeCanSubmit(taskNode3, dag, completeTaskList); Assert.assertEquals(canSubmit, true); // 2forbidden, 3 cannot be submit completeTaskList.putIfAbsent("2", new TaskInstance()); TaskNode nodey = dag.getNode("4"); nodey.setRunFlag(""); - canSubmit = DagHelper.taskNodeCanSubmit(taskNode3, dag, completeTaskList, null); + canSubmit = DagHelper.taskNodeCanSubmit(taskNode3, dag, completeTaskList); Assert.assertEquals(canSubmit, false); // 2/3 forbidden submit 5 TaskNode node3 = dag.getNode("3"); node3.setRunFlag(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN); TaskNode node5 = dag.getNode("5"); - canSubmit = DagHelper.taskNodeCanSubmit(node5, dag, completeTaskList, null); + canSubmit = DagHelper.taskNodeCanSubmit(node5, dag, completeTaskList); Assert.assertEquals(canSubmit, true); } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java index 4b039f1a939a..a25ae10ad3bf 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java @@ -130,11 +130,6 @@ public class MasterExecThread implements Runnable { */ private DAG dag; - /** - * all task nodes generated by process instance json - */ - private List allNodes; - /** * process service */ @@ -369,8 +364,6 @@ private void buildFlowDag() throws Exception { } // generate process dag dag = DagHelper.buildDagGraph(processDag); - - allNodes = DagHelper.getAllTaskNodesFromFlowJson(processInstance.getProcessInstanceJson()); } /** @@ -540,7 +533,7 @@ private boolean isTaskNodeNeedSkip(TaskNode taskNode){ private void setTaskNodeSkip(List taskNodesSkipList){ for(String skipNode : taskNodesSkipList){ skipTaskNodeList.putIfAbsent(skipNode, dag.getNode(skipNode)); - Collection postNodeList = DagHelper.getStartVertex(skipNode, dag, completeTaskList, allNodes); + Collection postNodeList = DagHelper.getStartVertex(skipNode, dag, completeTaskList); List postSkipList = new ArrayList<>(); for(String post : postNodeList){ TaskNode postNode = dag.getNode(post); @@ -562,10 +555,6 @@ private void setTaskNodeSkip(List taskNodesSkipList){ private List parseConditionTask(String nodeName){ List conditionTaskList = new ArrayList<>(); TaskNode taskNode = dag.getNode(nodeName); - if (taskNode == null) { - taskNode = DagHelper.findNodeByName(allNodes, nodeName); - } - if(!taskNode.isConditionsTask()){ return conditionTaskList; } @@ -604,7 +593,7 @@ private List parsePostNodeList(String previousNodeName){ if(taskNode != null && taskNode.isConditionsTask()){ return parseConditionTask(previousNodeName); } - Collection postNodeCollection = DagHelper.getStartVertex(previousNodeName, dag, completeTaskList, allNodes); + Collection postNodeCollection = DagHelper.getStartVertex(previousNodeName, dag, completeTaskList); List postSkipList = new ArrayList<>(); // delete success node, parse the past nodes // if conditions node, @@ -683,18 +672,13 @@ private DependResult isTaskDepsComplete(String taskName) { TaskNode taskNode = dag.getNode(taskName); List depNameList = taskNode.getDepList(); for(String depsNode : depNameList ){ - // in this condition, it needs to check the command type - if((!dag.containsNode(depsNode) && !processInstance.getCommandType().equals(CommandType.RESUME_FROM_FORCED_SUCCESS)) + if(!dag.containsNode(depsNode) || forbiddenTaskList.containsKey(depsNode) || skipTaskNodeList.containsKey(depsNode)){ continue; } // all the dependencies must be fully completed if(!completeTaskList.containsKey(depsNode)){ - // if dag doesn't contain it either, the depend will never be executed, so return failed - if (!dag.containsNode(depsNode)) { - return DependResult.FAILED; - } return DependResult.WAITING; } ExecutionStatus depTaskState = completeTaskList.get(depsNode).getState(); @@ -725,10 +709,6 @@ private DependResult isTaskDepsComplete(String taskName) { */ private boolean dependTaskSuccess(String dependNodeName, String nextNodeName){ TaskNode tmpNode = dag.getNode(dependNodeName); - // if dag doesn't contain depend node - if (tmpNode == null) { - tmpNode = DagHelper.findNodeByName(allNodes, dependNodeName); - } if(tmpNode.isConditionsTask()){ //condition task need check the branch to run diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterCommandTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterCommandTest.java index c36521b8554a..d541f43a3b6c 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterCommandTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterCommandTest.java @@ -121,7 +121,7 @@ public void testDagHelper(){ new ArrayList<>(), new ArrayList<>(), TaskDependType.TASK_POST); DAG dag = DagHelper.buildDagGraph(processDag); - Collection start = DagHelper.getStartVertex("1", dag, null, null); + Collection start = DagHelper.getStartVertex("1", dag, null); System.out.println(start.toString()); From 815f62ad578ab8de75a7b2247ffad00588c2ed0b Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Fri, 7 Aug 2020 23:25:30 +0800 Subject: [PATCH 15/49] =?UTF-8?q?=E5=A2=9E=E5=8A=A0=E4=BA=86=E4=BF=AE?= =?UTF-8?q?=E6=94=B9task=E7=8A=B6=E6=80=81=E7=9A=84=E5=8D=95=E5=85=83?= =?UTF-8?q?=E6=B5=8B=E8=AF=95?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../api/service/TaskInstanceService.java | 2 -- .../TaskInstanceControllerTest.java | 20 +++++++++++-- .../api/service/TaskInstanceServiceTest.java | 29 ++++++++++++++++++- 3 files changed, 46 insertions(+), 5 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java index 2140af497f06..f5df9a5da265 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java @@ -167,7 +167,6 @@ public Map forceSingleTaskSuccess(User loginUser, String project // check whether the task instance state type is failure if (!task.getState().typeIsFailure()) { - // FIXME: 这个status的msg中是包含参数的,需要处理一下(已完善,待测试) putMsg(result, Status.TASK_INSTANCE_STATE_OPETATION_ERROR, taskInstanceId, task.getState().toString()); return result; } @@ -179,7 +178,6 @@ public Map forceSingleTaskSuccess(User loginUser, String project putMsg(result, Status.SUCCESS); } else { - // FIXME: 或许应该再加一个状态码,如果按照process的执行逻辑,那么是不需要的 putMsg(result, Status.FORCE_TASK_SUCCESS_ERROR); } diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java index 60da75b55494..37b2668d076e 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java @@ -28,7 +28,12 @@ import org.springframework.util.LinkedMultiValueMap; import org.springframework.util.MultiValueMap; +import java.util.HashMap; +import java.util.Map; + +import static org.mockito.Mockito.when; import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get; +import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post; import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content; import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status; @@ -64,7 +69,18 @@ public void testQueryTaskListPaging() throws Exception { } @Test - public void forceSingleTaskSuccess() { - // TODO: 加入测试 + public void forceSingleTaskSuccess() throws Exception { + MultiValueMap paramsMap = new LinkedMultiValueMap<>(); + paramsMap.add("taskInstanceId","104"); + + MvcResult mvcResult = mockMvc.perform(post("/projects/{projectName}/task-instance/force-success","test") + .header(SESSION_ID, sessionId) + .params(paramsMap)) + .andExpect(status().isOk()) + .andExpect(content().contentType(MediaType.APPLICATION_JSON_UTF8)) + .andReturn(); + + Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class); + Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue()); } } diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java index 0d2ffb3bb653..43667d65eccc 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java @@ -45,6 +45,7 @@ import java.util.*; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mockingDetails; import static org.mockito.Mockito.when; @RunWith(MockitoJUnitRunner.Silent.class) @@ -192,6 +193,32 @@ private void putMsg(Map result, Status status, Object... statusP @Test public void forceSingleTaskSuccess() { - // TODO: 加入测试 + User user = getAdminUser(); + String projectName = "test"; + Project project = getProject(projectName); + int taskId = 1; + TaskInstance task = getTaskInstance(); + + Map tmpResult = new HashMap<>(5); + putMsg(tmpResult, Status.SUCCESS); + when(projectMapper.queryByName("test")).thenReturn(project); + when(projectService.checkProjectAndAuth(user, project, projectName)).thenReturn(tmpResult); + + // test task not found + when(taskInstanceMapper.selectById(Mockito.anyInt())).thenReturn(null); + Map taskNotFoundRes = taskInstanceService.forceSingleTaskSuccess(user, projectName, taskId); + Assert.assertEquals(Status.TASK_INSTANCE_NOT_FOUND, taskNotFoundRes.get(Constants.STATUS)); + + // test task instance state error + task.setState(ExecutionStatus.SUCCESS); + when(taskInstanceMapper.selectById(1)).thenReturn(task); + Map taskStateErrorRes = taskInstanceService.forceSingleTaskSuccess(user, projectName, taskId); + Assert.assertEquals(Status.TASK_INSTANCE_STATE_OPETATION_ERROR, taskStateErrorRes.get(Constants.STATUS)); + + // test success + task.setState(ExecutionStatus.FAILURE); + when(taskInstanceMapper.updateById(task)).thenReturn(1); + Map successRes = taskInstanceService.forceSingleTaskSuccess(user, projectName, taskId); + Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS)); } } \ No newline at end of file From b418b681e2e067ba5e907bb8b877e4462351bd69 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Sat, 22 Aug 2020 16:02:50 +0800 Subject: [PATCH 16/49] add taskInstanceMapperTest --- .../dao/mapper/TaskInstanceMapperTest.java | 82 +++++++++++++++++-- 1 file changed, 75 insertions(+), 7 deletions(-) diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapperTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapperTest.java index b224067a296c..b6351d99891a 100644 --- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapperTest.java +++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapperTest.java @@ -19,23 +19,23 @@ import com.baomidou.mybatisplus.core.metadata.IPage; import com.baomidou.mybatisplus.extension.plugins.pagination.Page; +import javafx.concurrent.Task; import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.Flag; import org.apache.dolphinscheduler.common.enums.TaskType; -import org.apache.dolphinscheduler.dao.entity.ExecuteStatusCount; -import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.dao.entity.*; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; +import org.mockito.internal.junit.ExceptionFactory; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.boot.test.context.SpringBootTest; import org.springframework.test.annotation.Rollback; import org.springframework.test.context.junit4.SpringRunner; import org.springframework.transaction.annotation.Transactional; +import java.util.ArrayList; import java.util.Date; import java.util.List; @@ -55,20 +55,39 @@ public class TaskInstanceMapperTest { @Autowired ProcessInstanceMapper processInstanceMapper; + @Autowired + ProcessInstanceMapMapper processInstanceMapMapper; + /** * insert * @return TaskInstance */ private TaskInstance insertOne(){ //insertOne + return insertOne("us task", 1, ExecutionStatus.RUNNING_EXEUTION, TaskType.SHELL.toString()); + } + + /** + * construct a task instance and then insert + * @param taskName + * @param processInstanceId + * @param state + * @param taskType + * @return + */ + private TaskInstance insertOne(String taskName, + int processInstanceId, + ExecutionStatus state, + String taskType) { TaskInstance taskInstance = new TaskInstance(); taskInstance.setFlag(Flag.YES); - taskInstance.setName("ut task"); - taskInstance.setState(ExecutionStatus.RUNNING_EXEUTION); + taskInstance.setName(taskName); + taskInstance.setState(state); taskInstance.setStartTime(new Date()); taskInstance.setEndTime(new Date()); taskInstance.setTaskJson("{}"); - taskInstance.setTaskType(TaskType.SHELL.toString()); + taskInstance.setProcessInstanceId(processInstanceId); + taskInstance.setTaskType(taskType); taskInstanceMapper.insert(taskInstance); return taskInstance; } @@ -297,4 +316,53 @@ public void testQueryTaskInstanceListPaging() { Assert.assertNotEquals(taskInstanceIPage.getTotal(), 0); } + + @Test + public void testQueryTaskByPIdAndStatusAndType() { + // insert three task instances with the same process instance id + List taskList = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + String name = "ut task" + String.valueOf(i); + taskList.add(insertOne(name, 66, ExecutionStatus.FAILURE, TaskType.SUB_PROCESS.toString())); + } + + // test query result + List resultArray = taskInstanceMapper.queryTaskByPIdAndStatusAndType(66, + new int[]{ExecutionStatus.FAILURE.ordinal(), ExecutionStatus.KILL.ordinal(), ExecutionStatus.NEED_FAULT_TOLERANCE.ordinal()}, + TaskType.SUB_PROCESS.toString()); + Assert.assertEquals(3, resultArray.size()); + + // delete + for (int i = 0; i < 3; i++) { + taskInstanceMapper.deleteById(taskList.get(i)); + } + } + + @Test + public void testQueryTaskBySubProcessTaskIdAndStatusAndType() { + TaskInstance parentTask = insertOne("parent-task",66, ExecutionStatus.FAILURE, TaskType.SUB_PROCESS.toString()); + + ProcessInstanceMap processInstanceMap = new ProcessInstanceMap(); + processInstanceMap.setParentProcessInstanceId(66); + processInstanceMap.setParentTaskInstanceId(parentTask.getId()); + processInstanceMap.setProcessInstanceId(67); + processInstanceMapMapper.insert(processInstanceMap); + + TaskInstance subTask1 = insertOne("sub1", 67, ExecutionStatus.SUCCESS, TaskType.SHELL.toString()); + TaskInstance subTask2 = insertOne("sub2", 67, ExecutionStatus.FORCED_SUCCESS, TaskType.SHELL.toString()); + + // test query result + List resultList = taskInstanceMapper.queryTaskBySubProcessTaskIdAndStatusAndType(parentTask.getId(), + new int[]{ExecutionStatus.FORCED_SUCCESS.ordinal()}, + null); + + Assert.assertEquals(1, resultList.size()); + Assert.assertEquals(subTask2.getId(), resultList.get(0).intValue()); + + // delete + taskInstanceMapper.deleteById(parentTask.getId()); + processInstanceMapMapper.deleteById(processInstanceMap.getId()); + taskInstanceMapper.deleteById(subTask1.getId()); + taskInstanceMapper.deleteById(subTask2.getId()); + } } \ No newline at end of file From 1f19cf2b50da301bad863fc902545558ce438c99 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Sat, 22 Aug 2020 18:48:15 +0800 Subject: [PATCH 17/49] update test of TaskInstanceController --- .../api/controller/TaskInstanceControllerTest.java | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java index 37b2668d076e..1f1cb9a0fb5a 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java @@ -17,12 +17,16 @@ package org.apache.dolphinscheduler.api.controller; import org.apache.dolphinscheduler.api.enums.Status; +import org.apache.dolphinscheduler.api.service.TaskInstanceService; import org.apache.dolphinscheduler.api.utils.Result; +import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.*; +import org.apache.dolphinscheduler.dao.entity.User; import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.springframework.boot.test.mock.mockito.MockBean; import org.springframework.http.MediaType; import org.springframework.test.web.servlet.MvcResult; import org.springframework.util.LinkedMultiValueMap; @@ -31,6 +35,7 @@ import java.util.HashMap; import java.util.Map; +import static org.mockito.ArgumentMatchers.*; import static org.mockito.Mockito.when; import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get; import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post; @@ -43,6 +48,9 @@ public class TaskInstanceControllerTest extends AbstractControllerTest{ private static Logger logger = LoggerFactory.getLogger(TaskInstanceControllerTest.class); + @MockBean + private TaskInstanceService taskInstanceService; + @Test public void testQueryTaskListPaging() throws Exception { @@ -73,6 +81,11 @@ public void forceSingleTaskSuccess() throws Exception { MultiValueMap paramsMap = new LinkedMultiValueMap<>(); paramsMap.add("taskInstanceId","104"); + Map mockResult = new HashMap<>(5); + mockResult.put(Constants.STATUS, Status.SUCCESS); + mockResult.put(Constants.MSG, Status.SUCCESS.getMsg()); + when(taskInstanceService.forceSingleTaskSuccess(any(User.class), anyString(), anyInt())).thenReturn(mockResult); + MvcResult mvcResult = mockMvc.perform(post("/projects/{projectName}/task-instance/force-success","test") .header(SESSION_ID, sessionId) .params(paramsMap)) From 295796a5c55657391c6efed2ac4ebd66c475c87e Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Sat, 22 Aug 2020 22:47:59 +0800 Subject: [PATCH 18/49] fix --- .../api/service/impl/DataAnalysisServiceImpl.java | 7 ++++++- .../dolphinscheduler/common/enums/ExecutionStatus.java | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/DataAnalysisServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/DataAnalysisServiceImpl.java index 21313b96d39b..e2cdec3affed 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/DataAnalysisServiceImpl.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/DataAnalysisServiceImpl.java @@ -26,6 +26,7 @@ import org.apache.dolphinscheduler.api.service.ProjectService; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.CommandType; +import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.UserType; import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.StringUtils; @@ -153,6 +154,8 @@ private Map countStateByProject(User loginUser, int projectId, S if (processInstanceStateCounts != null) { TaskCountDto taskCountResult = new TaskCountDto(processInstanceStateCounts); + // process state count needs to remove state of forced success + taskCountResult.removeStateFromCountList(ExecutionStatus.FORCED_SUCCESS); result.put(Constants.DATA_LIST, taskCountResult); putMsg(result, Status.SUCCESS); } @@ -243,7 +246,8 @@ public Map countCommandState(User loginUser, int projectId, Stri // init data map /** * START_PROCESS, START_CURRENT_TASK_PROCESS, RECOVER_TOLERANCE_FAULT_PROCESS, RECOVER_SUSPENDED_PROCESS, - START_FAILURE_TASK_PROCESS,COMPLEMENT_DATA,SCHEDULER, REPEAT_RUNNING,PAUSE,STOP,RECOVER_WAITTING_THREAD; + START_FAILURE_TASK_PROCESS,COMPLEMENT_DATA,SCHEDULER, REPEAT_RUNNING,PAUSE,STOP,RECOVER_WAITTING_THREAD, + RESUME_FROM_FORCED_SUCCESS; */ dataMap.put(CommandType.START_PROCESS, commonCommand); dataMap.put(CommandType.START_CURRENT_TASK_PROCESS, commonCommand); @@ -256,6 +260,7 @@ public Map countCommandState(User loginUser, int projectId, Stri dataMap.put(CommandType.PAUSE, commonCommand); dataMap.put(CommandType.STOP, commonCommand); dataMap.put(CommandType.RECOVER_WAITTING_THREAD, commonCommand); + dataMap.put(CommandType.RESUME_FROM_FORCED_SUCCESS, commonCommand); // put command state for (CommandCount executeStatusCount : commandStateCounts) { diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/ExecutionStatus.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/ExecutionStatus.java index 3dfaed3dc86a..a9ee584f2b2e 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/ExecutionStatus.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/ExecutionStatus.java @@ -55,7 +55,7 @@ public enum ExecutionStatus { KILL(9, "kill"), WAITTING_THREAD(10, "waiting thread"), WAITTING_DEPEND(11, "waiting depend node complete"), - DELAY_EXECUTION(12, "delay execution"), + DELAY_EXECUTION(12, "delay execution"), FORCED_SUCCESS(13, "forced success"); ExecutionStatus(int code, String descp) { From ae9eb160c109c8e67eaf4a7b9156f3506cfe1d54 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Sun, 23 Aug 2020 00:41:03 +0800 Subject: [PATCH 19/49] =?UTF-8?q?=E8=A7=A3=E5=86=B3=E6=9B=B4=E6=96=B0?= =?UTF-8?q?=E5=90=8E=E7=9A=84=E9=80=BB=E8=BE=91=E5=86=B2=E7=AA=81?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../api/service/impl/DataAnalysisServiceImpl.java | 9 ++++++--- .../dao/mapper/TaskInstanceMapperTest.java | 2 +- .../dolphinscheduler/service/process/ProcessService.java | 1 + 3 files changed, 8 insertions(+), 4 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/DataAnalysisServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/DataAnalysisServiceImpl.java index e2cdec3affed..ee75837da7f4 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/DataAnalysisServiceImpl.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/DataAnalysisServiceImpl.java @@ -122,12 +122,17 @@ public Map countTaskStateByProject(User loginUser, int projectId * @return process instance state count data */ public Map countProcessInstanceStateByProject(User loginUser, int projectId, String startDate, String endDate) { - return this.countStateByProject( + Map tmpResult = this.countStateByProject( loginUser, projectId, startDate, endDate, (start, end, projectIds) -> this.processInstanceMapper.countInstanceStateByUser(start, end, projectIds)); + // process state count needs to remove state of forced success + if (tmpResult.get(Constants.STATUS).equals(Status.SUCCESS)) { + ((TaskCountDto)tmpResult.get(Constants.DATA_LIST)).removeStateFromCountList(ExecutionStatus.FORCED_SUCCESS); + } + return tmpResult; } private Map countStateByProject(User loginUser, int projectId, String startDate, String endDate @@ -154,8 +159,6 @@ private Map countStateByProject(User loginUser, int projectId, S if (processInstanceStateCounts != null) { TaskCountDto taskCountResult = new TaskCountDto(processInstanceStateCounts); - // process state count needs to remove state of forced success - taskCountResult.removeStateFromCountList(ExecutionStatus.FORCED_SUCCESS); result.put(Constants.DATA_LIST, taskCountResult); putMsg(result, Status.SUCCESS); } diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapperTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapperTest.java index 4a71065b2efd..40b59605bf97 100644 --- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapperTest.java +++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapperTest.java @@ -64,7 +64,7 @@ public class TaskInstanceMapperTest { */ private TaskInstance insertOne(){ //insertOne - return insertOne("us task", 1, ExecutionStatus.RUNNING_EXEUTION, TaskType.SHELL.toString()); + return insertOne("us task", 1, ExecutionStatus.RUNNING_EXECUTION, TaskType.SHELL.toString()); } /** diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java index 7f9e6299b57e..48ab4aa6a1b9 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java @@ -36,6 +36,7 @@ import org.apache.dolphinscheduler.common.enums.Flag; import org.apache.dolphinscheduler.common.enums.ResourceType; import org.apache.dolphinscheduler.common.enums.TaskDependType; +import org.apache.dolphinscheduler.common.enums.TaskType; import org.apache.dolphinscheduler.common.enums.WarningType; import org.apache.dolphinscheduler.common.model.DateInterval; import org.apache.dolphinscheduler.common.model.TaskNode; From 1fc1218a34f2177bf92befd6c6338c955e65aa7e Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Sun, 23 Aug 2020 10:29:44 +0800 Subject: [PATCH 20/49] fix import problems --- .../dao/mapper/TaskInstanceMapperTest.java | 95 ++++++++++--------- 1 file changed, 51 insertions(+), 44 deletions(-) diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapperTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapperTest.java index 40b59605bf97..8673ffd45f02 100644 --- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapperTest.java +++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapperTest.java @@ -14,30 +14,35 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.dolphinscheduler.dao.mapper; -import com.baomidou.mybatisplus.core.metadata.IPage; -import com.baomidou.mybatisplus.extension.plugins.pagination.Page; -import javafx.concurrent.Task; import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.Flag; import org.apache.dolphinscheduler.common.enums.TaskType; -import org.apache.dolphinscheduler.dao.entity.*; +import org.apache.dolphinscheduler.dao.entity.ExecuteStatusCount; +import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.ProcessInstanceMap; +import org.apache.dolphinscheduler.dao.entity.TaskInstance; + +import java.util.ArrayList; +import java.util.Date; +import java.util.List; + import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; -import org.mockito.internal.junit.ExceptionFactory; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.boot.test.context.SpringBootTest; import org.springframework.test.annotation.Rollback; import org.springframework.test.context.junit4.SpringRunner; import org.springframework.transaction.annotation.Transactional; -import java.util.ArrayList; -import java.util.Date; -import java.util.List; +import com.baomidou.mybatisplus.core.metadata.IPage; +import com.baomidou.mybatisplus.extension.plugins.pagination.Page; @RunWith(SpringRunner.class) @SpringBootTest @@ -60,15 +65,17 @@ public class TaskInstanceMapperTest { /** * insert + * * @return TaskInstance */ - private TaskInstance insertOne(){ + private TaskInstance insertOne() { //insertOne return insertOne("us task", 1, ExecutionStatus.RUNNING_EXECUTION, TaskType.SHELL.toString()); } /** * construct a task instance and then insert + * * @param taskName * @param processInstanceId * @param state @@ -96,7 +103,7 @@ private TaskInstance insertOne(String taskName, * test update */ @Test - public void testUpdate(){ + public void testUpdate() { //insertOne TaskInstance taskInstance = insertOne(); //update @@ -109,7 +116,7 @@ public void testUpdate(){ * test delete */ @Test - public void testDelete(){ + public void testDelete() { TaskInstance taskInstance = insertOne(); int delete = taskInstanceMapper.deleteById(taskInstance.getId()); Assert.assertEquals(1, delete); @@ -136,8 +143,8 @@ public void testQueryTaskByProcessIdAndState() { task.setProcessInstanceId(110); taskInstanceMapper.updateById(task); List taskInstances = taskInstanceMapper.queryTaskByProcessIdAndState( - task.getProcessInstanceId(), - ExecutionStatus.RUNNING_EXECUTION.ordinal() + task.getProcessInstanceId(), + ExecutionStatus.RUNNING_EXECUTION.ordinal() ); taskInstanceMapper.deleteById(task.getId()); Assert.assertNotEquals(taskInstances.size(), 0); @@ -156,19 +163,19 @@ public void testFindValidTaskListByProcessId() { taskInstanceMapper.updateById(task2); List taskInstances = taskInstanceMapper.findValidTaskListByProcessId( - task.getProcessInstanceId(), - Flag.YES + task.getProcessInstanceId(), + Flag.YES ); task2.setFlag(Flag.NO); taskInstanceMapper.updateById(task2); List taskInstances1 = taskInstanceMapper.findValidTaskListByProcessId(task.getProcessInstanceId(), - Flag.NO); + Flag.NO); taskInstanceMapper.deleteById(task2.getId()); taskInstanceMapper.deleteById(task.getId()); Assert.assertNotEquals(taskInstances.size(), 0); - Assert.assertNotEquals(taskInstances1.size(), 0 ); + Assert.assertNotEquals(taskInstances1.size(), 0); } /** @@ -181,7 +188,7 @@ public void testQueryByHostAndStatus() { taskInstanceMapper.updateById(task); List taskInstances = taskInstanceMapper.queryByHostAndStatus( - task.getHost(), new int[]{ExecutionStatus.RUNNING_EXECUTION.ordinal()} + task.getHost(), new int[] {ExecutionStatus.RUNNING_EXECUTION.ordinal()} ); taskInstanceMapper.deleteById(task.getId()); Assert.assertNotEquals(taskInstances.size(), 0); @@ -197,9 +204,9 @@ public void testSetFailoverByHostAndStateArray() { taskInstanceMapper.updateById(task); int setResult = taskInstanceMapper.setFailoverByHostAndStateArray( - task.getHost(), - new int[]{ExecutionStatus.RUNNING_EXECUTION.ordinal()}, - ExecutionStatus.NEED_FAULT_TOLERANCE + task.getHost(), + new int[] {ExecutionStatus.RUNNING_EXECUTION.ordinal()}, + ExecutionStatus.NEED_FAULT_TOLERANCE ); taskInstanceMapper.deleteById(task.getId()); Assert.assertNotEquals(setResult, 0); @@ -215,8 +222,8 @@ public void testQueryByInstanceIdAndName() { taskInstanceMapper.updateById(task); TaskInstance taskInstance = taskInstanceMapper.queryByInstanceIdAndName( - task.getProcessInstanceId(), - task.getName() + task.getProcessInstanceId(), + task.getName() ); taskInstanceMapper.deleteById(task.getId()); Assert.assertNotEquals(taskInstance, null); @@ -236,12 +243,12 @@ public void testCountTask() { taskInstanceMapper.updateById(task); int countTask = taskInstanceMapper.countTask( - new Integer[0], - new int[0] + new Integer[0], + new int[0] ); int countTask2 = taskInstanceMapper.countTask( - new Integer[]{definition.getProjectId()}, - new int[]{task.getId()} + new Integer[] {definition.getProjectId()}, + new int[] {task.getId()} ); taskInstanceMapper.deleteById(task.getId()); processDefinitionMapper.deleteById(definition.getId()); @@ -266,8 +273,8 @@ public void testCountTaskInstanceStateByUser() { List count = taskInstanceMapper.countTaskInstanceStateByUser( - null, null, - new Integer[]{definition.getProjectId()} + null, null, + new Integer[] {definition.getProjectId()} ); processDefinitionMapper.deleteById(definition.getId()); @@ -298,17 +305,17 @@ public void testQueryTaskInstanceListPaging() { task.setProcessInstanceId(processInstance.getId()); taskInstanceMapper.updateById(task); - Page page = new Page(1,3); + Page page = new Page(1, 3); IPage taskInstanceIPage = taskInstanceMapper.queryTaskInstanceListPaging( - page, - definition.getProjectId(), - task.getProcessInstanceId(), - "", - "", - 0, - new int[0], - "", - null,null + page, + definition.getProjectId(), + task.getProcessInstanceId(), + "", + "", + 0, + new int[0], + "", + null, null ); processInstanceMapper.deleteById(processInstance.getId()); taskInstanceMapper.deleteById(task.getId()); @@ -328,8 +335,8 @@ public void testQueryTaskByPIdAndStatusAndType() { // test query result List resultArray = taskInstanceMapper.queryTaskByPIdAndStatusAndType(66, - new int[]{ExecutionStatus.FAILURE.ordinal(), ExecutionStatus.KILL.ordinal(), ExecutionStatus.NEED_FAULT_TOLERANCE.ordinal()}, - TaskType.SUB_PROCESS.toString()); + new int[] {ExecutionStatus.FAILURE.ordinal(), ExecutionStatus.KILL.ordinal(), ExecutionStatus.NEED_FAULT_TOLERANCE.ordinal()}, + TaskType.SUB_PROCESS.toString()); Assert.assertEquals(3, resultArray.size()); // delete @@ -340,7 +347,7 @@ public void testQueryTaskByPIdAndStatusAndType() { @Test public void testQueryTaskBySubProcessTaskIdAndStatusAndType() { - TaskInstance parentTask = insertOne("parent-task",66, ExecutionStatus.FAILURE, TaskType.SUB_PROCESS.toString()); + TaskInstance parentTask = insertOne("parent-task", 66, ExecutionStatus.FAILURE, TaskType.SUB_PROCESS.toString()); ProcessInstanceMap processInstanceMap = new ProcessInstanceMap(); processInstanceMap.setParentProcessInstanceId(66); @@ -353,8 +360,8 @@ public void testQueryTaskBySubProcessTaskIdAndStatusAndType() { // test query result List resultList = taskInstanceMapper.queryTaskBySubProcessTaskIdAndStatusAndType(parentTask.getId(), - new int[]{ExecutionStatus.FORCED_SUCCESS.ordinal()}, - null); + new int[] {ExecutionStatus.FORCED_SUCCESS.ordinal()}, + null); Assert.assertEquals(1, resultList.size()); Assert.assertEquals(subTask2.getId(), resultList.get(0).intValue()); From 640774595dfdf111acdaf9bb69c007f9317ffde3 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Sun, 23 Aug 2020 10:59:04 +0800 Subject: [PATCH 21/49] Update DataAnalysisServiceImpl.java --- .../api/service/impl/DataAnalysisServiceImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/DataAnalysisServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/DataAnalysisServiceImpl.java index ee75837da7f4..7527a1a5366b 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/DataAnalysisServiceImpl.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/DataAnalysisServiceImpl.java @@ -129,7 +129,7 @@ public Map countProcessInstanceStateByProject(User loginUser, in endDate, (start, end, projectIds) -> this.processInstanceMapper.countInstanceStateByUser(start, end, projectIds)); // process state count needs to remove state of forced success - if (tmpResult.get(Constants.STATUS).equals(Status.SUCCESS)) { + if (tmpResult.containsKey(Constants.STATUS) && tmpResult.get(Constants.STATUS).equals(Status.SUCCESS)) { ((TaskCountDto)tmpResult.get(Constants.DATA_LIST)).removeStateFromCountList(ExecutionStatus.FORCED_SUCCESS); } return tmpResult; From 31cd1f6cb2b47a94e87635e36def34fb314b065d Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Sun, 23 Aug 2020 15:14:18 +0800 Subject: [PATCH 22/49] adapt to code style --- .../controller/TaskInstanceController.java | 27 ++++-- .../api/dto/TaskCountDto.java | 4 +- .../api/service/ExecutorService.java | 41 +++++++-- .../TaskInstanceControllerTest.java | 26 +++--- .../api/service/TaskInstanceServiceTest.java | 7 +- .../common/enums/ExecutionStatus.java | 53 +++++------ .../utils/placeholder/BusinessTimeUtils.java | 90 ++++++++++--------- .../dolphinscheduler/dao/utils/DagHelper.java | 17 ++-- .../master/runner/MasterExecThread.java | 39 ++++---- .../service/process/ProcessService.java | 14 ++- 10 files changed, 180 insertions(+), 138 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java index a4a28a555d3b..71ada4fd732e 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java @@ -14,8 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.dolphinscheduler.api.controller; +import static org.apache.dolphinscheduler.api.enums.Status.FORCE_TASK_SUCCESS_ERROR; +import static org.apache.dolphinscheduler.api.enums.Status.QUERY_TASK_LIST_PAGING_ERROR; import org.apache.dolphinscheduler.api.exceptions.ApiException; import org.apache.dolphinscheduler.api.service.TaskInstanceService; @@ -24,18 +27,28 @@ import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.utils.ParameterUtils; import org.apache.dolphinscheduler.dao.entity.User; -import io.swagger.annotations.*; + +import java.util.Map; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.http.HttpStatus; -import org.springframework.web.bind.annotation.*; -import springfox.documentation.annotations.ApiIgnore; - -import java.util.Map; +import org.springframework.web.bind.annotation.GetMapping; +import org.springframework.web.bind.annotation.PathVariable; +import org.springframework.web.bind.annotation.PostMapping; +import org.springframework.web.bind.annotation.RequestAttribute; +import org.springframework.web.bind.annotation.RequestMapping; +import org.springframework.web.bind.annotation.RequestParam; +import org.springframework.web.bind.annotation.ResponseStatus; +import org.springframework.web.bind.annotation.RestController; -import static org.apache.dolphinscheduler.api.enums.Status.FORCE_TASK_SUCCESS_ERROR; -import static org.apache.dolphinscheduler.api.enums.Status.QUERY_TASK_LIST_PAGING_ERROR; +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiImplicitParam; +import io.swagger.annotations.ApiImplicitParams; +import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiParam; +import springfox.documentation.annotations.ApiIgnore; /** * task instance controller diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/dto/TaskCountDto.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/dto/TaskCountDto.java index 2340fc366dde..5ad87be07618 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/dto/TaskCountDto.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/dto/TaskCountDto.java @@ -123,7 +123,7 @@ private void countTaskDtos(List taskInstanceStateCounts) { // remove the specified state public void removeStateFromCountList(ExecutionStatus status) { - for(TaskStateCount count : this.taskCountDtos) { + for (TaskStateCount count : this.taskCountDtos) { if (count.getTaskStateType().equals(status)) { this.taskCountDtos.remove(count); break; @@ -131,7 +131,7 @@ public void removeStateFromCountList(ExecutionStatus status) { } } - public List getTaskCountDtos(){ + public List getTaskCountDtos() { return taskCountDtos; } diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java index 07ccd30a2236..c44a094b1ab0 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java @@ -14,35 +14,58 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.dolphinscheduler.api.service; +import static org.apache.dolphinscheduler.common.Constants.CMDPARAM_COMPLEMENT_DATA_END_DATE; +import static org.apache.dolphinscheduler.common.Constants.CMDPARAM_COMPLEMENT_DATA_START_DATE; +import static org.apache.dolphinscheduler.common.Constants.CMDPARAM_RECOVER_PROCESS_ID_STRING; +import static org.apache.dolphinscheduler.common.Constants.CMDPARAM_START_NODE_NAMES; +import static org.apache.dolphinscheduler.common.Constants.MAX_TASK_TIMEOUT; -import com.sun.org.apache.bcel.internal.generic.BREAKPOINT; import org.apache.dolphinscheduler.api.enums.ExecuteType; import org.apache.dolphinscheduler.api.enums.Status; import org.apache.dolphinscheduler.common.Constants; -import org.apache.dolphinscheduler.common.enums.*; +import org.apache.dolphinscheduler.common.enums.CommandType; +import org.apache.dolphinscheduler.common.enums.ExecutionStatus; +import org.apache.dolphinscheduler.common.enums.FailureStrategy; +import org.apache.dolphinscheduler.common.enums.Priority; +import org.apache.dolphinscheduler.common.enums.ReleaseState; +import org.apache.dolphinscheduler.common.enums.RunMode; +import org.apache.dolphinscheduler.common.enums.TaskDependType; +import org.apache.dolphinscheduler.common.enums.TaskType; +import org.apache.dolphinscheduler.common.enums.WarningType; import org.apache.dolphinscheduler.common.model.Server; import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.DateUtils; -import org.apache.dolphinscheduler.common.utils.*; +import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.StringUtils; -import org.apache.dolphinscheduler.dao.entity.*; +import org.apache.dolphinscheduler.dao.entity.Command; +import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.Project; +import org.apache.dolphinscheduler.dao.entity.Schedule; +import org.apache.dolphinscheduler.dao.entity.Tenant; +import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper; import org.apache.dolphinscheduler.dao.mapper.ProcessInstanceMapper; import org.apache.dolphinscheduler.dao.mapper.ProjectMapper; import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.quartz.cron.CronUtils; + +import java.text.ParseException; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Service; -import java.text.ParseException; -import java.util.*; - -import static org.apache.dolphinscheduler.common.Constants.*; - /** * executor service */ diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java index 1f1cb9a0fb5a..abaaa3ffd23a 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java @@ -14,14 +14,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.dolphinscheduler.api.controller; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.when; +import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get; +import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post; +import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content; +import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status; + import org.apache.dolphinscheduler.api.enums.Status; import org.apache.dolphinscheduler.api.service.TaskInstanceService; import org.apache.dolphinscheduler.api.utils.Result; import org.apache.dolphinscheduler.common.Constants; -import org.apache.dolphinscheduler.common.utils.*; +import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.dao.entity.User; + +import java.util.HashMap; +import java.util.Map; + import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; @@ -32,16 +46,6 @@ import org.springframework.util.LinkedMultiValueMap; import org.springframework.util.MultiValueMap; -import java.util.HashMap; -import java.util.Map; - -import static org.mockito.ArgumentMatchers.*; -import static org.mockito.Mockito.when; -import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get; -import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post; -import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content; -import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status; - /** * task instance controller test */ diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java index 2c3446e02fd6..f04fe5ccaa35 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.dolphinscheduler.api.service; import static org.mockito.ArgumentMatchers.eq; @@ -52,12 +53,6 @@ import org.slf4j.LoggerFactory; import org.springframework.boot.test.context.SpringBootTest; -import java.text.MessageFormat; -import java.util.*; - -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.mockingDetails; -import static org.mockito.Mockito.when; import com.baomidou.mybatisplus.extension.plugins.pagination.Page; @RunWith(MockitoJUnitRunner.Silent.class) diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/ExecutionStatus.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/ExecutionStatus.java index a9ee584f2b2e..b9eaee6c28c6 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/ExecutionStatus.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/ExecutionStatus.java @@ -75,31 +75,34 @@ public enum ExecutionStatus { } } - /** - * status is success - * @return status - */ - public boolean typeIsSuccess() { - return this == SUCCESS || this == FORCED_SUCCESS; - } - - /** - * status is failure - * @return status - */ - public boolean typeIsFailure() { - return this == FAILURE || this == NEED_FAULT_TOLERANCE || this == KILL; - } - - /** - * status is finished - * @return status - */ - public boolean typeIsFinished() { - - return typeIsSuccess() || typeIsFailure() || typeIsCancel() || typeIsPause() - || typeIsStop(); - } + /** + * status is success + * + * @return status + */ + public boolean typeIsSuccess() { + return this == SUCCESS || this == FORCED_SUCCESS; + } + + /** + * status is failure + * + * @return status + */ + public boolean typeIsFailure() { + return this == FAILURE || this == NEED_FAULT_TOLERANCE || this == KILL; + } + + /** + * status is finished + * + * @return status + */ + public boolean typeIsFinished() { + + return typeIsSuccess() || typeIsFailure() || typeIsCancel() || typeIsPause() + || typeIsStop(); + } /** * status is waiting thread diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/BusinessTimeUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/BusinessTimeUtils.java index 9350612f60f9..8c55e8521a5c 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/BusinessTimeUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/BusinessTimeUtils.java @@ -14,8 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.dolphinscheduler.common.utils.placeholder; +import static org.apache.commons.lang.time.DateUtils.addDays; +import static org.apache.dolphinscheduler.common.Constants.PARAMETER_FORMAT_DATE; +import static org.apache.dolphinscheduler.common.Constants.PARAMETER_FORMAT_TIME; +import static org.apache.dolphinscheduler.common.utils.DateUtils.format; + import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.CommandType; @@ -23,54 +29,50 @@ import java.util.HashMap; import java.util.Map; -import static org.apache.dolphinscheduler.common.Constants.PARAMETER_FORMAT_DATE; -import static org.apache.dolphinscheduler.common.Constants.PARAMETER_FORMAT_TIME; -import static org.apache.dolphinscheduler.common.utils.DateUtils.format; -import static org.apache.commons.lang.time.DateUtils.addDays; - /** * business time utils */ public class BusinessTimeUtils { - private BusinessTimeUtils() { - throw new IllegalStateException("BusinessTimeUtils class"); - } - /** - * get business time in parameters by different command types - * - * @param commandType command type - * @param runTime run time or schedule time - * @return business time - */ - public static Map getBusinessTime(CommandType commandType, Date runTime) { - Date businessDate = runTime; - switch (commandType) { - case COMPLEMENT_DATA: - break; - case START_PROCESS: - case START_CURRENT_TASK_PROCESS: - case RECOVER_TOLERANCE_FAULT_PROCESS: - case RECOVER_SUSPENDED_PROCESS: - case START_FAILURE_TASK_PROCESS: - case REPEAT_RUNNING: - case RESUME_FROM_FORCED_SUCCESS: - case SCHEDULER: - default: - businessDate = addDays(new Date(), -1); - if (runTime != null){ - /** - * If there is a scheduled time, take the scheduling time. Recovery from failed nodes, suspension of recovery, re-run for scheduling - */ - businessDate = addDays(runTime, -1); - } - break; + private BusinessTimeUtils() { + throw new IllegalStateException("BusinessTimeUtils class"); + } + + /** + * get business time in parameters by different command types + * + * @param commandType command type + * @param runTime run time or schedule time + * @return business time + */ + public static Map getBusinessTime(CommandType commandType, Date runTime) { + Date businessDate = runTime; + switch (commandType) { + case COMPLEMENT_DATA: + break; + case START_PROCESS: + case START_CURRENT_TASK_PROCESS: + case RECOVER_TOLERANCE_FAULT_PROCESS: + case RECOVER_SUSPENDED_PROCESS: + case START_FAILURE_TASK_PROCESS: + case REPEAT_RUNNING: + case RESUME_FROM_FORCED_SUCCESS: + case SCHEDULER: + default: + businessDate = addDays(new Date(), -1); + if (runTime != null) { + /** + * If there is a scheduled time, take the scheduling time. Recovery from failed nodes, suspension of recovery, re-run for scheduling + */ + businessDate = addDays(runTime, -1); + } + break; + } + Date businessCurrentDate = addDays(businessDate, 1); + Map result = new HashMap<>(); + result.put(Constants.PARAMETER_CURRENT_DATE, format(businessCurrentDate, PARAMETER_FORMAT_DATE)); + result.put(Constants.PARAMETER_BUSINESS_DATE, format(businessDate, PARAMETER_FORMAT_DATE)); + result.put(Constants.PARAMETER_DATETIME, format(businessCurrentDate, PARAMETER_FORMAT_TIME)); + return result; } - Date businessCurrentDate = addDays(businessDate, 1); - Map result = new HashMap<>(); - result.put(Constants.PARAMETER_CURRENT_DATE, format(businessCurrentDate, PARAMETER_FORMAT_DATE)); - result.put(Constants.PARAMETER_BUSINESS_DATE, format(businessDate, PARAMETER_FORMAT_DATE)); - result.put(Constants.PARAMETER_DATETIME, format(businessCurrentDate, PARAMETER_FORMAT_TIME)); - return result; - } } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java index a16c14ebc18d..f537fadf7521 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java @@ -14,26 +14,31 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.dao.utils; +package org.apache.dolphinscheduler.dao.utils; -import com.amazonaws.services.simpleworkflow.model.TaskList; import org.apache.dolphinscheduler.common.enums.TaskDependType; import org.apache.dolphinscheduler.common.graph.DAG; import org.apache.dolphinscheduler.common.model.TaskNode; import org.apache.dolphinscheduler.common.model.TaskNodeRelation; import org.apache.dolphinscheduler.common.process.ProcessDag; import org.apache.dolphinscheduler.common.utils.CollectionUtils; -import org.apache.dolphinscheduler.common.utils.*; +import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.StringUtils; import org.apache.dolphinscheduler.dao.entity.ProcessData; import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * dag tools */ diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java index 0e1e3d61802f..c0f3eea6f831 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java @@ -684,37 +684,38 @@ private void submitPostNode(String parentNodeName){ * determine whether the dependencies of the task node are complete * @return DependResult */ + @SuppressWarnings("checkstyle:WhitespaceAround") private DependResult isTaskDepsComplete(String taskName) { Collection startNodes = dag.getBeginNode(); // if vertex,returns true directly - if(startNodes.contains(taskName)){ + if (startNodes.contains(taskName)) { return DependResult.SUCCESS; } TaskNode taskNode = dag.getNode(taskName); List depNameList = taskNode.getDepList(); - for(String depsNode : depNameList ){ - if(!dag.containsNode(depsNode) - || forbiddenTaskList.containsKey(depsNode) - || skipTaskNodeList.containsKey(depsNode)){ + for (String depsNode : depNameList) { + if (!dag.containsNode(depsNode) + || forbiddenTaskList.containsKey(depsNode) + || skipTaskNodeList.containsKey(depsNode)) { continue; } // all the dependencies must be fully completed - if(!completeTaskList.containsKey(depsNode)){ + if (!completeTaskList.containsKey(depsNode)) { return DependResult.WAITING; } ExecutionStatus depTaskState = completeTaskList.get(depsNode).getState(); - if(depTaskState.typeIsPause() || depTaskState.typeIsCancel()){ + if (depTaskState.typeIsPause() || depTaskState.typeIsCancel()) { return DependResult.WAITING; } // ignore task state if current task is condition - if(taskNode.isConditionsTask()){ + if (taskNode.isConditionsTask()) { continue; } - if(!dependTaskSuccess(depsNode, taskName)){ + if (!dependTaskSuccess(depsNode, taskName)) { return DependResult.FAILED; } } @@ -730,18 +731,18 @@ private DependResult isTaskDepsComplete(String taskName) { * @param nextNodeName * @return */ - private boolean dependTaskSuccess(String dependNodeName, String nextNodeName){ + private boolean dependTaskSuccess(String dependNodeName, String nextNodeName) { TaskNode tmpNode = dag.getNode(dependNodeName); - if(tmpNode.isConditionsTask()){ + if (tmpNode.isConditionsTask()) { //condition task need check the branch to run List nextTaskList = parseConditionTask(dependNodeName); - if(!nextTaskList.contains(nextNodeName)){ + if (!nextTaskList.contains(nextNodeName)) { return false; } - }else { + } else { ExecutionStatus depTaskState = completeTaskList.get(dependNodeName).getState(); - if(depTaskState.typeIsFailure()){ + if (depTaskState.typeIsFailure()) { return false; } } @@ -1062,17 +1063,17 @@ private void runProcess(){ // failure priority is higher than pause // if a task fails, other suspended tasks need to be reset kill // check if there exists forced success nodes in errorTaskList - if(errorTaskList.size() > 0){ - for(Map.Entry entry: completeTaskList.entrySet()) { + if (errorTaskList.size() > 0) { + for (Map.Entry entry : completeTaskList.entrySet()) { TaskInstance completeTask = entry.getValue(); - if(completeTask.getState()== ExecutionStatus.PAUSE){ + if (completeTask.getState() == ExecutionStatus.PAUSE) { completeTask.setState(ExecutionStatus.KILL); completeTaskList.put(entry.getKey(), completeTask); processService.updateTaskInstance(completeTask); } } - for(Map.Entry entry: errorTaskList.entrySet()) { + for (Map.Entry entry : errorTaskList.entrySet()) { TaskInstance errorTask = entry.getValue(); TaskInstance currentTask = processService.findTaskInstanceById(errorTask.getId()); if (currentTask == null) { @@ -1089,7 +1090,7 @@ private void runProcess(){ } } } - if(canSubmitTaskToQueue()){ + if (canSubmitTaskToQueue()) { submitStandByTask(); } try { diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java index 48ab4aa6a1b9..816791a184b0 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java @@ -14,19 +14,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.dolphinscheduler.service.process; +import static java.util.stream.Collectors.toSet; import static org.apache.dolphinscheduler.common.Constants.CMDPARAM_COMPLEMENT_DATA_END_DATE; import static org.apache.dolphinscheduler.common.Constants.CMDPARAM_COMPLEMENT_DATA_START_DATE; import static org.apache.dolphinscheduler.common.Constants.CMDPARAM_EMPTY_SUB_PROCESS; +import static org.apache.dolphinscheduler.common.Constants.CMDPARAM_RECOVERY_START_NODE_STRING; import static org.apache.dolphinscheduler.common.Constants.CMDPARAM_RECOVER_PROCESS_ID_STRING; import static org.apache.dolphinscheduler.common.Constants.CMDPARAM_SUB_PROCESS; import static org.apache.dolphinscheduler.common.Constants.CMDPARAM_SUB_PROCESS_DEFINE_ID; import static org.apache.dolphinscheduler.common.Constants.CMDPARAM_SUB_PROCESS_PARENT_INSTANCE_ID; import static org.apache.dolphinscheduler.common.Constants.YYYY_MM_DD_HH_MM_SS; -import static java.util.stream.Collectors.toSet; - import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.AuthorizationType; import org.apache.dolphinscheduler.common.enums.CommandType; @@ -98,11 +99,6 @@ import org.springframework.stereotype.Component; import org.springframework.transaction.annotation.Transactional; -import java.util.*; -import java.util.stream.Collectors; - -import static java.util.stream.Collectors.toSet; -import static org.apache.dolphinscheduler.common.Constants.*; import com.cronutils.model.Cron; import com.fasterxml.jackson.databind.node.ObjectNode; @@ -1372,7 +1368,7 @@ public List findTaskIdByInstanceState(int instanceId, ExecutionStatus s * @param taskType task type * @return task instance id list */ - public List findTaskIdByInstanceStatusAndType(int processInstanceId, ExecutionStatus[] states, TaskType taskType){ + public List findTaskIdByInstanceStatusAndType(int processInstanceId, ExecutionStatus[] states, TaskType taskType) { int[] stateArray = new int[states.length]; for (int i = 0; i < states.length; i++) { stateArray[i] = states[i].ordinal(); @@ -1388,7 +1384,7 @@ public List findTaskIdByInstanceStatusAndType(int processInstanceId, Ex * @param taskType task type * @return task instance id list */ - public List findTaskIdInSubProcessByStatusAndType(int taskId, ExecutionStatus[] states, TaskType taskType){ + public List findTaskIdInSubProcessByStatusAndType(int taskId, ExecutionStatus[] states, TaskType taskType) { int[] stateArray = new int[states.length]; for (int i = 0; i < states.length; i++) { stateArray[i] = states[i].ordinal(); From d97cc3db86d68a5f0fc212ebca6940a3ca27045e Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Mon, 24 Aug 2020 16:03:39 +0800 Subject: [PATCH 23/49] fix vulnerability MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 防止logger注入的问题 --- .../dolphinscheduler/api/controller/TaskInstanceController.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java index 71ada4fd732e..ea0c3eeb76f6 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java @@ -136,7 +136,7 @@ public Result forceSingleTaskSuccess(@ApiIgnore @RequestAttribute(value = Consta @ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName, @RequestParam(value = "taskInstanceId") Integer taskInstanceId) { logger.info("force task success, login user: {}, project:{}, task instance id:{}", - loginUser.getUserName(), projectName, taskInstanceId); + loginUser.getUserName(), projectName.replaceAll("[\n|\r|\t]", "_"), taskInstanceId); Map result = taskInstanceService.forceSingleTaskSuccess(loginUser, projectName, taskInstanceId); return returnDataList(result); } From 6e5145f7c4130658663bbbec7b1c729783ab0acd Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Mon, 24 Aug 2020 16:19:52 +0800 Subject: [PATCH 24/49] fix some code smell problems 1. use Collections.isEmpt() instead of size() > 0 2. make if statements more readable --- .../api/service/ExecutorService.java | 2 +- .../master/runner/MasterExecThread.java | 28 +++++++++---------- .../service/process/ProcessService.java | 2 +- 3 files changed, 15 insertions(+), 17 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java index c44a094b1ab0..0f6b35822fe5 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java @@ -635,7 +635,7 @@ private Map checkResultAndAuth(User loginUser, String projectNam */ private boolean checkValidForcedSuccessTask(int processInstanceId) { List forcedSuccessList = processService.findTaskIdByInstanceState(processInstanceId, ExecutionStatus.FORCED_SUCCESS); - if (forcedSuccessList != null && forcedSuccessList.size() > 0) { + if (forcedSuccessList != null && !forcedSuccessList.isEmpty()) { return true; } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java index c0f3eea6f831..6d4906748b72 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java @@ -1180,34 +1180,32 @@ private boolean retryTaskIntervalOverTime(TaskInstance taskInstance){ /** * handling the list of tasks to be submitted */ - private void submitStandByTask(){ - for(Map.Entry entry: readyToSubmitTaskList.entrySet()) { + private void submitStandByTask() { + for (Map.Entry entry : readyToSubmitTaskList.entrySet()) { TaskInstance task = entry.getValue(); // stop tasks which is retrying if forced success happens if (task.taskCanRetry()) { TaskInstance tmpTask = processService.findTaskInstanceById(task.getId()); - if (tmpTask != null) { - if (tmpTask.getState().equals(ExecutionStatus.FORCED_SUCCESS)) { - task.setState(tmpTask.getState()); - logger.info("task: {} has been forced success, put it into complete task list and stop retrying", task.getName()); - removeTaskFromStandbyList(task); - completeTaskList.put(task.getName(), task); - submitPostNode(task.getName()); - continue; - } + if (tmpTask != null && tmpTask.getState().equals(ExecutionStatus.FORCED_SUCCESS)) { + task.setState(tmpTask.getState()); + logger.info("task: {} has been forced success, put it into complete task list and stop retrying", task.getName()); + removeTaskFromStandbyList(task); + completeTaskList.put(task.getName(), task); + submitPostNode(task.getName()); + continue; } } DependResult dependResult = getDependResultForTask(task); - if(DependResult.SUCCESS == dependResult){ - if(retryTaskIntervalOverTime(task)){ + if (DependResult.SUCCESS == dependResult) { + if (retryTaskIntervalOverTime(task)) { submitTaskExec(task); removeTaskFromStandbyList(task); } - }else if(DependResult.FAILED == dependResult){ + } else if (DependResult.FAILED == dependResult) { // if the dependency fails, the current node is not submitted and the state changes to failure. dependFailedTask.put(entry.getKey(), task); removeTaskFromStandbyList(task); - logger.info("task {},id:{} depend result : {}",task.getName(), task.getId(), dependResult); + logger.info("task {},id:{} depend result : {}", task.getName(), task.getId(), dependResult); } } } diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java index 816791a184b0..f6b5899848d0 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java @@ -790,7 +790,7 @@ public boolean haveForcedSuccessInSubProcess(int taskInstanceId) { List forcedSuccessList = this.findTaskIdInSubProcessByStatusAndType(taskInstanceId, new ExecutionStatus[]{ExecutionStatus.FORCED_SUCCESS}, null); - if (forcedSuccessList != null && forcedSuccessList.size() > 0) { + if (forcedSuccessList != null && !forcedSuccessList.isEmpty()) { return true; } From de19fabbc44cae53a44c4679ef92f58b14dc2525 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Wed, 26 Aug 2020 22:30:02 +0800 Subject: [PATCH 25/49] change back to original import order --- .../common/utils/placeholder/BusinessTimeUtils.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/BusinessTimeUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/BusinessTimeUtils.java index 8c55e8521a5c..f30560331e23 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/BusinessTimeUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/BusinessTimeUtils.java @@ -14,14 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.dolphinscheduler.common.utils.placeholder; -import static org.apache.commons.lang.time.DateUtils.addDays; -import static org.apache.dolphinscheduler.common.Constants.PARAMETER_FORMAT_DATE; -import static org.apache.dolphinscheduler.common.Constants.PARAMETER_FORMAT_TIME; -import static org.apache.dolphinscheduler.common.utils.DateUtils.format; - import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.CommandType; @@ -29,6 +23,11 @@ import java.util.HashMap; import java.util.Map; +import static org.apache.dolphinscheduler.common.Constants.PARAMETER_FORMAT_DATE; +import static org.apache.dolphinscheduler.common.Constants.PARAMETER_FORMAT_TIME; +import static org.apache.dolphinscheduler.common.utils.DateUtils.format; +import static org.apache.commons.lang.time.DateUtils.addDays; + /** * business time utils From 79bff32733495c88c13e781aa8bac1a7678fe914 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Wed, 26 Aug 2020 23:03:45 +0800 Subject: [PATCH 26/49] Update TaskInstanceService.java --- .../dolphinscheduler/api/service/TaskInstanceService.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java index 0815ae88651e..068fdcc564ad 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java @@ -188,6 +188,9 @@ public Map forceSingleTaskSuccess(User loginUser, String project putMsg(result, Status.FORCE_TASK_SUCCESS_ERROR); } + return result; + } + /*** * generate {@link org.apache.dolphinscheduler.api.enums.Status#REQUEST_PARAMS_NOT_VALID_ERROR} res with param name * @param result exist result map From 757f8e4f5894af62d22a442e7dcb80424a99d476 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Wed, 26 Aug 2020 23:49:39 +0800 Subject: [PATCH 27/49] fix spelling mistake --- .../main/java/org/apache/dolphinscheduler/api/enums/Status.java | 2 +- .../dolphinscheduler/api/service/TaskInstanceService.java | 2 +- .../dolphinscheduler/api/service/TaskInstanceServiceTest.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java index b254a9eeb3bc..48487d7b2c81 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java @@ -194,7 +194,7 @@ public enum Status { BATCH_MOVE_PROCESS_DEFINITION_ERROR(10160, "batch move process definition error", "移动工作流错误"), QUERY_WORKFLOW_LINEAGE_ERROR(10161, "query workflow lineage error", "查询血缘失败"), FORCE_TASK_SUCCESS_ERROR(10162, "force task success error", "强制成功任务实例错误"), - TASK_INSTANCE_STATE_OPETATION_ERROR(10163, "the status of task instance {0} is {1},Cannot perform force success operation", "任务实例[{0}]的状态是[{1}],无法执行强制成功操作"), + TASK_INSTANCE_STATE_OPERATION_ERROR(10163, "the status of task instance {0} is {1},Cannot perform force success operation", "任务实例[{0}]的状态是[{1}],无法执行强制成功操作"), diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java index 068fdcc564ad..0cad69032f49 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java @@ -174,7 +174,7 @@ public Map forceSingleTaskSuccess(User loginUser, String project // check whether the task instance state type is failure if (!task.getState().typeIsFailure()) { - putMsg(result, Status.TASK_INSTANCE_STATE_OPETATION_ERROR, taskInstanceId, task.getState().toString()); + putMsg(result, Status.TASK_INSTANCE_STATE_OPERATION_ERROR, taskInstanceId, task.getState().toString()); return result; } diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java index 855485751a6a..39ea85fa718d 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java @@ -237,7 +237,7 @@ public void forceSingleTaskSuccess() { task.setState(ExecutionStatus.SUCCESS); when(taskInstanceMapper.selectById(1)).thenReturn(task); Map taskStateErrorRes = taskInstanceService.forceSingleTaskSuccess(user, projectName, taskId); - Assert.assertEquals(Status.TASK_INSTANCE_STATE_OPETATION_ERROR, taskStateErrorRes.get(Constants.STATUS)); + Assert.assertEquals(Status.TASK_INSTANCE_STATE_OPERATION_ERROR, taskStateErrorRes.get(Constants.STATUS)); // test success task.setState(ExecutionStatus.FAILURE); From 211bcd5c86551459e7ba3cedaaedb34de47b9bed Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Fri, 28 Aug 2020 14:53:21 +0800 Subject: [PATCH 28/49] change bad varible name --- .../service/process/ProcessService.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java index f6b5899848d0..9d6fd22b1e03 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java @@ -1369,11 +1369,11 @@ public List findTaskIdByInstanceState(int instanceId, ExecutionStatus s * @return task instance id list */ public List findTaskIdByInstanceStatusAndType(int processInstanceId, ExecutionStatus[] states, TaskType taskType) { - int[] stateArray = new int[states.length]; + int[] statesArray = new int[states.length]; for (int i = 0; i < states.length; i++) { - stateArray[i] = states[i].ordinal(); + statesArray[i] = states[i].ordinal(); } - return taskInstanceMapper.queryTaskByPIdAndStatusAndType(processInstanceId, stateArray, taskType.toString()); + return taskInstanceMapper.queryTaskByPIdAndStatusAndType(processInstanceId, statesArray, taskType.toString()); } /** @@ -1385,14 +1385,14 @@ public List findTaskIdByInstanceStatusAndType(int processInstanceId, Ex * @return task instance id list */ public List findTaskIdInSubProcessByStatusAndType(int taskId, ExecutionStatus[] states, TaskType taskType) { - int[] stateArray = new int[states.length]; + int[] statesArray = new int[states.length]; for (int i = 0; i < states.length; i++) { - stateArray[i] = states[i].ordinal(); + statesArray[i] = states[i].ordinal(); } if (taskType == null) { - return taskInstanceMapper.queryTaskBySubProcessTaskIdAndStatusAndType(taskId, stateArray, null); + return taskInstanceMapper.queryTaskBySubProcessTaskIdAndStatusAndType(taskId, statesArray, null); } - return taskInstanceMapper.queryTaskBySubProcessTaskIdAndStatusAndType(taskId, stateArray, taskType.toString()); + return taskInstanceMapper.queryTaskBySubProcessTaskIdAndStatusAndType(taskId, statesArray, taskType.toString()); } /** From 92e70b94cc5f81d0eb075095c40c86e415c55ee2 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Fri, 28 Aug 2020 17:41:15 +0800 Subject: [PATCH 29/49] improvement --- .../api/controller/TaskInstanceController.java | 8 ++++---- .../api/controller/TaskInstanceControllerTest.java | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java index ea0c3eeb76f6..03fd2790729e 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java @@ -118,16 +118,16 @@ public Result queryTaskListPaging(@ApiIgnore @RequestAttribute(value = Constants } /** - * change one single task instance's state from failure to forced success + * change one single task instance's state from FAILURE to FORCED_SUCCESS * * @param loginUser login user * @param projectName project name * @param taskInstanceId task instance id * @return the result code and msg */ - @ApiOperation(value = "force-success", notes = "") + @ApiOperation(value = "force-success", notes = "FORCE_SINGLE_TASK_SUCCESS") @ApiImplicitParams({ - @ApiImplicitParam(name = "taskInstanceId", value = "TASK_INTSTANCE_ID", required = true, dataType = "Int", example = "2") + @ApiImplicitParam(name = "taskInstanceId", value = "TASK_INSTANCE_ID", required = true, dataType = "Int", example = "12") }) @PostMapping(value = "/force-success") @ResponseStatus(HttpStatus.OK) @@ -136,7 +136,7 @@ public Result forceSingleTaskSuccess(@ApiIgnore @RequestAttribute(value = Consta @ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName, @RequestParam(value = "taskInstanceId") Integer taskInstanceId) { logger.info("force task success, login user: {}, project:{}, task instance id:{}", - loginUser.getUserName(), projectName.replaceAll("[\n|\r|\t]", "_"), taskInstanceId); + loginUser.getUserName(), projectName, taskInstanceId); Map result = taskInstanceService.forceSingleTaskSuccess(loginUser, projectName, taskInstanceId); return returnDataList(result); } diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java index abaaa3ffd23a..02f234b31644 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java @@ -81,7 +81,7 @@ public void testQueryTaskListPaging() throws Exception { } @Test - public void forceSingleTaskSuccess() throws Exception { + public void testForceSingleTaskSuccess() throws Exception { MultiValueMap paramsMap = new LinkedMultiValueMap<>(); paramsMap.add("taskInstanceId","104"); From 83548e63baca45c3a3f3444043a0b297159994ea Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Sat, 29 Aug 2020 11:18:50 +0800 Subject: [PATCH 30/49] add test of executorService --- .../api/service/ExecutorService2Test.java | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java index a4c0c6bfe7ce..0a8a99c21531 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java @@ -14,22 +14,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.dolphinscheduler.api.service; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import org.apache.dolphinscheduler.api.enums.ExecuteType; import org.apache.dolphinscheduler.api.enums.Status; import org.apache.dolphinscheduler.api.service.impl.ProjectServiceImpl; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.CommandType; +import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.Priority; import org.apache.dolphinscheduler.common.enums.ReleaseState; import org.apache.dolphinscheduler.common.enums.RunMode; +import org.apache.dolphinscheduler.common.enums.TaskType; import org.apache.dolphinscheduler.common.model.Server; import org.apache.dolphinscheduler.dao.entity.Command; import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.Project; import org.apache.dolphinscheduler.dao.entity.Schedule; import org.apache.dolphinscheduler.dao.entity.Tenant; @@ -81,12 +87,16 @@ public class ExecutorService2Test { private int processDefinitionId = 1; + private int processInstanceId = 1; + private int tenantId = 1; private int userId = 1; private ProcessDefinition processDefinition = new ProcessDefinition(); + private ProcessInstance processInstance = new ProcessInstance(); + private User loginUser = new User(); private String projectName = "projectName"; @@ -106,6 +116,13 @@ public void init() { processDefinition.setTenantId(tenantId); processDefinition.setUserId(userId); + // processInstance + processInstance.setId(processInstanceId); + processInstance.setProcessDefinitionId(processDefinitionId); + processInstance.setState(ExecutionStatus.FAILURE); + processInstance.setExecutorId(userId); + processInstance.setTenantId(tenantId); + // project project.setName(projectName); @@ -119,6 +136,8 @@ public void init() { Mockito.when(processService.getTenantForProcess(tenantId, userId)).thenReturn(new Tenant()); Mockito.when(processService.createCommand(any(Command.class))).thenReturn(1); Mockito.when(monitorService.getServerListFromZK(true)).thenReturn(getMasterServersList()); + Mockito.when(processService.findProcessInstanceDetailById(processInstanceId)).thenReturn(processInstance); + Mockito.when(processService.findProcessDefineById(processDefinitionId)).thenReturn(processDefinition); } /** @@ -231,6 +250,25 @@ public void testNoMsterServers() throws ParseException { } + @Test + public void testExecute() { + List mockRes = new ArrayList<>(); + mockRes.add(1); + mockRes.add(2); + Mockito.when(processService.findTaskIdByInstanceStatusAndType(anyInt(), any(ExecutionStatus[].class), any(TaskType.class))) + .thenReturn(mockRes); + + Mockito.when(processService.haveForcedSuccessInSubProcess(anyInt())) + .thenReturn(true); + + Mockito.when(processService.verifyIsNeedCreateCommand(any(Command.class))) + .thenReturn(true); + + Map result = executorService.execute(loginUser, projectName, processInstanceId, ExecuteType.RESUME_FROM_FORCED_SUCCESS); + Assert.assertEquals(result.get(Constants.STATUS), Status.SUCCESS); + verify(processService, times(1)).createCommand(any(Command.class)); + } + private List getMasterServersList() { List masterServerList = new ArrayList<>(); Server masterServer1 = new Server(); From d2a43b0c258c25ab31be19dd19428112c3cd40d5 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Sat, 29 Aug 2020 14:57:23 +0800 Subject: [PATCH 31/49] fix bug about CONDITION-task 1. condition shoudn't be in error list 2. condition need more checks when rusume_from_forced_success --- .../dolphinscheduler/dao/utils/DagHelper.java | 25 ++++++++++++++----- .../master/runner/MasterExecThread.java | 2 +- 2 files changed, 20 insertions(+), 7 deletions(-) diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java index f537fadf7521..3ad64d010d7c 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java @@ -22,6 +22,7 @@ import org.apache.dolphinscheduler.common.model.TaskNode; import org.apache.dolphinscheduler.common.model.TaskNodeRelation; import org.apache.dolphinscheduler.common.process.ProcessDag; +import org.apache.dolphinscheduler.common.task.conditions.ConditionsParameters; import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.StringUtils; @@ -268,20 +269,32 @@ public static TaskNode findNodeByName(List nodeDetails, String nodeNam * @return start Vertex list */ public static Collection getStartVertex(String parentNodeName, DAG dag, - Map completeTaskList){ + Map completeTaskList) { - if(completeTaskList == null){ + if (completeTaskList == null) { completeTaskList = new HashMap<>(); } Collection startVertexs = null; - if(StringUtils.isNotEmpty(parentNodeName)){ - startVertexs = dag.getSubsequentNodes(parentNodeName); - }else{ + if (StringUtils.isNotEmpty(parentNodeName)) { + TaskNode task = dag.getNode(parentNodeName); + if (task.isConditionsTask() && completeTaskList.containsKey(parentNodeName)) { + ConditionsParameters conditionsParameters = JSONUtils.parseObject(task.getConditionResult(), ConditionsParameters.class); + TaskInstance taskInstance = completeTaskList.get(parentNodeName); + if (taskInstance.getState().typeIsSuccess()) { + startVertexs = conditionsParameters.getSuccessNode(); + } else if (taskInstance.getState().typeIsFailure()) { + startVertexs = conditionsParameters.getFailedNode(); + } + } + else { + startVertexs = dag.getSubsequentNodes(parentNodeName); + } + } else { startVertexs = dag.getBeginNode(); } List tmpStartVertexs = new ArrayList<>(); - if(startVertexs!= null){ + if (startVertexs != null) { tmpStartVertexs.addAll(startVertexs); } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java index 6d4906748b72..dd4de8b635cd 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java @@ -405,7 +405,7 @@ private void initTaskQueue(){ if(task.isTaskComplete()){ completeTaskList.put(task.getName(), task); } - if(task.getState().typeIsFailure() && !task.taskCanRetry()){ + if (task.getState().typeIsFailure() && !task.taskCanRetry() && !task.isConditionsTask()) { errorTaskList.put(task.getName(), task); } } From 3eeea22671726d6797ef70e72b41db72fefd314b Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Sat, 29 Aug 2020 20:58:31 +0800 Subject: [PATCH 32/49] improve test of DAGhelper --- .../api/service/ExecutorService2Test.java | 2 +- .../dolphinscheduler/dao/utils/DagHelper.java | 25 ++++++++----- .../dao/utils/DagHelperTest.java | 36 +++++++++++++++++++ 3 files changed, 54 insertions(+), 9 deletions(-) diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java index 0a8a99c21531..8068974e5e09 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java @@ -265,7 +265,7 @@ public void testExecute() { .thenReturn(true); Map result = executorService.execute(loginUser, projectName, processInstanceId, ExecuteType.RESUME_FROM_FORCED_SUCCESS); - Assert.assertEquals(result.get(Constants.STATUS), Status.SUCCESS); + Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS)); verify(processService, times(1)).createCommand(any(Command.class)); } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java index 3ad64d010d7c..e5c4b8194caf 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java @@ -32,6 +32,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -278,13 +279,7 @@ public static Collection getStartVertex(String parentNodeName, DAG getStartVertex(String parentNodeName, DAG tmpStartVertexs = new ArrayList<>(); + Set tmpStartVertexs = new HashSet<>(); if (startVertexs != null) { tmpStartVertexs.addAll(startVertexs); } @@ -317,6 +312,20 @@ public static Collection getStartVertex(String parentNodeName, DAG parseConditionTask(String parentNodeName, TaskNode task, Map completeTaskList) { + ConditionsParameters conditionsParameters = JSONUtils.parseObject(task.getConditionResult(), ConditionsParameters.class); + TaskInstance taskInstance = completeTaskList.get(parentNodeName); + if (taskInstance.getState().typeIsSuccess()) { + return conditionsParameters.getSuccessNode(); + } else if (taskInstance.getState().typeIsFailure()) { + return conditionsParameters.getFailedNode(); + } + return null; + } + /** * the task can be submit when all the depends nodes are forbidden or complete * @param taskNode taskNode diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/utils/DagHelperTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/utils/DagHelperTest.java index 14dfe0b75005..9e569df61caf 100644 --- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/utils/DagHelperTest.java +++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/utils/DagHelperTest.java @@ -19,7 +19,9 @@ import com.fasterxml.jackson.core.JsonProcessingException; import org.apache.dolphinscheduler.common.Constants; +import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.TaskDependType; +import org.apache.dolphinscheduler.common.enums.TaskType; import org.apache.dolphinscheduler.common.graph.DAG; import org.apache.dolphinscheduler.common.model.TaskNode; import org.apache.dolphinscheduler.common.model.TaskNodeRelation; @@ -34,6 +36,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; /** * dag helper test @@ -79,6 +82,7 @@ public void testTaskNodeCanSubmit() throws JsonProcessingException { /** * 1->2->3->5 * 4->3 + * 3->6 * @return dag * @throws JsonProcessingException if error throws JsonProcessingException */ @@ -120,6 +124,14 @@ private DAG generateDag() throws JsonProcess node5.setDepList(dep5); taskNodeList.add(node5); + TaskNode node6 = new TaskNode(); + node6.setId("6"); + node6.setName("6"); + List dep6 = new ArrayList<>(); + dep6.add("3"); + node6.setDepList(dep6); + taskNodeList.add(node6); + List startNodes = new ArrayList<>(); List recoveryNodes = new ArrayList<>(); List destTaskNodeList = DagHelper.generateFlowNodeListByStartNode(taskNodeList, @@ -148,4 +160,28 @@ public void testBuildDagGraph() { Assert.assertNotNull(dag); } + @Test + public void testGetStartVertex() throws JsonProcessingException { + // let 1->2->3->5, 3->5, 4->3 + DAG dag = generateDag(); + + // test when completeList is null + Assert.assertEquals(2, DagHelper.getStartVertex(null, dag, null).size()); + Assert.assertEquals(1, DagHelper.getStartVertex("1", dag, null).size()); + + // test when 3 is CONDITIONS and 1,2,3,4 all completed + Map completeTaskList = new ConcurrentHashMap<>(); + completeTaskList.putIfAbsent("1", new TaskInstance()); + completeTaskList.putIfAbsent("2", new TaskInstance()); + completeTaskList.putIfAbsent("4", new TaskInstance()); + TaskInstance task3 = new TaskInstance(); + task3.setState(ExecutionStatus.SUCCESS); + completeTaskList.putIfAbsent("3", task3); + + dag.getNode("3").setConditionResult("{\"successNode\":[\"5\"],\"failedNode\":[\"6\"]}"); + dag.getNode("3").setType(TaskType.CONDITIONS.toString()); + + Assert.assertEquals(1, DagHelper.getStartVertex(null, dag, completeTaskList).size()); + Assert.assertEquals(true, DagHelper.getStartVertex(null, dag, completeTaskList).contains("5")); + } } From 58b4e1f4561ecc70f645a1b39764b53d65f830d4 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Sat, 29 Aug 2020 22:30:15 +0800 Subject: [PATCH 33/49] fix vulnerability --- .../api/controller/TaskInstanceController.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java index 03fd2790729e..fe4686d64d05 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java @@ -26,6 +26,7 @@ import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.utils.ParameterUtils; +import org.apache.dolphinscheduler.common.utils.StringUtils; import org.apache.dolphinscheduler.dao.entity.User; import java.util.Map; @@ -135,8 +136,10 @@ public Result queryTaskListPaging(@ApiIgnore @RequestAttribute(value = Constants public Result forceSingleTaskSuccess(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser, @ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName, @RequestParam(value = "taskInstanceId") Integer taskInstanceId) { + String userNameReplace = StringUtils.replaceNRTtoUnderline(loginUser.getUserName()); + String projectNameReplace = StringUtils.replaceNRTtoUnderline(projectName); logger.info("force task success, login user: {}, project:{}, task instance id:{}", - loginUser.getUserName(), projectName, taskInstanceId); + userNameReplace, projectNameReplace, taskInstanceId); Map result = taskInstanceService.forceSingleTaskSuccess(loginUser, projectName, taskInstanceId); return returnDataList(result); } From bbfd35be0581141d24938d381da5b85aad4288be Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Sun, 30 Aug 2020 00:53:11 +0800 Subject: [PATCH 34/49] improve test coverage --- .../api/service/ExecutorService2Test.java | 30 ++++++++++++++----- .../api/service/TaskInstanceServiceTest.java | 21 ++++++++++--- .../dolphinscheduler/dao/utils/DagHelper.java | 3 +- .../dao/utils/DagHelperTest.java | 10 +++++++ 4 files changed, 51 insertions(+), 13 deletions(-) diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java index 8068974e5e09..2438e806cc4d 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java @@ -255,18 +255,32 @@ public void testExecute() { List mockRes = new ArrayList<>(); mockRes.add(1); mockRes.add(2); + Mockito.when(processService.verifyIsNeedCreateCommand(any(Command.class))) + .thenReturn(true); + + // check execute type error + processInstance.setState(ExecutionStatus.SUCCESS); + Map checkExeTypeRes = executorService.execute(loginUser, projectName, processInstanceId, ExecuteType.RESUME_FROM_FORCED_SUCCESS); + Assert.assertEquals(Status.PROCESS_INSTANCE_STATE_OPERATION_ERROR, checkExeTypeRes.get(Constants.STATUS)); + + // no valid forced success task + processInstance.setState(ExecutionStatus.FAILURE); + Map noValidTaskRes = executorService.execute(loginUser, projectName, processInstanceId, ExecuteType.RESUME_FROM_FORCED_SUCCESS); + Assert.assertEquals(Status.NO_VALID_FORCED_SUCCESS_TASK, noValidTaskRes.get(Constants.STATUS)); + + // have forced success in sub-process Mockito.when(processService.findTaskIdByInstanceStatusAndType(anyInt(), any(ExecutionStatus[].class), any(TaskType.class))) .thenReturn(mockRes); - Mockito.when(processService.haveForcedSuccessInSubProcess(anyInt())) .thenReturn(true); - - Mockito.when(processService.verifyIsNeedCreateCommand(any(Command.class))) - .thenReturn(true); - - Map result = executorService.execute(loginUser, projectName, processInstanceId, ExecuteType.RESUME_FROM_FORCED_SUCCESS); - Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS)); - verify(processService, times(1)).createCommand(any(Command.class)); + Map successRes1 = executorService.execute(loginUser, projectName, processInstanceId, ExecuteType.RESUME_FROM_FORCED_SUCCESS); + Assert.assertEquals(Status.SUCCESS, successRes1.get(Constants.STATUS)); + + // test success + Mockito.when(processService.findTaskIdByInstanceState(processInstanceId, ExecutionStatus.FORCED_SUCCESS)).thenReturn(mockRes); + Map successRes = executorService.execute(loginUser, projectName, processInstanceId, ExecuteType.RESUME_FROM_FORCED_SUCCESS); + Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS)); + verify(processService, times(2)).createCommand(any(Command.class)); } private List getMasterServersList() { diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java index 39ea85fa718d..ac38535253db 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java @@ -223,12 +223,19 @@ public void forceSingleTaskSuccess() { int taskId = 1; TaskInstance task = getTaskInstance(); - Map tmpResult = new HashMap<>(5); - putMsg(tmpResult, Status.SUCCESS); - when(projectMapper.queryByName("test")).thenReturn(project); - when(projectService.checkProjectAndAuth(user, project, projectName)).thenReturn(tmpResult); + Map mockSuccess = new HashMap<>(5); + putMsg(mockSuccess, Status.SUCCESS); + when(projectMapper.queryByName(projectName)).thenReturn(project); + + // user auth failed + Map mockFailure = new HashMap<>(5); + putMsg(mockFailure, Status.USER_NO_OPERATION_PROJECT_PERM, user.getUserName(), projectName); + when(projectService.checkProjectAndAuth(user, project, projectName)).thenReturn(mockFailure); + Map authFailRes = taskInstanceService.forceSingleTaskSuccess(user, projectName, taskId); + Assert.assertNotSame(Status.SUCCESS, authFailRes.get(Constants.STATUS)); // test task not found + when(projectService.checkProjectAndAuth(user, project, projectName)).thenReturn(mockSuccess); when(taskInstanceMapper.selectById(Mockito.anyInt())).thenReturn(null); Map taskNotFoundRes = taskInstanceService.forceSingleTaskSuccess(user, projectName, taskId); Assert.assertEquals(Status.TASK_INSTANCE_NOT_FOUND, taskNotFoundRes.get(Constants.STATUS)); @@ -239,6 +246,12 @@ public void forceSingleTaskSuccess() { Map taskStateErrorRes = taskInstanceService.forceSingleTaskSuccess(user, projectName, taskId); Assert.assertEquals(Status.TASK_INSTANCE_STATE_OPERATION_ERROR, taskStateErrorRes.get(Constants.STATUS)); + // test error + task.setState(ExecutionStatus.FAILURE); + when(taskInstanceMapper.updateById(task)).thenReturn(0); + Map errorRes = taskInstanceService.forceSingleTaskSuccess(user, projectName, taskId); + Assert.assertEquals(Status.FORCE_TASK_SUCCESS_ERROR, errorRes.get(Constants.STATUS)); + // test success task.setState(ExecutionStatus.FAILURE); when(taskInstanceMapper.updateById(task)).thenReturn(1); diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java index e5c4b8194caf..769ed79dcbcf 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java @@ -31,6 +31,7 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -323,7 +324,7 @@ private static Collection parseConditionTask(String parentNodeName, Task } else if (taskInstance.getState().typeIsFailure()) { return conditionsParameters.getFailedNode(); } - return null; + return Collections.emptyList(); } /** diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/utils/DagHelperTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/utils/DagHelperTest.java index 9e569df61caf..f170ad78512f 100644 --- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/utils/DagHelperTest.java +++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/utils/DagHelperTest.java @@ -181,7 +181,17 @@ public void testGetStartVertex() throws JsonProcessingException { dag.getNode("3").setConditionResult("{\"successNode\":[\"5\"],\"failedNode\":[\"6\"]}"); dag.getNode("3").setType(TaskType.CONDITIONS.toString()); + // test success branch Assert.assertEquals(1, DagHelper.getStartVertex(null, dag, completeTaskList).size()); Assert.assertEquals(true, DagHelper.getStartVertex(null, dag, completeTaskList).contains("5")); + + // test failure branch + task3.setState(ExecutionStatus.FAILURE); + Assert.assertEquals(1, DagHelper.getStartVertex(null, dag, completeTaskList).size()); + Assert.assertEquals(true, DagHelper.getStartVertex(null, dag, completeTaskList).contains("6")); + + // test state error + task3.setState(ExecutionStatus.PAUSE); + Assert.assertEquals(0, DagHelper.getStartVertex(null, dag, completeTaskList).size()); } } From 8ba41dc1313886541110928d4e7a18d2c913a1d5 Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Tue, 8 Sep 2020 12:51:29 +0800 Subject: [PATCH 35/49] fix code smells --- .../api/controller/TaskInstanceController.java | 2 +- .../org/apache/dolphinscheduler/dao/utils/DagHelper.java | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java index fe4686d64d05..02098cc7cc43 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java @@ -133,7 +133,7 @@ public Result queryTaskListPaging(@ApiIgnore @RequestAttribute(value = Constants @PostMapping(value = "/force-success") @ResponseStatus(HttpStatus.OK) @ApiException(FORCE_TASK_SUCCESS_ERROR) - public Result forceSingleTaskSuccess(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser, + public Result forceSingleTaskSuccess(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser, @ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName, @RequestParam(value = "taskInstanceId") Integer taskInstanceId) { String userNameReplace = StringUtils.replaceNRTtoUnderline(loginUser.getUserName()); diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java index 769ed79dcbcf..38b469a08257 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java @@ -278,13 +278,12 @@ public static Collection getStartVertex(String parentNodeName, DAG startVertexs = null; if (StringUtils.isNotEmpty(parentNodeName)) { + startVertexs = dag.getSubsequentNodes(parentNodeName); + // if task is CONDITION node TaskNode task = dag.getNode(parentNodeName); if (task.isConditionsTask() && completeTaskList.containsKey(parentNodeName)) { startVertexs = parseConditionTask(parentNodeName, task, completeTaskList); } - else { - startVertexs = dag.getSubsequentNodes(parentNodeName); - } } else { startVertexs = dag.getBeginNode(); } From 8e4037c38595ab70ef08d1f23fb375f5313240be Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Tue, 8 Sep 2020 14:07:14 +0800 Subject: [PATCH 36/49] Update MasterExecThread.java --- .../dolphinscheduler/server/master/runner/MasterExecThread.java | 1 - 1 file changed, 1 deletion(-) diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java index dd4de8b635cd..728f3497a6db 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java @@ -684,7 +684,6 @@ private void submitPostNode(String parentNodeName){ * determine whether the dependencies of the task node are complete * @return DependResult */ - @SuppressWarnings("checkstyle:WhitespaceAround") private DependResult isTaskDepsComplete(String taskName) { Collection startNodes = dag.getBeginNode(); From a224d6fc3c3f47d1619f3620d45041f6714e157d Mon Sep 17 00:00:00 2001 From: Zhou Zheng <1606079777@qq.comom> Date: Tue, 8 Sep 2020 15:09:03 +0800 Subject: [PATCH 37/49] Update DagHelper.java --- .../java/org/apache/dolphinscheduler/dao/utils/DagHelper.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java index 38b469a08257..6192d56dfba3 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java @@ -279,7 +279,7 @@ public static Collection getStartVertex(String parentNodeName, DAG startVertexs = null; if (StringUtils.isNotEmpty(parentNodeName)) { startVertexs = dag.getSubsequentNodes(parentNodeName); - // if task is CONDITION node + // if task is CONDITION node and complete then parse it TaskNode task = dag.getNode(parentNodeName); if (task.isConditionsTask() && completeTaskList.containsKey(parentNodeName)) { startVertexs = parseConditionTask(parentNodeName, task, completeTaskList); From 4f0c74dcc1b855ac96fb8a1be49b7c7080e4c4d6 Mon Sep 17 00:00:00 2001 From: "Zhou.Z" <1606079777@qq.comom> Date: Wed, 14 Oct 2020 12:43:07 +0800 Subject: [PATCH 38/49] Update Status.java --- .../main/java/org/apache/dolphinscheduler/api/enums/Status.java | 1 + 1 file changed, 1 insertion(+) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java index 7a16257b4d56..1743aff85106 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java @@ -253,6 +253,7 @@ public enum Status { IMPORT_PROCESS_DEFINE_ERROR(50029,"import process definition error", "导入工作流定义错误"), NO_VALID_FORCED_SUCCESS_TASK(50030, "there is no valid forced success node in process instance {0}", "工作流实例[{0}]中不包含有效的强制成功的任务实例"), + HDFS_NOT_STARTUP(60001, "hdfs not startup", "hdfs未启用"), /** From 7817601bd5ca5af60e4b022326df48a747e6463b Mon Sep 17 00:00:00 2001 From: "Zhou.Z" <1606079777@qq.comom> Date: Mon, 26 Oct 2020 12:30:16 +0800 Subject: [PATCH 39/49] Update Status.java --- .../main/java/org/apache/dolphinscheduler/api/enums/Status.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java index be0644821ca3..7761e60ebcbf 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java @@ -198,7 +198,7 @@ public enum Status { CHECK_TENANT_CODE_ERROR(10164, "Please enter the English tenant code", "请输入英文租户编码"), FORCE_TASK_SUCCESS_ERROR(10165, "force task success error", "强制成功任务实例错误"), TASK_INSTANCE_STATE_OPERATION_ERROR(10166, "the status of task instance {0} is {1},Cannot perform force success operation", "任务实例[{0}]的状态是[{1}],无法执行强制成功操作"), - + UDF_FUNCTION_NOT_EXIST(20001, "UDF function not found", "UDF函数不存在"), UDF_FUNCTION_EXISTS(20002, "UDF function already exists", "UDF函数已存在"), RESOURCE_NOT_EXIST(20004, "resource not exist", "资源不存在"), From 972c6d893b8715a4cd8192308cfa35c39b542971 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9B=B9=E8=81=9A=E9=98=B3?= Date: Thu, 3 Dec 2020 23:18:59 +0800 Subject: [PATCH 40/49] start from force success --- .../instance/pages/list/_source/list.vue | 47 +++++++++++++++++-- .../src/js/module/i18n/locale/en_US.js | 1 + .../src/js/module/i18n/locale/zh_CN.js | 1 + 3 files changed, 45 insertions(+), 4 deletions(-) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue index cda59968cbb0..d1ffb81133da 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue @@ -55,12 +55,12 @@ {{$t('Executor')}} - -
+ +
{{$t('host')}}
- + {{$t('Operation')}} @@ -99,7 +99,7 @@ {{item.host}} - - +
+ + + + {{item.count}} + + + + @@ -422,6 +448,18 @@ }) } }, + /** + * Start from force success + * @param START_FROM_FORCED_SUCCESS + */ + _startFromForceSuccess (item, index) { + this._countDownFn({ + id: item.id, + executeType: 'START_FROM_FORCED_SUCCESS', + index: index, + buttonType: 'forced' + }) + }, /** * pause * @param PAUSE @@ -570,6 +608,7 @@ } }, created () { + }, mounted () { }, diff --git a/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js b/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js index a901bf31c366..23348f77f7a4 100755 --- a/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js +++ b/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js @@ -33,6 +33,7 @@ export default { 'View history': 'View history', 'View log': 'View log', 'Force success': 'Force success', + 'Start From Force Success': 'Start From Force Success', 'Enter this child node': 'Enter this child node', 'Node name': 'Node name', 'Run flag': 'Run flag', diff --git a/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js b/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js index bc080d5c0ea6..6b7da2265759 100755 --- a/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js +++ b/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js @@ -33,6 +33,7 @@ export default { 'View history': '查看历史', 'View log': '查看日志', 'Force success': '强制成功', + 'Start From Force Success': '从强制成功处继续', 'Enter this child node': '进入该子节点', 'Node name': '节点名称', 'Please enter name (required)': '请输入名称(必填)', From cb647e9331a1f8fe963a650643a2990d91205603 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9B=B9=E8=81=9A=E9=98=B3?= Date: Thu, 3 Dec 2020 23:38:35 +0800 Subject: [PATCH 41/49] correct param name --- .../pages/projects/pages/instance/pages/list/_source/list.vue | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue index d1ffb81133da..604d23baad0e 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue @@ -455,7 +455,7 @@ _startFromForceSuccess (item, index) { this._countDownFn({ id: item.id, - executeType: 'START_FROM_FORCED_SUCCESS', + executeType: 'RESUME_FROM_FORCED_SUCCESS', index: index, buttonType: 'forced' }) From cc2068bcec2e1ebc2d297ece81f212bbda891d1e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9B=B9=E8=81=9A=E9=98=B3?= Date: Thu, 3 Dec 2020 23:45:38 +0800 Subject: [PATCH 42/49] correct names --- .../pages/instance/pages/list/_source/list.vue | 12 ++++++------ .../src/js/module/i18n/locale/en_US.js | 2 +- .../src/js/module/i18n/locale/zh_CN.js | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue index 604d23baad0e..30e9e43fcc60 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue @@ -129,8 +129,8 @@ shape="circle" size="xsmall" data-toggle="tooltip" - :title="$t('Start From Force Success')" - @click="_startFromForceSuccess(item,$index)" + :title="$t('Resume From Forced Success')" + @click="_resumeFromForcedSuccess(item,$index)" icon="ans-icon-notice-empty" :disabled="item.state !== 'FORCED_SUCCESS'"> - + Date: Sat, 19 Dec 2020 19:00:50 +0800 Subject: [PATCH 43/49] add pasue icon button and adjust operation button style --- .../instance/pages/list/_source/list.vue | 65 +++++++++++-------- .../pages/instance/pages/list/index.vue | 17 +---- 2 files changed, 40 insertions(+), 42 deletions(-) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue index 4d4c03b32b07..4f7bbccaa51e 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue @@ -17,7 +17,7 @@