|
@@ -159,34 +159,38 @@ public class WorkflowExecuteThread {
|
|
|
private Map<Integer, TaskInstance> taskInstanceMap = new ConcurrentHashMap<>();
|
|
|
|
|
|
|
|
|
- * running TaskNode, taskId as key
|
|
|
+ * running taskProcessor, taskCode as key, taskProcessor as value
|
|
|
+ * only on taskProcessor per taskCode
|
|
|
*/
|
|
|
- private final Map<Integer, ITaskProcessor> activeTaskProcessorMaps = new ConcurrentHashMap<>();
|
|
|
+ private final Map<Long, ITaskProcessor> activeTaskProcessorMaps = new ConcurrentHashMap<>();
|
|
|
|
|
|
|
|
|
* valid task map, taskCode as key, taskId as value
|
|
|
+ * in a DAG, only one taskInstance per taskCode is valid
|
|
|
*/
|
|
|
- private Map<String, Integer> validTaskMap = new ConcurrentHashMap<>();
|
|
|
+ private Map<Long, Integer> validTaskMap = new ConcurrentHashMap<>();
|
|
|
|
|
|
|
|
|
- * error task map, taskCode as key, taskId as value
|
|
|
+ * error task map, taskCode as key, taskInstanceId as value
|
|
|
+ * in a DAG, only one taskInstance per taskCode is valid
|
|
|
*/
|
|
|
- private Map<String, Integer> errorTaskMap = new ConcurrentHashMap<>();
|
|
|
+ private Map<Long, Integer> errorTaskMap = new ConcurrentHashMap<>();
|
|
|
|
|
|
|
|
|
- * complete task map, taskCode as key, taskId as value
|
|
|
+ * complete task map, taskCode as key, taskInstanceId as value
|
|
|
+ * in a DAG, only one taskInstance per taskCode is valid
|
|
|
*/
|
|
|
- private Map<String, Integer> completeTaskMap = new ConcurrentHashMap<>();
|
|
|
+ private Map<Long, Integer> completeTaskMap = new ConcurrentHashMap<>();
|
|
|
|
|
|
|
|
|
* depend failed task map, taskCode as key, taskId as value
|
|
|
*/
|
|
|
- private Map<String, Integer> dependFailedTaskMap = new ConcurrentHashMap<>();
|
|
|
+ private Map<Long, Integer> dependFailedTaskMap = new ConcurrentHashMap<>();
|
|
|
|
|
|
|
|
|
* forbidden task map, code as key
|
|
|
*/
|
|
|
- private Map<String, TaskNode> forbiddenTaskMap = new ConcurrentHashMap<>();
|
|
|
+ private Map<Long, TaskNode> forbiddenTaskMap = new ConcurrentHashMap<>();
|
|
|
|
|
|
|
|
|
* skip task map, code as key
|
|
@@ -208,6 +212,12 @@ public class WorkflowExecuteThread {
|
|
|
*/
|
|
|
private PeerTaskInstancePriorityQueue readyToSubmitTaskQueue = new PeerTaskInstancePriorityQueue();
|
|
|
|
|
|
+
|
|
|
+ * wait to retry taskInstance map, taskCode as key, taskInstance as value
|
|
|
+ * before retry, the taskInstance id is 0
|
|
|
+ */
|
|
|
+ private Map<Long, TaskInstance> waitToRetryTaskInstanceMap = new ConcurrentHashMap<>();
|
|
|
+
|
|
|
|
|
|
* state wheel execute thread
|
|
|
*/
|
|
@@ -317,6 +327,9 @@ public class WorkflowExecuteThread {
|
|
|
case WAIT_TASK_GROUP:
|
|
|
result = checkForceStartAndWakeUp(stateEvent);
|
|
|
break;
|
|
|
+ case TASK_RETRY:
|
|
|
+ result = taskRetryEventHandler(stateEvent);
|
|
|
+ break;
|
|
|
default:
|
|
|
break;
|
|
|
}
|
|
@@ -330,8 +343,8 @@ public class WorkflowExecuteThread {
|
|
|
private boolean checkForceStartAndWakeUp(StateEvent stateEvent) {
|
|
|
TaskGroupQueue taskGroupQueue = this.processService.loadTaskGroupQueue(stateEvent.getTaskInstanceId());
|
|
|
if (taskGroupQueue.getForceStart() == Flag.YES.getCode()) {
|
|
|
- ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(stateEvent.getTaskInstanceId());
|
|
|
TaskInstance taskInstance = this.processService.findTaskInstanceById(stateEvent.getTaskInstanceId());
|
|
|
+ ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskInstance.getTaskCode());
|
|
|
ProcessInstance processInstance = this.processService.findProcessInstanceById(taskInstance.getProcessInstanceId());
|
|
|
taskProcessor.init(taskInstance, processInstance);
|
|
|
taskProcessor.action(TaskAction.DISPATCH);
|
|
@@ -341,8 +354,8 @@ public class WorkflowExecuteThread {
|
|
|
if (taskGroupQueue.getInQueue() == Flag.YES.getCode()) {
|
|
|
boolean acquireTaskGroup = processService.acquireTaskGroupAgain(taskGroupQueue);
|
|
|
if (acquireTaskGroup) {
|
|
|
- ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(stateEvent.getTaskInstanceId());
|
|
|
TaskInstance taskInstance = this.processService.findTaskInstanceById(stateEvent.getTaskInstanceId());
|
|
|
+ ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskInstance.getTaskCode());
|
|
|
ProcessInstance processInstance = this.processService.findProcessInstanceById(taskInstance.getProcessInstanceId());
|
|
|
taskProcessor.init(taskInstance, processInstance);
|
|
|
taskProcessor.action(TaskAction.DISPATCH);
|
|
@@ -363,7 +376,7 @@ public class WorkflowExecuteThread {
|
|
|
}
|
|
|
TaskTimeoutStrategy taskTimeoutStrategy = taskInstance.getTaskDefine().getTimeoutNotifyStrategy();
|
|
|
if (TaskTimeoutStrategy.FAILED == taskTimeoutStrategy) {
|
|
|
- ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(stateEvent.getTaskInstanceId());
|
|
|
+ ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskInstance.getTaskCode());
|
|
|
taskProcessor.action(TaskAction.TIMEOUT);
|
|
|
} else {
|
|
|
processAlertManager.sendTaskTimeoutAlert(processInstance, taskInstance, taskInstance.getTaskDefine());
|
|
@@ -387,79 +400,57 @@ public class WorkflowExecuteThread {
|
|
|
return true;
|
|
|
}
|
|
|
|
|
|
- if (task.getState().typeIsFinished() && !completeTaskMap.containsKey(Long.toString(task.getTaskCode()))) {
|
|
|
+ if (task.getState().typeIsFinished()) {
|
|
|
+ if (completeTaskMap.containsKey(task.getTaskCode()) && completeTaskMap.get(task.getTaskCode()) == task.getId()) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
taskFinished(task);
|
|
|
if (task.getTaskGroupId() > 0) {
|
|
|
-
|
|
|
- TaskInstance nextTaskInstance = this.processService.releaseTaskGroup(task);
|
|
|
- if (nextTaskInstance != null) {
|
|
|
- if (nextTaskInstance.getProcessInstanceId() == task.getProcessInstanceId()) {
|
|
|
- StateEvent nextEvent = new StateEvent();
|
|
|
- nextEvent.setProcessInstanceId(this.processInstance.getId());
|
|
|
- nextEvent.setTaskInstanceId(nextTaskInstance.getId());
|
|
|
- nextEvent.setType(StateEventType.WAIT_TASK_GROUP);
|
|
|
- this.stateEvents.add(nextEvent);
|
|
|
- } else {
|
|
|
- ProcessInstance processInstance = this.processService.findProcessInstanceById(nextTaskInstance.getProcessInstanceId());
|
|
|
- this.processService.sendStartTask2Master(processInstance, nextTaskInstance.getId(),
|
|
|
- org.apache.dolphinscheduler.remote.command.CommandType.TASK_WAKEUP_EVENT_REQUEST);
|
|
|
- }
|
|
|
- }
|
|
|
+ releaseTaskGroup(task);
|
|
|
}
|
|
|
- } else if (activeTaskProcessorMaps.containsKey(stateEvent.getTaskInstanceId())) {
|
|
|
- ITaskProcessor iTaskProcessor = activeTaskProcessorMaps.get(stateEvent.getTaskInstanceId());
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ if (activeTaskProcessorMaps.containsKey(task.getTaskCode())) {
|
|
|
+ ITaskProcessor iTaskProcessor = activeTaskProcessorMaps.get(task.getTaskCode());
|
|
|
iTaskProcessor.action(TaskAction.RUN);
|
|
|
|
|
|
- if (iTaskProcessor.taskState().typeIsFinished()) {
|
|
|
- task = processService.findTaskInstanceById(stateEvent.getTaskInstanceId());
|
|
|
+ if (iTaskProcessor.taskInstance().getState().typeIsFinished()) {
|
|
|
taskFinished(task);
|
|
|
}
|
|
|
- } else {
|
|
|
- logger.error("state handler error: {}", stateEvent);
|
|
|
+ return true;
|
|
|
}
|
|
|
+ logger.error("state handler error: {}", stateEvent);
|
|
|
+
|
|
|
return true;
|
|
|
}
|
|
|
|
|
|
- private void taskFinished(TaskInstance task) {
|
|
|
- logger.info("work flow {} task {} state:{} ",
|
|
|
+ private void taskFinished(TaskInstance taskInstance) {
|
|
|
+ logger.info("work flow {} task id:{} code:{} state:{} ",
|
|
|
processInstance.getId(),
|
|
|
- task.getId(),
|
|
|
- task.getState());
|
|
|
- if (task.taskCanRetry()) {
|
|
|
- addTaskToStandByList(task);
|
|
|
- if (!task.retryTaskIntervalOverTime()) {
|
|
|
- logger.info("failure task will be submitted: process id: {}, task instance id: {} state:{} retry times:{} / {}, interval:{}",
|
|
|
- processInstance.getId(),
|
|
|
- task.getId(),
|
|
|
- task.getState(),
|
|
|
- task.getRetryTimes(),
|
|
|
- task.getMaxRetryTimes(),
|
|
|
- task.getRetryInterval());
|
|
|
- stateWheelExecuteThread.addTask4TimeoutCheck(task);
|
|
|
- stateWheelExecuteThread.addTask4RetryCheck(task);
|
|
|
- } else {
|
|
|
- submitStandByTask();
|
|
|
- stateWheelExecuteThread.removeTask4TimeoutCheck(task);
|
|
|
- stateWheelExecuteThread.removeTask4RetryCheck(task);
|
|
|
- }
|
|
|
- return;
|
|
|
- }
|
|
|
+ taskInstance.getId(),
|
|
|
+ taskInstance.getTaskCode(),
|
|
|
+ taskInstance.getState());
|
|
|
|
|
|
- completeTaskMap.put(Long.toString(task.getTaskCode()), task.getId());
|
|
|
- activeTaskProcessorMaps.remove(task.getId());
|
|
|
- stateWheelExecuteThread.removeTask4TimeoutCheck(task);
|
|
|
- stateWheelExecuteThread.removeTask4RetryCheck(task);
|
|
|
+ activeTaskProcessorMaps.remove(taskInstance.getTaskCode());
|
|
|
+ stateWheelExecuteThread.removeTask4TimeoutCheck(processInstance, taskInstance);
|
|
|
+ stateWheelExecuteThread.removeTask4RetryCheck(processInstance, taskInstance);
|
|
|
+ stateWheelExecuteThread.removeTask4StateCheck(processInstance, taskInstance);
|
|
|
|
|
|
- if (task.getState().typeIsSuccess()) {
|
|
|
- processInstance.setVarPool(task.getVarPool());
|
|
|
+ if (taskInstance.getState().typeIsSuccess()) {
|
|
|
+ completeTaskMap.put(taskInstance.getTaskCode(), taskInstance.getId());
|
|
|
+ processInstance.setVarPool(taskInstance.getVarPool());
|
|
|
processService.saveProcessInstance(processInstance);
|
|
|
- submitPostNode(Long.toString(task.getTaskCode()));
|
|
|
- } else if (task.getState().typeIsFailure()) {
|
|
|
- if (task.isConditionsTask()
|
|
|
- || DagHelper.haveConditionsAfterNode(Long.toString(task.getTaskCode()), dag)) {
|
|
|
- submitPostNode(Long.toString(task.getTaskCode()));
|
|
|
+ submitPostNode(Long.toString(taskInstance.getTaskCode()));
|
|
|
+ } else if (taskInstance.taskCanRetry()) {
|
|
|
+
|
|
|
+ retryTaskInstance(taskInstance);
|
|
|
+ } else if (taskInstance.getState().typeIsFailure()) {
|
|
|
+ completeTaskMap.put(taskInstance.getTaskCode(), taskInstance.getId());
|
|
|
+ if (taskInstance.isConditionsTask()
|
|
|
+ || DagHelper.haveConditionsAfterNode(Long.toString(taskInstance.getTaskCode()), dag)) {
|
|
|
+ submitPostNode(Long.toString(taskInstance.getTaskCode()));
|
|
|
} else {
|
|
|
- errorTaskMap.put(Long.toString(task.getTaskCode()), task.getId());
|
|
|
+ errorTaskMap.put(taskInstance.getTaskCode(), taskInstance.getId());
|
|
|
if (processInstance.getFailureStrategy() == FailureStrategy.END) {
|
|
|
killAllTasks();
|
|
|
}
|
|
@@ -468,6 +459,73 @@ public class WorkflowExecuteThread {
|
|
|
this.updateProcessInstanceState();
|
|
|
}
|
|
|
|
|
|
+
|
|
|
+ * release task group
|
|
|
+ * @param taskInstance
|
|
|
+ */
|
|
|
+ private void releaseTaskGroup(TaskInstance taskInstance) {
|
|
|
+ if (taskInstance.getTaskGroupId() > 0) {
|
|
|
+ TaskInstance nextTaskInstance = this.processService.releaseTaskGroup(taskInstance);
|
|
|
+ if (nextTaskInstance != null) {
|
|
|
+ if (nextTaskInstance.getProcessInstanceId() == taskInstance.getProcessInstanceId()) {
|
|
|
+ StateEvent nextEvent = new StateEvent();
|
|
|
+ nextEvent.setProcessInstanceId(this.processInstance.getId());
|
|
|
+ nextEvent.setTaskInstanceId(nextTaskInstance.getId());
|
|
|
+ nextEvent.setType(StateEventType.WAIT_TASK_GROUP);
|
|
|
+ this.stateEvents.add(nextEvent);
|
|
|
+ } else {
|
|
|
+ ProcessInstance processInstance = this.processService.findProcessInstanceById(nextTaskInstance.getProcessInstanceId());
|
|
|
+ this.processService.sendStartTask2Master(processInstance, nextTaskInstance.getId(),
|
|
|
+ org.apache.dolphinscheduler.remote.command.CommandType.TASK_WAKEUP_EVENT_REQUEST);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ * crate new task instance to retry, different objects from the original
|
|
|
+ * @param taskInstance
|
|
|
+ */
|
|
|
+ private void retryTaskInstance(TaskInstance taskInstance) {
|
|
|
+ if (!taskInstance.taskCanRetry()) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ TaskInstance newTaskInstance = cloneRetryTaskInstance(taskInstance);
|
|
|
+ if (newTaskInstance == null) {
|
|
|
+ logger.error("retry fail, new taskInstancce is null, task code:{}, task id:{}", taskInstance.getTaskCode(), taskInstance.getId());
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ waitToRetryTaskInstanceMap.put(newTaskInstance.getTaskCode(), newTaskInstance);
|
|
|
+ if (!taskInstance.retryTaskIntervalOverTime()) {
|
|
|
+ logger.info("failure task will be submitted: process id: {}, task instance code: {} state:{} retry times:{} / {}, interval:{}",
|
|
|
+ processInstance.getId(),
|
|
|
+ newTaskInstance.getTaskCode(),
|
|
|
+ newTaskInstance.getState(),
|
|
|
+ newTaskInstance.getRetryTimes(),
|
|
|
+ newTaskInstance.getMaxRetryTimes(),
|
|
|
+ newTaskInstance.getRetryInterval());
|
|
|
+ stateWheelExecuteThread.addTask4TimeoutCheck(processInstance, newTaskInstance);
|
|
|
+ stateWheelExecuteThread.addTask4RetryCheck(processInstance, newTaskInstance);
|
|
|
+ } else {
|
|
|
+ addTaskToStandByList(newTaskInstance);
|
|
|
+ submitStandByTask();
|
|
|
+ waitToRetryTaskInstanceMap.remove(newTaskInstance.getTaskCode());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ * handle task retry event
|
|
|
+ * @param stateEvent
|
|
|
+ * @return
|
|
|
+ */
|
|
|
+ private boolean taskRetryEventHandler(StateEvent stateEvent) {
|
|
|
+ TaskInstance taskInstance = waitToRetryTaskInstanceMap.get(stateEvent.getTaskCode());
|
|
|
+ addTaskToStandByList(taskInstance);
|
|
|
+ submitStandByTask();
|
|
|
+ waitToRetryTaskInstanceMap.remove(stateEvent.getTaskCode());
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
|
|
|
* update process instance
|
|
|
*/
|
|
@@ -492,9 +550,9 @@ public class WorkflowExecuteThread {
|
|
|
processService.packageTaskInstance(taskInstance, processInstance);
|
|
|
taskInstanceMap.put(taskInstance.getId(), taskInstance);
|
|
|
|
|
|
- validTaskMap.remove(Long.toString(taskInstance.getTaskCode()));
|
|
|
+ validTaskMap.remove(taskInstance.getTaskCode());
|
|
|
if (Flag.YES == taskInstance.getFlag()) {
|
|
|
- validTaskMap.put(Long.toString(taskInstance.getTaskCode()), taskInstance.getId());
|
|
|
+ validTaskMap.put(taskInstance.getTaskCode(), taskInstance.getId());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -561,6 +619,20 @@ public class WorkflowExecuteThread {
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
+ public TaskInstance getActiveTaskInstanceByTaskCode(long taskCode) {
|
|
|
+ if (activeTaskProcessorMaps.containsKey(taskCode)) {
|
|
|
+ return activeTaskProcessorMaps.get(taskCode).taskInstance();
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ public TaskInstance getRetryTaskInstanceByTaskCode(long taskCode) {
|
|
|
+ if (waitToRetryTaskInstanceMap.containsKey(taskCode)) {
|
|
|
+ return waitToRetryTaskInstanceMap.get(taskCode);
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
private boolean processStateChangeHandler(StateEvent stateEvent) {
|
|
|
try {
|
|
|
logger.info("process:{} state {} change to {}", processInstance.getId(), processInstance.getState(), stateEvent.getExecutionStatus());
|
|
@@ -736,7 +808,7 @@ public class WorkflowExecuteThread {
|
|
|
|
|
|
taskNodeList.forEach(taskNode -> {
|
|
|
if (taskNode.isForbidden()) {
|
|
|
- forbiddenTaskMap.put(Long.toString(taskNode.getCode()), taskNode);
|
|
|
+ forbiddenTaskMap.put(taskNode.getCode(), taskNode);
|
|
|
}
|
|
|
});
|
|
|
|
|
@@ -767,17 +839,39 @@ public class WorkflowExecuteThread {
|
|
|
if (!isNewProcessInstance()) {
|
|
|
List<TaskInstance> validTaskInstanceList = processService.findValidTaskListByProcessId(processInstance.getId());
|
|
|
for (TaskInstance task : validTaskInstanceList) {
|
|
|
- validTaskMap.put(Long.toString(task.getTaskCode()), task.getId());
|
|
|
+ if (validTaskMap.containsKey(task.getTaskCode())) {
|
|
|
+ int oldTaskInstanceId = validTaskMap.get(task.getTaskCode());
|
|
|
+ TaskInstance oldTaskInstance = taskInstanceMap.get(oldTaskInstanceId);
|
|
|
+ if (!oldTaskInstance.getState().typeIsFinished() && task.getState().typeIsFinished()) {
|
|
|
+ task.setFlag(Flag.NO);
|
|
|
+ processService.updateTaskInstance(task);
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ logger.warn("have same taskCode taskInstance when init task queue, taskCode:{}", task.getTaskCode());
|
|
|
+ }
|
|
|
+
|
|
|
+ validTaskMap.put(task.getTaskCode(), task.getId());
|
|
|
taskInstanceMap.put(task.getId(), task);
|
|
|
|
|
|
if (task.isTaskComplete()) {
|
|
|
- completeTaskMap.put(Long.toString(task.getTaskCode()), task.getId());
|
|
|
+ completeTaskMap.put(task.getTaskCode(), task.getId());
|
|
|
+ continue;
|
|
|
}
|
|
|
if (task.isConditionsTask() || DagHelper.haveConditionsAfterNode(Long.toString(task.getTaskCode()), dag)) {
|
|
|
continue;
|
|
|
}
|
|
|
- if (task.getState().typeIsFailure() && !task.taskCanRetry()) {
|
|
|
- errorTaskMap.put(Long.toString(task.getTaskCode()), task.getId());
|
|
|
+ if (task.taskCanRetry()) {
|
|
|
+ if (task.getState() == ExecutionStatus.NEED_FAULT_TOLERANCE) {
|
|
|
+
|
|
|
+ TaskInstance tolerantTaskInstance = cloneTolerantTaskInstance(task);
|
|
|
+ addTaskToStandByList(tolerantTaskInstance);
|
|
|
+ } else {
|
|
|
+ retryTaskInstance(task);
|
|
|
+ }
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ if (task.getState().typeIsFailure()) {
|
|
|
+ errorTaskMap.put(task.getTaskCode(), task.getId());
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -832,19 +926,32 @@ public class WorkflowExecuteThread {
|
|
|
taskInstance.getId(), taskInstance.getName());
|
|
|
return null;
|
|
|
}
|
|
|
- validTaskMap.put(Long.toString(taskInstance.getTaskCode()), taskInstance.getId());
|
|
|
+
|
|
|
+
|
|
|
+ if (validTaskMap.containsKey(taskInstance.getTaskCode())) {
|
|
|
+ int oldTaskInstanceId = validTaskMap.get(taskInstance.getTaskCode());
|
|
|
+ if (taskInstance.getId() != oldTaskInstanceId) {
|
|
|
+ TaskInstance oldTaskInstance = taskInstanceMap.get(oldTaskInstanceId);
|
|
|
+ oldTaskInstance.setFlag(Flag.NO);
|
|
|
+ processService.updateTaskInstance(oldTaskInstance);
|
|
|
+ validTaskMap.remove(taskInstance.getTaskCode());
|
|
|
+ activeTaskProcessorMaps.remove(taskInstance.getTaskCode());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ validTaskMap.put(taskInstance.getTaskCode(), taskInstance.getId());
|
|
|
taskInstanceMap.put(taskInstance.getId(), taskInstance);
|
|
|
- activeTaskProcessorMaps.put(taskInstance.getId(), taskProcessor);
|
|
|
+ activeTaskProcessorMaps.put(taskInstance.getTaskCode(), taskProcessor);
|
|
|
taskProcessor.action(TaskAction.RUN);
|
|
|
|
|
|
- stateWheelExecuteThread.addTask4TimeoutCheck(taskInstance);
|
|
|
- stateWheelExecuteThread.addTask4RetryCheck(taskInstance);
|
|
|
+ stateWheelExecuteThread.addTask4TimeoutCheck(processInstance, taskInstance);
|
|
|
+ stateWheelExecuteThread.addTask4StateCheck(processInstance, taskInstance);
|
|
|
|
|
|
- if (taskProcessor.taskState().typeIsFinished()) {
|
|
|
+ if (taskProcessor.taskInstance().getState().typeIsFinished()) {
|
|
|
StateEvent stateEvent = new StateEvent();
|
|
|
stateEvent.setProcessInstanceId(this.processInstance.getId());
|
|
|
stateEvent.setTaskInstanceId(taskInstance.getId());
|
|
|
- stateEvent.setExecutionStatus(taskProcessor.taskState());
|
|
|
+ stateEvent.setExecutionStatus(taskProcessor.taskInstance().getState());
|
|
|
stateEvent.setType(StateEventType.TASK_STATE_CHANGE);
|
|
|
this.stateEvents.add(stateEvent);
|
|
|
}
|
|
@@ -898,78 +1005,129 @@ public class WorkflowExecuteThread {
|
|
|
*/
|
|
|
private TaskInstance createTaskInstance(ProcessInstance processInstance, TaskNode taskNode) {
|
|
|
TaskInstance taskInstance = findTaskIfExists(taskNode.getCode(), taskNode.getVersion());
|
|
|
- if (taskInstance == null) {
|
|
|
- taskInstance = new TaskInstance();
|
|
|
- taskInstance.setTaskCode(taskNode.getCode());
|
|
|
- taskInstance.setTaskDefinitionVersion(taskNode.getVersion());
|
|
|
-
|
|
|
- taskInstance.setName(taskNode.getName());
|
|
|
-
|
|
|
- taskInstance.setState(ExecutionStatus.SUBMITTED_SUCCESS);
|
|
|
-
|
|
|
- taskInstance.setProcessInstanceId(processInstance.getId());
|
|
|
-
|
|
|
- taskInstance.setTaskType(taskNode.getType().toUpperCase());
|
|
|
-
|
|
|
- taskInstance.setAlertFlag(Flag.NO);
|
|
|
-
|
|
|
-
|
|
|
- taskInstance.setStartTime(null);
|
|
|
-
|
|
|
-
|
|
|
- taskInstance.setFlag(Flag.YES);
|
|
|
-
|
|
|
-
|
|
|
- taskInstance.setDryRun(processInstance.getDryRun());
|
|
|
-
|
|
|
-
|
|
|
- taskInstance.setRetryTimes(0);
|
|
|
-
|
|
|
-
|
|
|
- taskInstance.setMaxRetryTimes(taskNode.getMaxRetryTimes());
|
|
|
-
|
|
|
-
|
|
|
- taskInstance.setRetryInterval(taskNode.getRetryInterval());
|
|
|
-
|
|
|
-
|
|
|
- taskInstance.setTaskParams(taskNode.getTaskParams());
|
|
|
-
|
|
|
-
|
|
|
- taskInstance.setTaskGroupId(taskNode.getTaskGroupId());
|
|
|
- taskInstance.setTaskGroupPriority(taskNode.getTaskGroupPriority());
|
|
|
-
|
|
|
-
|
|
|
- if (taskNode.getTaskInstancePriority() == null) {
|
|
|
- taskInstance.setTaskInstancePriority(Priority.MEDIUM);
|
|
|
- } else {
|
|
|
- taskInstance.setTaskInstancePriority(taskNode.getTaskInstancePriority());
|
|
|
- }
|
|
|
+ if (taskInstance != null) {
|
|
|
+ return taskInstance;
|
|
|
+ }
|
|
|
+
|
|
|
+ return newTaskInstance(processInstance, taskNode);
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ * clone a new taskInstance for retry and reset some logic fields
|
|
|
+ * @return
|
|
|
+ */
|
|
|
+ public TaskInstance cloneRetryTaskInstance(TaskInstance taskInstance) {
|
|
|
+ TaskNode taskNode = dag.getNode(Long.toString(taskInstance.getTaskCode()));
|
|
|
+ if (taskNode == null) {
|
|
|
+ logger.error("taskNode is null, code:{}", taskInstance.getTaskCode());
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ TaskInstance newTaskInstance = newTaskInstance(processInstance, taskNode);
|
|
|
+ newTaskInstance.setTaskDefine(taskInstance.getTaskDefine());
|
|
|
+ newTaskInstance.setProcessDefine(taskInstance.getProcessDefine());
|
|
|
+ newTaskInstance.setProcessInstance(processInstance);
|
|
|
+ newTaskInstance.setRetryTimes(taskInstance.getRetryTimes() + 1);
|
|
|
+
|
|
|
+ newTaskInstance.setState(taskInstance.getState());
|
|
|
+ newTaskInstance.setEndTime(taskInstance.getEndTime());
|
|
|
+ return newTaskInstance;
|
|
|
+ }
|
|
|
|
|
|
- String processWorkerGroup = processInstance.getWorkerGroup();
|
|
|
- processWorkerGroup = StringUtils.isBlank(processWorkerGroup) ? DEFAULT_WORKER_GROUP : processWorkerGroup;
|
|
|
- String taskWorkerGroup = StringUtils.isBlank(taskNode.getWorkerGroup()) ? processWorkerGroup : taskNode.getWorkerGroup();
|
|
|
+
|
|
|
+ * clone a new taskInstance for tolerant and reset some logic fields
|
|
|
+ * @return
|
|
|
+ */
|
|
|
+ public TaskInstance cloneTolerantTaskInstance(TaskInstance taskInstance) {
|
|
|
+ TaskNode taskNode = dag.getNode(Long.toString(taskInstance.getTaskCode()));
|
|
|
+ if (taskNode == null) {
|
|
|
+ logger.error("taskNode is null, code:{}", taskInstance.getTaskCode());
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ TaskInstance newTaskInstance = newTaskInstance(processInstance, taskNode);
|
|
|
+ newTaskInstance.setTaskDefine(taskInstance.getTaskDefine());
|
|
|
+ newTaskInstance.setProcessDefine(taskInstance.getProcessDefine());
|
|
|
+ newTaskInstance.setProcessInstance(processInstance);
|
|
|
+ newTaskInstance.setRetryTimes(taskInstance.getRetryTimes());
|
|
|
+ newTaskInstance.setState(taskInstance.getState());
|
|
|
+ return newTaskInstance;
|
|
|
+ }
|
|
|
|
|
|
- Long processEnvironmentCode = Objects.isNull(processInstance.getEnvironmentCode()) ? -1 : processInstance.getEnvironmentCode();
|
|
|
- Long taskEnvironmentCode = Objects.isNull(taskNode.getEnvironmentCode()) ? processEnvironmentCode : taskNode.getEnvironmentCode();
|
|
|
+
|
|
|
+ * new a taskInstance
|
|
|
+ * @param processInstance
|
|
|
+ * @param taskNode
|
|
|
+ * @return
|
|
|
+ */
|
|
|
+ public TaskInstance newTaskInstance(ProcessInstance processInstance, TaskNode taskNode) {
|
|
|
+ TaskInstance taskInstance = new TaskInstance();
|
|
|
+ taskInstance.setTaskCode(taskNode.getCode());
|
|
|
+ taskInstance.setTaskDefinitionVersion(taskNode.getVersion());
|
|
|
+
|
|
|
+ taskInstance.setName(taskNode.getName());
|
|
|
+
|
|
|
+ taskInstance.setState(ExecutionStatus.SUBMITTED_SUCCESS);
|
|
|
+
|
|
|
+ taskInstance.setProcessInstanceId(processInstance.getId());
|
|
|
+
|
|
|
+ taskInstance.setTaskType(taskNode.getType().toUpperCase());
|
|
|
+
|
|
|
+ taskInstance.setAlertFlag(Flag.NO);
|
|
|
+
|
|
|
+
|
|
|
+ taskInstance.setStartTime(null);
|
|
|
+
|
|
|
+
|
|
|
+ taskInstance.setFlag(Flag.YES);
|
|
|
+
|
|
|
+
|
|
|
+ taskInstance.setDryRun(processInstance.getDryRun());
|
|
|
+
|
|
|
+
|
|
|
+ taskInstance.setRetryTimes(0);
|
|
|
+
|
|
|
+
|
|
|
+ taskInstance.setMaxRetryTimes(taskNode.getMaxRetryTimes());
|
|
|
+
|
|
|
+
|
|
|
+ taskInstance.setRetryInterval(taskNode.getRetryInterval());
|
|
|
+
|
|
|
+
|
|
|
+ taskInstance.setTaskParams(taskNode.getTaskParams());
|
|
|
+
|
|
|
+
|
|
|
+ taskInstance.setTaskGroupId(taskNode.getTaskGroupId());
|
|
|
+ taskInstance.setTaskGroupPriority(taskNode.getTaskGroupPriority());
|
|
|
+
|
|
|
+
|
|
|
+ if (taskNode.getTaskInstancePriority() == null) {
|
|
|
+ taskInstance.setTaskInstancePriority(Priority.MEDIUM);
|
|
|
+ } else {
|
|
|
+ taskInstance.setTaskInstancePriority(taskNode.getTaskInstancePriority());
|
|
|
+ }
|
|
|
|
|
|
- if (!processWorkerGroup.equals(DEFAULT_WORKER_GROUP) && taskWorkerGroup.equals(DEFAULT_WORKER_GROUP)) {
|
|
|
- taskInstance.setWorkerGroup(processWorkerGroup);
|
|
|
- taskInstance.setEnvironmentCode(processEnvironmentCode);
|
|
|
- } else {
|
|
|
- taskInstance.setWorkerGroup(taskWorkerGroup);
|
|
|
- taskInstance.setEnvironmentCode(taskEnvironmentCode);
|
|
|
- }
|
|
|
+ String processWorkerGroup = processInstance.getWorkerGroup();
|
|
|
+ processWorkerGroup = StringUtils.isBlank(processWorkerGroup) ? DEFAULT_WORKER_GROUP : processWorkerGroup;
|
|
|
+ String taskWorkerGroup = StringUtils.isBlank(taskNode.getWorkerGroup()) ? processWorkerGroup : taskNode.getWorkerGroup();
|
|
|
|
|
|
- if (!taskInstance.getEnvironmentCode().equals(-1L)) {
|
|
|
- Environment environment = processService.findEnvironmentByCode(taskInstance.getEnvironmentCode());
|
|
|
- if (Objects.nonNull(environment) && StringUtils.isNotEmpty(environment.getConfig())) {
|
|
|
- taskInstance.setEnvironmentConfig(environment.getConfig());
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- taskInstance.setDelayTime(taskNode.getDelayTime());
|
|
|
+ Long processEnvironmentCode = Objects.isNull(processInstance.getEnvironmentCode()) ? -1 : processInstance.getEnvironmentCode();
|
|
|
+ Long taskEnvironmentCode = Objects.isNull(taskNode.getEnvironmentCode()) ? processEnvironmentCode : taskNode.getEnvironmentCode();
|
|
|
+
|
|
|
+ if (!processWorkerGroup.equals(DEFAULT_WORKER_GROUP) && taskWorkerGroup.equals(DEFAULT_WORKER_GROUP)) {
|
|
|
+ taskInstance.setWorkerGroup(processWorkerGroup);
|
|
|
+ taskInstance.setEnvironmentCode(processEnvironmentCode);
|
|
|
+ } else {
|
|
|
+ taskInstance.setWorkerGroup(taskWorkerGroup);
|
|
|
+ taskInstance.setEnvironmentCode(taskEnvironmentCode);
|
|
|
}
|
|
|
|
|
|
+ if (!taskInstance.getEnvironmentCode().equals(-1L)) {
|
|
|
+ Environment environment = processService.findEnvironmentByCode(taskInstance.getEnvironmentCode());
|
|
|
+ if (Objects.nonNull(environment) && StringUtils.isNotEmpty(environment.getConfig())) {
|
|
|
+ taskInstance.setEnvironmentConfig(environment.getConfig());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ taskInstance.setDelayTime(taskNode.getDelayTime());
|
|
|
return taskInstance;
|
|
|
}
|
|
|
|
|
@@ -978,7 +1136,7 @@ public class WorkflowExecuteThread {
|
|
|
Map<String, TaskInstance> allTaskInstance = new HashMap<>();
|
|
|
if (CollectionUtils.isNotEmpty(preTask)) {
|
|
|
for (String preTaskCode : preTask) {
|
|
|
- Integer taskId = completeTaskMap.get(preTaskCode);
|
|
|
+ Integer taskId = completeTaskMap.get(Long.parseLong(preTaskCode));
|
|
|
if (taskId == null) {
|
|
|
continue;
|
|
|
}
|
|
@@ -1073,7 +1231,7 @@ public class WorkflowExecuteThread {
|
|
|
continue;
|
|
|
}
|
|
|
|
|
|
- if (completeTaskMap.containsKey(Long.toString(task.getTaskCode()))) {
|
|
|
+ if (task.getId() > 0 && completeTaskMap.containsKey(task.getTaskCode())) {
|
|
|
logger.info("task {} has already run success", task.getName());
|
|
|
continue;
|
|
|
}
|
|
@@ -1106,10 +1264,11 @@ public class WorkflowExecuteThread {
|
|
|
for (String depsNode : indirectDepCodeList) {
|
|
|
if (dag.containsNode(depsNode) && !skipTaskNodeMap.containsKey(depsNode)) {
|
|
|
|
|
|
- if (!completeTaskMap.containsKey(depsNode)) {
|
|
|
+ Long despNodeTaskCode = Long.parseLong(depsNode);
|
|
|
+ if (!completeTaskMap.containsKey(despNodeTaskCode)) {
|
|
|
return DependResult.WAITING;
|
|
|
}
|
|
|
- Integer depsTaskId = completeTaskMap.get(depsNode);
|
|
|
+ Integer depsTaskId = completeTaskMap.get(despNodeTaskCode);
|
|
|
ExecutionStatus depTaskState = taskInstanceMap.get(depsTaskId).getState();
|
|
|
if (depTaskState.typeIsPause() || depTaskState.typeIsCancel()) {
|
|
|
return DependResult.NON_EXEC;
|
|
@@ -1138,7 +1297,7 @@ public class WorkflowExecuteThread {
|
|
|
TaskNode taskNode = dag.getNode(taskCode);
|
|
|
List<String> depCodeList = taskNode.getDepList();
|
|
|
for (String depsNode : depCodeList) {
|
|
|
- if (forbiddenTaskMap.containsKey(depsNode)) {
|
|
|
+ if (forbiddenTaskMap.containsKey(Long.parseLong(depsNode))) {
|
|
|
setIndirectDepList(depsNode, indirectDepCodeList);
|
|
|
} else {
|
|
|
indirectDepCodeList.add(depsNode);
|
|
@@ -1157,7 +1316,8 @@ public class WorkflowExecuteThread {
|
|
|
return false;
|
|
|
}
|
|
|
} else {
|
|
|
- Integer taskInstanceId = completeTaskMap.get(dependNodeName);
|
|
|
+ long taskCode = Long.parseLong(dependNodeName);
|
|
|
+ Integer taskInstanceId = completeTaskMap.get(taskCode);
|
|
|
ExecutionStatus depTaskState = taskInstanceMap.get(taskInstanceId).getState();
|
|
|
if (depTaskState.typeIsFailure()) {
|
|
|
return false;
|
|
@@ -1228,7 +1388,9 @@ public class WorkflowExecuteThread {
|
|
|
return true;
|
|
|
}
|
|
|
if (processInstance.getFailureStrategy() == FailureStrategy.CONTINUE) {
|
|
|
- return readyToSubmitTaskQueue.size() == 0 && activeTaskProcessorMaps.size() == 0;
|
|
|
+ return readyToSubmitTaskQueue.size() == 0
|
|
|
+ && activeTaskProcessorMaps.size() == 0
|
|
|
+ && waitToRetryTaskInstanceMap.size() == 0;
|
|
|
}
|
|
|
}
|
|
|
return false;
|
|
@@ -1310,7 +1472,7 @@ public class WorkflowExecuteThread {
|
|
|
|
|
|
if (state == ExecutionStatus.RUNNING_EXECUTION) {
|
|
|
List<TaskInstance> killTasks = getCompleteTaskByState(ExecutionStatus.KILL);
|
|
|
- if (readyToSubmitTaskQueue.size() > 0) {
|
|
|
+ if (readyToSubmitTaskQueue.size() > 0 || waitToRetryTaskInstanceMap.size() > 0) {
|
|
|
|
|
|
return ExecutionStatus.RUNNING_EXECUTION;
|
|
|
} else if (CollectionUtils.isNotEmpty(killTasks)) {
|
|
@@ -1445,19 +1607,23 @@ public class WorkflowExecuteThread {
|
|
|
private void killAllTasks() {
|
|
|
logger.info("kill called on process instance id: {}, num: {}", processInstance.getId(),
|
|
|
activeTaskProcessorMaps.size());
|
|
|
- for (int taskId : activeTaskProcessorMaps.keySet()) {
|
|
|
- TaskInstance taskInstance = processService.findTaskInstanceById(taskId);
|
|
|
+ for (long taskCode : activeTaskProcessorMaps.keySet()) {
|
|
|
+ ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskCode);
|
|
|
+ Integer taskInstanceId = validTaskMap.get(taskCode);
|
|
|
+ if (taskInstanceId == null || taskInstanceId.equals(0)) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ TaskInstance taskInstance = processService.findTaskInstanceById(taskInstanceId);
|
|
|
if (taskInstance == null || taskInstance.getState().typeIsFinished()) {
|
|
|
continue;
|
|
|
}
|
|
|
- ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskId);
|
|
|
taskProcessor.action(TaskAction.STOP);
|
|
|
- if (taskProcessor.taskState().typeIsFinished()) {
|
|
|
+ if (taskProcessor.taskInstance().getState().typeIsFinished()) {
|
|
|
StateEvent stateEvent = new StateEvent();
|
|
|
stateEvent.setType(StateEventType.TASK_STATE_CHANGE);
|
|
|
stateEvent.setProcessInstanceId(this.processInstance.getId());
|
|
|
stateEvent.setTaskInstanceId(taskInstance.getId());
|
|
|
- stateEvent.setExecutionStatus(taskProcessor.taskState());
|
|
|
+ stateEvent.setExecutionStatus(taskProcessor.taskInstance().getState());
|
|
|
this.addStateEvent(stateEvent);
|
|
|
}
|
|
|
}
|
|
@@ -1485,7 +1651,7 @@ public class WorkflowExecuteThread {
|
|
|
task.setState(retryTask.getState());
|
|
|
logger.info("task: {} has been forced success, put it into complete task list and stop retrying", task.getName());
|
|
|
removeTaskFromStandbyList(task);
|
|
|
- completeTaskMap.put(Long.toString(task.getTaskCode()), task.getId());
|
|
|
+ completeTaskMap.put(task.getTaskCode(), task.getId());
|
|
|
taskInstanceMap.put(task.getId(), task);
|
|
|
submitPostNode(Long.toString(task.getTaskCode()));
|
|
|
continue;
|
|
@@ -1499,21 +1665,15 @@ public class WorkflowExecuteThread {
|
|
|
}
|
|
|
DependResult dependResult = getDependResultForTask(task);
|
|
|
if (DependResult.SUCCESS == dependResult) {
|
|
|
- if (task.retryTaskIntervalOverTime()) {
|
|
|
- int originalId = task.getId();
|
|
|
- TaskInstance taskInstance = submitTaskExec(task);
|
|
|
- if (taskInstance == null) {
|
|
|
- this.taskFailedSubmit = true;
|
|
|
- } else {
|
|
|
- removeTaskFromStandbyList(task);
|
|
|
- if (taskInstance.getId() != originalId) {
|
|
|
- activeTaskProcessorMaps.remove(originalId);
|
|
|
- }
|
|
|
- }
|
|
|
+ TaskInstance taskInstance = submitTaskExec(task);
|
|
|
+ if (taskInstance == null) {
|
|
|
+ this.taskFailedSubmit = true;
|
|
|
+ } else {
|
|
|
+ removeTaskFromStandbyList(task);
|
|
|
}
|
|
|
} else if (DependResult.FAILED == dependResult) {
|
|
|
|
|
|
- dependFailedTaskMap.put(Long.toString(task.getTaskCode()), task.getId());
|
|
|
+ dependFailedTaskMap.put(task.getTaskCode(), task.getId());
|
|
|
removeTaskFromStandbyList(task);
|
|
|
logger.info("task {},id:{} depend result : {}", task.getName(), task.getId(), dependResult);
|
|
|
} else if (DependResult.NON_EXEC == dependResult) {
|