|
|
@ -35,16 +35,14 @@ import org.apache.dolphinscheduler.common.enums.CommandType; |
|
|
|
import org.apache.dolphinscheduler.common.enums.FailureStrategy; |
|
|
|
import org.apache.dolphinscheduler.common.enums.FailureStrategy; |
|
|
|
import org.apache.dolphinscheduler.common.enums.Flag; |
|
|
|
import org.apache.dolphinscheduler.common.enums.Flag; |
|
|
|
import org.apache.dolphinscheduler.common.enums.Priority; |
|
|
|
import org.apache.dolphinscheduler.common.enums.Priority; |
|
|
|
import org.apache.dolphinscheduler.common.enums.StateEvent; |
|
|
|
|
|
|
|
import org.apache.dolphinscheduler.common.enums.StateEventType; |
|
|
|
import org.apache.dolphinscheduler.common.enums.StateEventType; |
|
|
|
import org.apache.dolphinscheduler.common.enums.TaskDependType; |
|
|
|
import org.apache.dolphinscheduler.common.enums.TaskDependType; |
|
|
|
import org.apache.dolphinscheduler.common.enums.TaskGroupQueueStatus; |
|
|
|
import org.apache.dolphinscheduler.common.enums.TaskGroupQueueStatus; |
|
|
|
import org.apache.dolphinscheduler.common.enums.TimeoutFlag; |
|
|
|
|
|
|
|
import org.apache.dolphinscheduler.service.expand.CuringParamsService; |
|
|
|
|
|
|
|
import org.apache.dolphinscheduler.common.graph.DAG; |
|
|
|
import org.apache.dolphinscheduler.common.graph.DAG; |
|
|
|
import org.apache.dolphinscheduler.common.model.TaskNode; |
|
|
|
import org.apache.dolphinscheduler.common.model.TaskNode; |
|
|
|
import org.apache.dolphinscheduler.common.model.TaskNodeRelation; |
|
|
|
import org.apache.dolphinscheduler.common.model.TaskNodeRelation; |
|
|
|
import org.apache.dolphinscheduler.common.process.ProcessDag; |
|
|
|
import org.apache.dolphinscheduler.common.process.ProcessDag; |
|
|
|
|
|
|
|
import org.apache.dolphinscheduler.common.thread.ThreadUtils; |
|
|
|
import org.apache.dolphinscheduler.common.utils.DateUtils; |
|
|
|
import org.apache.dolphinscheduler.common.utils.DateUtils; |
|
|
|
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
|
|
|
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
|
|
|
import org.apache.dolphinscheduler.common.utils.LoggerUtils; |
|
|
|
import org.apache.dolphinscheduler.common.utils.LoggerUtils; |
|
|
@ -63,20 +61,23 @@ import org.apache.dolphinscheduler.dao.utils.DagHelper; |
|
|
|
import org.apache.dolphinscheduler.plugin.task.api.enums.DependResult; |
|
|
|
import org.apache.dolphinscheduler.plugin.task.api.enums.DependResult; |
|
|
|
import org.apache.dolphinscheduler.plugin.task.api.enums.Direct; |
|
|
|
import org.apache.dolphinscheduler.plugin.task.api.enums.Direct; |
|
|
|
import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus; |
|
|
|
import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus; |
|
|
|
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy; |
|
|
|
|
|
|
|
import org.apache.dolphinscheduler.plugin.task.api.model.Property; |
|
|
|
import org.apache.dolphinscheduler.plugin.task.api.model.Property; |
|
|
|
import org.apache.dolphinscheduler.plugin.task.api.parameters.BlockingParameters; |
|
|
|
|
|
|
|
import org.apache.dolphinscheduler.remote.command.HostUpdateCommand; |
|
|
|
import org.apache.dolphinscheduler.remote.command.HostUpdateCommand; |
|
|
|
import org.apache.dolphinscheduler.remote.utils.Host; |
|
|
|
import org.apache.dolphinscheduler.remote.utils.Host; |
|
|
|
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
|
|
|
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
|
|
|
import org.apache.dolphinscheduler.server.master.dispatch.executor.NettyExecutorManager; |
|
|
|
import org.apache.dolphinscheduler.server.master.dispatch.executor.NettyExecutorManager; |
|
|
|
import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics; |
|
|
|
import org.apache.dolphinscheduler.server.master.event.StateEvent; |
|
|
|
|
|
|
|
import org.apache.dolphinscheduler.server.master.event.StateEventHandleError; |
|
|
|
|
|
|
|
import org.apache.dolphinscheduler.server.master.event.StateEventHandleException; |
|
|
|
|
|
|
|
import org.apache.dolphinscheduler.server.master.event.StateEventHandler; |
|
|
|
|
|
|
|
import org.apache.dolphinscheduler.server.master.event.StateEventHandlerManager; |
|
|
|
import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics; |
|
|
|
import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics; |
|
|
|
import org.apache.dolphinscheduler.server.master.runner.task.ITaskProcessor; |
|
|
|
import org.apache.dolphinscheduler.server.master.runner.task.ITaskProcessor; |
|
|
|
import org.apache.dolphinscheduler.server.master.runner.task.TaskAction; |
|
|
|
import org.apache.dolphinscheduler.server.master.runner.task.TaskAction; |
|
|
|
import org.apache.dolphinscheduler.server.master.runner.task.TaskProcessorFactory; |
|
|
|
import org.apache.dolphinscheduler.server.master.runner.task.TaskProcessorFactory; |
|
|
|
import org.apache.dolphinscheduler.service.alert.ProcessAlertManager; |
|
|
|
import org.apache.dolphinscheduler.service.alert.ProcessAlertManager; |
|
|
|
import org.apache.dolphinscheduler.service.corn.CronUtils; |
|
|
|
import org.apache.dolphinscheduler.service.corn.CronUtils; |
|
|
|
|
|
|
|
import org.apache.dolphinscheduler.service.expand.CuringParamsService; |
|
|
|
import org.apache.dolphinscheduler.service.process.ProcessService; |
|
|
|
import org.apache.dolphinscheduler.service.process.ProcessService; |
|
|
|
import org.apache.dolphinscheduler.service.queue.PeerTaskInstancePriorityQueue; |
|
|
|
import org.apache.dolphinscheduler.service.queue.PeerTaskInstancePriorityQueue; |
|
|
|
|
|
|
|
|
|
|
@ -283,16 +284,37 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
if (!isStart) { |
|
|
|
if (!isStart) { |
|
|
|
return; |
|
|
|
return; |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
StateEvent stateEvent = null; |
|
|
|
while (!this.stateEvents.isEmpty()) { |
|
|
|
while (!this.stateEvents.isEmpty()) { |
|
|
|
try { |
|
|
|
try { |
|
|
|
StateEvent stateEvent = this.stateEvents.peek(); |
|
|
|
stateEvent = this.stateEvents.peek(); |
|
|
|
LoggerUtils.setWorkflowAndTaskInstanceIDMDC(stateEvent.getProcessInstanceId(), stateEvent.getTaskInstanceId()); |
|
|
|
LoggerUtils.setWorkflowAndTaskInstanceIDMDC(stateEvent.getProcessInstanceId(), |
|
|
|
if (stateEventHandler(stateEvent)) { |
|
|
|
stateEvent.getTaskInstanceId()); |
|
|
|
|
|
|
|
// if state handle success then will remove this state, otherwise will retry this state next time.
|
|
|
|
|
|
|
|
// The state should always handle success except database error.
|
|
|
|
|
|
|
|
checkProcessInstance(stateEvent); |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
StateEventHandler stateEventHandler = |
|
|
|
|
|
|
|
StateEventHandlerManager.getStateEventHandler(stateEvent.getType()) |
|
|
|
|
|
|
|
.orElseThrow(() -> new StateEventHandleError("Cannot find handler for the given state event")); |
|
|
|
|
|
|
|
if (stateEventHandler.handleStateEvent(this, stateEvent)) { |
|
|
|
this.stateEvents.remove(stateEvent); |
|
|
|
this.stateEvents.remove(stateEvent); |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
} catch (StateEventHandleError stateEventHandleError) { |
|
|
|
|
|
|
|
logger.error("State event handle error, will remove this event: {}", stateEvent, stateEventHandleError); |
|
|
|
|
|
|
|
this.stateEvents.remove(stateEvent); |
|
|
|
|
|
|
|
ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS); |
|
|
|
|
|
|
|
} catch (StateEventHandleException stateEventHandleException) { |
|
|
|
|
|
|
|
logger.error("State event handle error, will retry this event: {}", |
|
|
|
|
|
|
|
stateEvent, |
|
|
|
|
|
|
|
stateEventHandleException); |
|
|
|
|
|
|
|
ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS); |
|
|
|
} catch (Exception e) { |
|
|
|
} catch (Exception e) { |
|
|
|
// we catch the exception here, since if the state event handle failed, the state event will still keep in the stateEvents queue.
|
|
|
|
// we catch the exception here, since if the state event handle failed, the state event will still keep in the stateEvents queue.
|
|
|
|
logger.error("state handle error:", e); |
|
|
|
logger.error("State event handle error, get a unknown exception, will retry this event: {}", |
|
|
|
|
|
|
|
stateEvent, |
|
|
|
|
|
|
|
e); |
|
|
|
|
|
|
|
ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS); |
|
|
|
} finally { |
|
|
|
} finally { |
|
|
|
LoggerUtils.removeWorkflowAndTaskInstanceIdMDC(); |
|
|
|
LoggerUtils.removeWorkflowAndTaskInstanceIdMDC(); |
|
|
|
} |
|
|
|
} |
|
|
@ -330,58 +352,14 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
return this.processInstance; |
|
|
|
return this.processInstance; |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
private boolean stateEventHandler(StateEvent stateEvent) { |
|
|
|
public boolean checkForceStartAndWakeUp(StateEvent stateEvent) { |
|
|
|
logger.info("process event: {}", stateEvent); |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
if (!checkProcessInstance(stateEvent)) { |
|
|
|
|
|
|
|
return false; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
boolean result = false; |
|
|
|
|
|
|
|
switch (stateEvent.getType()) { |
|
|
|
|
|
|
|
case PROCESS_STATE_CHANGE: |
|
|
|
|
|
|
|
measureProcessState(stateEvent); |
|
|
|
|
|
|
|
result = processStateChangeHandler(stateEvent); |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
case TASK_STATE_CHANGE: |
|
|
|
|
|
|
|
measureTaskState(stateEvent); |
|
|
|
|
|
|
|
result = taskStateChangeHandler(stateEvent); |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
case PROCESS_TIMEOUT: |
|
|
|
|
|
|
|
ProcessInstanceMetrics.incProcessInstanceTimeout(); |
|
|
|
|
|
|
|
result = processTimeout(); |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
case TASK_TIMEOUT: |
|
|
|
|
|
|
|
TaskMetrics.incTaskTimeout(); |
|
|
|
|
|
|
|
result = taskTimeout(stateEvent); |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
case WAIT_TASK_GROUP: |
|
|
|
|
|
|
|
result = checkForceStartAndWakeUp(stateEvent); |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
case TASK_RETRY: |
|
|
|
|
|
|
|
TaskMetrics.incTaskRetry(); |
|
|
|
|
|
|
|
result = taskRetryEventHandler(stateEvent); |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
case PROCESS_BLOCKED: |
|
|
|
|
|
|
|
result = processBlockHandler(stateEvent); |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
default: |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
if (result) { |
|
|
|
|
|
|
|
this.stateEvents.remove(stateEvent); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
return result; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
private boolean checkForceStartAndWakeUp(StateEvent stateEvent) { |
|
|
|
|
|
|
|
TaskGroupQueue taskGroupQueue = this.processService.loadTaskGroupQueue(stateEvent.getTaskInstanceId()); |
|
|
|
TaskGroupQueue taskGroupQueue = this.processService.loadTaskGroupQueue(stateEvent.getTaskInstanceId()); |
|
|
|
if (taskGroupQueue.getForceStart() == Flag.YES.getCode()) { |
|
|
|
if (taskGroupQueue.getForceStart() == Flag.YES.getCode()) { |
|
|
|
TaskInstance taskInstance = this.processService.findTaskInstanceById(stateEvent.getTaskInstanceId()); |
|
|
|
TaskInstance taskInstance = this.processService.findTaskInstanceById(stateEvent.getTaskInstanceId()); |
|
|
|
ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskInstance.getTaskCode()); |
|
|
|
ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskInstance.getTaskCode()); |
|
|
|
taskProcessor.action(TaskAction.DISPATCH); |
|
|
|
taskProcessor.action(TaskAction.DISPATCH); |
|
|
|
this.processService.updateTaskGroupQueueStatus(taskGroupQueue.getTaskId(), TaskGroupQueueStatus.ACQUIRE_SUCCESS.getCode()); |
|
|
|
this.processService.updateTaskGroupQueueStatus(taskGroupQueue.getTaskId(), |
|
|
|
|
|
|
|
TaskGroupQueueStatus.ACQUIRE_SUCCESS.getCode()); |
|
|
|
return true; |
|
|
|
return true; |
|
|
|
} |
|
|
|
} |
|
|
|
if (taskGroupQueue.getInQueue() == Flag.YES.getCode()) { |
|
|
|
if (taskGroupQueue.getInQueue() == Flag.YES.getCode()) { |
|
|
@ -396,76 +374,17 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
return false; |
|
|
|
return false; |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
private boolean taskTimeout(StateEvent stateEvent) { |
|
|
|
public void processTimeout() { |
|
|
|
if (!checkTaskInstanceByStateEvent(stateEvent)) { |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
TaskInstance taskInstance = taskInstanceMap.get(stateEvent.getTaskInstanceId()); |
|
|
|
|
|
|
|
if (TimeoutFlag.CLOSE == taskInstance.getTaskDefine().getTimeoutFlag()) { |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
TaskTimeoutStrategy taskTimeoutStrategy = taskInstance.getTaskDefine().getTimeoutNotifyStrategy(); |
|
|
|
|
|
|
|
if (TaskTimeoutStrategy.FAILED == taskTimeoutStrategy || TaskTimeoutStrategy.WARNFAILED == taskTimeoutStrategy) { |
|
|
|
|
|
|
|
ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskInstance.getTaskCode()); |
|
|
|
|
|
|
|
taskProcessor.action(TaskAction.TIMEOUT); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
if (TaskTimeoutStrategy.WARN == taskTimeoutStrategy || TaskTimeoutStrategy.WARNFAILED == taskTimeoutStrategy) { |
|
|
|
|
|
|
|
ProjectUser projectUser = processService.queryProjectWithUserByProcessInstanceId(processInstance.getId()); |
|
|
|
|
|
|
|
processAlertManager.sendTaskTimeoutAlert(processInstance, taskInstance, projectUser); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
private boolean processTimeout() { |
|
|
|
|
|
|
|
ProjectUser projectUser = processService.queryProjectWithUserByProcessInstanceId(processInstance.getId()); |
|
|
|
ProjectUser projectUser = processService.queryProjectWithUserByProcessInstanceId(processInstance.getId()); |
|
|
|
this.processAlertManager.sendProcessTimeoutAlert(this.processInstance, projectUser); |
|
|
|
this.processAlertManager.sendProcessTimeoutAlert(this.processInstance, projectUser); |
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
private boolean taskStateChangeHandler(StateEvent stateEvent) { |
|
|
|
|
|
|
|
if (!checkTaskInstanceByStateEvent(stateEvent)) { |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
Optional<TaskInstance> taskInstanceOptional = getTaskInstance(stateEvent.getTaskInstanceId()); |
|
|
|
|
|
|
|
TaskInstance task = taskInstanceOptional.orElseThrow( |
|
|
|
|
|
|
|
() -> new RuntimeException("Cannot find task instance by task instance id: " + stateEvent.getTaskInstanceId())); |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
if (task.getState() == null) { |
|
|
|
|
|
|
|
logger.error("task state is null, state handler error: {}", stateEvent); |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
if (task.getState().typeIsFinished()) { |
|
|
|
|
|
|
|
if (completeTaskMap.containsKey(task.getTaskCode()) && completeTaskMap.get(task.getTaskCode()) == task.getId()) { |
|
|
|
|
|
|
|
logger.warn("The task instance is already complete, stateEvent: {}", stateEvent); |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
taskFinished(task); |
|
|
|
|
|
|
|
if (task.getTaskGroupId() > 0) { |
|
|
|
|
|
|
|
releaseTaskGroup(task); |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
if (activeTaskProcessorMaps.containsKey(task.getTaskCode())) { |
|
|
|
|
|
|
|
ITaskProcessor iTaskProcessor = activeTaskProcessorMaps.get(task.getTaskCode()); |
|
|
|
|
|
|
|
iTaskProcessor.action(TaskAction.RUN); |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
if (iTaskProcessor.taskInstance().getState().typeIsFinished()) { |
|
|
|
public void taskTimeout(TaskInstance taskInstance) { |
|
|
|
if (iTaskProcessor.taskInstance().getState() != task.getState()) { |
|
|
|
ProjectUser projectUser = processService.queryProjectWithUserByProcessInstanceId(processInstance.getId()); |
|
|
|
task.setState(iTaskProcessor.taskInstance().getState()); |
|
|
|
processAlertManager.sendTaskTimeoutAlert(processInstance, taskInstance, projectUser); |
|
|
|
} |
|
|
|
|
|
|
|
taskFinished(task); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
logger.error("state handler error: {}", stateEvent); |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
private void taskFinished(TaskInstance taskInstance) { |
|
|
|
public void taskFinished(TaskInstance taskInstance) throws StateEventHandleException { |
|
|
|
logger.info("TaskInstance finished task code:{} state:{} ", |
|
|
|
logger.info("TaskInstance finished task code:{} state:{} ", |
|
|
|
taskInstance.getTaskCode(), |
|
|
|
taskInstance.getTaskCode(), |
|
|
|
taskInstance.getState()); |
|
|
|
taskInstance.getState()); |
|
|
@ -512,7 +431,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
* |
|
|
|
* |
|
|
|
* @param taskInstance |
|
|
|
* @param taskInstance |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
private void releaseTaskGroup(TaskInstance taskInstance) { |
|
|
|
public void releaseTaskGroup(TaskInstance taskInstance) { |
|
|
|
if (taskInstance.getTaskGroupId() > 0) { |
|
|
|
if (taskInstance.getTaskGroupId() > 0) { |
|
|
|
TaskInstance nextTaskInstance = this.processService.releaseTaskGroup(taskInstance); |
|
|
|
TaskInstance nextTaskInstance = this.processService.releaseTaskGroup(taskInstance); |
|
|
|
if (nextTaskInstance != null) { |
|
|
|
if (nextTaskInstance != null) { |
|
|
@ -536,13 +455,15 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
* |
|
|
|
* |
|
|
|
* @param taskInstance |
|
|
|
* @param taskInstance |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
private void retryTaskInstance(TaskInstance taskInstance) { |
|
|
|
private void retryTaskInstance(TaskInstance taskInstance) throws StateEventHandleException { |
|
|
|
if (!taskInstance.taskCanRetry()) { |
|
|
|
if (!taskInstance.taskCanRetry()) { |
|
|
|
return; |
|
|
|
return; |
|
|
|
} |
|
|
|
} |
|
|
|
TaskInstance newTaskInstance = cloneRetryTaskInstance(taskInstance); |
|
|
|
TaskInstance newTaskInstance = cloneRetryTaskInstance(taskInstance); |
|
|
|
if (newTaskInstance == null) { |
|
|
|
if (newTaskInstance == null) { |
|
|
|
logger.error("retry fail, new taskInstance is null, task code:{}, task id:{}", taskInstance.getTaskCode(), taskInstance.getId()); |
|
|
|
logger.error("retry fail, new taskInstance is null, task code:{}, task id:{}", |
|
|
|
|
|
|
|
taskInstance.getTaskCode(), |
|
|
|
|
|
|
|
taskInstance.getId()); |
|
|
|
return; |
|
|
|
return; |
|
|
|
} |
|
|
|
} |
|
|
|
waitToRetryTaskInstanceMap.put(newTaskInstance.getTaskCode(), newTaskInstance); |
|
|
|
waitToRetryTaskInstanceMap.put(newTaskInstance.getTaskCode(), newTaskInstance); |
|
|
@ -563,20 +484,6 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
|
|
|
|
* 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 |
|
|
|
* update process instance |
|
|
|
*/ |
|
|
|
*/ |
|
|
@ -610,45 +517,23 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
/** |
|
|
|
/** |
|
|
|
* check process instance by state event |
|
|
|
* check process instance by state event |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
public boolean checkProcessInstance(StateEvent stateEvent) { |
|
|
|
public void checkProcessInstance(StateEvent stateEvent) throws StateEventHandleError { |
|
|
|
if (this.processInstance.getId() != stateEvent.getProcessInstanceId()) { |
|
|
|
if (this.processInstance.getId() != stateEvent.getProcessInstanceId()) { |
|
|
|
logger.error("mismatch process instance id: {}, state event:{}", |
|
|
|
throw new StateEventHandleError("The event doesn't contains process instance id"); |
|
|
|
this.processInstance.getId(), |
|
|
|
|
|
|
|
stateEvent); |
|
|
|
|
|
|
|
return false; |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
/** |
|
|
|
* check if task instance exist by state event |
|
|
|
* check if task instance exist by state event |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
public boolean checkTaskInstanceByStateEvent(StateEvent stateEvent) { |
|
|
|
public void checkTaskInstanceByStateEvent(StateEvent stateEvent) throws StateEventHandleError { |
|
|
|
if (stateEvent.getTaskInstanceId() == 0) { |
|
|
|
if (stateEvent.getTaskInstanceId() == 0) { |
|
|
|
logger.error("task instance id null, state event:{}", stateEvent); |
|
|
|
throw new StateEventHandleError("The taskInstanceId is 0"); |
|
|
|
return false; |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
if (!taskInstanceMap.containsKey(stateEvent.getTaskInstanceId())) { |
|
|
|
if (!taskInstanceMap.containsKey(stateEvent.getTaskInstanceId())) { |
|
|
|
logger.error("mismatch task instance id, event:{}", stateEvent); |
|
|
|
throw new StateEventHandleError("Cannot find the taskInstance from taskInstanceMap"); |
|
|
|
return false; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
|
|
|
|
* check if task instance exist by task code |
|
|
|
|
|
|
|
*/ |
|
|
|
|
|
|
|
public boolean checkTaskInstanceByCode(long taskCode) { |
|
|
|
|
|
|
|
if (taskInstanceMap.isEmpty()) { |
|
|
|
|
|
|
|
return false; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
for (TaskInstance taskInstance : taskInstanceMap.values()) { |
|
|
|
|
|
|
|
if (taskInstance.getTaskCode() == taskCode) { |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
return false; |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
/** |
|
|
@ -697,58 +582,13 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
return Optional.empty(); |
|
|
|
return Optional.empty(); |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
private boolean processStateChangeHandler(StateEvent stateEvent) { |
|
|
|
public void processBlock() { |
|
|
|
try { |
|
|
|
|
|
|
|
logger.info("process:{} state {} change to {}", processInstance.getId(), processInstance.getState(), stateEvent.getExecutionStatus()); |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
if (stateEvent.getExecutionStatus() == ExecutionStatus.STOP) { |
|
|
|
|
|
|
|
// serial wait execution type needs to wake up the waiting process
|
|
|
|
|
|
|
|
if (processDefinition.getExecutionType().typeIsSerialWait() || processDefinition.getExecutionType().typeIsSerialPriority()) { |
|
|
|
|
|
|
|
endProcess(); |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
this.updateProcessInstanceState(stateEvent); |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
if (processComplementData()) { |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
if (stateEvent.getExecutionStatus().typeIsFinished()) { |
|
|
|
|
|
|
|
endProcess(); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
if (processInstance.getState() == ExecutionStatus.READY_STOP) { |
|
|
|
|
|
|
|
killAllTasks(); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} catch (Exception e) { |
|
|
|
|
|
|
|
logger.error("process state change error:", e); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
private boolean processBlockHandler(StateEvent stateEvent) { |
|
|
|
|
|
|
|
try { |
|
|
|
|
|
|
|
Optional<TaskInstance> taskInstanceOptional = getTaskInstance(stateEvent.getTaskInstanceId()); |
|
|
|
|
|
|
|
TaskInstance task = taskInstanceOptional.orElseThrow( |
|
|
|
|
|
|
|
() -> new RuntimeException("Cannot find taskInstance by taskInstanceId:" + stateEvent.getTaskInstanceId())); |
|
|
|
|
|
|
|
if (!checkTaskInstanceByStateEvent(stateEvent)) { |
|
|
|
|
|
|
|
logger.error("task {} is not a blocking task", task.getTaskCode()); |
|
|
|
|
|
|
|
return false; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
BlockingParameters parameters = JSONUtils.parseObject(task.getTaskParams(), BlockingParameters.class); |
|
|
|
|
|
|
|
if (parameters.isAlertWhenBlocking()) { |
|
|
|
|
|
|
|
ProjectUser projectUser = processService.queryProjectWithUserByProcessInstanceId(processInstance.getId()); |
|
|
|
ProjectUser projectUser = processService.queryProjectWithUserByProcessInstanceId(processInstance.getId()); |
|
|
|
processAlertManager.sendProcessBlockingAlert(processInstance, projectUser); |
|
|
|
processAlertManager.sendProcessBlockingAlert(processInstance, projectUser); |
|
|
|
logger.info("processInstance {} block alert send successful!", processInstance.getId()); |
|
|
|
logger.info("processInstance {} block alert send successful!", processInstance.getId()); |
|
|
|
} |
|
|
|
} |
|
|
|
} catch (Exception e) { |
|
|
|
|
|
|
|
logger.error("sending blocking message error:", e); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
private boolean processComplementData() throws Exception { |
|
|
|
public boolean processComplementData() { |
|
|
|
if (!needComplementProcess()) { |
|
|
|
if (!needComplementProcess()) { |
|
|
|
return false; |
|
|
|
return false; |
|
|
|
} |
|
|
|
} |
|
|
@ -946,7 +786,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
/** |
|
|
|
/** |
|
|
|
* init task queue |
|
|
|
* init task queue |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
private void initTaskQueue() { |
|
|
|
private void initTaskQueue() throws StateEventHandleException { |
|
|
|
|
|
|
|
|
|
|
|
taskFailedSubmit = false; |
|
|
|
taskFailedSubmit = false; |
|
|
|
activeTaskProcessorMaps.clear(); |
|
|
|
activeTaskProcessorMaps.clear(); |
|
|
@ -955,7 +795,8 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
errorTaskMap.clear(); |
|
|
|
errorTaskMap.clear(); |
|
|
|
|
|
|
|
|
|
|
|
if (!isNewProcessInstance()) { |
|
|
|
if (!isNewProcessInstance()) { |
|
|
|
List<TaskInstance> validTaskInstanceList = processService.findValidTaskListByProcessId(processInstance.getId()); |
|
|
|
List<TaskInstance> validTaskInstanceList = |
|
|
|
|
|
|
|
processService.findValidTaskListByProcessId(processInstance.getId()); |
|
|
|
for (TaskInstance task : validTaskInstanceList) { |
|
|
|
for (TaskInstance task : validTaskInstanceList) { |
|
|
|
if (validTaskMap.containsKey(task.getTaskCode())) { |
|
|
|
if (validTaskMap.containsKey(task.getTaskCode())) { |
|
|
|
int oldTaskInstanceId = validTaskMap.get(task.getTaskCode()); |
|
|
|
int oldTaskInstanceId = validTaskMap.get(task.getTaskCode()); |
|
|
@ -965,7 +806,8 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
processService.updateTaskInstance(task); |
|
|
|
processService.updateTaskInstance(task); |
|
|
|
continue; |
|
|
|
continue; |
|
|
|
} |
|
|
|
} |
|
|
|
logger.warn("have same taskCode taskInstance when init task queue, taskCode:{}", task.getTaskCode()); |
|
|
|
logger.warn("have same taskCode taskInstance when init task queue, taskCode:{}", |
|
|
|
|
|
|
|
task.getTaskCode()); |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
validTaskMap.put(task.getTaskCode(), task.getId()); |
|
|
|
validTaskMap.put(task.getTaskCode(), task.getId()); |
|
|
@ -1113,6 +955,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
Host host = new Host(taskInstance.getHost()); |
|
|
|
Host host = new Host(taskInstance.getHost()); |
|
|
|
nettyExecutorManager.doExecute(host, hostUpdateCommand.convert2Command()); |
|
|
|
nettyExecutorManager.doExecute(host, hostUpdateCommand.convert2Command()); |
|
|
|
} catch (Exception e) { |
|
|
|
} catch (Exception e) { |
|
|
|
|
|
|
|
// Do we need to catch this exception?
|
|
|
|
logger.error("notify process host update", e); |
|
|
|
logger.error("notify process host update", e); |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
@ -1366,8 +1209,9 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
return validTaskInstanceList; |
|
|
|
return validTaskInstanceList; |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
private void submitPostNode(String parentNodeCode) { |
|
|
|
private void submitPostNode(String parentNodeCode) throws StateEventHandleException { |
|
|
|
Set<String> submitTaskNodeList = DagHelper.parsePostNodes(parentNodeCode, skipTaskNodeMap, dag, getCompleteTaskInstanceMap()); |
|
|
|
Set<String> submitTaskNodeList = |
|
|
|
|
|
|
|
DagHelper.parsePostNodes(parentNodeCode, skipTaskNodeMap, dag, getCompleteTaskInstanceMap()); |
|
|
|
List<TaskInstance> taskInstances = new ArrayList<>(); |
|
|
|
List<TaskInstance> taskInstances = new ArrayList<>(); |
|
|
|
for (String taskNode : submitTaskNodeList) { |
|
|
|
for (String taskNode : submitTaskNodeList) { |
|
|
|
TaskNode taskNodeObject = dag.getNode(taskNode); |
|
|
|
TaskNode taskNodeObject = dag.getNode(taskNode); |
|
|
@ -1710,34 +1554,19 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
return true; |
|
|
|
return true; |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
try { |
|
|
|
|
|
|
|
Map<String, String> cmdParam = JSONUtils.toMap(processInstance.getCommandParam()); |
|
|
|
Map<String, String> cmdParam = JSONUtils.toMap(processInstance.getCommandParam()); |
|
|
|
Date endTime = DateUtils.getScheduleDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_END_DATE)); |
|
|
|
Date endTime = DateUtils.getScheduleDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_END_DATE)); |
|
|
|
return processInstance.getScheduleTime().equals(endTime); |
|
|
|
return processInstance.getScheduleTime().equals(endTime); |
|
|
|
} catch (Exception e) { |
|
|
|
|
|
|
|
logger.error("complement end failed ", e); |
|
|
|
|
|
|
|
return false; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
/** |
|
|
|
* updateProcessInstance process instance state |
|
|
|
* updateProcessInstance process instance state |
|
|
|
* after each batch of tasks is executed, the status of the process instance is updated |
|
|
|
* after each batch of tasks is executed, the status of the process instance is updated |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
private void updateProcessInstanceState() { |
|
|
|
private void updateProcessInstanceState() throws StateEventHandleException { |
|
|
|
ExecutionStatus state = getProcessInstanceState(processInstance); |
|
|
|
ExecutionStatus state = getProcessInstanceState(processInstance); |
|
|
|
if (processInstance.getState() != state) { |
|
|
|
if (processInstance.getState() != state) { |
|
|
|
logger.info( |
|
|
|
updateWorkflowInstanceStatesToDB(state); |
|
|
|
"work flow process instance [id: {}, name:{}], state change from {} to {}, cmd type: {}", |
|
|
|
|
|
|
|
processInstance.getId(), processInstance.getName(), |
|
|
|
|
|
|
|
processInstance.getState(), state, |
|
|
|
|
|
|
|
processInstance.getCommandType()); |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
processInstance.setState(state); |
|
|
|
|
|
|
|
if (state.typeIsFinished()) { |
|
|
|
|
|
|
|
processInstance.setEndTime(new Date()); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
processService.updateProcessInstance(processInstance); |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
StateEvent stateEvent = new StateEvent(); |
|
|
|
StateEvent stateEvent = new StateEvent(); |
|
|
|
stateEvent.setExecutionStatus(processInstance.getState()); |
|
|
|
stateEvent.setExecutionStatus(processInstance.getState()); |
|
|
@ -1752,20 +1581,33 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
/** |
|
|
|
/** |
|
|
|
* stateEvent's execution status as process instance state |
|
|
|
* stateEvent's execution status as process instance state |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
private void updateProcessInstanceState(StateEvent stateEvent) { |
|
|
|
public void updateProcessInstanceState(StateEvent stateEvent) throws StateEventHandleException { |
|
|
|
ExecutionStatus state = stateEvent.getExecutionStatus(); |
|
|
|
ExecutionStatus state = stateEvent.getExecutionStatus(); |
|
|
|
if (processInstance.getState() != state) { |
|
|
|
updateWorkflowInstanceStatesToDB(state); |
|
|
|
logger.info( |
|
|
|
} |
|
|
|
"work flow process instance [id: {}, name:{}], state change from {} to {}, cmd type: {}", |
|
|
|
|
|
|
|
processInstance.getId(), processInstance.getName(), |
|
|
|
private void updateWorkflowInstanceStatesToDB(ExecutionStatus newStates) throws StateEventHandleException { |
|
|
|
processInstance.getState(), state, |
|
|
|
ExecutionStatus originStates = processInstance.getState(); |
|
|
|
|
|
|
|
if (originStates != newStates) { |
|
|
|
|
|
|
|
logger.info("work flow process instance [id: {}, name:{}], state change from {} to {}, cmd type: {}", |
|
|
|
|
|
|
|
processInstance.getId(), |
|
|
|
|
|
|
|
processInstance.getName(), |
|
|
|
|
|
|
|
originStates, |
|
|
|
|
|
|
|
newStates, |
|
|
|
processInstance.getCommandType()); |
|
|
|
processInstance.getCommandType()); |
|
|
|
|
|
|
|
|
|
|
|
processInstance.setState(state); |
|
|
|
processInstance.setState(newStates); |
|
|
|
if (state.typeIsFinished()) { |
|
|
|
if (newStates.typeIsFinished()) { |
|
|
|
processInstance.setEndTime(new Date()); |
|
|
|
processInstance.setEndTime(new Date()); |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
try { |
|
|
|
processService.updateProcessInstance(processInstance); |
|
|
|
processService.updateProcessInstance(processInstance); |
|
|
|
|
|
|
|
} catch (Exception ex) { |
|
|
|
|
|
|
|
// recover the status
|
|
|
|
|
|
|
|
processInstance.setState(originStates); |
|
|
|
|
|
|
|
processInstance.setEndTime(null); |
|
|
|
|
|
|
|
throw new StateEventHandleException("Update process instance status to DB error", ex); |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
@ -1784,19 +1626,17 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
* |
|
|
|
* |
|
|
|
* @param taskInstance task instance |
|
|
|
* @param taskInstance task instance |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
private void addTaskToStandByList(TaskInstance taskInstance) { |
|
|
|
public void addTaskToStandByList(TaskInstance taskInstance) { |
|
|
|
try { |
|
|
|
|
|
|
|
if (readyToSubmitTaskQueue.contains(taskInstance)) { |
|
|
|
if (readyToSubmitTaskQueue.contains(taskInstance)) { |
|
|
|
logger.warn("task was found in ready submit queue, task code:{}", taskInstance.getTaskCode()); |
|
|
|
logger.warn("task was found in ready submit queue, task code:{}", taskInstance.getTaskCode()); |
|
|
|
return; |
|
|
|
return; |
|
|
|
} |
|
|
|
} |
|
|
|
logger.info("add task to stand by list, task name:{}, task id:{}, task code:{}", |
|
|
|
logger.info("add task to stand by list, task name:{}, task id:{}, task code:{}", |
|
|
|
taskInstance.getName(), taskInstance.getId(), taskInstance.getTaskCode()); |
|
|
|
taskInstance.getName(), |
|
|
|
|
|
|
|
taskInstance.getId(), |
|
|
|
|
|
|
|
taskInstance.getTaskCode()); |
|
|
|
TaskMetrics.incTaskSubmit(); |
|
|
|
TaskMetrics.incTaskSubmit(); |
|
|
|
readyToSubmitTaskQueue.put(taskInstance); |
|
|
|
readyToSubmitTaskQueue.put(taskInstance); |
|
|
|
} catch (Exception e) { |
|
|
|
|
|
|
|
logger.error("add task instance to readyToSubmitTaskQueue, taskName:{}, task id:{}", taskInstance.getName(), taskInstance.getId(), e); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
/** |
|
|
@ -1805,13 +1645,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
* @param taskInstance task instance |
|
|
|
* @param taskInstance task instance |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
private void removeTaskFromStandbyList(TaskInstance taskInstance) { |
|
|
|
private void removeTaskFromStandbyList(TaskInstance taskInstance) { |
|
|
|
try { |
|
|
|
|
|
|
|
readyToSubmitTaskQueue.remove(taskInstance); |
|
|
|
readyToSubmitTaskQueue.remove(taskInstance); |
|
|
|
} catch (Exception e) { |
|
|
|
|
|
|
|
logger.error("remove task instance from readyToSubmitTaskQueue error, task id:{}, Name: {}", |
|
|
|
|
|
|
|
taskInstance.getId(), |
|
|
|
|
|
|
|
taskInstance.getName(), e); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
/** |
|
|
@ -1831,8 +1665,9 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
/** |
|
|
|
/** |
|
|
|
* close the on going tasks |
|
|
|
* close the on going tasks |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
private void killAllTasks() { |
|
|
|
public void killAllTasks() { |
|
|
|
logger.info("kill called on process instance id: {}, num: {}", processInstance.getId(), |
|
|
|
logger.info("kill called on process instance id: {}, num: {}", |
|
|
|
|
|
|
|
processInstance.getId(), |
|
|
|
activeTaskProcessorMaps.size()); |
|
|
|
activeTaskProcessorMaps.size()); |
|
|
|
|
|
|
|
|
|
|
|
if (readyToSubmitTaskQueue.size() > 0) { |
|
|
|
if (readyToSubmitTaskQueue.size() > 0) { |
|
|
@ -1868,7 +1703,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
/** |
|
|
|
/** |
|
|
|
* handling the list of tasks to be submitted |
|
|
|
* handling the list of tasks to be submitted |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
private void submitStandByTask() { |
|
|
|
public void submitStandByTask() throws StateEventHandleException { |
|
|
|
int length = readyToSubmitTaskQueue.size(); |
|
|
|
int length = readyToSubmitTaskQueue.size(); |
|
|
|
for (int i = 0; i < length; i++) { |
|
|
|
for (int i = 0; i < length; i++) { |
|
|
|
TaskInstance task = readyToSubmitTaskQueue.peek(); |
|
|
|
TaskInstance task = readyToSubmitTaskQueue.peek(); |
|
|
@ -2027,6 +1862,19 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
throw new Exception("resubmit error, taskProcessor is null, task code: " + taskCode); |
|
|
|
throw new Exception("resubmit error, taskProcessor is null, task code: " + taskCode); |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
public Map<Long, Integer> getCompleteTaskMap() { |
|
|
|
|
|
|
|
return completeTaskMap; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
public Map<Long, ITaskProcessor> getActiveTaskProcessMap() { |
|
|
|
|
|
|
|
return activeTaskProcessorMaps; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
public Map<Long, TaskInstance> getWaitToRetryTaskInstanceMap() { |
|
|
|
|
|
|
|
return waitToRetryTaskInstanceMap; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
private void setGlobalParamIfCommanded(ProcessDefinition processDefinition, Map<String, String> cmdParam) { |
|
|
|
private void setGlobalParamIfCommanded(ProcessDefinition processDefinition, Map<String, String> cmdParam) { |
|
|
|
// get start params from command param
|
|
|
|
// get start params from command param
|
|
|
|
Map<String, String> startParamMap = new HashMap<>(); |
|
|
|
Map<String, String> startParamMap = new HashMap<>(); |
|
|
@ -2061,46 +1909,4 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> { |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
private void measureProcessState(StateEvent processStateEvent) { |
|
|
|
|
|
|
|
if (processStateEvent.getExecutionStatus().typeIsFinished()) { |
|
|
|
|
|
|
|
ProcessInstanceMetrics.incProcessInstanceFinish(); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
switch (processStateEvent.getExecutionStatus()) { |
|
|
|
|
|
|
|
case STOP: |
|
|
|
|
|
|
|
ProcessInstanceMetrics.incProcessInstanceStop(); |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
case SUCCESS: |
|
|
|
|
|
|
|
ProcessInstanceMetrics.incProcessInstanceSuccess(); |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
case FAILURE: |
|
|
|
|
|
|
|
ProcessInstanceMetrics.incProcessInstanceFailure(); |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
default: |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
private void measureTaskState(StateEvent taskStateEvent) { |
|
|
|
|
|
|
|
if (taskStateEvent == null || taskStateEvent.getExecutionStatus() == null) { |
|
|
|
|
|
|
|
// the event is broken
|
|
|
|
|
|
|
|
logger.warn("The task event is broken..., taskEvent: {}", taskStateEvent); |
|
|
|
|
|
|
|
return; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
if (taskStateEvent.getExecutionStatus().typeIsFinished()) { |
|
|
|
|
|
|
|
TaskMetrics.incTaskFinish(); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
switch (taskStateEvent.getExecutionStatus()) { |
|
|
|
|
|
|
|
case STOP: |
|
|
|
|
|
|
|
TaskMetrics.incTaskStop(); |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
case SUCCESS: |
|
|
|
|
|
|
|
TaskMetrics.incTaskSuccess(); |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
case FAILURE: |
|
|
|
|
|
|
|
TaskMetrics.incTaskFailure(); |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
default: |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
} |
|
|
|
} |