You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dolphinscheduler.apache.org by we...@apache.org on 2022/08/10 03:00:29 UTC

[dolphinscheduler] branch dev updated: Split ExecutionStatus to WorkflowExecutionStatus and TaskExecutionStatus (#11340)

This is an automated email from the ASF dual-hosted git repository.

wenjun pushed a commit to branch dev
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git


The following commit(s) were added to refs/heads/dev by this push:
     new 8774415197 Split ExecutionStatus to WorkflowExecutionStatus and TaskExecutionStatus (#11340)
8774415197 is described below

commit 877441519711c729951ba811f91cdb088d4f1e96
Author: Wenjun Ruan <we...@apache.org>
AuthorDate: Wed Aug 10 11:00:23 2022 +0800

    Split ExecutionStatus to WorkflowExecutionStatus and TaskExecutionStatus (#11340)
---
 .../dolphinscheduler/alert/AlertSenderService.java |  45 +-
 .../alert/runner/AlertSenderServiceTest.java       |  62 +-
 .../api/controller/ProcessInstanceController.java  | 127 ++--
 .../api/controller/TaskInstanceController.java     |  30 +-
 .../dolphinscheduler/api/dto/TaskCountDto.java     |  30 +-
 .../dolphinscheduler/api/dto/TaskStateCount.java   |  59 +-
 .../api/service/ProcessInstanceService.java        |   4 +-
 .../api/service/TaskInstanceService.java           |   4 +-
 .../api/service/impl/DataAnalysisServiceImpl.java  |  86 +--
 .../api/service/impl/ExecutorServiceImpl.java      | 193 +++---
 .../service/impl/ProcessInstanceServiceImpl.java   | 224 ++++---
 .../api/service/impl/TaskInstanceServiceImpl.java  |  53 +-
 .../controller/ProcessInstanceControllerTest.java  |  63 +-
 .../api/controller/TaskInstanceControllerTest.java |   8 +-
 .../api/service/DataAnalysisServiceTest.java       |  75 +--
 .../api/service/ExecutorServiceTest.java           |  75 ++-
 .../api/service/ProcessInstanceServiceTest.java    | 312 +++++-----
 .../api/service/TaskInstanceServiceTest.java       | 120 ++--
 .../apache/dolphinscheduler/common/Constants.java  |  61 +-
 .../common/enums/TaskStateType.java                |  72 ---
 .../common/enums/WorkflowExecutionStatus.java      | 134 +++++
 .../dolphinscheduler/common/utils/HadoopUtils.java |  87 +--
 .../org/apache/dolphinscheduler/dao/AlertDao.java  |  26 +-
 .../dao/entity/ExecuteStatusCount.java             |  40 +-
 .../dao/entity/ProcessAlertContent.java            | 215 +------
 .../dao/entity/ProcessInstance.java                | 447 +-------------
 .../dao/entity/TaskAlertContent.java               | 110 +---
 .../dolphinscheduler/dao/entity/TaskInstance.java  | 434 +-------------
 .../dao/mapper/ProcessInstanceMapper.java          |  25 +-
 .../dao/mapper/TaskInstanceMapper.java             |  12 +-
 .../dolphinscheduler/dao/utils/DagHelper.java      |  58 +-
 .../dao/mapper/ProcessInstanceMapperTest.java      |  80 +--
 .../dao/mapper/TaskInstanceMapperTest.java         |  47 +-
 .../dolphinscheduler/dao/utils/DagHelperTest.java  |  51 +-
 .../builder/TaskExecutionContextBuilder.java       |  12 +-
 .../cache/ProcessInstanceExecCacheManager.java     |   4 +-
 .../master/consumer/TaskPriorityQueueConsumer.java |   2 +-
 .../server/master/event/StateEvent.java            |  31 +-
 .../server/master/event/TaskDelayEventHandler.java |  24 +-
 .../master/event/TaskDispatchEventHandler.java     |  14 +-
 .../event/TaskRejectByWorkerEventHandler.java      |  20 +-
 .../master/event/TaskResultEventHandler.java       |  32 +-
 .../master/event/TaskRetryStateEventHandler.java   |  14 +-
 .../master/event/TaskRunningEventHandler.java      |  30 +-
 .../event/{StateEvent.java => TaskStateEvent.java} |  35 +-
 .../server/master/event/TaskStateEventHandler.java |  33 +-
 .../master/event/TaskTimeoutStateEventHandler.java |  20 +-
 .../master/event/WorkflowStartEventHandler.java    |  36 +-
 .../{StateEvent.java => WorkflowStateEvent.java}   |  33 +-
 .../master/event/WorkflowStateEventHandler.java    |  32 +-
 .../master/processor/StateEventProcessor.java      |  58 +-
 .../master/processor/TaskEventProcessor.java       |  30 +-
 .../processor/queue/StateEventResponseService.java |  40 +-
 .../server/master/processor/queue/TaskEvent.java   |   6 +-
 .../registry/MasterConnectionStateListener.java    |  10 +-
 .../master/registry/MasterRegistryClient.java      |  13 +-
 .../master/runner/StateWheelExecuteThread.java     | 143 +++--
 .../master/runner/WorkflowExecuteRunnable.java     | 530 ++++++++---------
 .../master/runner/WorkflowExecuteThreadPool.java   |  56 +-
 .../master/runner/task/BaseTaskProcessor.java      |  75 ++-
 .../master/runner/task/BlockingTaskProcessor.java  |  46 +-
 .../master/runner/task/CommonTaskProcessor.java    |  27 +-
 .../master/runner/task/ConditionTaskProcessor.java |  40 +-
 .../master/runner/task/DependentTaskProcessor.java |  17 +-
 .../master/runner/task/SubTaskProcessor.java       |  59 +-
 .../master/runner/task/SwitchTaskProcessor.java    |  33 +-
 .../master/service/MasterFailoverService.java      |  51 +-
 .../master/service/WorkerFailoverService.java      |  89 +--
 .../master/utils/DataQualityResultOperator.java    |  26 +-
 .../server/master/utils/DependentExecute.java      |  45 +-
 .../server/master/BlockingTaskTest.java            |  73 ++-
 .../server/master/ConditionsTaskTest.java          |  30 +-
 .../server/master/DependentTaskTest.java           | 119 ++--
 .../server/master/SubProcessTaskTest.java          |  43 +-
 .../server/master/SwitchTaskTest.java              |  17 +-
 .../consumer/TaskPriorityQueueConsumerTest.java    |  24 +-
 .../master/processor/TaskAckProcessorTest.java     |  33 +-
 .../processor/TaskKillResponseProcessorTest.java   |  15 +-
 .../processor/queue/TaskResponseServiceTest.java   |  24 +-
 .../master/runner/MasterTaskExecThreadTest.java    |  42 +-
 .../master/runner/WorkflowExecuteRunnableTest.java |  23 +-
 .../runner/task/CommonTaskProcessorTest.java       |  11 +-
 .../server/master/service/FailoverServiceTest.java |  43 +-
 .../remote/command/HostUpdateCommand.java          |  25 +-
 .../remote/command/StateEventResponseCommand.java  |  39 +-
 .../remote/command/TaskEventChangeCommand.java     |  41 +-
 .../remote/command/TaskExecuteAckCommand.java      |   7 +-
 .../command/TaskExecuteRunningAckMessage.java      |  40 +-
 .../remote/command/TaskExecuteRunningCommand.java  |   4 +-
 .../remote/command/TaskKillResponseCommand.java    |  75 +--
 .../remote/command/TaskRejectAckCommand.java       |   7 +-
 ...and.java => TaskStateEventResponseCommand.java} |  35 +-
 ...d.java => WorkflowStateEventChangeCommand.java} |  76 +--
 .../command/alert/AlertSendRequestCommand.java     |  59 +-
 .../command/alert/AlertSendResponseCommand.java    |  33 +-
 .../command/alert/AlertSendResponseResult.java     |  33 +-
 .../command/log/GetLogBytesRequestCommand.java     |  21 +-
 .../command/log/GetLogBytesResponseCommand.java    |  21 +-
 .../command/log/RemoveTaskLogRequestCommand.java   |  21 +-
 .../command/log/RemoveTaskLogResponseCommand.java  |  25 +-
 .../command/log/RollViewLogRequestCommand.java     |  39 +-
 .../command/log/RollViewLogResponseCommand.java    |  21 +-
 .../remote/command/log/ViewLogRequestCommand.java  |  21 +-
 .../remote/command/log/ViewLogResponseCommand.java |  21 +-
 .../remote/dto/TaskInstanceExecuteDto.java         |   4 +-
 .../remote/dto/WorkflowExecuteDto.java             |  10 +-
 .../alert/AlertSendResponseCommandTest.java        |   8 +-
 .../command/log/GetLogBytesRequestCommandTest.java |   3 +-
 .../log/GetLogBytesResponseCommandTest.java        |   3 +-
 .../server/utils/ProcessUtils.java                 |  34 +-
 .../server/utils/ProcessUtilsTest.java             |  10 +-
 .../service/alert/ProcessAlertManager.java         |  44 +-
 .../service/process/ProcessService.java            |  25 +-
 .../service/process/ProcessServiceImpl.java        | 645 +++++++++++----------
 .../service/alert/AlertClientServiceTest.java      |  84 +--
 .../service/alert/ProcessAlertManagerTest.java     |   9 +-
 .../plugin/task/api/AbstractTask.java              |  13 +-
 .../plugin/task/api/TaskExecutionContext.java      |   9 +-
 .../plugin/task/api/enums/ExecutionStatus.java     | 206 -------
 .../plugin/task/api/enums/TaskExecutionStatus.java | 126 ++++
 .../plugin/task/api/model/DependentItem.java       |  62 +-
 .../plugin/task/api/enums/ExecutionStatusTest.java |  32 -
 .../plugin/task/pigeon/PigeonTaskTest.java         |  52 +-
 .../worker/processor/TaskDispatchProcessor.java    |  57 +-
 .../processor/TaskExecuteResultAckProcessor.java   |  21 +-
 .../processor/TaskExecuteRunningAckProcessor.java  |   9 +-
 .../server/worker/processor/TaskKillProcessor.java |  13 +-
 .../worker/processor/TaskRejectAckProcessor.java   |  18 +-
 .../server/worker/runner/TaskExecuteThread.java    |  82 ++-
 .../processor/TaskDispatchProcessorTest.java       |  56 +-
 130 files changed, 3295 insertions(+), 4736 deletions(-)

diff --git a/dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/java/org/apache/dolphinscheduler/alert/AlertSenderService.java b/dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/java/org/apache/dolphinscheduler/alert/AlertSenderService.java
index 798c717d59..9af74e668c 100644
--- a/dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/java/org/apache/dolphinscheduler/alert/AlertSenderService.java
+++ b/dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/java/org/apache/dolphinscheduler/alert/AlertSenderService.java
@@ -54,6 +54,7 @@ import com.google.common.collect.Lists;
 
 @Service
 public final class AlertSenderService extends Thread {
+
     private static final Logger logger = LoggerFactory.getLogger(AlertSenderService.class);
 
     private final AlertDao alertDao;
@@ -89,33 +90,36 @@ public final class AlertSenderService extends Thread {
 
     public void send(List<Alert> alerts) {
         for (Alert alert : alerts) {
-            //get alert group from alert
+            // get alert group from alert
             int alertId = Optional.ofNullable(alert.getId()).orElse(0);
             int alertGroupId = Optional.ofNullable(alert.getAlertGroupId()).orElse(0);
             List<AlertPluginInstance> alertInstanceList = alertDao.listInstanceByAlertGroupId(alertGroupId);
             if (CollectionUtils.isEmpty(alertInstanceList)) {
                 logger.error("send alert msg fail,no bind plugin instance.");
                 List<AlertResult> alertResults = Lists.newArrayList(new AlertResult("false",
-                                                                                    "no bind plugin instance"));
+                        "no bind plugin instance"));
                 alertDao.updateAlert(AlertStatus.EXECUTION_FAILURE, JSONUtils.toJsonString(alertResults), alertId);
                 continue;
             }
             AlertData alertData = AlertData.builder()
-                .id(alertId)
-                .content(alert.getContent())
-                .log(alert.getLog())
-                .title(alert.getTitle())
-                .warnType(alert.getWarningType().getCode())
-                .alertType(alert.getAlertType().getCode())
-                .build();
+                    .id(alertId)
+                    .content(alert.getContent())
+                    .log(alert.getLog())
+                    .title(alert.getTitle())
+                    .warnType(alert.getWarningType().getCode())
+                    .alertType(alert.getAlertType().getCode())
+                    .build();
 
             int sendSuccessCount = 0;
             List<AlertResult> alertResults = new ArrayList<>();
             for (AlertPluginInstance instance : alertInstanceList) {
                 AlertResult alertResult = this.alertResultHandler(instance, alertData);
                 if (alertResult != null) {
-                    AlertStatus sendStatus = Boolean.parseBoolean(String.valueOf(alertResult.getStatus())) ? AlertStatus.EXECUTION_SUCCESS : AlertStatus.EXECUTION_FAILURE;
-                    alertDao.addAlertSendStatus(sendStatus, JSONUtils.toJsonString(alertResult), alertId, instance.getId());
+                    AlertStatus sendStatus = Boolean.parseBoolean(String.valueOf(alertResult.getStatus()))
+                            ? AlertStatus.EXECUTION_SUCCESS
+                            : AlertStatus.EXECUTION_FAILURE;
+                    alertDao.addAlertSendStatus(sendStatus, JSONUtils.toJsonString(alertResult), alertId,
+                            instance.getId());
                     if (sendStatus.equals(AlertStatus.EXECUTION_SUCCESS)) {
                         sendSuccessCount++;
                         AlertServerMetrics.incAlertSuccessCount();
@@ -157,7 +161,7 @@ public final class AlertSenderService extends Thread {
         if (CollectionUtils.isEmpty(alertInstanceList)) {
             AlertSendResponseResult alertSendResponseResult = new AlertSendResponseResult();
             String message = String.format("Alert GroupId %s send error : not found alert instance", alertGroupId);
-            alertSendResponseResult.setStatus(false);
+            alertSendResponseResult.setSuccess(false);
             alertSendResponseResult.setMessage(message);
             sendResponseResults.add(alertSendResponseResult);
             logger.error("Alert GroupId {} send error : not found alert instance", alertGroupId);
@@ -169,7 +173,7 @@ public final class AlertSenderService extends Thread {
             if (alertResult != null) {
                 AlertSendResponseResult alertSendResponseResult = new AlertSendResponseResult(
                         Boolean.parseBoolean(String.valueOf(alertResult.getStatus())), alertResult.getMessage());
-                sendResponseStatus = sendResponseStatus && alertSendResponseResult.getStatus();
+                sendResponseStatus = sendResponseStatus && alertSendResponseResult.isSuccess();
                 sendResponseResults.add(alertSendResponseResult);
             }
         }
@@ -190,8 +194,8 @@ public final class AlertSenderService extends Thread {
         Optional<AlertChannel> alertChannelOptional = alertPluginManager.getAlertChannel(instance.getPluginDefineId());
         if (!alertChannelOptional.isPresent()) {
             String message = String.format("Alert Plugin %s send error: the channel doesn't exist, pluginDefineId: %s",
-                                           pluginInstanceName,
-                                           pluginDefineId);
+                    pluginInstanceName,
+                    pluginDefineId);
             logger.error("Alert Plugin {} send error : not found plugin {}", pluginInstanceName, pluginDefineId);
             return new AlertResult("false", message);
         }
@@ -231,16 +235,17 @@ public final class AlertSenderService extends Thread {
         }
 
         if (!sendWarning) {
-            logger.info("Alert Plugin {} send ignore warning type not match: plugin warning type is {}, alert data warning type is {}",
+            logger.info(
+                    "Alert Plugin {} send ignore warning type not match: plugin warning type is {}, alert data warning type is {}",
                     pluginInstanceName, warningType.getCode(), alertData.getWarnType());
             return null;
         }
 
         AlertInfo alertInfo = AlertInfo.builder()
-            .alertData(alertData)
-            .alertParams(paramsMap)
-            .alertPluginInstanceId(instance.getId())
-            .build();
+                .alertData(alertData)
+                .alertParams(paramsMap)
+                .alertPluginInstanceId(instance.getId())
+                .build();
         int waitTimeout = alertConfig.getWaitTimeout();
         try {
             AlertResult alertResult;
diff --git a/dolphinscheduler-alert/dolphinscheduler-alert-server/src/test/java/org/apache/dolphinscheduler/alert/runner/AlertSenderServiceTest.java b/dolphinscheduler-alert/dolphinscheduler-alert-server/src/test/java/org/apache/dolphinscheduler/alert/runner/AlertSenderServiceTest.java
index cdc2f83443..3e38721fba 100644
--- a/dolphinscheduler-alert/dolphinscheduler-alert-server/src/test/java/org/apache/dolphinscheduler/alert/runner/AlertSenderServiceTest.java
+++ b/dolphinscheduler-alert/dolphinscheduler-alert-server/src/test/java/org/apache/dolphinscheduler/alert/runner/AlertSenderServiceTest.java
@@ -48,6 +48,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class AlertSenderServiceTest {
+
     private static final Logger logger = LoggerFactory.getLogger(AlertSenderServiceTest.class);
 
     @Mock
@@ -74,22 +75,23 @@ public class AlertSenderServiceTest {
         String title = "alert mail test title";
         String content = "alert mail test content";
 
-        //1.alert instance does not exist
+        // 1.alert instance does not exist
         when(alertDao.listInstanceByAlertGroupId(alertGroupId)).thenReturn(null);
         when(alertConfig.getWaitTimeout()).thenReturn(0);
 
-        AlertSendResponseCommand alertSendResponseCommand = alertSenderService.syncHandler(alertGroupId, title, content, WarningType.ALL.getCode());
-        Assert.assertFalse(alertSendResponseCommand.getResStatus());
-        alertSendResponseCommand.getResResults().forEach(result ->
-            logger.info("alert send response result, status:{}, message:{}", result.getStatus(), result.getMessage()));
+        AlertSendResponseCommand alertSendResponseCommand =
+                alertSenderService.syncHandler(alertGroupId, title, content, WarningType.ALL.getCode());
+        Assert.assertFalse(alertSendResponseCommand.isSuccess());
+        alertSendResponseCommand.getResResults().forEach(result -> logger
+                .info("alert send response result, status:{}, message:{}", result.isSuccess(), result.getMessage()));
 
-        //2.alert plugin does not exist
+        // 2.alert plugin does not exist
         int pluginDefineId = 1;
         String pluginInstanceParams = "alert-instance-mail-params";
         String pluginInstanceName = "alert-instance-mail";
         List<AlertPluginInstance> alertInstanceList = new ArrayList<>();
         AlertPluginInstance alertPluginInstance = new AlertPluginInstance(
-            pluginDefineId, pluginInstanceParams, pluginInstanceName);
+                pluginDefineId, pluginInstanceParams, pluginInstanceName);
         alertInstanceList.add(alertPluginInstance);
         when(alertDao.listInstanceByAlertGroupId(1)).thenReturn(alertInstanceList);
 
@@ -97,35 +99,38 @@ public class AlertSenderServiceTest {
         PluginDefine pluginDefine = new PluginDefine(pluginName, "1", null);
         when(pluginDao.getPluginDefineById(pluginDefineId)).thenReturn(pluginDefine);
 
-        alertSendResponseCommand = alertSenderService.syncHandler(alertGroupId, title, content, WarningType.ALL.getCode());
-        Assert.assertFalse(alertSendResponseCommand.getResStatus());
-        alertSendResponseCommand.getResResults().forEach(result ->
-            logger.info("alert send response result, status:{}, message:{}", result.getStatus(), result.getMessage()));
+        alertSendResponseCommand =
+                alertSenderService.syncHandler(alertGroupId, title, content, WarningType.ALL.getCode());
+        Assert.assertFalse(alertSendResponseCommand.isSuccess());
+        alertSendResponseCommand.getResResults().forEach(result -> logger
+                .info("alert send response result, status:{}, message:{}", result.isSuccess(), result.getMessage()));
 
-        //3.alert result value is null
+        // 3.alert result value is null
         AlertChannel alertChannelMock = mock(AlertChannel.class);
         when(alertChannelMock.process(Mockito.any())).thenReturn(null);
         when(alertPluginManager.getAlertChannel(1)).thenReturn(Optional.of(alertChannelMock));
         when(alertConfig.getWaitTimeout()).thenReturn(0);
 
-        alertSendResponseCommand = alertSenderService.syncHandler(alertGroupId, title, content, WarningType.ALL.getCode());
-        Assert.assertFalse(alertSendResponseCommand.getResStatus());
-        alertSendResponseCommand.getResResults().forEach(result ->
-            logger.info("alert send response result, status:{}, message:{}", result.getStatus(), result.getMessage()));
+        alertSendResponseCommand =
+                alertSenderService.syncHandler(alertGroupId, title, content, WarningType.ALL.getCode());
+        Assert.assertFalse(alertSendResponseCommand.isSuccess());
+        alertSendResponseCommand.getResResults().forEach(result -> logger
+                .info("alert send response result, status:{}, message:{}", result.isSuccess(), result.getMessage()));
 
-        //4.abnormal information inside the alert plug-in code
+        // 4.abnormal information inside the alert plug-in code
         AlertResult alertResult = new AlertResult();
         alertResult.setStatus(String.valueOf(false));
         alertResult.setMessage("Abnormal information inside the alert plug-in code");
         when(alertChannelMock.process(Mockito.any())).thenReturn(alertResult);
         when(alertPluginManager.getAlertChannel(1)).thenReturn(Optional.of(alertChannelMock));
 
-        alertSendResponseCommand = alertSenderService.syncHandler(alertGroupId, title, content, WarningType.ALL.getCode());
-        Assert.assertFalse(alertSendResponseCommand.getResStatus());
-        alertSendResponseCommand.getResResults().forEach(result ->
-            logger.info("alert send response result, status:{}, message:{}", result.getStatus(), result.getMessage()));
+        alertSendResponseCommand =
+                alertSenderService.syncHandler(alertGroupId, title, content, WarningType.ALL.getCode());
+        Assert.assertFalse(alertSendResponseCommand.isSuccess());
+        alertSendResponseCommand.getResResults().forEach(result -> logger
+                .info("alert send response result, status:{}, message:{}", result.isSuccess(), result.getMessage()));
 
-        //5.alert plugin send success
+        // 5.alert plugin send success
         alertResult = new AlertResult();
         alertResult.setStatus(String.valueOf(true));
         alertResult.setMessage(String.format("Alert Plugin %s send success", pluginInstanceName));
@@ -133,10 +138,11 @@ public class AlertSenderServiceTest {
         when(alertPluginManager.getAlertChannel(1)).thenReturn(Optional.of(alertChannelMock));
         when(alertConfig.getWaitTimeout()).thenReturn(5000);
 
-        alertSendResponseCommand = alertSenderService.syncHandler(alertGroupId, title, content, WarningType.ALL.getCode());
-        Assert.assertTrue(alertSendResponseCommand.getResStatus());
-        alertSendResponseCommand.getResResults().forEach(result ->
-            logger.info("alert send response result, status:{}, message:{}", result.getStatus(), result.getMessage()));
+        alertSendResponseCommand =
+                alertSenderService.syncHandler(alertGroupId, title, content, WarningType.ALL.getCode());
+        Assert.assertTrue(alertSendResponseCommand.isSuccess());
+        alertSendResponseCommand.getResResults().forEach(result -> logger
+                .info("alert send response result, status:{}, message:{}", result.isSuccess(), result.getMessage()));
 
     }
 
@@ -153,14 +159,14 @@ public class AlertSenderServiceTest {
         alert.setWarningType(WarningType.FAILURE);
         alertList.add(alert);
 
-//        alertSenderService = new AlertSenderService();
+        // alertSenderService = new AlertSenderService();
 
         int pluginDefineId = 1;
         String pluginInstanceParams = "alert-instance-mail-params";
         String pluginInstanceName = "alert-instance-mail";
         List<AlertPluginInstance> alertInstanceList = new ArrayList<>();
         AlertPluginInstance alertPluginInstance = new AlertPluginInstance(
-            pluginDefineId, pluginInstanceParams, pluginInstanceName);
+                pluginDefineId, pluginInstanceParams, pluginInstanceName);
         alertInstanceList.add(alertPluginInstance);
         when(alertDao.listInstanceByAlertGroupId(alertGroupId)).thenReturn(alertInstanceList);
 
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceController.java
index 90711242f1..a5ccd5b329 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceController.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceController.java
@@ -17,29 +17,24 @@
 
 package org.apache.dolphinscheduler.api.controller;
 
-import static org.apache.dolphinscheduler.api.enums.Status.BATCH_DELETE_PROCESS_INSTANCE_BY_IDS_ERROR;
-import static org.apache.dolphinscheduler.api.enums.Status.DELETE_PROCESS_INSTANCE_BY_ID_ERROR;
-import static org.apache.dolphinscheduler.api.enums.Status.ENCAPSULATION_PROCESS_INSTANCE_GANTT_STRUCTURE_ERROR;
-import static org.apache.dolphinscheduler.api.enums.Status.QUERY_PARENT_PROCESS_INSTANCE_DETAIL_INFO_BY_SUB_PROCESS_INSTANCE_ID_ERROR;
-import static org.apache.dolphinscheduler.api.enums.Status.QUERY_PROCESS_INSTANCE_ALL_VARIABLES_ERROR;
-import static org.apache.dolphinscheduler.api.enums.Status.QUERY_PROCESS_INSTANCE_BY_ID_ERROR;
-import static org.apache.dolphinscheduler.api.enums.Status.QUERY_PROCESS_INSTANCE_LIST_PAGING_ERROR;
-import static org.apache.dolphinscheduler.api.enums.Status.QUERY_SUB_PROCESS_INSTANCE_DETAIL_INFO_BY_TASK_ID_ERROR;
-import static org.apache.dolphinscheduler.api.enums.Status.QUERY_TASK_LIST_BY_PROCESS_INSTANCE_ID_ERROR;
-import static org.apache.dolphinscheduler.api.enums.Status.UPDATE_PROCESS_INSTANCE_ERROR;
-
+import io.swagger.annotations.*;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.dolphinscheduler.api.aspect.AccessLogAnnotation;
 import org.apache.dolphinscheduler.api.enums.Status;
 import org.apache.dolphinscheduler.api.exceptions.ApiException;
 import org.apache.dolphinscheduler.api.service.ProcessInstanceService;
 import org.apache.dolphinscheduler.api.utils.Result;
 import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
 import org.apache.dolphinscheduler.common.utils.ParameterUtils;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
 import org.apache.dolphinscheduler.dao.entity.User;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
-
-import org.apache.commons.lang3.StringUtils;
+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.io.IOException;
 import java.text.MessageFormat;
@@ -48,27 +43,7 @@ import java.util.HashMap;
 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.http.HttpStatus;
-import org.springframework.web.bind.annotation.DeleteMapping;
-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.PutMapping;
-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 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;
+import static org.apache.dolphinscheduler.api.enums.Status.*;
 
 /**
  * process instance controller
@@ -81,7 +56,7 @@ public class ProcessInstanceController extends BaseController {
     private static final Logger logger = LoggerFactory.getLogger(ProcessInstanceController.class);
 
     @Autowired
-    ProcessInstanceService processInstanceService;
+    private ProcessInstanceService processInstanceService;
 
     /**
      * query process instance list paging
@@ -101,15 +76,15 @@ public class ProcessInstanceController extends BaseController {
      */
     @ApiOperation(value = "queryProcessInstanceListPaging", notes = "QUERY_PROCESS_INSTANCE_LIST_NOTES")
     @ApiImplicitParams({
-        @ApiImplicitParam(name = "processDefineCode", value = "PROCESS_DEFINITION_CODE", dataType = "Long", example = "100"),
-        @ApiImplicitParam(name = "searchVal", value = "SEARCH_VAL", type = "String"),
-        @ApiImplicitParam(name = "executorName", value = "EXECUTOR_NAME", type = "String"),
-        @ApiImplicitParam(name = "stateType", value = "EXECUTION_STATUS", type = "ExecutionStatus"),
-        @ApiImplicitParam(name = "host", value = "HOST", type = "String"),
-        @ApiImplicitParam(name = "startDate", value = "START_DATE", type = "String"),
-        @ApiImplicitParam(name = "endDate", value = "END_DATE", type = "String"),
-        @ApiImplicitParam(name = "pageNo", value = "PAGE_NO", required = true, dataType = "Int", example = "1"),
-        @ApiImplicitParam(name = "pageSize", value = "PAGE_SIZE", required = true, dataType = "Int", example = "10")
+            @ApiImplicitParam(name = "processDefineCode", value = "PROCESS_DEFINITION_CODE", dataType = "Long", example = "100"),
+            @ApiImplicitParam(name = "searchVal", value = "SEARCH_VAL", type = "String"),
+            @ApiImplicitParam(name = "executorName", value = "EXECUTOR_NAME", type = "String"),
+            @ApiImplicitParam(name = "stateType", value = "EXECUTION_STATUS", type = "ExecutionStatus"),
+            @ApiImplicitParam(name = "host", value = "HOST", type = "String"),
+            @ApiImplicitParam(name = "startDate", value = "START_DATE", type = "String"),
+            @ApiImplicitParam(name = "endDate", value = "END_DATE", type = "String"),
+            @ApiImplicitParam(name = "pageNo", value = "PAGE_NO", required = true, dataType = "Int", example = "1"),
+            @ApiImplicitParam(name = "pageSize", value = "PAGE_SIZE", required = true, dataType = "Int", example = "10")
     })
     @GetMapping()
     @ResponseStatus(HttpStatus.OK)
@@ -120,7 +95,7 @@ public class ProcessInstanceController extends BaseController {
                                            @RequestParam(value = "processDefineCode", required = false, defaultValue = "0") long processDefineCode,
                                            @RequestParam(value = "searchVal", required = false) String searchVal,
                                            @RequestParam(value = "executorName", required = false) String executorName,
-                                           @RequestParam(value = "stateType", required = false) ExecutionStatus stateType,
+                                           @RequestParam(value = "stateType", required = false) WorkflowExecutionStatus stateType,
                                            @RequestParam(value = "host", required = false) String host,
                                            @RequestParam(value = "startDate", required = false) String startTime,
                                            @RequestParam(value = "endDate", required = false) String endTime,
@@ -133,7 +108,8 @@ public class ProcessInstanceController extends BaseController {
             return result;
         }
         searchVal = ParameterUtils.handleEscapes(searchVal);
-        result = processInstanceService.queryProcessInstanceList(loginUser, projectCode, processDefineCode, startTime, endTime,
+        result = processInstanceService.queryProcessInstanceList(loginUser, projectCode, processDefineCode, startTime,
+                endTime,
                 searchVal, executorName, stateType, host, otherParamsJson, pageNo, pageSize);
         return result;
     }
@@ -148,7 +124,7 @@ public class ProcessInstanceController extends BaseController {
      */
     @ApiOperation(value = "queryTaskListByProcessId", notes = "QUERY_TASK_LIST_BY_PROCESS_INSTANCE_ID_NOTES")
     @ApiImplicitParams({
-        @ApiImplicitParam(name = "id", value = "PROCESS_INSTANCE_ID", required = true, dataType = "Int", example = "100")
+            @ApiImplicitParam(name = "id", value = "PROCESS_INSTANCE_ID", required = true, dataType = "Int", example = "100")
     })
     @GetMapping(value = "/{id}/tasks")
     @ResponseStatus(HttpStatus.OK)
@@ -177,15 +153,15 @@ public class ProcessInstanceController extends BaseController {
      */
     @ApiOperation(value = "updateProcessInstance", notes = "UPDATE_PROCESS_INSTANCE_NOTES")
     @ApiImplicitParams({
-        @ApiImplicitParam(name = "taskRelationJson", value = "TASK_RELATION_JSON", type = "String"),
-        @ApiImplicitParam(name = "taskDefinitionJson", value = "TASK_DEFINITION_JSON", type = "String"),
-        @ApiImplicitParam(name = "id", value = "PROCESS_INSTANCE_ID", required = true, dataType = "Int", example = "1"),
-        @ApiImplicitParam(name = "scheduleTime", value = "SCHEDULE_TIME", type = "String"),
-        @ApiImplicitParam(name = "syncDefine", value = "SYNC_DEFINE", required = true, type = "Boolean", example = "false"),
-        @ApiImplicitParam(name = "globalParams", value = "PROCESS_GLOBAL_PARAMS", type = "String", example = "[]"),
-        @ApiImplicitParam(name = "locations", value = "PROCESS_INSTANCE_LOCATIONS", type = "String"),
-        @ApiImplicitParam(name = "timeout", value = "PROCESS_TIMEOUT", type = "Int", example = "0"),
-        @ApiImplicitParam(name = "tenantCode", value = "TENANT_CODE", type = "String", example = "default")
+            @ApiImplicitParam(name = "taskRelationJson", value = "TASK_RELATION_JSON", type = "String"),
+            @ApiImplicitParam(name = "taskDefinitionJson", value = "TASK_DEFINITION_JSON", type = "String"),
+            @ApiImplicitParam(name = "id", value = "PROCESS_INSTANCE_ID", required = true, dataType = "Int", example = "1"),
+            @ApiImplicitParam(name = "scheduleTime", value = "SCHEDULE_TIME", type = "String"),
+            @ApiImplicitParam(name = "syncDefine", value = "SYNC_DEFINE", required = true, type = "Boolean", example = "false"),
+            @ApiImplicitParam(name = "globalParams", value = "PROCESS_GLOBAL_PARAMS", type = "String", example = "[]"),
+            @ApiImplicitParam(name = "locations", value = "PROCESS_INSTANCE_LOCATIONS", type = "String"),
+            @ApiImplicitParam(name = "timeout", value = "PROCESS_TIMEOUT", type = "Int", example = "0"),
+            @ApiImplicitParam(name = "tenantCode", value = "TENANT_CODE", type = "String", example = "default")
     })
     @PutMapping(value = "/{id}")
     @ResponseStatus(HttpStatus.OK)
@@ -203,7 +179,8 @@ public class ProcessInstanceController extends BaseController {
                                         @RequestParam(value = "timeout", required = false, defaultValue = "0") int timeout,
                                         @RequestParam(value = "tenantCode", required = true) String tenantCode) {
         Map<String, Object> result = processInstanceService.updateProcessInstance(loginUser, projectCode, id,
-            taskRelationJson, taskDefinitionJson, scheduleTime, syncDefine, globalParams, locations, timeout, tenantCode);
+                taskRelationJson, taskDefinitionJson, scheduleTime, syncDefine, globalParams, locations, timeout,
+                tenantCode);
         return returnDataList(result);
     }
 
@@ -217,7 +194,7 @@ public class ProcessInstanceController extends BaseController {
      */
     @ApiOperation(value = "queryProcessInstanceById", notes = "QUERY_PROCESS_INSTANCE_BY_ID_NOTES")
     @ApiImplicitParams({
-        @ApiImplicitParam(name = "id", value = "PROCESS_INSTANCE_ID", required = true, dataType = "Int", example = "100")
+            @ApiImplicitParam(name = "id", value = "PROCESS_INSTANCE_ID", required = true, dataType = "Int", example = "100")
     })
     @GetMapping(value = "/{id}")
     @ResponseStatus(HttpStatus.OK)
@@ -242,9 +219,9 @@ public class ProcessInstanceController extends BaseController {
      */
     @ApiOperation(value = "queryTopNLongestRunningProcessInstance", notes = "QUERY_TOPN_LONGEST_RUNNING_PROCESS_INSTANCE_NOTES")
     @ApiImplicitParams({
-        @ApiImplicitParam(name = "size", value = "PROCESS_INSTANCE_SIZE", required = true, dataType = "Int", example = "10"),
-        @ApiImplicitParam(name = "startTime", value = "PROCESS_INSTANCE_START_TIME", required = true, dataType = "String"),
-        @ApiImplicitParam(name = "endTime", value = "PROCESS_INSTANCE_END_TIME", required = true, dataType = "String"),
+            @ApiImplicitParam(name = "size", value = "PROCESS_INSTANCE_SIZE", required = true, dataType = "Int", example = "10"),
+            @ApiImplicitParam(name = "startTime", value = "PROCESS_INSTANCE_START_TIME", required = true, dataType = "String"),
+            @ApiImplicitParam(name = "endTime", value = "PROCESS_INSTANCE_END_TIME", required = true, dataType = "String"),
     })
     @GetMapping(value = "/top-n")
     @ResponseStatus(HttpStatus.OK)
@@ -255,7 +232,8 @@ public class ProcessInstanceController extends BaseController {
                                                                           @RequestParam("size") Integer size,
                                                                           @RequestParam(value = "startTime", required = true) String startTime,
                                                                           @RequestParam(value = "endTime", required = true) String endTime) {
-        Map<String, Object> result = processInstanceService.queryTopNLongestRunningProcessInstance(loginUser, projectCode, size, startTime, endTime);
+        Map<String, Object> result = processInstanceService.queryTopNLongestRunningProcessInstance(loginUser,
+                projectCode, size, startTime, endTime);
         return returnDataList(result);
     }
 
@@ -270,7 +248,7 @@ public class ProcessInstanceController extends BaseController {
      */
     @ApiOperation(value = "deleteProcessInstanceById", notes = "DELETE_PROCESS_INSTANCE_BY_ID_NOTES")
     @ApiImplicitParams({
-        @ApiImplicitParam(name = "id", value = "PROCESS_INSTANCE_ID", required = true, dataType = "Int", example = "100")
+            @ApiImplicitParam(name = "id", value = "PROCESS_INSTANCE_ID", required = true, dataType = "Int", example = "100")
     })
     @DeleteMapping(value = "/{id}")
     @ResponseStatus(HttpStatus.OK)
@@ -293,7 +271,7 @@ public class ProcessInstanceController extends BaseController {
      */
     @ApiOperation(value = "querySubProcessInstanceByTaskCode", notes = "QUERY_SUBPROCESS_INSTANCE_BY_TASK_CODE_NOTES")
     @ApiImplicitParams({
-        @ApiImplicitParam(name = "taskCode", value = "TASK_CODE", required = true, dataType = "Long", example = "100")
+            @ApiImplicitParam(name = "taskCode", value = "TASK_CODE", required = true, dataType = "Long", example = "100")
     })
     @GetMapping(value = "/query-sub-by-parent")
     @ResponseStatus(HttpStatus.OK)
@@ -302,7 +280,8 @@ public class ProcessInstanceController extends BaseController {
     public Result querySubProcessInstanceByTaskId(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
                                                   @ApiParam(name = "projectCode", value = "PROJECT_CODE", required = true) @PathVariable long projectCode,
                                                   @RequestParam("taskId") Integer taskId) {
-        Map<String, Object> result = processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectCode, taskId);
+        Map<String, Object> result =
+                processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectCode, taskId);
         return returnDataList(result);
     }
 
@@ -316,7 +295,7 @@ public class ProcessInstanceController extends BaseController {
      */
     @ApiOperation(value = "queryParentInstanceBySubId", notes = "QUERY_PARENT_PROCESS_INSTANCE_BY_SUB_PROCESS_INSTANCE_ID_NOTES")
     @ApiImplicitParams({
-        @ApiImplicitParam(name = "subId", value = "SUB_PROCESS_INSTANCE_ID", required = true, dataType = "Int", example = "100")
+            @ApiImplicitParam(name = "subId", value = "SUB_PROCESS_INSTANCE_ID", required = true, dataType = "Int", example = "100")
     })
     @GetMapping(value = "/query-parent-by-sub")
     @ResponseStatus(HttpStatus.OK)
@@ -338,7 +317,7 @@ public class ProcessInstanceController extends BaseController {
      */
     @ApiOperation(value = "viewVariables", notes = "QUERY_PROCESS_INSTANCE_GLOBAL_VARIABLES_AND_LOCAL_VARIABLES_NOTES")
     @ApiImplicitParams({
-        @ApiImplicitParam(name = "id", value = "PROCESS_INSTANCE_ID", required = true, dataType = "Int", example = "100")
+            @ApiImplicitParam(name = "id", value = "PROCESS_INSTANCE_ID", required = true, dataType = "Int", example = "100")
     })
     @GetMapping(value = "/{id}/view-variables")
     @ResponseStatus(HttpStatus.OK)
@@ -361,7 +340,7 @@ public class ProcessInstanceController extends BaseController {
      */
     @ApiOperation(value = "vieGanttTree", notes = "VIEW_GANTT_NOTES")
     @ApiImplicitParams({
-        @ApiImplicitParam(name = "id", value = "PROCESS_INSTANCE_ID", required = true, dataType = "Int", example = "100")
+            @ApiImplicitParam(name = "id", value = "PROCESS_INSTANCE_ID", required = true, dataType = "Int", example = "100")
     })
     @GetMapping(value = "/{id}/view-gantt")
     @ResponseStatus(HttpStatus.OK)
@@ -385,8 +364,8 @@ public class ProcessInstanceController extends BaseController {
      */
     @ApiOperation(value = "batchDeleteProcessInstanceByIds", notes = "BATCH_DELETE_PROCESS_INSTANCE_BY_IDS_NOTES")
     @ApiImplicitParams({
-        @ApiImplicitParam(name = "projectCode", value = "PROJECT_CODE", required = true, dataType = "Int"),
-        @ApiImplicitParam(name = "processInstanceIds", value = "PROCESS_INSTANCE_IDS", required = true, dataType = "String"),
+            @ApiImplicitParam(name = "projectCode", value = "PROJECT_CODE", required = true, dataType = "Int"),
+            @ApiImplicitParam(name = "processInstanceIds", value = "PROCESS_INSTANCE_IDS", required = true, dataType = "String"),
     })
     @PostMapping(value = "/batch-delete")
     @ResponseStatus(HttpStatus.OK)
@@ -404,13 +383,15 @@ public class ProcessInstanceController extends BaseController {
             for (String strProcessInstanceId : processInstanceIdArray) {
                 int processInstanceId = Integer.parseInt(strProcessInstanceId);
                 try {
-                    Map<String, Object> deleteResult = processInstanceService.deleteProcessInstanceById(loginUser, projectCode, processInstanceId);
+                    Map<String, Object> deleteResult =
+                            processInstanceService.deleteProcessInstanceById(loginUser, projectCode, processInstanceId);
                     if (!Status.SUCCESS.equals(deleteResult.get(Constants.STATUS))) {
                         deleteFailedIdList.add((String) deleteResult.get(Constants.MSG));
                         logger.error((String) deleteResult.get(Constants.MSG));
                     }
                 } catch (Exception e) {
-                    deleteFailedIdList.add(MessageFormat.format(Status.PROCESS_INSTANCE_ERROR.getMsg(), strProcessInstanceId));
+                    deleteFailedIdList
+                            .add(MessageFormat.format(Status.PROCESS_INSTANCE_ERROR.getMsg(), strProcessInstanceId));
                 }
             }
         }
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 e007c55807..411fd417e6 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
@@ -27,10 +27,10 @@ import org.apache.dolphinscheduler.api.utils.Result;
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.utils.ParameterUtils;
 import org.apache.dolphinscheduler.dao.entity.User;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 
 import java.util.Map;
 
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.http.HttpStatus;
 import org.springframework.web.bind.annotation.GetMapping;
@@ -58,7 +58,7 @@ import springfox.documentation.annotations.ApiIgnore;
 public class TaskInstanceController extends BaseController {
 
     @Autowired
-    TaskInstanceService taskInstanceService;
+    private TaskInstanceService taskInstanceService;
 
     /**
      * query task list paging
@@ -78,17 +78,17 @@ public class TaskInstanceController extends BaseController {
      */
     @ApiOperation(value = "queryTaskListPaging", notes = "QUERY_TASK_INSTANCE_LIST_PAGING_NOTES")
     @ApiImplicitParams({
-        @ApiImplicitParam(name = "processInstanceId", value = "PROCESS_INSTANCE_ID", required = false, dataType = "Int", example = "100"),
-        @ApiImplicitParam(name = "processInstanceName", value = "PROCESS_INSTANCE_NAME", required = false, type = "String"),
-        @ApiImplicitParam(name = "searchVal", value = "SEARCH_VAL", type = "String"),
-        @ApiImplicitParam(name = "taskName", value = "TASK_NAME", type = "String"),
-        @ApiImplicitParam(name = "executorName", value = "EXECUTOR_NAME", type = "String"),
-        @ApiImplicitParam(name = "stateType", value = "EXECUTION_STATUS", type = "ExecutionStatus"),
-        @ApiImplicitParam(name = "host", value = "HOST", type = "String"),
-        @ApiImplicitParam(name = "startDate", value = "START_DATE", type = "String"),
-        @ApiImplicitParam(name = "endDate", value = "END_DATE", type = "String"),
-        @ApiImplicitParam(name = "pageNo", value = "PAGE_NO", required = true, dataType = "Int", example = "1"),
-        @ApiImplicitParam(name = "pageSize", value = "PAGE_SIZE", required = true, dataType = "Int", example = "20")
+            @ApiImplicitParam(name = "processInstanceId", value = "PROCESS_INSTANCE_ID", required = false, dataType = "Int", example = "100"),
+            @ApiImplicitParam(name = "processInstanceName", value = "PROCESS_INSTANCE_NAME", required = false, type = "String"),
+            @ApiImplicitParam(name = "searchVal", value = "SEARCH_VAL", type = "String"),
+            @ApiImplicitParam(name = "taskName", value = "TASK_NAME", type = "String"),
+            @ApiImplicitParam(name = "executorName", value = "EXECUTOR_NAME", type = "String"),
+            @ApiImplicitParam(name = "stateType", value = "EXECUTION_STATUS", type = "ExecutionStatus"),
+            @ApiImplicitParam(name = "host", value = "HOST", type = "String"),
+            @ApiImplicitParam(name = "startDate", value = "START_DATE", type = "String"),
+            @ApiImplicitParam(name = "endDate", value = "END_DATE", type = "String"),
+            @ApiImplicitParam(name = "pageNo", value = "PAGE_NO", required = true, dataType = "Int", example = "1"),
+            @ApiImplicitParam(name = "pageSize", value = "PAGE_SIZE", required = true, dataType = "Int", example = "20")
     })
     @GetMapping()
     @ResponseStatus(HttpStatus.OK)
@@ -101,7 +101,7 @@ public class TaskInstanceController extends BaseController {
                                       @RequestParam(value = "searchVal", required = false) String searchVal,
                                       @RequestParam(value = "taskName", required = false) String taskName,
                                       @RequestParam(value = "executorName", required = false) String executorName,
-                                      @RequestParam(value = "stateType", required = false) ExecutionStatus stateType,
+                                      @RequestParam(value = "stateType", required = false) TaskExecutionStatus stateType,
                                       @RequestParam(value = "host", required = false) String host,
                                       @RequestParam(value = "startDate", required = false) String startTime,
                                       @RequestParam(value = "endDate", required = false) String endTime,
@@ -127,7 +127,7 @@ public class TaskInstanceController extends BaseController {
      */
     @ApiOperation(value = "force-success", notes = "FORCE_TASK_SUCCESS")
     @ApiImplicitParams({
-        @ApiImplicitParam(name = "id", value = "TASK_INSTANCE_ID", required = true, dataType = "Int", example = "12")
+            @ApiImplicitParam(name = "id", value = "TASK_INSTANCE_ID", required = true, dataType = "Int", example = "12")
     })
     @PostMapping(value = "/{id}/force-success")
     @ResponseStatus(HttpStatus.OK)
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 96082aab45..6a7d78fc4a 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
@@ -17,17 +17,16 @@
 
 package org.apache.dolphinscheduler.api.dto;
 
+import lombok.Data;
 import org.apache.dolphinscheduler.dao.entity.ExecuteStatusCount;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
 
-/**
- * task count dto
- */
+@Data
 public class TaskCountDto {
 
     /**
@@ -45,10 +44,10 @@ public class TaskCountDto {
     }
 
     private void countTaskDtos(List<ExecuteStatusCount> taskInstanceStateCounts) {
-        Map<ExecutionStatus, Integer> statusCountMap = taskInstanceStateCounts.stream()
-                .collect(Collectors.toMap(ExecuteStatusCount::getExecutionStatus, ExecuteStatusCount::getCount, Integer::sum));
+        Map<TaskExecutionStatus, Integer> statusCountMap = taskInstanceStateCounts.stream()
+                .collect(Collectors.toMap(ExecuteStatusCount::getState, ExecuteStatusCount::getCount, Integer::sum));
 
-        taskCountDtos = Arrays.stream(ExecutionStatus.values())
+        taskCountDtos = Arrays.stream(TaskExecutionStatus.values())
                 .map(status -> new TaskStateCount(status, statusCountMap.getOrDefault(status, 0)))
                 .collect(Collectors.toList());
 
@@ -58,7 +57,7 @@ public class TaskCountDto {
     }
 
     // remove the specified state
-    public void removeStateFromCountList(ExecutionStatus status) {
+    public void removeStateFromCountList(TaskExecutionStatus status) {
         for (TaskStateCount count : this.taskCountDtos) {
             if (count.getTaskStateType().equals(status)) {
                 this.taskCountDtos.remove(count);
@@ -67,19 +66,4 @@ public class TaskCountDto {
         }
     }
 
-    public List<TaskStateCount> getTaskCountDtos() {
-        return taskCountDtos;
-    }
-
-    public void setTaskCountDtos(List<TaskStateCount> taskCountDtos) {
-        this.taskCountDtos = taskCountDtos;
-    }
-
-    public int getTotalCount() {
-        return totalCount;
-    }
-
-    public void setTotalCount(int totalCount) {
-        this.totalCount = totalCount;
-    }
 }
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/dto/TaskStateCount.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/dto/TaskStateCount.java
index 2fe5e9ec23..95dc18a49f 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/dto/TaskStateCount.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/dto/TaskStateCount.java
@@ -17,58 +17,17 @@
 
 package org.apache.dolphinscheduler.api.dto;
 
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
-
-/**
- * task state count
- */
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
+
+@Data
+@NoArgsConstructor
+@AllArgsConstructor
 public class TaskStateCount {
 
+    private TaskExecutionStatus taskStateType;
     private int count;
-    private ExecutionStatus taskStateType;
-
-    public TaskStateCount(ExecutionStatus taskStateType, int count) {
-        this.taskStateType = taskStateType;
-        this.count = count;
-    }
-
-    public int getCount() {
-        return count;
-    }
-
-    public void setCount(int count) {
-        this.count = count;
-    }
-
-    public ExecutionStatus getTaskStateType() {
-        return taskStateType;
-    }
-
-    public void setTaskStateType(ExecutionStatus taskStateType) {
-        this.taskStateType = taskStateType;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-
-        TaskStateCount that = (TaskStateCount) o;
-
-        if (count != that.count) {
-            return false;
-        }
-        return taskStateType == that.taskStateType;
-    }
 
-    @Override
-    public int hashCode() {
-        int result = count;
-        result = 31 * result + (taskStateType != null ? taskStateType.hashCode() : 0);
-        return result;
-    }
 }
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java
index 73a5b35ff6..ca03353740 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java
@@ -23,10 +23,10 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.dolphinscheduler.api.utils.Result;
+import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
 import org.apache.dolphinscheduler.dao.entity.User;
 import org.apache.dolphinscheduler.plugin.task.api.enums.DependResult;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 
 /**
  * process instance service
@@ -78,7 +78,7 @@ public interface ProcessInstanceService {
                                     String endDate,
                                     String searchVal,
                                     String executorName,
-                                    ExecutionStatus stateType,
+                                    WorkflowExecutionStatus stateType,
                                     String host,
                                     String otherParamsJson,
                                     Integer pageNo,
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 0d16022799..f532b7d046 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
@@ -19,7 +19,7 @@ package org.apache.dolphinscheduler.api.service;
 
 import org.apache.dolphinscheduler.api.utils.Result;
 import org.apache.dolphinscheduler.dao.entity.User;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 
 import java.util.Map;
 
@@ -53,7 +53,7 @@ public interface TaskInstanceService {
                                String startDate,
                                String endDate,
                                String searchVal,
-                               ExecutionStatus stateType,
+                               TaskExecutionStatus stateType,
                                String host,
                                Integer pageNo,
                                Integer pageSize);
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 c914bcd3d0..c7222c2a0d 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
@@ -19,14 +19,12 @@ package org.apache.dolphinscheduler.api.service.impl;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.tuple.Pair;
-import org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant;
 import org.apache.dolphinscheduler.api.dto.CommandStateCount;
 import org.apache.dolphinscheduler.api.dto.DefineUserDto;
 import org.apache.dolphinscheduler.api.dto.TaskCountDto;
 import org.apache.dolphinscheduler.api.enums.Status;
 import org.apache.dolphinscheduler.api.service.DataAnalysisService;
 import org.apache.dolphinscheduler.api.service.ProjectService;
-import org.apache.dolphinscheduler.api.utils.Result;
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.AuthorizationType;
 import org.apache.dolphinscheduler.common.enums.CommandType;
@@ -44,7 +42,7 @@ 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.dao.mapper.TaskInstanceMapper;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -53,11 +51,8 @@ import org.springframework.stereotype.Service;
 
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -109,7 +104,8 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
      * @return task state count data
      */
     @Override
-    public Map<String, Object> countTaskStateByProject(User loginUser, long projectCode, String startDate, String endDate) {
+    public Map<String, Object> countTaskStateByProject(User loginUser, long projectCode, String startDate,
+                                                       String endDate) {
 
         return countStateByProject(
                 loginUser,
@@ -129,17 +125,20 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
      * @return process instance state count data
      */
     @Override
-    public Map<String, Object> countProcessInstanceStateByProject(User loginUser, long projectCode, String startDate, String endDate) {
+    public Map<String, Object> countProcessInstanceStateByProject(User loginUser, long projectCode, String startDate,
+                                                                  String endDate) {
         Map<String, Object> result = this.countStateByProject(
                 loginUser,
                 projectCode,
                 startDate,
                 endDate,
-                (start, end, projectCodes) -> this.processInstanceMapper.countInstanceStateByProjectCodes(start, end, projectCodes));
+                (start, end, projectCodes) -> this.processInstanceMapper.countInstanceStateByProjectCodes(start, end,
+                        projectCodes));
 
         // process state count needs to remove state of forced success
         if (result.containsKey(Constants.STATUS) && result.get(Constants.STATUS).equals(Status.SUCCESS)) {
-            ((TaskCountDto) result.get(Constants.DATA_LIST)).removeStateFromCountList(ExecutionStatus.FORCED_SUCCESS);
+            ((TaskCountDto) result.get(Constants.DATA_LIST))
+                    .removeStateFromCountList(TaskExecutionStatus.FORCED_SUCCESS);
         }
         return result;
     }
@@ -152,12 +151,12 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
      * @param startDate   start date
      * @param endDate     end date
      */
-    private Map<String, Object> countStateByProject(User loginUser, long projectCode, String startDate, String endDate
-            , TriFunction<Date, Date, Long[], List<ExecuteStatusCount>> instanceStateCounter) {
+    private Map<String, Object> countStateByProject(User loginUser, long projectCode, String startDate, String endDate,
+                                                    TriFunction<Date, Date, Long[], List<ExecuteStatusCount>> instanceStateCounter) {
         Map<String, Object> result = new HashMap<>();
         if (projectCode != 0) {
             Project project = projectMapper.queryByCode(projectCode);
-            result = projectService.checkProjectAndAuth(loginUser, project, projectCode,PROJECT_OVERVIEW);
+            result = projectService.checkProjectAndAuth(loginUser, project, projectCode, PROJECT_OVERVIEW);
             if (result.get(Constants.STATUS) != Status.SUCCESS) {
                 return result;
             }
@@ -177,7 +176,8 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
         if (projectIds.getRight() != null) {
             return projectIds.getRight();
         }
-        Long[] projectCodeArray = projectCode == 0 ? getProjectCodesArrays(projectIds.getLeft()) : new Long[]{projectCode};
+        Long[] projectCodeArray =
+                projectCode == 0 ? getProjectCodesArrays(projectIds.getLeft()) : new Long[]{projectCode};
         List<ExecuteStatusCount> processInstanceStateCounts = new ArrayList<>();
 
         if (projectCodeArray.length != 0 || loginUser.getUserType() == UserType.ADMIN_USER) {
@@ -192,7 +192,6 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
         return result;
     }
 
-
     /**
      * statistics the process definition quantities of a certain person
      * <p>
@@ -207,7 +206,7 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
         Map<String, Object> result = new HashMap<>();
         if (projectCode != 0) {
             Project project = projectMapper.queryByCode(projectCode);
-            result = projectService.checkProjectAndAuth(loginUser, project, projectCode,PROJECT_OVERVIEW);
+            result = projectService.checkProjectAndAuth(loginUser, project, projectCode, PROJECT_OVERVIEW);
             if (result.get(Constants.STATUS) != Status.SUCCESS) {
                 return result;
             }
@@ -222,7 +221,8 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
             putMsg(result, Status.SUCCESS);
             return result;
         }
-        Long[] projectCodeArray = projectCode == 0 ? getProjectCodesArrays(projectIds.getLeft()) : new Long[]{projectCode};
+        Long[] projectCodeArray =
+                projectCode == 0 ? getProjectCodesArrays(projectIds.getLeft()) : new Long[]{projectCode};
         if (projectCodeArray.length != 0 || loginUser.getUserType() == UserType.ADMIN_USER) {
             defineGroupByUsers = processDefinitionMapper.countDefinitionByProjectCodes(projectCodeArray);
         }
@@ -233,7 +233,6 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
         return result;
     }
 
-
     /**
      * statistical command status data
      *
@@ -252,28 +251,31 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
         Date end = null;
         Pair<Set<Integer>, Map<String, Object>> projectIds = getProjectIds(loginUser, result);
         if (projectIds.getRight() != null) {
-            List<CommandStateCount> noData = Arrays.stream(CommandType.values()).map(commandType -> new CommandStateCount(0, 0, commandType)).collect(Collectors.toList());
+            List<CommandStateCount> noData = Arrays.stream(CommandType.values())
+                    .map(commandType -> new CommandStateCount(0, 0, commandType)).collect(Collectors.toList());
             result.put(Constants.DATA_LIST, noData);
             putMsg(result, Status.SUCCESS);
             return result;
         }
         Long[] projectCodeArray = getProjectCodesArrays(projectIds.getLeft());
         // count normal command state
-        Map<CommandType, Integer> normalCountCommandCounts = commandMapper.countCommandState(start, end, projectCodeArray)
-                .stream()
-                .collect(Collectors.toMap(CommandCount::getCommandType, CommandCount::getCount));
+        Map<CommandType, Integer> normalCountCommandCounts =
+                commandMapper.countCommandState(start, end, projectCodeArray)
+                        .stream()
+                        .collect(Collectors.toMap(CommandCount::getCommandType, CommandCount::getCount));
 
         // count error command state
-        Map<CommandType, Integer> errorCommandCounts = errorCommandMapper.countCommandState(start, end, projectCodeArray)
-                .stream()
-                .collect(Collectors.toMap(CommandCount::getCommandType, CommandCount::getCount));
+        Map<CommandType, Integer> errorCommandCounts =
+                errorCommandMapper.countCommandState(start, end, projectCodeArray)
+                        .stream()
+                        .collect(Collectors.toMap(CommandCount::getCommandType, CommandCount::getCount));
 
         List<CommandStateCount> list = Arrays.stream(CommandType.values())
                 .map(commandType -> new CommandStateCount(
                         errorCommandCounts.getOrDefault(commandType, 0),
                         normalCountCommandCounts.getOrDefault(commandType, 0),
-                        commandType)
-                ).collect(Collectors.toList());
+                        commandType))
+                .collect(Collectors.toList());
 
         result.put(Constants.DATA_LIST, list);
         putMsg(result, Status.SUCCESS);
@@ -281,7 +283,8 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
     }
 
     private Pair<Set<Integer>, Map<String, Object>> getProjectIds(User loginUser, Map<String, Object> result) {
-        Set<Integer> projectIds = resourcePermissionCheckService.userOwnedResourceIdsAcquisition(AuthorizationType.PROJECTS, loginUser.getId(), logger);
+        Set<Integer> projectIds = resourcePermissionCheckService
+                .userOwnedResourceIdsAcquisition(AuthorizationType.PROJECTS, loginUser.getId(), logger);
         if (projectIds.isEmpty()) {
             List<ExecuteStatusCount> taskInstanceStateCounts = new ArrayList<>();
             result.put(Constants.DATA_LIST, new TaskCountDto(taskInstanceStateCounts));
@@ -308,7 +311,7 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
     public Map<String, Object> countQueueState(User loginUser) {
         Map<String, Object> result = new HashMap<>();
 
-        //TODO need to add detail data info
+        // TODO need to add detail data info
         Map<String, Integer> dataMap = new HashMap<>();
         dataMap.put("taskQueue", 0);
         dataMap.put("taskKill", 0);
@@ -318,15 +321,19 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
     }
 
     @Override
-    public List<ExecuteStatusCount> countTaskInstanceAllStatesByProjectCodes(Date startTime, Date endTime, Long[] projectCodes) {
-        Optional<List<ExecuteStatusCount>> startTimeStates = Optional.ofNullable(this.taskInstanceMapper.countTaskInstanceStateByProjectCodes(startTime, endTime, projectCodes));
+    public List<ExecuteStatusCount> countTaskInstanceAllStatesByProjectCodes(Date startTime, Date endTime,
+                                                                             Long[] projectCodes) {
+        Optional<List<ExecuteStatusCount>> startTimeStates = Optional.ofNullable(
+                this.taskInstanceMapper.countTaskInstanceStateByProjectCodes(startTime, endTime, projectCodes));
 
-        List<ExecutionStatus> allState = Arrays.stream(ExecutionStatus.values()).collect(Collectors.toList());
-        List<ExecutionStatus> needRecountState;
+        List<TaskExecutionStatus> allState = Arrays.stream(TaskExecutionStatus.values()).collect(Collectors.toList());
+        List<TaskExecutionStatus> needRecountState;
         if (startTimeStates.isPresent() && startTimeStates.get().size() != 0) {
-            List<ExecutionStatus> instanceState = startTimeStates.get().stream().map(ExecuteStatusCount::getExecutionStatus).collect(Collectors.toList());
-            //value 0 state need to recount by submit time
-            needRecountState = allState.stream().filter(ele -> !instanceState.contains(ele)).collect(Collectors.toList());
+            List<TaskExecutionStatus> instanceState =
+                    startTimeStates.get().stream().map(ExecuteStatusCount::getState).collect(Collectors.toList());
+            // value 0 state need to recount by submit time
+            needRecountState =
+                    allState.stream().filter(ele -> !instanceState.contains(ele)).collect(Collectors.toList());
             if (needRecountState.size() == 0) {
                 return startTimeStates.get();
             }
@@ -334,10 +341,11 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
             needRecountState = allState;
         }
 
-        //use submit time to recount when 0
-        //if have any issues with this code, should change to specified states 0 8 9 17 not state count is 0
+        // use submit time to recount when 0
+        // if have any issues with this code, should change to specified states 0 8 9 17 not state count is 0
         List<ExecuteStatusCount> recounts = this.taskInstanceMapper
-                .countTaskInstanceStateByProjectCodesAndStatesBySubmitTime(startTime, endTime, projectCodes, needRecountState);
+                .countTaskInstanceStateByProjectCodesAndStatesBySubmitTime(startTime, endTime, projectCodes,
+                        needRecountState);
         startTimeStates.orElseGet(ArrayList::new).addAll(recounts);
 
         return startTimeStates.orElse(null);
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ExecutorServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ExecutorServiceImpl.java
index 473acf3f39..7cf164aaaa 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ExecutorServiceImpl.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ExecutorServiceImpl.java
@@ -35,17 +35,7 @@ import org.apache.dolphinscheduler.api.service.ExecutorService;
 import org.apache.dolphinscheduler.api.service.MonitorService;
 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.ComplementDependentMode;
-import org.apache.dolphinscheduler.common.enums.CycleEnum;
-import org.apache.dolphinscheduler.common.enums.FailureStrategy;
-import org.apache.dolphinscheduler.common.enums.Flag;
-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.TaskGroupQueueStatus;
-import org.apache.dolphinscheduler.common.enums.WarningType;
+import org.apache.dolphinscheduler.common.enums.*;
 import org.apache.dolphinscheduler.common.model.Server;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
@@ -67,8 +57,7 @@ import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
 import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
 import org.apache.dolphinscheduler.dao.mapper.TaskGroupQueueMapper;
 import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
-import org.apache.dolphinscheduler.remote.command.StateEventChangeCommand;
+import org.apache.dolphinscheduler.remote.command.WorkflowStateEventChangeCommand;
 import org.apache.dolphinscheduler.remote.command.WorkflowExecutingDataRequestCommand;
 import org.apache.dolphinscheduler.remote.command.WorkflowExecutingDataResponseCommand;
 import org.apache.dolphinscheduler.remote.dto.WorkflowExecuteDto;
@@ -172,9 +161,9 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
                                                    Map<String, String> startParams, Integer expectedParallelismNumber,
                                                    int dryRun, ComplementDependentMode complementDependentMode) {
         Project project = projectMapper.queryByCode(projectCode);
-        //check user access for project
+        // check user access for project
         Map<String, Object> result =
-            projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_START);
+                projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_START);
         if (result.get(Constants.STATUS) != Status.SUCCESS) {
             return result;
         }
@@ -186,14 +175,15 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
 
         // check process define release state
         ProcessDefinition processDefinition = processDefinitionMapper.queryByCode(processDefinitionCode);
-        result = checkProcessDefinitionValid(projectCode, processDefinition, processDefinitionCode, processDefinition.getVersion());
+        result = checkProcessDefinitionValid(projectCode, processDefinition, processDefinitionCode,
+                processDefinition.getVersion());
         if (result.get(Constants.STATUS) != Status.SUCCESS) {
             return result;
         }
 
         if (!checkTenantSuitable(processDefinition)) {
             logger.error("there is not any valid tenant for the process definition: id:{},name:{}, ",
-                processDefinition.getId(), processDefinition.getName());
+                    processDefinition.getId(), processDefinition.getName());
             putMsg(result, Status.TENANT_NOT_SUITABLE);
             return result;
         }
@@ -211,9 +201,11 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
          * create command
          */
         int create =
-            this.createCommand(commandType, processDefinition.getCode(), taskDependType, failureStrategy, startNodeList,
-                cronTime, warningType, loginUser.getId(), warningGroupId, runMode, processInstancePriority, workerGroup,
-                environmentCode, startParams, expectedParallelismNumber, dryRun, complementDependentMode);
+                this.createCommand(commandType, processDefinition.getCode(), taskDependType, failureStrategy,
+                        startNodeList,
+                        cronTime, warningType, loginUser.getId(), warningGroupId, runMode, processInstancePriority,
+                        workerGroup,
+                        environmentCode, startParams, expectedParallelismNumber, dryRun, complementDependentMode);
 
         if (create > 0) {
             processDefinition.setWarningGroupId(warningGroupId);
@@ -303,20 +295,21 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
     public boolean checkSubProcessDefinitionValid(ProcessDefinition processDefinition) {
         // query all subprocesses under the current process
         List<ProcessTaskRelation> processTaskRelations =
-            processTaskRelationMapper.queryDownstreamByProcessDefinitionCode(processDefinition.getCode());
+                processTaskRelationMapper.queryDownstreamByProcessDefinitionCode(processDefinition.getCode());
         if (processTaskRelations.isEmpty()) {
             return true;
         }
         Set<Long> relationCodes =
-            processTaskRelations.stream().map(ProcessTaskRelation::getPostTaskCode).collect(Collectors.toSet());
+                processTaskRelations.stream().map(ProcessTaskRelation::getPostTaskCode).collect(Collectors.toSet());
         List<TaskDefinition> taskDefinitions = taskDefinitionMapper.queryByCodeList(relationCodes);
 
         // find out the process definition code
         Set<Long> processDefinitionCodeSet = new HashSet<>();
         taskDefinitions.stream()
-            .filter(task -> TaskConstants.TASK_TYPE_SUB_PROCESS.equalsIgnoreCase(task.getTaskType())).forEach(
-                taskDefinition -> processDefinitionCodeSet.add(Long.valueOf(
-                    JSONUtils.getNodeString(taskDefinition.getTaskParams(), Constants.CMD_PARAM_SUB_PROCESS_DEFINE_CODE))));
+                .filter(task -> TaskConstants.TASK_TYPE_SUB_PROCESS.equalsIgnoreCase(task.getTaskType())).forEach(
+                        taskDefinition -> processDefinitionCodeSet.add(Long.valueOf(
+                                JSONUtils.getNodeString(taskDefinition.getTaskParams(),
+                                        Constants.CMD_PARAM_SUB_PROCESS_DEFINE_CODE))));
         if (processDefinitionCodeSet.isEmpty()) {
             return true;
         }
@@ -324,11 +317,11 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
         // check sub releaseState
         List<ProcessDefinition> processDefinitions = processDefinitionMapper.queryByCodes(processDefinitionCodeSet);
         return processDefinitions.stream()
-            .filter(definition -> definition.getReleaseState().equals(ReleaseState.OFFLINE)).collect(Collectors.toSet())
-            .isEmpty();
+                .filter(definition -> definition.getReleaseState().equals(ReleaseState.OFFLINE))
+                .collect(Collectors.toSet())
+                .isEmpty();
     }
 
-
     /**
      * do action to process instance:pause, stop, repeat, recover from pause, recover from stop
      *
@@ -339,12 +332,13 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
      * @return execute result code
      */
     @Override
-    public Map<String, Object> execute(User loginUser, long projectCode, Integer processInstanceId, ExecuteType executeType) {
+    public Map<String, Object> execute(User loginUser, long projectCode, Integer processInstanceId,
+                                       ExecuteType executeType) {
         Project project = projectMapper.queryByCode(projectCode);
-        //check user access for project
+        // check user access for project
 
         Map<String, Object> result = projectService.checkProjectAndAuth(loginUser, project, projectCode,
-            ApiFuncIdentificationConstant.map.get(executeType));
+                ApiFuncIdentificationConstant.map.get(executeType));
         if (result.get(Constants.STATUS) != Status.SUCCESS) {
             return result;
         }
@@ -361,12 +355,13 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
         }
 
         ProcessDefinition processDefinition =
-            processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
-                processInstance.getProcessDefinitionVersion());
+                processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
+                        processInstance.getProcessDefinitionVersion());
         if (executeType != ExecuteType.STOP && executeType != ExecuteType.PAUSE) {
             result =
-                checkProcessDefinitionValid(projectCode, processDefinition, processInstance.getProcessDefinitionCode(),
-                    processInstance.getProcessDefinitionVersion());
+                    checkProcessDefinitionValid(projectCode, processDefinition,
+                            processInstance.getProcessDefinitionCode(),
+                            processInstance.getProcessDefinitionVersion());
             if (result.get(Constants.STATUS) != Status.SUCCESS) {
                 return result;
             }
@@ -378,14 +373,14 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
         }
         if (!checkTenantSuitable(processDefinition)) {
             logger.error("there is not any valid tenant for the process definition: id:{},name:{}, ",
-                processDefinition.getId(), processDefinition.getName());
+                    processDefinition.getId(), processDefinition.getName());
             putMsg(result, Status.TENANT_NOT_SUITABLE);
         }
 
-        //get the startParams user specified at the first starting while repeat running is needed
+        // get the startParams user specified at the first starting while repeat running is needed
         Map<String, Object> commandMap =
-            JSONUtils.parseObject(processInstance.getCommandParam(), new TypeReference<Map<String, Object>>() {
-            });
+                JSONUtils.parseObject(processInstance.getCommandParam(), new TypeReference<Map<String, Object>>() {
+                });
         String startParams = null;
         if (MapUtils.isNotEmpty(commandMap) && executeType == ExecuteType.REPEAT_RUNNING) {
             Object startParamsJson = commandMap.get(Constants.CMD_PARAM_START_PARAMS);
@@ -397,30 +392,33 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
         switch (executeType) {
             case REPEAT_RUNNING:
                 result = insertCommand(loginUser, processInstanceId, processDefinition.getCode(),
-                    processDefinition.getVersion(), CommandType.REPEAT_RUNNING, startParams);
+                        processDefinition.getVersion(), CommandType.REPEAT_RUNNING, startParams);
                 break;
             case RECOVER_SUSPENDED_PROCESS:
                 result = insertCommand(loginUser, processInstanceId, processDefinition.getCode(),
-                    processDefinition.getVersion(), CommandType.RECOVER_SUSPENDED_PROCESS, startParams);
+                        processDefinition.getVersion(), CommandType.RECOVER_SUSPENDED_PROCESS, startParams);
                 break;
             case START_FAILURE_TASK_PROCESS:
                 result = insertCommand(loginUser, processInstanceId, processDefinition.getCode(),
-                    processDefinition.getVersion(), CommandType.START_FAILURE_TASK_PROCESS, startParams);
+                        processDefinition.getVersion(), CommandType.START_FAILURE_TASK_PROCESS, startParams);
                 break;
             case STOP:
-                if (processInstance.getState() == ExecutionStatus.READY_STOP) {
+                if (processInstance.getState() == WorkflowExecutionStatus.READY_STOP) {
                     putMsg(result, Status.PROCESS_INSTANCE_ALREADY_CHANGED, processInstance.getName(),
-                        processInstance.getState());
+                            processInstance.getState());
                 } else {
                     result =
-                        updateProcessInstancePrepare(processInstance, CommandType.STOP, ExecutionStatus.READY_STOP);
+                            updateProcessInstancePrepare(processInstance, CommandType.STOP,
+                                    WorkflowExecutionStatus.READY_STOP);
                 }
                 break;
             case PAUSE:
-                if (processInstance.getState() == ExecutionStatus.READY_PAUSE) {
-                    putMsg(result, Status.PROCESS_INSTANCE_ALREADY_CHANGED, processInstance.getName(), processInstance.getState());
+                if (processInstance.getState() == WorkflowExecutionStatus.READY_PAUSE) {
+                    putMsg(result, Status.PROCESS_INSTANCE_ALREADY_CHANGED, processInstance.getName(),
+                            processInstance.getState());
                 } else {
-                    result = updateProcessInstancePrepare(processInstance, CommandType.PAUSE, ExecutionStatus.READY_PAUSE);
+                    result = updateProcessInstancePrepare(processInstance, CommandType.PAUSE,
+                            WorkflowExecutionStatus.READY_PAUSE);
                 }
                 break;
             default:
@@ -458,7 +456,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
      */
     private boolean checkTenantSuitable(ProcessDefinition processDefinition) {
         Tenant tenant =
-            processService.getTenantForProcess(processDefinition.getTenantId(), processDefinition.getUserId());
+                processService.getTenantForProcess(processDefinition.getTenantId(), processDefinition.getUserId());
         return tenant != null;
     }
 
@@ -472,27 +470,27 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
     private Map<String, Object> checkExecuteType(ProcessInstance processInstance, ExecuteType executeType) {
 
         Map<String, Object> result = new HashMap<>();
-        ExecutionStatus executionStatus = processInstance.getState();
+        WorkflowExecutionStatus executionStatus = processInstance.getState();
         boolean checkResult = false;
         switch (executeType) {
             case PAUSE:
             case STOP:
-                if (executionStatus.typeIsRunning()) {
+                if (executionStatus.isRunning()) {
                     checkResult = true;
                 }
                 break;
             case REPEAT_RUNNING:
-                if (executionStatus.typeIsFinished()) {
+                if (executionStatus.isFinished()) {
                     checkResult = true;
                 }
                 break;
             case START_FAILURE_TASK_PROCESS:
-                if (executionStatus.typeIsFailure()) {
+                if (executionStatus.isFailure()) {
                     checkResult = true;
                 }
                 break;
             case RECOVER_SUSPENDED_PROCESS:
-                if (executionStatus.typeIsPause() || executionStatus.typeIsCancel()) {
+                if (executionStatus.isPause() || executionStatus.isStop()) {
                     checkResult = true;
                 }
                 break;
@@ -501,7 +499,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
         }
         if (!checkResult) {
             putMsg(result, Status.PROCESS_INSTANCE_STATE_OPERATION_ERROR, processInstance.getName(),
-                executionStatus.toString(), executeType.toString());
+                    executionStatus.toString(), executeType.toString());
         } else {
             putMsg(result, Status.SUCCESS);
         }
@@ -517,7 +515,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
      * @return update result
      */
     private Map<String, Object> updateProcessInstancePrepare(ProcessInstance processInstance, CommandType commandType,
-                                                             ExecutionStatus executionStatus) {
+                                                             WorkflowExecutionStatus executionStatus) {
         Map<String, Object> result = new HashMap<>();
 
         processInstance.setCommandType(commandType);
@@ -527,12 +525,12 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
 
         // determine whether the process is normal
         if (update > 0) {
-            // directly send the process instance state change event to target master, not guarantee the event send success
-            StateEventChangeCommand stateEventChangeCommand = new StateEventChangeCommand(
-                    processInstance.getId(), 0, processInstance.getState(), processInstance.getId(), 0
-            );
+            // directly send the process instance state change event to target master, not guarantee the event send
+            // success
+            WorkflowStateEventChangeCommand workflowStateEventChangeCommand = new WorkflowStateEventChangeCommand(
+                    processInstance.getId(), 0, processInstance.getState(), processInstance.getId(), 0);
             Host host = new Host(processInstance.getHost());
-            stateEventCallbackService.sendResult(host, stateEventChangeCommand.convert2Command());
+            stateEventCallbackService.sendResult(host, workflowStateEventChangeCommand.convert2Command());
             putMsg(result, Status.SUCCESS);
         } else {
             putMsg(result, Status.EXECUTE_PROCESS_INSTANCE_ERROR);
@@ -556,7 +554,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
         taskGroupQueue.setForceStart(Flag.YES.getCode());
         processService.updateTaskGroupQueue(taskGroupQueue);
         processService.sendStartTask2Master(processInstance, taskGroupQueue.getTaskId(),
-            org.apache.dolphinscheduler.remote.command.CommandType.TASK_FORCE_STATE_EVENT_REQUEST);
+                org.apache.dolphinscheduler.remote.command.CommandType.TASK_FORCE_STATE_EVENT_REQUEST);
         putMsg(result, Status.SUCCESS);
         return result;
     }
@@ -575,7 +573,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
                                               int processVersion, CommandType commandType, String startParams) {
         Map<String, Object> result = new HashMap<>();
 
-        //To add startParams only when repeat running is needed
+        // To add startParams only when repeat running is needed
         Map<String, Object> cmdParam = new HashMap<>();
         cmdParam.put(CMD_PARAM_RECOVER_PROCESS_ID_STRING, instanceId);
         if (!StringUtils.isEmpty(startParams)) {
@@ -636,7 +634,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
                     if (processDefinitionTmp.getReleaseState() != ReleaseState.ONLINE) {
                         putMsg(result, Status.PROCESS_DEFINE_NOT_RELEASE, processDefinitionTmp.getName());
                         logger.info("not release process definition id: {} , name : {}", processDefinitionTmp.getId(),
-                            processDefinitionTmp.getName());
+                                processDefinitionTmp.getName());
                         return result;
                     }
                 }
@@ -722,9 +720,10 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
             }
             try {
                 return createComplementCommandList(schedule, runMode, command, expectedParallelismNumber,
-                    complementDependentMode);
+                        complementDependentMode);
             } catch (CronParseException cronParseException) {
-                // We catch the exception here just to make compiler happy, since we have already validated the schedule cron expression before
+                // We catch the exception here just to make compiler happy, since we have already validated the schedule
+                // cron expression before
                 return 0;
             }
         } else {
@@ -743,8 +742,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
      */
     protected int createComplementCommandList(String scheduleTimeParam, RunMode runMode, Command command,
                                               Integer expectedParallelismNumber,
-                                              ComplementDependentMode complementDependentMode)
-        throws CronParseException {
+                                              ComplementDependentMode complementDependentMode) throws CronParseException {
         int createCount = 0;
         String startDate = null;
         String endDate = null;
@@ -758,7 +756,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
             dateList = removeDuplicates(dateList);
         }
         if (scheduleParam.containsKey(CMDPARAM_COMPLEMENT_DATA_START_DATE) && scheduleParam.containsKey(
-            CMDPARAM_COMPLEMENT_DATA_END_DATE)) {
+                CMDPARAM_COMPLEMENT_DATA_END_DATE)) {
             startDate = scheduleParam.get(CMDPARAM_COMPLEMENT_DATA_START_DATE);
             endDate = scheduleParam.get(CMDPARAM_COMPLEMENT_DATA_END_DATE);
         }
@@ -777,11 +775,11 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
 
                     // dependent process definition
                     List<Schedule> schedules = processService.queryReleaseSchedulerListByProcessDefinitionCode(
-                        command.getProcessDefinitionCode());
+                            command.getProcessDefinitionCode());
 
                     if (schedules.isEmpty() || complementDependentMode == ComplementDependentMode.OFF_MODE) {
                         logger.info("process code: {} complement dependent in off mode or schedule's size is 0, skip "
-                            + "dependent complement data", command.getProcessDefinitionCode());
+                                + "dependent complement data", command.getProcessDefinitionCode());
                     } else {
                         dependentProcessDefinitionCreateCount += createComplementDependentCommand(schedules, command);
                     }
@@ -791,10 +789,10 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
             case RUN_MODE_PARALLEL: {
                 if (startDate != null && endDate != null) {
                     List<Schedule> schedules = processService.queryReleaseSchedulerListByProcessDefinitionCode(
-                        command.getProcessDefinitionCode());
+                            command.getProcessDefinitionCode());
                     List<ZonedDateTime> listDate = new ArrayList<>(
-                        CronUtils.getSelfFireDateList(DateUtils.stringToZoneDateTime(startDate),
-                            DateUtils.stringToZoneDateTime(endDate), schedules));
+                            CronUtils.getSelfFireDateList(DateUtils.stringToZoneDateTime(startDate),
+                                    DateUtils.stringToZoneDateTime(endDate), schedules));
                     int listDateSize = listDate.size();
                     createCount = listDate.size();
                     if (!CollectionUtils.isEmpty(listDate)) {
@@ -821,18 +819,21 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
                                 endDateIndex += singleCommandItems;
                             }
 
-                            cmdParam.put(CMDPARAM_COMPLEMENT_DATA_START_DATE, DateUtils.dateToString(listDate.get(startDateIndex)));
-                            cmdParam.put(CMDPARAM_COMPLEMENT_DATA_END_DATE, DateUtils.dateToString(listDate.get(endDateIndex)));
+                            cmdParam.put(CMDPARAM_COMPLEMENT_DATA_START_DATE,
+                                    DateUtils.dateToString(listDate.get(startDateIndex)));
+                            cmdParam.put(CMDPARAM_COMPLEMENT_DATA_END_DATE,
+                                    DateUtils.dateToString(listDate.get(endDateIndex)));
                             command.setCommandParam(JSONUtils.toJsonString(cmdParam));
                             processService.createCommand(command);
 
                             if (schedules.isEmpty() || complementDependentMode == ComplementDependentMode.OFF_MODE) {
                                 logger.info(
-                                    "process code: {} complement dependent in off mode or schedule's size is 0, skip "
-                                        + "dependent complement data", command.getProcessDefinitionCode());
+                                        "process code: {} complement dependent in off mode or schedule's size is 0, skip "
+                                                + "dependent complement data",
+                                        command.getProcessDefinitionCode());
                             } else {
                                 dependentProcessDefinitionCreateCount +=
-                                    createComplementDependentCommand(schedules, command);
+                                        createComplementDependentCommand(schedules, command);
                             }
                         }
                     }
@@ -858,7 +859,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
                 break;
         }
         logger.info("create complement command count: {}, create dependent complement command count: {}", createCount,
-            dependentProcessDefinitionCreateCount);
+                dependentProcessDefinitionCreateCount);
         return createCount;
     }
 
@@ -877,8 +878,8 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
         }
 
         List<DependentProcessDefinition> dependentProcessDefinitionList =
-            getComplementDependentDefinitionList(dependentCommand.getProcessDefinitionCode(),
-                CronUtils.getMaxCycle(schedules.get(0).getCrontab()), dependentCommand.getWorkerGroup());
+                getComplementDependentDefinitionList(dependentCommand.getProcessDefinitionCode(),
+                        CronUtils.getMaxCycle(schedules.get(0).getCrontab()), dependentCommand.getWorkerGroup());
 
         dependentCommand.setTaskDependType(TaskDependType.TASK_POST);
         for (DependentProcessDefinition dependentProcessDefinition : dependentProcessDefinitionList) {
@@ -900,10 +901,10 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
                                                                                   CycleEnum processDefinitionCycle,
                                                                                   String workerGroup) {
         List<DependentProcessDefinition> dependentProcessDefinitionList =
-            processService.queryDependentProcessDefinitionByProcessDefinitionCode(processDefinitionCode);
+                processService.queryDependentProcessDefinitionByProcessDefinitionCode(processDefinitionCode);
 
         return checkDependentProcessDefinitionValid(dependentProcessDefinitionList, processDefinitionCycle,
-            workerGroup);
+                workerGroup);
     }
 
     /**
@@ -912,21 +913,22 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
      * worker group
      */
     private List<DependentProcessDefinition> checkDependentProcessDefinitionValid(
-        List<DependentProcessDefinition> dependentProcessDefinitionList, CycleEnum processDefinitionCycle,
-        String workerGroup) {
+                                                                                  List<DependentProcessDefinition> dependentProcessDefinitionList,
+                                                                                  CycleEnum processDefinitionCycle,
+                                                                                  String workerGroup) {
         List<DependentProcessDefinition> validDependentProcessDefinitionList = new ArrayList<>();
 
         List<Long> processDefinitionCodeList =
-            dependentProcessDefinitionList.stream().map(DependentProcessDefinition::getProcessDefinitionCode)
-                .collect(Collectors.toList());
+                dependentProcessDefinitionList.stream().map(DependentProcessDefinition::getProcessDefinitionCode)
+                        .collect(Collectors.toList());
 
         Map<Long, String> processDefinitionWorkerGroupMap =
-            processService.queryWorkerGroupByProcessDefinitionCodes(processDefinitionCodeList);
+                processService.queryWorkerGroupByProcessDefinitionCodes(processDefinitionCodeList);
 
         for (DependentProcessDefinition dependentProcessDefinition : dependentProcessDefinitionList) {
             if (dependentProcessDefinition.getDependentCycle() == processDefinitionCycle) {
-                if (processDefinitionWorkerGroupMap.get(dependentProcessDefinition.getProcessDefinitionCode())
-                    == null) {
+                if (processDefinitionWorkerGroupMap
+                        .get(dependentProcessDefinition.getProcessDefinitionCode()) == null) {
                     dependentProcessDefinition.setWorkerGroup(workerGroup);
                 }
 
@@ -981,7 +983,8 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
      */
     private String removeDuplicates(String scheduleTimeList) {
         if (StringUtils.isNotEmpty(scheduleTimeList)) {
-            Set<String> dateSet  = Arrays.stream(scheduleTimeList.split(COMMA)).map(String::trim).collect(Collectors.toSet());
+            Set<String> dateSet =
+                    Arrays.stream(scheduleTimeList.split(COMMA)).map(String::trim).collect(Collectors.toSet());
             return String.join(COMMA, dateSet);
         }
         return null;
@@ -1001,11 +1004,13 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
         Host host = new Host(processInstance.getHost());
         WorkflowExecutingDataRequestCommand requestCommand = new WorkflowExecutingDataRequestCommand();
         requestCommand.setProcessInstanceId(processInstanceId);
-        org.apache.dolphinscheduler.remote.command.Command command = stateEventCallbackService.sendSync(host, requestCommand.convert2Command());
+        org.apache.dolphinscheduler.remote.command.Command command =
+                stateEventCallbackService.sendSync(host, requestCommand.convert2Command());
         if (command == null) {
             return null;
         }
-        WorkflowExecutingDataResponseCommand responseCommand = JSONUtils.parseObject(command.getBody(), WorkflowExecutingDataResponseCommand.class);
+        WorkflowExecutingDataResponseCommand responseCommand =
+                JSONUtils.parseObject(command.getBody(), WorkflowExecutingDataResponseCommand.class);
         return responseCommand.getWorkflowExecuteDto();
     }
 }
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProcessInstanceServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProcessInstanceServiceImpl.java
index 99427b954a..833a734377 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProcessInstanceServiceImpl.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProcessInstanceServiceImpl.java
@@ -17,31 +17,20 @@
 
 package org.apache.dolphinscheduler.api.service.impl;
 
-import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.INSTANCE_DELETE;
-import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.INSTANCE_UPDATE;
-import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_INSTANCE;
-import static org.apache.dolphinscheduler.common.Constants.DATA_LIST;
-import static org.apache.dolphinscheduler.common.Constants.DEPENDENT_SPLIT;
-import static org.apache.dolphinscheduler.common.Constants.GLOBAL_PARAMS;
-import static org.apache.dolphinscheduler.common.Constants.LOCAL_PARAMS;
-import static org.apache.dolphinscheduler.common.Constants.PROCESS_INSTANCE_STATE;
-import static org.apache.dolphinscheduler.common.Constants.TASK_LIST;
-import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_DEPENDENT;
-
+import com.baomidou.mybatisplus.core.metadata.IPage;
+import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.dolphinscheduler.api.dto.gantt.GanttDto;
 import org.apache.dolphinscheduler.api.dto.gantt.Task;
 import org.apache.dolphinscheduler.api.enums.Status;
 import org.apache.dolphinscheduler.api.exceptions.ServiceException;
-import org.apache.dolphinscheduler.api.service.ExecutorService;
-import org.apache.dolphinscheduler.api.service.LoggerService;
-import org.apache.dolphinscheduler.api.service.ProcessDefinitionService;
-import org.apache.dolphinscheduler.api.service.ProcessInstanceService;
-import org.apache.dolphinscheduler.api.service.ProjectService;
-import org.apache.dolphinscheduler.api.service.UsersService;
+import org.apache.dolphinscheduler.api.service.*;
 import org.apache.dolphinscheduler.api.utils.PageInfo;
 import org.apache.dolphinscheduler.api.utils.Result;
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.Flag;
+import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
 import org.apache.dolphinscheduler.common.graph.DAG;
 import org.apache.dolphinscheduler.common.model.TaskNode;
 import org.apache.dolphinscheduler.common.model.TaskNodeRelation;
@@ -49,57 +38,30 @@ import org.apache.dolphinscheduler.common.utils.DateUtils;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
 import org.apache.dolphinscheduler.common.utils.ParameterUtils;
 import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils;
-import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
-import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
-import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelationLog;
-import org.apache.dolphinscheduler.dao.entity.Project;
-import org.apache.dolphinscheduler.dao.entity.ResponseTaskLog;
-import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
-import org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog;
-import org.apache.dolphinscheduler.dao.entity.TaskInstance;
-import org.apache.dolphinscheduler.dao.entity.Tenant;
-import org.apache.dolphinscheduler.dao.entity.User;
-import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionLogMapper;
-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.dao.mapper.ScheduleMapper;
-import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionLogMapper;
-import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
-import org.apache.dolphinscheduler.dao.mapper.TaskInstanceMapper;
-import org.apache.dolphinscheduler.dao.mapper.TenantMapper;
+import org.apache.dolphinscheduler.dao.entity.*;
+import org.apache.dolphinscheduler.dao.mapper.*;
 import org.apache.dolphinscheduler.plugin.task.api.enums.DependResult;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.plugin.task.api.model.Property;
 import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
 import org.apache.dolphinscheduler.service.expand.CuringParamsService;
 import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.apache.dolphinscheduler.service.task.TaskPluginManager;
-
-import org.apache.commons.collections.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Service;
+import org.springframework.transaction.annotation.Transactional;
 
 import java.io.BufferedReader;
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
+import java.util.*;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Service;
-import org.springframework.transaction.annotation.Transactional;
-
-import com.baomidou.mybatisplus.core.metadata.IPage;
-import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
+import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.*;
+import static org.apache.dolphinscheduler.common.Constants.*;
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_DEPENDENT;
 
 /**
  * process instance service impl
@@ -165,10 +127,12 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
      * return top n SUCCESS process instance order by running time which started between startTime and endTime
      */
     @Override
-    public Map<String, Object> queryTopNLongestRunningProcessInstance(User loginUser, long projectCode, int size, String startTime, String endTime) {
+    public Map<String, Object> queryTopNLongestRunningProcessInstance(User loginUser, long projectCode, int size,
+                                                                      String startTime, String endTime) {
         Project project = projectMapper.queryByCode(projectCode);
-        //check user access for project
-        Map<String, Object> result = projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE);
+        // check user access for project
+        Map<String, Object> result =
+                projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE);
         if (result.get(Constants.STATUS) != Status.SUCCESS) {
             return result;
         }
@@ -196,7 +160,8 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
             return result;
         }
 
-        List<ProcessInstance> processInstances = processInstanceMapper.queryTopNProcessInstance(size, start, end, ExecutionStatus.SUCCESS, projectCode);
+        List<ProcessInstance> processInstances = processInstanceMapper.queryTopNProcessInstance(size, start, end,
+                WorkflowExecutionStatus.SUCCESS, projectCode);
         result.put(DATA_LIST, processInstances);
         putMsg(result, Status.SUCCESS);
         return result;
@@ -213,15 +178,17 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
     @Override
     public Map<String, Object> queryProcessInstanceById(User loginUser, long projectCode, Integer processId) {
         Project project = projectMapper.queryByCode(projectCode);
-        //check user access for project
-        Map<String, Object> result = projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE);
+        // check user access for project
+        Map<String, Object> result =
+                projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE);
         if (result.get(Constants.STATUS) != Status.SUCCESS) {
             return result;
         }
         ProcessInstance processInstance = processService.findProcessInstanceDetailById(processId);
 
-        ProcessDefinition processDefinition = processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
-            processInstance.getProcessDefinitionVersion());
+        ProcessDefinition processDefinition =
+                processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
+                        processInstance.getProcessDefinitionVersion());
 
         if (processDefinition == null || projectCode != processDefinition.getProjectCode()) {
             putMsg(result, Status.PROCESS_DEFINE_NOT_EXIST, processId);
@@ -252,29 +219,32 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
      * @return process instance list
      */
     @Override
-    public Result queryProcessInstanceList(User loginUser, long projectCode, long processDefineCode, String startDate, String endDate, String searchVal, String executorName,
-                                           ExecutionStatus stateType, String host, String otherParamsJson, Integer pageNo, Integer pageSize) {
+    public Result queryProcessInstanceList(User loginUser, long projectCode, long processDefineCode, String startDate,
+                                           String endDate, String searchVal, String executorName,
+                                           WorkflowExecutionStatus stateType, String host, String otherParamsJson,
+                                           Integer pageNo, Integer pageSize) {
 
         Result result = new Result();
         Project project = projectMapper.queryByCode(projectCode);
-        //check user access for project
-        Map<String, Object> checkResult = projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE);
+        // check user access for project
+        Map<String, Object> checkResult =
+                projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE);
         Status resultEnum = (Status) checkResult.get(Constants.STATUS);
         if (resultEnum != Status.SUCCESS) {
-            putMsg(result,resultEnum);
+            putMsg(result, resultEnum);
             return result;
         }
 
         int[] statusArray = null;
         // filter by state
         if (stateType != null) {
-            statusArray = new int[]{stateType.ordinal()};
+            statusArray = new int[]{stateType.getCode()};
         }
 
         Map<String, Object> checkAndParseDateResult = checkAndParseDateParameters(startDate, endDate);
         resultEnum = (Status) checkAndParseDateResult.get(Constants.STATUS);
         if (resultEnum != Status.SUCCESS) {
-            putMsg(result,resultEnum);
+            putMsg(result, resultEnum);
             return result;
         }
         Date start = (Date) checkAndParseDateResult.get(Constants.START_TIME);
@@ -285,7 +255,7 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
         int executorId = usersService.getUserIdByName(executorName);
 
         IPage<ProcessInstance> processInstanceList = processInstanceMapper.queryProcessInstanceListPaging(page,
-            project.getCode(), processDefineCode, searchVal, executorId, statusArray, host, start, end);
+                project.getCode(), processDefineCode, searchVal, executorId, statusArray, host, start, end);
 
         List<ProcessInstance> processInstances = processInstanceList.getRecords();
         List<Integer> userIds = Collections.emptyList();
@@ -299,7 +269,8 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
         }
 
         for (ProcessInstance processInstance : processInstances) {
-            processInstance.setDuration(DateUtils.format2Duration(processInstance.getStartTime(), processInstance.getEndTime()));
+            processInstance.setDuration(
+                    DateUtils.format2Duration(processInstance.getStartTime(), processInstance.getEndTime()));
             User executor = idToUserMap.get(processInstance.getExecutorId());
             if (null != executor) {
                 processInstance.setExecutorName(executor.getUserName());
@@ -323,15 +294,18 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
      * @throws IOException io exception
      */
     @Override
-    public Map<String, Object> queryTaskListByProcessId(User loginUser, long projectCode, Integer processId) throws IOException {
+    public Map<String, Object> queryTaskListByProcessId(User loginUser, long projectCode,
+                                                        Integer processId) throws IOException {
         Project project = projectMapper.queryByCode(projectCode);
-        //check user access for project
-        Map<String, Object> result = projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE);
+        // check user access for project
+        Map<String, Object> result =
+                projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE);
         if (result.get(Constants.STATUS) != Status.SUCCESS) {
             return result;
         }
         ProcessInstance processInstance = processService.findProcessInstanceDetailById(processId);
-        ProcessDefinition processDefinition = processDefineMapper.queryByCode(processInstance.getProcessDefinitionCode());
+        ProcessDefinition processDefinition =
+                processDefineMapper.queryByCode(processInstance.getProcessDefinitionCode());
         if (processDefinition != null && projectCode != processDefinition.getProjectCode()) {
             putMsg(result, Status.PROCESS_INSTANCE_NOT_EXIST, processId);
             return result;
@@ -354,7 +328,7 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
         for (TaskInstance taskInstance : taskInstanceList) {
             if (TASK_TYPE_DEPENDENT.equalsIgnoreCase(taskInstance.getTaskType())) {
                 Result<ResponseTaskLog> logResult = loggerService.queryLog(
-                    taskInstance.getId(), Constants.LOG_QUERY_SKIP_LINE_NUMBER, Constants.LOG_QUERY_LIMIT);
+                        taskInstance.getId(), Constants.LOG_QUERY_SKIP_LINE_NUMBER, Constants.LOG_QUERY_LIMIT);
                 if (logResult.getCode() == Status.SUCCESS.ordinal()) {
                     String log = logResult.getData().getMessage();
                     Map<String, DependResult> resultMap = parseLogForDependentResult(log);
@@ -372,7 +346,7 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
         }
 
         BufferedReader br = new BufferedReader(new InputStreamReader(new ByteArrayInputStream(log.getBytes(
-            StandardCharsets.UTF_8)), StandardCharsets.UTF_8));
+                StandardCharsets.UTF_8)), StandardCharsets.UTF_8));
         String line;
         while ((line = br.readLine()) != null) {
             if (line.contains(DEPENDENT_SPLIT)) {
@@ -404,8 +378,9 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
     @Override
     public Map<String, Object> querySubProcessInstanceByTaskId(User loginUser, long projectCode, Integer taskId) {
         Project project = projectMapper.queryByCode(projectCode);
-        //check user access for project
-        Map<String, Object> result = projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE);
+        // check user access for project
+        Map<String, Object> result =
+                projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE);
         if (result.get(Constants.STATUS) != Status.SUCCESS) {
             return result;
         }
@@ -428,7 +403,7 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
         }
 
         ProcessInstance subWorkflowInstance = processService.findSubProcessInstance(
-            taskInstance.getProcessInstanceId(), taskInstance.getId());
+                taskInstance.getProcessInstanceId(), taskInstance.getId());
         if (subWorkflowInstance == null) {
             putMsg(result, Status.SUB_PROCESS_INSTANCE_NOT_EXIST, taskId);
             return result;
@@ -458,31 +433,35 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
      */
     @Transactional
     @Override
-    public Map<String, Object> updateProcessInstance(User loginUser, long projectCode, Integer processInstanceId, String taskRelationJson,
-                                                     String taskDefinitionJson, String scheduleTime, Boolean syncDefine, String globalParams,
+    public Map<String, Object> updateProcessInstance(User loginUser, long projectCode, Integer processInstanceId,
+                                                     String taskRelationJson,
+                                                     String taskDefinitionJson, String scheduleTime, Boolean syncDefine,
+                                                     String globalParams,
                                                      String locations, int timeout, String tenantCode) {
         Project project = projectMapper.queryByCode(projectCode);
-        //check user access for project
-        Map<String, Object> result = projectService.checkProjectAndAuth(loginUser, project, projectCode,INSTANCE_UPDATE);
+        // check user access for project
+        Map<String, Object> result =
+                projectService.checkProjectAndAuth(loginUser, project, projectCode, INSTANCE_UPDATE);
         if (result.get(Constants.STATUS) != Status.SUCCESS) {
             return result;
         }
-        //check process instance exists
+        // check process instance exists
         ProcessInstance processInstance = processService.findProcessInstanceDetailById(processInstanceId);
         if (processInstance == null) {
             putMsg(result, Status.PROCESS_INSTANCE_NOT_EXIST, processInstanceId);
             return result;
         }
-        //check process instance exists in project
-        ProcessDefinition processDefinition0 = processDefineMapper.queryByCode(processInstance.getProcessDefinitionCode());
+        // check process instance exists in project
+        ProcessDefinition processDefinition0 =
+                processDefineMapper.queryByCode(processInstance.getProcessDefinitionCode());
         if (processDefinition0 != null && projectCode != processDefinition0.getProjectCode()) {
             putMsg(result, Status.PROCESS_INSTANCE_NOT_EXIST, processInstanceId);
             return result;
         }
-        //check process instance status
-        if (!processInstance.getState().typeIsFinished()) {
+        // check process instance status
+        if (!processInstance.getState().isFinished()) {
             putMsg(result, Status.PROCESS_INSTANCE_STATE_OPERATION_ERROR,
-                processInstance.getName(), processInstance.getState().toString(), "update");
+                    processInstance.getName(), processInstance.getState().toString(), "update");
             return result;
         }
 
@@ -516,9 +495,11 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
             putMsg(result, Status.UPDATE_TASK_DEFINITION_ERROR);
             throw new ServiceException(Status.UPDATE_TASK_DEFINITION_ERROR);
         }
-        ProcessDefinition processDefinition = processDefineMapper.queryByCode(processInstance.getProcessDefinitionCode());
-        List<ProcessTaskRelationLog> taskRelationList = JSONUtils.toList(taskRelationJson, ProcessTaskRelationLog.class);
-        //check workflow json is valid
+        ProcessDefinition processDefinition =
+                processDefineMapper.queryByCode(processInstance.getProcessDefinitionCode());
+        List<ProcessTaskRelationLog> taskRelationList =
+                JSONUtils.toList(taskRelationJson, ProcessTaskRelationLog.class);
+        // check workflow json is valid
         result = processDefinitionService.checkProcessNodeList(taskRelationJson, taskDefinitionLogs);
         if (result.get(Constants.STATUS) != Status.SUCCESS) {
             return result;
@@ -532,7 +513,8 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
             }
             tenantId = tenant.getId();
         }
-        processDefinition.set(projectCode, processDefinition.getName(), processDefinition.getDescription(), globalParams, locations, timeout, tenantId);
+        processDefinition.set(projectCode, processDefinition.getName(), processDefinition.getDescription(),
+                globalParams, locations, timeout, tenantId);
         processDefinition.setUpdateTime(new Date());
         int insertVersion = processService.saveProcessDefine(loginUser, processDefinition, syncDefine, Boolean.FALSE);
         if (insertVersion == 0) {
@@ -540,7 +522,7 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
             throw new ServiceException(Status.UPDATE_PROCESS_DEFINITION_ERROR);
         }
         int insertResult = processService.saveTaskRelation(loginUser, processDefinition.getProjectCode(),
-            processDefinition.getCode(), insertVersion, taskRelationList, taskDefinitionLogs, syncDefine);
+                processDefinition.getCode(), insertVersion, taskRelationList, taskDefinitionLogs, syncDefine);
         if (insertResult == Constants.EXIT_CODE_SUCCESS) {
             putMsg(result, Status.SUCCESS);
             result.put(Constants.DATA_LIST, processDefinition);
@@ -561,15 +543,18 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
     /**
      * update process instance attributes
      */
-    private void setProcessInstance(ProcessInstance processInstance, String tenantCode, String scheduleTime, String globalParams, int timeout, String timezone) {
+    private void setProcessInstance(ProcessInstance processInstance, String tenantCode, String scheduleTime,
+                                    String globalParams, int timeout, String timezone) {
         Date schedule = processInstance.getScheduleTime();
         if (scheduleTime != null) {
             schedule = DateUtils.stringToDate(scheduleTime);
         }
         processInstance.setScheduleTime(schedule);
         List<Property> globalParamList = JSONUtils.toList(globalParams, Property.class);
-        Map<String, String> globalParamMap = globalParamList.stream().collect(Collectors.toMap(Property::getProp, Property::getValue));
-        globalParams = curingGlobalParamsService.curingGlobalParams(processInstance.getId(), globalParamMap, globalParamList, processInstance.getCmdTypeIfComplement(), schedule, timezone);
+        Map<String, String> globalParamMap =
+                globalParamList.stream().collect(Collectors.toMap(Property::getProp, Property::getValue));
+        globalParams = curingGlobalParamsService.curingGlobalParams(processInstance.getId(), globalParamMap,
+                globalParamList, processInstance.getCmdTypeIfComplement(), schedule, timezone);
         processInstance.setTimeout(timeout);
         processInstance.setTenantCode(tenantCode);
         processInstance.setGlobalParams(globalParams);
@@ -586,8 +571,9 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
     @Override
     public Map<String, Object> queryParentInstanceBySubId(User loginUser, long projectCode, Integer subId) {
         Project project = projectMapper.queryByCode(projectCode);
-        //check user access for project
-        Map<String, Object> result = projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE);
+        // check user access for project
+        Map<String, Object> result =
+                projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE);
         if (result.get(Constants.STATUS) != Status.SUCCESS) {
             return result;
         }
@@ -626,8 +612,9 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
     @Transactional
     public Map<String, Object> deleteProcessInstanceById(User loginUser, long projectCode, Integer processInstanceId) {
         Project project = projectMapper.queryByCode(projectCode);
-        //check user access for project
-        Map<String, Object> result = projectService.checkProjectAndAuth(loginUser, project, projectCode,INSTANCE_DELETE);
+        // check user access for project
+        Map<String, Object> result =
+                projectService.checkProjectAndAuth(loginUser, project, projectCode, INSTANCE_DELETE);
         if (result.get(Constants.STATUS) != Status.SUCCESS) {
             return result;
         }
@@ -636,14 +623,15 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
             putMsg(result, Status.PROCESS_INSTANCE_NOT_EXIST, String.valueOf(processInstanceId));
             return result;
         }
-        //check process instance status
-        if (!processInstance.getState().typeIsFinished()) {
+        // check process instance status
+        if (!processInstance.getState().isFinished()) {
             putMsg(result, Status.PROCESS_INSTANCE_STATE_OPERATION_ERROR,
                     processInstance.getName(), processInstance.getState().toString(), "delete");
             return result;
         }
 
-        ProcessDefinition processDefinition = processDefineMapper.queryByCode(processInstance.getProcessDefinitionCode());
+        ProcessDefinition processDefinition =
+                processDefineMapper.queryByCode(processInstance.getProcessDefinitionCode());
         if (processDefinition != null && projectCode != processDefinition.getProjectCode()) {
             putMsg(result, Status.PROCESS_INSTANCE_NOT_EXIST, String.valueOf(processInstanceId));
             return result;
@@ -689,7 +677,8 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
             throw new RuntimeException("workflow instance is null");
         }
 
-        ProcessDefinition processDefinition = processDefineMapper.queryByCode(processInstance.getProcessDefinitionCode());
+        ProcessDefinition processDefinition =
+                processDefineMapper.queryByCode(processInstance.getProcessDefinitionCode());
         if (processDefinition != null && projectCode != processDefinition.getProjectCode()) {
             putMsg(result, Status.PROCESS_INSTANCE_NOT_EXIST, processInstanceId);
             return result;
@@ -701,14 +690,15 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
             timezone = commandParam.get(Constants.SCHEDULE_TIMEZONE);
         }
         Map<String, String> timeParams = BusinessTimeUtils
-            .getBusinessTime(processInstance.getCmdTypeIfComplement(),
-                processInstance.getScheduleTime(), timezone);
+                .getBusinessTime(processInstance.getCmdTypeIfComplement(),
+                        processInstance.getScheduleTime(), timezone);
         String userDefinedParams = processInstance.getGlobalParams();
         // global params
         List<Property> globalParams = new ArrayList<>();
 
         // global param string
-        String globalParamStr = ParameterUtils.convertParameterPlaceholders(JSONUtils.toJsonString(globalParams), timeParams);
+        String globalParamStr =
+                ParameterUtils.convertParameterPlaceholders(JSONUtils.toJsonString(globalParams), timeParams);
         globalParams = JSONUtils.toList(globalParamStr, Property.class);
         for (Property property : globalParams) {
             timeParams.put(property.getProp(), property.getValue());
@@ -733,12 +723,14 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
     /**
      * get local params
      */
-    private Map<String, Map<String, Object>> getLocalParams(ProcessInstance processInstance, Map<String, String> timeParams) {
+    private Map<String, Map<String, Object>> getLocalParams(ProcessInstance processInstance,
+                                                            Map<String, String> timeParams) {
         Map<String, Map<String, Object>> localUserDefParams = new HashMap<>();
-        List<TaskInstance> taskInstanceList = taskInstanceMapper.findValidTaskListByProcessId(processInstance.getId(), Flag.YES);
+        List<TaskInstance> taskInstanceList =
+                taskInstanceMapper.findValidTaskListByProcessId(processInstance.getId(), Flag.YES);
         for (TaskInstance taskInstance : taskInstanceList) {
             TaskDefinitionLog taskDefinitionLog = taskDefinitionLogMapper.queryByDefinitionCodeAndVersion(
-                taskInstance.getTaskCode(), taskInstance.getTaskDefinitionVersion());
+                    taskInstance.getTaskCode(), taskInstance.getTaskDefinitionVersion());
 
             String localParams = JSONUtils.getNodeString(taskDefinitionLog.getTaskParams(), LOCAL_PARAMS);
             if (!StringUtils.isEmpty(localParams)) {
@@ -774,23 +766,23 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
         }
 
         ProcessDefinition processDefinition = processDefinitionLogMapper.queryByDefinitionCodeAndVersion(
-            processInstance.getProcessDefinitionCode(),
-            processInstance.getProcessDefinitionVersion()
-        );
+                processInstance.getProcessDefinitionCode(),
+                processInstance.getProcessDefinitionVersion());
         if (processDefinition == null || projectCode != processDefinition.getProjectCode()) {
             putMsg(result, Status.PROCESS_INSTANCE_NOT_EXIST, processInstanceId);
             return result;
         }
         GanttDto ganttDto = new GanttDto();
         DAG<String, TaskNode, TaskNodeRelation> dag = processService.genDagGraph(processDefinition);
-        //topological sort
+        // topological sort
         List<String> nodeList = dag.topologicalSort();
 
         ganttDto.setTaskNames(nodeList);
 
         List<Task> taskList = new ArrayList<>();
         for (String node : nodeList) {
-            TaskInstance taskInstance = taskInstanceMapper.queryByInstanceIdAndCode(processInstanceId, Long.parseLong(node));
+            TaskInstance taskInstance =
+                    taskInstanceMapper.queryByInstanceIdAndCode(processInstanceId, Long.parseLong(node));
             if (taskInstance == null) {
                 continue;
             }
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java
index 1364915203..73a2e3f96a 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java
@@ -37,7 +37,7 @@ import org.apache.dolphinscheduler.dao.entity.User;
 import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
 import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
 import org.apache.dolphinscheduler.dao.mapper.TaskInstanceMapper;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.service.process.ProcessService;
 
 import java.util.Date;
@@ -99,35 +99,36 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
      */
     @Override
     public Result queryTaskListPaging(User loginUser,
-                                                   long projectCode,
-                                                   Integer processInstanceId,
-                                                   String processInstanceName,
-                                                   String taskName,
-                                                   String executorName,
-                                                   String startDate,
-                                                   String endDate,
-                                                   String searchVal,
-                                                   ExecutionStatus stateType,
-                                                   String host,
-                                                   Integer pageNo,
-                                                   Integer pageSize) {
+                                      long projectCode,
+                                      Integer processInstanceId,
+                                      String processInstanceName,
+                                      String taskName,
+                                      String executorName,
+                                      String startDate,
+                                      String endDate,
+                                      String searchVal,
+                                      TaskExecutionStatus stateType,
+                                      String host,
+                                      Integer pageNo,
+                                      Integer pageSize) {
         Result result = new Result();
         Project project = projectMapper.queryByCode(projectCode);
-        //check user access for project
-        Map<String, Object> checkResult = projectService.checkProjectAndAuth(loginUser, project, projectCode, TASK_INSTANCE);
+        // check user access for project
+        Map<String, Object> checkResult =
+                projectService.checkProjectAndAuth(loginUser, project, projectCode, TASK_INSTANCE);
         Status status = (Status) checkResult.get(Constants.STATUS);
         if (status != Status.SUCCESS) {
-            putMsg(result,status);
+            putMsg(result, status);
             return result;
         }
         int[] statusArray = null;
         if (stateType != null) {
-            statusArray = new int[]{stateType.ordinal()};
+            statusArray = new int[]{stateType.getCode()};
         }
         Map<String, Object> checkAndParseDateResult = checkAndParseDateParameters(startDate, endDate);
         status = (Status) checkAndParseDateResult.get(Constants.STATUS);
         if (status != Status.SUCCESS) {
-            putMsg(result,status);
+            putMsg(result, status);
             return result;
         }
         Date start = (Date) checkAndParseDateResult.get(Constants.START_TIME);
@@ -136,13 +137,14 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
         PageInfo<Map<String, Object>> pageInfo = new PageInfo<>(pageNo, pageSize);
         int executorId = usersService.getUserIdByName(executorName);
         IPage<TaskInstance> taskInstanceIPage = taskInstanceMapper.queryTaskInstanceListPaging(
-            page, project.getCode(), processInstanceId, processInstanceName, searchVal, taskName, executorId, statusArray, host, start, end
-        );
+                page, project.getCode(), processInstanceId, processInstanceName, searchVal, taskName, executorId,
+                statusArray, host, start, end);
         Set<String> exclusionSet = new HashSet<>();
         exclusionSet.add(Constants.CLASS);
         exclusionSet.add("taskJson");
         List<TaskInstance> taskInstanceList = taskInstanceIPage.getRecords();
-        List<Integer> executorIds = taskInstanceList.stream().map(TaskInstance::getExecutorId).distinct().collect(Collectors.toList());
+        List<Integer> executorIds =
+                taskInstanceList.stream().map(TaskInstance::getExecutorId).distinct().collect(Collectors.toList());
         List<User> users = usersService.queryUser(executorIds);
         Map<Integer, User> userMap = users.stream().collect(Collectors.toMap(User::getId, v -> v));
         for (TaskInstance taskInstance : taskInstanceList) {
@@ -171,8 +173,9 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
     @Override
     public Map<String, Object> forceTaskSuccess(User loginUser, long projectCode, Integer taskInstanceId) {
         Project project = projectMapper.queryByCode(projectCode);
-        //check user access for project
-        Map<String, Object> result = projectService.checkProjectAndAuth(loginUser, project, projectCode,FORCED_SUCCESS);
+        // check user access for project
+        Map<String, Object> result =
+                projectService.checkProjectAndAuth(loginUser, project, projectCode, FORCED_SUCCESS);
         if (result.get(Constants.STATUS) != Status.SUCCESS) {
             return result;
         }
@@ -191,13 +194,13 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
         }
 
         // check whether the task instance state type is failure or cancel
-        if (!task.getState().typeIsFailure() && !task.getState().typeIsCancel()) {
+        if (!task.getState().isFailure() && !task.getState().isKill()) {
             putMsg(result, Status.TASK_INSTANCE_STATE_OPERATION_ERROR, taskInstanceId, task.getState().toString());
             return result;
         }
 
         // change the state of the task instance
-        task.setState(ExecutionStatus.FORCED_SUCCESS);
+        task.setState(TaskExecutionStatus.FORCED_SUCCESS);
         int changedNum = taskInstanceMapper.updateById(task);
         if (changedNum > 0) {
             processService.forceProcessInstanceSuccessByTaskInstanceId(taskInstanceId);
diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceControllerTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceControllerTest.java
index 963eda2b69..8f4fd9902d 100644
--- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceControllerTest.java
+++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceControllerTest.java
@@ -24,8 +24,8 @@ import org.apache.dolphinscheduler.api.enums.Status;
 import org.apache.dolphinscheduler.api.service.ProcessInstanceService;
 import org.apache.dolphinscheduler.api.utils.Result;
 import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -56,13 +56,14 @@ public class ProcessInstanceControllerTest extends AbstractControllerTest {
         mockResult.setCode(Status.SUCCESS.getCode());
         PowerMockito.when(processInstanceService
                 .queryProcessInstanceList(Mockito.any(), Mockito.anyLong(), Mockito.anyLong(), Mockito.any(),
-                        Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any()))
+                        Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(),
+                        Mockito.any(), Mockito.any()))
                 .thenReturn(mockResult);
 
         MultiValueMap<String, String> paramsMap = new LinkedMultiValueMap<>();
         paramsMap.add("processDefineCode", "91");
         paramsMap.add("searchVal", "cxc");
-        paramsMap.add("stateType", String.valueOf(ExecutionStatus.SUCCESS));
+        paramsMap.add("stateType", WorkflowExecutionStatus.SUCCESS.name());
         paramsMap.add("host", "192.168.1.13");
         paramsMap.add("startDate", "2019-12-15 00:00:00");
         paramsMap.add("endDate", "2019-12-16 00:00:00");
@@ -84,7 +85,8 @@ public class ProcessInstanceControllerTest extends AbstractControllerTest {
     public void testQueryTaskListByProcessId() throws Exception {
         Map<String, Object> mockResult = new HashMap<>();
         mockResult.put(Constants.STATUS, Status.PROJECT_NOT_FOUND);
-        PowerMockito.when(processInstanceService.queryTaskListByProcessId(Mockito.any(), Mockito.anyLong(), Mockito.any()))
+        PowerMockito
+                .when(processInstanceService.queryTaskListByProcessId(Mockito.any(), Mockito.anyLong(), Mockito.any()))
                 .thenReturn(mockResult);
 
         MvcResult mvcResult = mockMvc.perform(get("/projects/{projectCode}/process-instances/{id}/tasks", "1113", "123")
@@ -103,12 +105,15 @@ public class ProcessInstanceControllerTest extends AbstractControllerTest {
         Map<String, Object> mockResult = new HashMap<>();
         mockResult.put(Constants.STATUS, Status.SUCCESS);
         PowerMockito.when(processInstanceService
-                .updateProcessInstance(Mockito.any(), Mockito.anyLong(), Mockito.anyInt(), Mockito.anyString(), Mockito.anyString(), Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyString(),
-                        Mockito.anyString(), Mockito.anyInt(), Mockito.anyString())).thenReturn(mockResult);
+                .updateProcessInstance(Mockito.any(), Mockito.anyLong(), Mockito.anyInt(), Mockito.anyString(),
+                        Mockito.anyString(), Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyString(),
+                        Mockito.anyString(), Mockito.anyInt(), Mockito.anyString()))
+                .thenReturn(mockResult);
 
-        String json = "[{\"name\":\"\",\"pre_task_code\":0,\"pre_task_version\":0,\"post_task_code\":123456789,\"post_task_version\":1,"
-                + "\"condition_type\":0,\"condition_params\":\"{}\"},{\"name\":\"\",\"pre_task_code\":123456789,\"pre_task_version\":1,"
-                + "\"post_task_code\":123451234,\"post_task_version\":1,\"condition_type\":0,\"condition_params\":\"{}\"}]";
+        String json =
+                "[{\"name\":\"\",\"pre_task_code\":0,\"pre_task_version\":0,\"post_task_code\":123456789,\"post_task_version\":1,"
+                        + "\"condition_type\":0,\"condition_params\":\"{}\"},{\"name\":\"\",\"pre_task_code\":123456789,\"pre_task_version\":1,"
+                        + "\"post_task_code\":123451234,\"post_task_version\":1,\"condition_type\":0,\"condition_params\":\"{}\"}]";
 
         String locations = "{\"tasks-36196\":{\"name\":\"ssh_test1\",\"targetarr\":\"\",\"x\":141,\"y\":70}}";
 
@@ -136,7 +141,9 @@ public class ProcessInstanceControllerTest extends AbstractControllerTest {
     public void testQueryProcessInstanceById() throws Exception {
         Map<String, Object> mockResult = new HashMap<>();
         mockResult.put(Constants.STATUS, Status.SUCCESS);
-        PowerMockito.when(processInstanceService.queryProcessInstanceById(Mockito.any(), Mockito.anyLong(), Mockito.anyInt())).thenReturn(mockResult);
+        PowerMockito.when(
+                processInstanceService.queryProcessInstanceById(Mockito.any(), Mockito.anyLong(), Mockito.anyInt()))
+                .thenReturn(mockResult);
         MvcResult mvcResult = mockMvc.perform(get("/projects/{projectCode}/process-instances/{id}", "1113", "123")
                 .header(SESSION_ID, sessionId))
                 .andExpect(status().isOk())
@@ -152,11 +159,13 @@ public class ProcessInstanceControllerTest extends AbstractControllerTest {
     public void testQuerySubProcessInstanceByTaskId() throws Exception {
         Map<String, Object> mockResult = new HashMap<>();
         mockResult.put(Constants.STATUS, Status.TASK_INSTANCE_NOT_EXISTS);
-        PowerMockito.when(processInstanceService.querySubProcessInstanceByTaskId(Mockito.any(), Mockito.anyLong(), Mockito.anyInt())).thenReturn(mockResult);
+        PowerMockito.when(processInstanceService.querySubProcessInstanceByTaskId(Mockito.any(), Mockito.anyLong(),
+                Mockito.anyInt())).thenReturn(mockResult);
 
-        MvcResult mvcResult = mockMvc.perform(get("/projects/{projectCode}/process-instances/query-sub-by-parent", "1113")
-                .header(SESSION_ID, sessionId)
-                .param("taskId", "1203"))
+        MvcResult mvcResult = mockMvc
+                .perform(get("/projects/{projectCode}/process-instances/query-sub-by-parent", "1113")
+                        .header(SESSION_ID, sessionId)
+                        .param("taskId", "1203"))
                 .andExpect(status().isOk())
                 .andExpect(content().contentType(MediaType.APPLICATION_JSON))
                 .andReturn();
@@ -170,11 +179,14 @@ public class ProcessInstanceControllerTest extends AbstractControllerTest {
     public void testQueryParentInstanceBySubId() throws Exception {
         Map<String, Object> mockResult = new HashMap<>();
         mockResult.put(Constants.STATUS, Status.PROCESS_INSTANCE_NOT_SUB_PROCESS_INSTANCE);
-        PowerMockito.when(processInstanceService.queryParentInstanceBySubId(Mockito.any(), Mockito.anyLong(), Mockito.anyInt())).thenReturn(mockResult);
+        PowerMockito.when(
+                processInstanceService.queryParentInstanceBySubId(Mockito.any(), Mockito.anyLong(), Mockito.anyInt()))
+                .thenReturn(mockResult);
 
-        MvcResult mvcResult = mockMvc.perform(get("/projects/{projectCode}/process-instances/query-parent-by-sub", "1113")
-                .header(SESSION_ID, sessionId)
-                .param("subId", "1204"))
+        MvcResult mvcResult = mockMvc
+                .perform(get("/projects/{projectCode}/process-instances/query-parent-by-sub", "1113")
+                        .header(SESSION_ID, sessionId)
+                        .param("subId", "1204"))
                 .andExpect(status().isOk())
                 .andExpect(content().contentType(MediaType.APPLICATION_JSON))
                 .andReturn();
@@ -188,9 +200,10 @@ public class ProcessInstanceControllerTest extends AbstractControllerTest {
     public void testViewVariables() throws Exception {
         Map<String, Object> mockResult = new HashMap<>();
         mockResult.put(Constants.STATUS, Status.SUCCESS);
-        PowerMockito.when(processInstanceService.viewVariables(1113L,123)).thenReturn(mockResult);
-        MvcResult mvcResult = mockMvc.perform(get("/projects/{projectCode}/process-instances/{id}/view-variables", "1113", "123")
-                .header(SESSION_ID, sessionId))
+        PowerMockito.when(processInstanceService.viewVariables(1113L, 123)).thenReturn(mockResult);
+        MvcResult mvcResult = mockMvc
+                .perform(get("/projects/{projectCode}/process-instances/{id}/view-variables", "1113", "123")
+                        .header(SESSION_ID, sessionId))
                 .andExpect(status().isOk())
                 .andExpect(content().contentType(MediaType.APPLICATION_JSON))
                 .andReturn();
@@ -203,7 +216,9 @@ public class ProcessInstanceControllerTest extends AbstractControllerTest {
     public void testDeleteProcessInstanceById() throws Exception {
         Map<String, Object> mockResult = new HashMap<>();
         mockResult.put(Constants.STATUS, Status.SUCCESS);
-        PowerMockito.when(processInstanceService.deleteProcessInstanceById(Mockito.any(), Mockito.anyLong(), Mockito.anyInt())).thenReturn(mockResult);
+        PowerMockito.when(
+                processInstanceService.deleteProcessInstanceById(Mockito.any(), Mockito.anyLong(), Mockito.anyInt()))
+                .thenReturn(mockResult);
 
         MvcResult mvcResult = mockMvc.perform(delete("/projects/{projectCode}/process-instances/{id}", "1113", "123")
                 .header(SESSION_ID, sessionId))
@@ -221,7 +236,9 @@ public class ProcessInstanceControllerTest extends AbstractControllerTest {
         Map<String, Object> mockResult = new HashMap<>();
         mockResult.put(Constants.STATUS, Status.PROCESS_INSTANCE_NOT_EXIST);
 
-        PowerMockito.when(processInstanceService.deleteProcessInstanceById(Mockito.any(), Mockito.anyLong(), Mockito.anyInt())).thenReturn(mockResult);
+        PowerMockito.when(
+                processInstanceService.deleteProcessInstanceById(Mockito.any(), Mockito.anyLong(), Mockito.anyInt()))
+                .thenReturn(mockResult);
         MvcResult mvcResult = mockMvc.perform(post("/projects/{projectCode}/process-instances/batch-delete", "1113")
                 .header(SESSION_ID, sessionId)
                 .param("processInstanceIds", "1205,1206"))
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 e58234f25f..852520f1f3 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
@@ -34,11 +34,11 @@ import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.dao.entity.User;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.junit.Assert;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -51,6 +51,7 @@ import org.springframework.util.LinkedMultiValueMap;
 import org.springframework.util.MultiValueMap;
 
 public class TaskInstanceControllerTest extends AbstractControllerTest {
+
     @InjectMocks
     private TaskInstanceController taskInstanceController;
 
@@ -68,10 +69,11 @@ public class TaskInstanceControllerTest extends AbstractControllerTest {
         result.setCode(Status.SUCCESS.getCode());
         result.setMsg(Status.SUCCESS.getMsg());
 
-        when(taskInstanceService.queryTaskListPaging(any(), eq(1L),  eq(1), eq(""), eq(""), eq(""),any(), any(),
+        when(taskInstanceService.queryTaskListPaging(any(), eq(1L), eq(1), eq(""), eq(""), eq(""), any(), any(),
                 eq(""), Mockito.any(), eq("192.168.xx.xx"), any(), any())).thenReturn(result);
         Result taskResult = taskInstanceController.queryTaskListPaging(null, 1L, 1, "", "",
-                "", "", ExecutionStatus.SUCCESS,"192.168.xx.xx", "2020-01-01 00:00:00", "2020-01-02 00:00:00",pageNo, pageSize);
+                "", "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", "2020-01-01 00:00:00", "2020-01-02 00:00:00",
+                pageNo, pageSize);
         Assert.assertEquals(Integer.valueOf(Status.SUCCESS.getCode()), taskResult.getCode());
     }
 
diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/DataAnalysisServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/DataAnalysisServiceTest.java
index 9d318b6cef..1ae0455f61 100644
--- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/DataAnalysisServiceTest.java
+++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/DataAnalysisServiceTest.java
@@ -21,7 +21,6 @@ import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationCon
 
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.ArgumentMatchers.anyLong;
 
 import org.apache.dolphinscheduler.api.dto.CommandStateCount;
@@ -45,7 +44,6 @@ 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.dao.mapper.TaskInstanceMapper;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 
 import java.text.MessageFormat;
 import java.util.ArrayList;
@@ -56,6 +54,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -120,7 +119,7 @@ public class DataAnalysisServiceTest {
         project.setName("test");
         resultMap = new HashMap<>();
         Mockito.when(projectMapper.selectById(1)).thenReturn(project);
-        Mockito.when(projectService.hasProjectAndPerm(user, project, resultMap,PROJECT_OVERVIEW)).thenReturn(true);
+        Mockito.when(projectService.hasProjectAndPerm(user, project, resultMap, PROJECT_OVERVIEW)).thenReturn(true);
 
         Mockito.when(projectMapper.queryByCode(1L)).thenReturn(project);
     }
@@ -142,15 +141,15 @@ public class DataAnalysisServiceTest {
         Mockito.when(projectService.checkProjectAndAuth(any(), any(), anyLong(), any())).thenReturn(result);
         Mockito.when(projectMapper.queryByCode(1L)).thenReturn(getProject("test"));
 
-        //SUCCESS
+        // SUCCESS
         Mockito.when(taskInstanceMapper.countTaskInstanceStateByProjectCodes(DateUtils.stringToDate(startDate),
-            DateUtils.stringToDate(endDate),
-            new Long[] {1L})).thenReturn(getTaskInstanceStateCounts());
+                DateUtils.stringToDate(endDate),
+                new Long[]{1L})).thenReturn(getTaskInstanceStateCounts());
         Mockito.when(projectMapper.selectById(Mockito.any())).thenReturn(getProject("test"));
         Mockito.when(projectService.hasProjectAndPerm(Mockito.any(),
-            Mockito.any(),
-            (Map<String, Object>) Mockito.any(),
-            Mockito.any())).thenReturn(true);
+                Mockito.any(),
+                (Map<String, Object>) Mockito.any(),
+                Mockito.any())).thenReturn(true);
 
         result = dataAnalysisServiceImpl.countTaskStateByProject(user, 1, startDate, endDate);
         Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
@@ -164,7 +163,7 @@ public class DataAnalysisServiceTest {
         // checkProject false
         Map<String, Object> failResult = new HashMap<>();
         putMsg(failResult, Status.PROJECT_NOT_FOUND, 1);
-        Mockito.when(projectService.checkProjectAndAuth(any(), any(), anyLong(),any())).thenReturn(failResult);
+        Mockito.when(projectService.checkProjectAndAuth(any(), any(), anyLong(), any())).thenReturn(failResult);
         failResult = dataAnalysisServiceImpl.countTaskStateByProject(user, 1, startDate, endDate);
         Assert.assertEquals(Status.PROJECT_NOT_FOUND, failResult.get(Constants.STATUS));
     }
@@ -173,7 +172,7 @@ public class DataAnalysisServiceTest {
     public void testCountTaskStateByProject_paramValid() {
         Map<String, Object> result = new HashMap<>();
         putMsg(result, Status.SUCCESS, null);
-        Mockito.when(projectService.checkProjectAndAuth(any(), any(), anyLong(),any())).thenReturn(result);
+        Mockito.when(projectService.checkProjectAndAuth(any(), any(), anyLong(), any())).thenReturn(result);
         Mockito.when(projectMapper.queryByCode(1L)).thenReturn(getProject("test"));
 
         // when date in illegal format then return error message
@@ -199,19 +198,20 @@ public class DataAnalysisServiceTest {
     public void testCountTaskStateByProject_allCountZero() {
         Map<String, Object> result = new HashMap<>();
         putMsg(result, Status.SUCCESS, null);
-        Mockito.when(projectService.checkProjectAndAuth(any(), any(), anyLong(),any())).thenReturn(result);
+        Mockito.when(projectService.checkProjectAndAuth(any(), any(), anyLong(), any())).thenReturn(result);
         Mockito.when(projectMapper.queryByCode(1L)).thenReturn(getProject("test"));
 
         // when general user doesn't have any task then return all count are 0
         user.setUserType(UserType.GENERAL_USER);
-        Mockito.when(resourcePermissionCheckService.userOwnedResourceIdsAcquisition(AuthorizationType.PROJECTS, 1, serviceLogger))
+        Mockito.when(resourcePermissionCheckService.userOwnedResourceIdsAcquisition(AuthorizationType.PROJECTS, 1,
+                serviceLogger))
                 .thenReturn(projectIds());
         Mockito.when(taskInstanceMapper.countTaskInstanceStateByProjectCodes(any(), any(), any())).thenReturn(
                 Collections.emptyList());
         result = dataAnalysisServiceImpl.countTaskStateByProject(user, 1, null, null);
         assertThat(result.get(Constants.DATA_LIST)).extracting("totalCount").isEqualTo(0);
         assertThat(result.get(Constants.DATA_LIST)).extracting("taskCountDtos").asList().hasSameSizeAs(
-                ExecutionStatus.values());
+                TaskExecutionStatus.values());
         assertThat(result.get(Constants.DATA_LIST)).extracting("taskCountDtos").asList().extracting(
                 "count").allMatch(count -> count.equals(0));
     }
@@ -220,13 +220,15 @@ public class DataAnalysisServiceTest {
     public void testCountTaskStateByProject_noData() {
         Map<String, Object> result = new HashMap<>();
         putMsg(result, Status.SUCCESS, null);
-        Mockito.when(projectService.checkProjectAndAuth(any(), any(), anyLong(),any())).thenReturn(result);
+        Mockito.when(projectService.checkProjectAndAuth(any(), any(), anyLong(), any())).thenReturn(result);
         Mockito.when(projectMapper.queryByCode(1L)).thenReturn(getProject("test"));
-        Mockito.when(resourcePermissionCheckService.userOwnedResourceIdsAcquisition(AuthorizationType.PROJECTS, 1, serviceLogger)).thenReturn(projectIds());
+        Mockito.when(resourcePermissionCheckService.userOwnedResourceIdsAcquisition(AuthorizationType.PROJECTS, 1,
+                serviceLogger)).thenReturn(projectIds());
 
         // when instanceStateCounter return null, then return nothing
         user.setUserType(UserType.GENERAL_USER);
-        PowerMockito.when(taskInstanceMapper.countTaskInstanceStateByProjectCodes(any(), any(), any())).thenReturn(null);
+        PowerMockito.when(taskInstanceMapper.countTaskInstanceStateByProjectCodes(any(), any(), any()))
+                .thenReturn(null);
         result = dataAnalysisServiceImpl.countTaskStateByProject(user, 1, null, null);
         Assert.assertNull(result.get(Constants.DATA_LIST));
     }
@@ -238,7 +240,7 @@ public class DataAnalysisServiceTest {
 
         Mockito.when(projectMapper.queryByCode(1L)).thenReturn(getProject("test"));
 
-        //checkProject false
+        // checkProject false
         Map<String, Object> failResult = new HashMap<>();
         putMsg(failResult, Status.PROJECT_NOT_FOUND, 1);
         Mockito.when(projectService.checkProjectAndAuth(any(), any(), anyLong(), any())).thenReturn(failResult);
@@ -249,14 +251,14 @@ public class DataAnalysisServiceTest {
         putMsg(result, Status.SUCCESS, null);
         Mockito.when(projectService.checkProjectAndAuth(any(), any(), anyLong(), any())).thenReturn(result);
 
-        //SUCCESS
+        // SUCCESS
         Mockito.when(processInstanceMapper.countInstanceStateByProjectCodes(DateUtils.stringToDate(startDate),
-            DateUtils.stringToDate(endDate),
-            new Long[] {1L})).thenReturn(getTaskInstanceStateCounts());
+                DateUtils.stringToDate(endDate),
+                new Long[]{1L})).thenReturn(getTaskInstanceStateCounts());
         Mockito.when(projectService.hasProjectAndPerm(Mockito.any(),
-            Mockito.any(),
-            (Map<String, Object>) Mockito.any(),
-            Mockito.any())).thenReturn(true);
+                Mockito.any(),
+                (Map<String, Object>) Mockito.any(),
+                Mockito.any())).thenReturn(true);
 
         result = dataAnalysisServiceImpl.countProcessInstanceStateByProject(user, 1, startDate, endDate);
         Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
@@ -268,15 +270,17 @@ public class DataAnalysisServiceTest {
 
         Map<String, Object> result = new HashMap<>();
         putMsg(result, Status.SUCCESS, null);
-        Mockito.when(projectService.checkProjectAndAuth(any(), any(), anyLong(),any())).thenReturn(result);
+        Mockito.when(projectService.checkProjectAndAuth(any(), any(), anyLong(), any())).thenReturn(result);
 
         Mockito.when(processDefinitionMapper.countDefinitionByProjectCodes(
                 Mockito.any(Long[].class))).thenReturn(new ArrayList<DefinitionGroupByUser>());
-        Mockito.when(resourcePermissionCheckService.userOwnedResourceIdsAcquisition(AuthorizationType.PROJECTS, 1, serviceLogger)).thenReturn(projectIds());
+        Mockito.when(resourcePermissionCheckService.userOwnedResourceIdsAcquisition(AuthorizationType.PROJECTS, 1,
+                serviceLogger)).thenReturn(projectIds());
         result = dataAnalysisServiceImpl.countDefinitionByUser(user, 0);
         Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
 
-        Mockito.when(resourcePermissionCheckService.userOwnedResourceIdsAcquisition(AuthorizationType.PROJECTS, 1, serviceLogger)).thenReturn(Collections.emptySet());
+        Mockito.when(resourcePermissionCheckService.userOwnedResourceIdsAcquisition(AuthorizationType.PROJECTS, 1,
+                serviceLogger)).thenReturn(Collections.emptySet());
         result = dataAnalysisServiceImpl.countDefinitionByUser(user, 0);
         Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
     }
@@ -299,12 +303,15 @@ public class DataAnalysisServiceTest {
         // when no command found then return all count are 0
         Mockito.when(commandMapper.countCommandState(any(), any(), any())).thenReturn(Collections.emptyList());
         Mockito.when(errorCommandMapper.countCommandState(any(), any(), any())).thenReturn(Collections.emptyList());
-        Mockito.when(resourcePermissionCheckService.userOwnedResourceIdsAcquisition(AuthorizationType.PROJECTS, 1, serviceLogger)).thenReturn(projectIds());
+        Mockito.when(resourcePermissionCheckService.userOwnedResourceIdsAcquisition(AuthorizationType.PROJECTS, 1,
+                serviceLogger)).thenReturn(projectIds());
 
         Map<String, Object> result5 = dataAnalysisServiceImpl.countCommandState(user);
         assertThat(result5).containsEntry(Constants.STATUS, Status.SUCCESS);
-        assertThat(result5.get(Constants.DATA_LIST)).asList().extracting("errorCount").allMatch(count -> count.equals(0));
-        assertThat(result5.get(Constants.DATA_LIST)).asList().extracting("normalCount").allMatch(count -> count.equals(0));
+        assertThat(result5.get(Constants.DATA_LIST)).asList().extracting("errorCount")
+                .allMatch(count -> count.equals(0));
+        assertThat(result5.get(Constants.DATA_LIST)).asList().extracting("normalCount")
+                .allMatch(count -> count.equals(0));
 
         // when command found then return combination result
         CommandCount normalCommandCount = new CommandCount();
@@ -313,8 +320,10 @@ public class DataAnalysisServiceTest {
         CommandCount errorCommandCount = new CommandCount();
         errorCommandCount.setCommandType(CommandType.START_PROCESS);
         errorCommandCount.setCount(5);
-        Mockito.when(commandMapper.countCommandState(any(), any(), any())).thenReturn(Collections.singletonList(normalCommandCount));
-        Mockito.when(errorCommandMapper.countCommandState(any(), any(), any())).thenReturn(Collections.singletonList(errorCommandCount));
+        Mockito.when(commandMapper.countCommandState(any(), any(), any()))
+                .thenReturn(Collections.singletonList(normalCommandCount));
+        Mockito.when(errorCommandMapper.countCommandState(any(), any(), any()))
+                .thenReturn(Collections.singletonList(errorCommandCount));
 
         Map<String, Object> result6 = dataAnalysisServiceImpl.countCommandState(user);
 
@@ -347,7 +356,7 @@ public class DataAnalysisServiceTest {
     private List<ExecuteStatusCount> getTaskInstanceStateCounts() {
         List<ExecuteStatusCount> taskInstanceStateCounts = new ArrayList<>(1);
         ExecuteStatusCount executeStatusCount = new ExecuteStatusCount();
-        executeStatusCount.setExecutionStatus(ExecutionStatus.RUNNING_EXECUTION);
+        executeStatusCount.setState(TaskExecutionStatus.RUNNING_EXECUTION);
         taskInstanceStateCounts.add(executeStatusCount);
 
         return taskInstanceStateCounts;
diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorServiceTest.java
index 44e149285f..9affbd2c2f 100644
--- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorServiceTest.java
+++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorServiceTest.java
@@ -45,7 +45,6 @@ import org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationMapper;
 import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
 import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
 import org.apache.dolphinscheduler.dao.mapper.TaskGroupQueueMapper;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.api.permission.ResourcePermissionCheckService;
 import org.apache.dolphinscheduler.service.process.ProcessService;
 
@@ -72,6 +71,7 @@ import org.slf4j.LoggerFactory;
  */
 @RunWith(MockitoJUnitRunner.Silent.class)
 public class ExecutorServiceTest {
+
     private static final Logger logger = LoggerFactory.getLogger(ExecutorServiceTest.class);
 
     private static final Logger baseServiceLogger = LoggerFactory.getLogger(BaseServiceImpl.class);
@@ -153,7 +153,7 @@ public class ExecutorServiceTest {
 
         // processInstance
         processInstance.setId(processInstanceId);
-        processInstance.setState(ExecutionStatus.FAILURE);
+        processInstance.setState(WorkflowExecutionStatus.FAILURE);
         processInstance.setExecutorId(userId);
         processInstance.setTenantId(tenantId);
         processInstance.setProcessDefinitionVersion(1);
@@ -173,7 +173,8 @@ public class ExecutorServiceTest {
 
         // mock
         Mockito.when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_START)).thenReturn(checkProjectAndAuth());
+        Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_START))
+                .thenReturn(checkProjectAndAuth());
         Mockito.when(processDefinitionMapper.queryByCode(processDefinitionCode)).thenReturn(processDefinition);
         Mockito.when(processService.getTenantForProcess(tenantId, userId)).thenReturn(new Tenant());
         Mockito.when(processService.createCommand(any(Command.class))).thenReturn(1);
@@ -185,7 +186,7 @@ public class ExecutorServiceTest {
     }
 
     @Test
-    public void testForceStartTaskInstance(){
+    public void testForceStartTaskInstance() {
 
         Map<String, Object> result = executorService.forceStartTaskInstance(loginUser, taskQueueId);
         Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
@@ -197,9 +198,12 @@ public class ExecutorServiceTest {
     @Test
     public void testNoComplement() {
 
-        Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode)).thenReturn(zeroSchedulerList());
+        Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode))
+                .thenReturn(zeroSchedulerList());
         Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode,
-                processDefinitionCode, "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}", CommandType.START_PROCESS,
+                processDefinitionCode,
+                "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}",
+                CommandType.START_PROCESS,
                 null, null,
                 null, null, 0,
                 RunMode.RUN_MODE_SERIAL,
@@ -216,13 +220,16 @@ public class ExecutorServiceTest {
     @Test
     public void testComplementWithStartNodeList() {
 
-        Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode)).thenReturn(zeroSchedulerList());
+        Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode))
+                .thenReturn(zeroSchedulerList());
         Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode,
-                processDefinitionCode, "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}", CommandType.START_PROCESS,
+                processDefinitionCode,
+                "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}",
+                CommandType.START_PROCESS,
                 null, "n1,n2",
                 null, null, 0,
                 RunMode.RUN_MODE_SERIAL,
-                Priority.LOW, Constants.DEFAULT_WORKER_GROUP, 100L,110, null, 0, Constants.DRY_RUN_FLAG_NO,
+                Priority.LOW, Constants.DEFAULT_WORKER_GROUP, 100L, 110, null, 0, Constants.DRY_RUN_FLAG_NO,
                 ComplementDependentMode.OFF_MODE);
         Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
         verify(processService, times(1)).createCommand(any(Command.class));
@@ -235,13 +242,16 @@ public class ExecutorServiceTest {
     @Test
     public void testDateError() {
 
-        Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode)).thenReturn(zeroSchedulerList());
+        Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode))
+                .thenReturn(zeroSchedulerList());
         Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode,
-                processDefinitionCode, "{\"complementStartDate\":\"2022-01-07 12:12:12\",\"complementEndDate\":\"2022-01-06 12:12:12\"}", CommandType.COMPLEMENT_DATA,
+                processDefinitionCode,
+                "{\"complementStartDate\":\"2022-01-07 12:12:12\",\"complementEndDate\":\"2022-01-06 12:12:12\"}",
+                CommandType.COMPLEMENT_DATA,
                 null, null,
                 null, null, 0,
                 RunMode.RUN_MODE_SERIAL,
-                Priority.LOW, Constants.DEFAULT_WORKER_GROUP,100L, 110, null, 0, Constants.DRY_RUN_FLAG_NO,
+                Priority.LOW, Constants.DEFAULT_WORKER_GROUP, 100L, 110, null, 0, Constants.DRY_RUN_FLAG_NO,
                 ComplementDependentMode.OFF_MODE);
         Assert.assertEquals(Status.START_PROCESS_INSTANCE_ERROR, result.get(Constants.STATUS));
         verify(processService, times(0)).createCommand(any(Command.class));
@@ -253,13 +263,16 @@ public class ExecutorServiceTest {
     @Test
     public void testSerial() {
 
-        Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode)).thenReturn(zeroSchedulerList());
+        Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode))
+                .thenReturn(zeroSchedulerList());
         Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode,
-                processDefinitionCode, "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}", CommandType.COMPLEMENT_DATA,
+                processDefinitionCode,
+                "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}",
+                CommandType.COMPLEMENT_DATA,
                 null, null,
                 null, null, 0,
                 RunMode.RUN_MODE_SERIAL,
-                Priority.LOW, Constants.DEFAULT_WORKER_GROUP,100L, 110, null, 0, Constants.DRY_RUN_FLAG_NO,
+                Priority.LOW, Constants.DEFAULT_WORKER_GROUP, 100L, 110, null, 0, Constants.DRY_RUN_FLAG_NO,
                 ComplementDependentMode.OFF_MODE);
         Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
         verify(processService, times(1)).createCommand(any(Command.class));
@@ -271,13 +284,16 @@ public class ExecutorServiceTest {
     @Test
     public void testParallelWithOutSchedule() {
 
-        Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode)).thenReturn(zeroSchedulerList());
+        Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode))
+                .thenReturn(zeroSchedulerList());
         Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode,
-                processDefinitionCode, "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}", CommandType.COMPLEMENT_DATA,
+                processDefinitionCode,
+                "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}",
+                CommandType.COMPLEMENT_DATA,
                 null, null,
                 null, null, 0,
                 RunMode.RUN_MODE_PARALLEL,
-                Priority.LOW, Constants.DEFAULT_WORKER_GROUP,100L, 110, null, 0, Constants.DRY_RUN_FLAG_NO,
+                Priority.LOW, Constants.DEFAULT_WORKER_GROUP, 100L, 110, null, 0, Constants.DRY_RUN_FLAG_NO,
                 ComplementDependentMode.OFF_MODE);
         Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
         verify(processService, times(31)).createCommand(any(Command.class));
@@ -290,13 +306,16 @@ public class ExecutorServiceTest {
     @Test
     public void testParallelWithSchedule() {
 
-        Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode)).thenReturn(oneSchedulerList());
+        Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode))
+                .thenReturn(oneSchedulerList());
         Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode,
-                processDefinitionCode, "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}", CommandType.COMPLEMENT_DATA,
+                processDefinitionCode,
+                "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}",
+                CommandType.COMPLEMENT_DATA,
                 null, null,
                 null, null, 0,
                 RunMode.RUN_MODE_PARALLEL,
-                Priority.LOW, Constants.DEFAULT_WORKER_GROUP, 100L,110, null, 15, Constants.DRY_RUN_FLAG_NO,
+                Priority.LOW, Constants.DEFAULT_WORKER_GROUP, 100L, 110, null, 15, Constants.DRY_RUN_FLAG_NO,
                 ComplementDependentMode.OFF_MODE);
         Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
         verify(processService, times(15)).createCommand(any(Command.class));
@@ -308,11 +327,13 @@ public class ExecutorServiceTest {
         Mockito.when(monitorService.getServerListFromRegistry(true)).thenReturn(new ArrayList<>());
 
         Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode,
-                processDefinitionCode, "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}", CommandType.COMPLEMENT_DATA,
+                processDefinitionCode,
+                "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}",
+                CommandType.COMPLEMENT_DATA,
                 null, null,
                 null, null, 0,
                 RunMode.RUN_MODE_PARALLEL,
-                Priority.LOW, Constants.DEFAULT_WORKER_GROUP, 100L,110, null, 0, Constants.DRY_RUN_FLAG_NO,
+                Priority.LOW, Constants.DEFAULT_WORKER_GROUP, 100L, 110, null, 0, Constants.DRY_RUN_FLAG_NO,
                 ComplementDependentMode.OFF_MODE);
         Assert.assertEquals(result.get(Constants.STATUS), Status.MASTER_NOT_EXISTS);
 
@@ -321,8 +342,10 @@ public class ExecutorServiceTest {
     @Test
     public void testExecuteRepeatRunning() {
         Mockito.when(processService.verifyIsNeedCreateCommand(any(Command.class))).thenReturn(true);
-        Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectCode, RERUN )).thenReturn(checkProjectAndAuth());
-        Map<String, Object> result = executorService.execute(loginUser, projectCode, processInstanceId, ExecuteType.REPEAT_RUNNING);
+        Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectCode, RERUN))
+                .thenReturn(checkProjectAndAuth());
+        Map<String, Object> result =
+                executorService.execute(loginUser, projectCode, processInstanceId, ExecuteType.REPEAT_RUNNING);
         Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
     }
 
@@ -418,5 +441,5 @@ public class ExecutorServiceTest {
         Assert.assertEquals("2,3", result.get(1));
         Assert.assertEquals("4,4", result.get(2));
     }
-    
+
 }
diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessInstanceServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessInstanceServiceTest.java
index dde556e453..73289dbabf 100644
--- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessInstanceServiceTest.java
+++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessInstanceServiceTest.java
@@ -36,6 +36,8 @@ import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.CommandType;
 import org.apache.dolphinscheduler.common.enums.Flag;
 import org.apache.dolphinscheduler.common.enums.UserType;
+import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.service.expand.CuringParamsService;
 import org.apache.dolphinscheduler.common.graph.DAG;
 import org.apache.dolphinscheduler.common.model.TaskNode;
@@ -61,7 +63,6 @@ import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
 import org.apache.dolphinscheduler.dao.mapper.TaskInstanceMapper;
 import org.apache.dolphinscheduler.dao.mapper.TenantMapper;
 import org.apache.dolphinscheduler.plugin.task.api.enums.DependResult;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.apache.dolphinscheduler.service.task.TaskPluginManager;
 import org.junit.Assert;
@@ -131,29 +132,31 @@ public class ProcessInstanceServiceTest {
     @Mock
     CuringParamsService curingGlobalParamsService;
 
-
     private String shellJson = "[{\"name\":\"\",\"preTaskCode\":0,\"preTaskVersion\":0,\"postTaskCode\":123456789,"
-        + "\"postTaskVersion\":1,\"conditionType\":0,\"conditionParams\":\"{}\"},{\"name\":\"\",\"preTaskCode\":123456789,"
-        + "\"preTaskVersion\":1,\"postTaskCode\":123451234,\"postTaskVersion\":1,\"conditionType\":0,\"conditionParams\":\"{}\"}]";
-
-    private String taskJson = "[{\"name\":\"shell1\",\"description\":\"\",\"taskType\":\"SHELL\",\"taskParams\":{\"resourceList\":[],"
-        + "\"localParams\":[],\"rawScript\":\"echo 1\",\"conditionResult\":{\"successNode\":[\"\"],\"failedNode\":[\"\"]},\"dependence\":{}},"
-        + "\"flag\":\"NORMAL\",\"taskPriority\":\"MEDIUM\",\"workerGroup\":\"default\",\"failRetryTimes\":\"0\",\"failRetryInterval\":\"1\","
-        + "\"timeoutFlag\":\"CLOSE\",\"timeoutNotifyStrategy\":\"\",\"timeout\":null,\"delayTime\":\"0\"},{\"name\":\"shell2\",\"description\":\"\","
-        + "\"taskType\":\"SHELL\",\"taskParams\":{\"resourceList\":[],\"localParams\":[],\"rawScript\":\"echo 2\",\"conditionResult\":{\"successNode\""
-        + ":[\"\"],\"failedNode\":[\"\"]},\"dependence\":{}},\"flag\":\"NORMAL\",\"taskPriority\":\"MEDIUM\",\"workerGroup\":\"default\","
-        + "\"failRetryTimes\":\"0\",\"failRetryInterval\":\"1\",\"timeoutFlag\":\"CLOSE\",\"timeoutNotifyStrategy\":\"\",\"timeout\":null,\"delayTime\":\"0\"}]";
-
-    private String taskRelationJson = "[{\"name\":\"\",\"preTaskCode\":4254865123776,\"preTaskVersion\":1,\"postTaskCode\":4254862762304,\"postTaskVersion\":1,\"conditionType\":0,"
-        + "\"conditionParams\":{}},{\"name\":\"\",\"preTaskCode\":0,\"preTaskVersion\":0,\"postTaskCode\":4254865123776,\"postTaskVersion\":1,\"conditionType\":0,\"conditionParams\":{}}]";
-
-    private String taskDefinitionJson = "[{\"code\":4254862762304,\"name\":\"test1\",\"version\":1,\"description\":\"\",\"delayTime\":0,\"taskType\":\"SHELL\",\"taskParams\":{\"resourceList\":[],"
-        + "\"localParams\":[],\"rawScript\":\"echo 1\",\"dependence\":{},\"conditionResult\":{\"successNode\":[],\"failedNode\":[]},\"waitStartTimeout\":{},\"switchResult\":{}},\"flag\":\"YES\","
-        + "\"taskPriority\":\"MEDIUM\",\"workerGroup\":\"default\",\"failRetryTimes\":0,\"failRetryInterval\":1,\"timeoutFlag\":\"CLOSE\",\"timeoutNotifyStrategy\":null,\"timeout\":0,"
-        + "\"environmentCode\":-1},{\"code\":4254865123776,\"name\":\"test2\",\"version\":1,\"description\":\"\",\"delayTime\":0,\"taskType\":\"SHELL\",\"taskParams\":{\"resourceList\":[],"
-        + "\"localParams\":[],\"rawScript\":\"echo 2\",\"dependence\":{},\"conditionResult\":{\"successNode\":[],\"failedNode\":[]},\"waitStartTimeout\":{},\"switchResult\":{}},\"flag\":\"YES\","
-        + "\"taskPriority\":\"MEDIUM\",\"workerGroup\":\"default\",\"failRetryTimes\":0,\"failRetryInterval\":1,\"timeoutFlag\":\"CLOSE\",\"timeoutNotifyStrategy\":\"WARN\",\"timeout\":0,"
-        + "\"environmentCode\":-1}]";
+            + "\"postTaskVersion\":1,\"conditionType\":0,\"conditionParams\":\"{}\"},{\"name\":\"\",\"preTaskCode\":123456789,"
+            + "\"preTaskVersion\":1,\"postTaskCode\":123451234,\"postTaskVersion\":1,\"conditionType\":0,\"conditionParams\":\"{}\"}]";
+
+    private String taskJson =
+            "[{\"name\":\"shell1\",\"description\":\"\",\"taskType\":\"SHELL\",\"taskParams\":{\"resourceList\":[],"
+                    + "\"localParams\":[],\"rawScript\":\"echo 1\",\"conditionResult\":{\"successNode\":[\"\"],\"failedNode\":[\"\"]},\"dependence\":{}},"
+                    + "\"flag\":\"NORMAL\",\"taskPriority\":\"MEDIUM\",\"workerGroup\":\"default\",\"failRetryTimes\":\"0\",\"failRetryInterval\":\"1\","
+                    + "\"timeoutFlag\":\"CLOSE\",\"timeoutNotifyStrategy\":\"\",\"timeout\":null,\"delayTime\":\"0\"},{\"name\":\"shell2\",\"description\":\"\","
+                    + "\"taskType\":\"SHELL\",\"taskParams\":{\"resourceList\":[],\"localParams\":[],\"rawScript\":\"echo 2\",\"conditionResult\":{\"successNode\""
+                    + ":[\"\"],\"failedNode\":[\"\"]},\"dependence\":{}},\"flag\":\"NORMAL\",\"taskPriority\":\"MEDIUM\",\"workerGroup\":\"default\","
+                    + "\"failRetryTimes\":\"0\",\"failRetryInterval\":\"1\",\"timeoutFlag\":\"CLOSE\",\"timeoutNotifyStrategy\":\"\",\"timeout\":null,\"delayTime\":\"0\"}]";
+
+    private String taskRelationJson =
+            "[{\"name\":\"\",\"preTaskCode\":4254865123776,\"preTaskVersion\":1,\"postTaskCode\":4254862762304,\"postTaskVersion\":1,\"conditionType\":0,"
+                    + "\"conditionParams\":{}},{\"name\":\"\",\"preTaskCode\":0,\"preTaskVersion\":0,\"postTaskCode\":4254865123776,\"postTaskVersion\":1,\"conditionType\":0,\"conditionParams\":{}}]";
+
+    private String taskDefinitionJson =
+            "[{\"code\":4254862762304,\"name\":\"test1\",\"version\":1,\"description\":\"\",\"delayTime\":0,\"taskType\":\"SHELL\",\"taskParams\":{\"resourceList\":[],"
+                    + "\"localParams\":[],\"rawScript\":\"echo 1\",\"dependence\":{},\"conditionResult\":{\"successNode\":[],\"failedNode\":[]},\"waitStartTimeout\":{},\"switchResult\":{}},\"flag\":\"YES\","
+                    + "\"taskPriority\":\"MEDIUM\",\"workerGroup\":\"default\",\"failRetryTimes\":0,\"failRetryInterval\":1,\"timeoutFlag\":\"CLOSE\",\"timeoutNotifyStrategy\":null,\"timeout\":0,"
+                    + "\"environmentCode\":-1},{\"code\":4254865123776,\"name\":\"test2\",\"version\":1,\"description\":\"\",\"delayTime\":0,\"taskType\":\"SHELL\",\"taskParams\":{\"resourceList\":[],"
+                    + "\"localParams\":[],\"rawScript\":\"echo 2\",\"dependence\":{},\"conditionResult\":{\"successNode\":[],\"failedNode\":[]},\"waitStartTimeout\":{},\"switchResult\":{}},\"flag\":\"YES\","
+                    + "\"taskPriority\":\"MEDIUM\",\"workerGroup\":\"default\",\"failRetryTimes\":0,\"failRetryInterval\":1,\"timeoutFlag\":\"CLOSE\",\"timeoutNotifyStrategy\":\"WARN\",\"timeout\":0,"
+                    + "\"environmentCode\":-1}]";
 
     @Test
     public void testQueryProcessInstanceList() {
@@ -163,12 +166,13 @@ public class ProcessInstanceServiceTest {
         Map<String, Object> result = new HashMap<>();
         putMsg(result, Status.PROJECT_NOT_FOUND, projectCode);
 
-        //project auth fail
+        // project auth fail
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
         when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE)).thenReturn(result);
-        Result proejctAuthFailRes = processInstanceService.queryProcessInstanceList(loginUser, projectCode, 46, "2020-01-01 00:00:00",
-            "2020-01-02 00:00:00", "", "test_user", ExecutionStatus.SUBMITTED_SUCCESS,
-            "192.168.xx.xx", "",1, 10);
+        Result proejctAuthFailRes =
+                processInstanceService.queryProcessInstanceList(loginUser, projectCode, 46, "2020-01-01 00:00:00",
+                        "2020-01-02 00:00:00", "", "test_user", WorkflowExecutionStatus.SUBMITTED_SUCCESS,
+                        "192.168.xx.xx", "", 1, 10);
         Assert.assertEquals(Status.PROJECT_NOT_FOUND.getCode(), (int) proejctAuthFailRes.getCode());
 
         Date start = DateUtils.stringToDate("2020-01-01 00:00:00");
@@ -182,57 +186,64 @@ public class ProcessInstanceServiceTest {
         // data parameter check
         putMsg(result, Status.SUCCESS, projectCode);
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE)).thenReturn(result);
+        when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE)).thenReturn(result);
         when(processDefineMapper.selectById(Mockito.anyInt())).thenReturn(getProcessDefinition());
-        when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class)
-            , Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(),
-            eq("192.168.xx.xx"), Mockito.any(), Mockito.any())).thenReturn(pageReturn);
-
-        Result dataParameterRes = processInstanceService.queryProcessInstanceList(loginUser, projectCode, 1, "20200101 00:00:00",
-            "20200102 00:00:00", "", loginUser.getUserName(), ExecutionStatus.SUBMITTED_SUCCESS,
-            "192.168.xx.xx", "",1, 10);
+        when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class), Mockito.any(), Mockito.any(),
+                Mockito.any(), Mockito.any(), Mockito.any(),
+                eq("192.168.xx.xx"), Mockito.any(), Mockito.any())).thenReturn(pageReturn);
+
+        Result dataParameterRes =
+                processInstanceService.queryProcessInstanceList(loginUser, projectCode, 1, "20200101 00:00:00",
+                        "20200102 00:00:00", "", loginUser.getUserName(), WorkflowExecutionStatus.SUBMITTED_SUCCESS,
+                        "192.168.xx.xx", "", 1, 10);
         Assert.assertEquals(Status.REQUEST_PARAMS_NOT_VALID_ERROR.getCode(), (int) dataParameterRes.getCode());
 
-        //project auth success
+        // project auth success
         putMsg(result, Status.SUCCESS, projectCode);
 
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE)).thenReturn(result);
+        when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE)).thenReturn(result);
         when(usersService.queryUser(loginUser.getId())).thenReturn(loginUser);
         when(usersService.getUserIdByName(loginUser.getUserName())).thenReturn(loginUser.getId());
-        when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1L), eq(""), eq(-1), Mockito.any(),
-            eq("192.168.xx.xx"), eq(start), eq(end))).thenReturn(pageReturn);
+        when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()),
+                eq(1L), eq(""), eq(-1), Mockito.any(),
+                eq("192.168.xx.xx"), eq(start), eq(end))).thenReturn(pageReturn);
         when(usersService.queryUser(processInstance.getExecutorId())).thenReturn(loginUser);
 
-        Result successRes = processInstanceService.queryProcessInstanceList(loginUser, projectCode, 1, "2020-01-01 00:00:00",
-            "2020-01-02 00:00:00", "", loginUser.getUserName(), ExecutionStatus.SUBMITTED_SUCCESS,
-            "192.168.xx.xx", "",1, 10);
-        Assert.assertEquals(Status.SUCCESS.getCode(), (int)successRes.getCode());
+        Result successRes =
+                processInstanceService.queryProcessInstanceList(loginUser, projectCode, 1, "2020-01-01 00:00:00",
+                        "2020-01-02 00:00:00", "", loginUser.getUserName(), WorkflowExecutionStatus.SUBMITTED_SUCCESS,
+                        "192.168.xx.xx", "", 1, 10);
+        Assert.assertEquals(Status.SUCCESS.getCode(), (int) successRes.getCode());
 
         // data parameter empty
-        when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1L), eq(""), eq(-1), Mockito.any(),
-            eq("192.168.xx.xx"), eq(null), eq(null))).thenReturn(pageReturn);
+        when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()),
+                eq(1L), eq(""), eq(-1), Mockito.any(),
+                eq("192.168.xx.xx"), eq(null), eq(null))).thenReturn(pageReturn);
         successRes = processInstanceService.queryProcessInstanceList(loginUser, projectCode, 1, "",
-            "", "", loginUser.getUserName(), ExecutionStatus.SUBMITTED_SUCCESS,
-            "192.168.xx.xx", "",1, 10);
-        Assert.assertEquals(Status.SUCCESS.getCode(), (int)successRes.getCode());
+                "", "", loginUser.getUserName(), WorkflowExecutionStatus.SUBMITTED_SUCCESS,
+                "192.168.xx.xx", "", 1, 10);
+        Assert.assertEquals(Status.SUCCESS.getCode(), (int) successRes.getCode());
 
-        //executor null
+        // executor null
         when(usersService.queryUser(loginUser.getId())).thenReturn(null);
         when(usersService.getUserIdByName(loginUser.getUserName())).thenReturn(-1);
-        Result executorExistRes = processInstanceService.queryProcessInstanceList(loginUser, projectCode, 1, "2020-01-01 00:00:00",
-            "2020-01-02 00:00:00", "", "admin", ExecutionStatus.SUBMITTED_SUCCESS,
-            "192.168.xx.xx", "",1, 10);
-
-        Assert.assertEquals(Status.SUCCESS.getCode(), (int)executorExistRes.getCode());
-
-        //executor name empty
-        when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1L), eq(""), eq(0), Mockito.any(),
-            eq("192.168.xx.xx"), eq(start), eq(end))).thenReturn(pageReturn);
-        Result executorEmptyRes = processInstanceService.queryProcessInstanceList(loginUser, projectCode, 1, "2020-01-01 00:00:00",
-            "2020-01-02 00:00:00", "", "", ExecutionStatus.SUBMITTED_SUCCESS,
-            "192.168.xx.xx", "",1, 10);
-        Assert.assertEquals(Status.SUCCESS.getCode(), (int)executorEmptyRes.getCode());
+        Result executorExistRes =
+                processInstanceService.queryProcessInstanceList(loginUser, projectCode, 1, "2020-01-01 00:00:00",
+                        "2020-01-02 00:00:00", "", "admin", WorkflowExecutionStatus.SUBMITTED_SUCCESS,
+                        "192.168.xx.xx", "", 1, 10);
+
+        Assert.assertEquals(Status.SUCCESS.getCode(), (int) executorExistRes.getCode());
+
+        // executor name empty
+        when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()),
+                eq(1L), eq(""), eq(0), Mockito.any(),
+                eq("192.168.xx.xx"), eq(start), eq(end))).thenReturn(pageReturn);
+        Result executorEmptyRes =
+                processInstanceService.queryProcessInstanceList(loginUser, projectCode, 1, "2020-01-01 00:00:00",
+                        "2020-01-02 00:00:00", "", "", WorkflowExecutionStatus.SUBMITTED_SUCCESS,
+                        "192.168.xx.xx", "", 1, 10);
+        Assert.assertEquals(Status.SUCCESS.getCode(), (int) executorEmptyRes.getCode());
 
     }
 
@@ -249,21 +260,23 @@ public class ProcessInstanceServiceTest {
         Date start = DateUtils.stringToDate(startTime);
         Date end = DateUtils.stringToDate(endTime);
 
-        //project auth fail
+        // project auth fail
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE)).thenReturn(result);
-        Map<String, Object> proejctAuthFailRes = processInstanceService.queryTopNLongestRunningProcessInstance(loginUser, projectCode, size, startTime, endTime);
+        when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE)).thenReturn(result);
+        Map<String, Object> proejctAuthFailRes = processInstanceService
+                .queryTopNLongestRunningProcessInstance(loginUser, projectCode, size, startTime, endTime);
         Assert.assertEquals(Status.PROJECT_NOT_FOUND, proejctAuthFailRes.get(Constants.STATUS));
 
-        //project auth success
+        // project auth success
         putMsg(result, Status.SUCCESS, projectCode);
         ProcessInstance processInstance = getProcessInstance();
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE)).thenReturn(result);
+        when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE)).thenReturn(result);
         when(usersService.queryUser(loginUser.getId())).thenReturn(loginUser);
         when(usersService.getUserIdByName(loginUser.getUserName())).thenReturn(loginUser.getId());
         when(usersService.queryUser(processInstance.getExecutorId())).thenReturn(loginUser);
-        Map<String, Object> successRes = processInstanceService.queryTopNLongestRunningProcessInstance(loginUser, projectCode, size, startTime, endTime);
+        Map<String, Object> successRes = processInstanceService.queryTopNLongestRunningProcessInstance(loginUser,
+                projectCode, size, startTime, endTime);
 
         Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS));
     }
@@ -276,30 +289,31 @@ public class ProcessInstanceServiceTest {
         Map<String, Object> result = new HashMap<>();
         putMsg(result, Status.PROJECT_NOT_FOUND, projectCode);
 
-        //project auth fail
+        // project auth fail
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE)).thenReturn(result);
-        Map<String, Object> proejctAuthFailRes = processInstanceService.queryProcessInstanceById(loginUser, projectCode, 1);
+        when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE)).thenReturn(result);
+        Map<String, Object> proejctAuthFailRes =
+                processInstanceService.queryProcessInstanceById(loginUser, projectCode, 1);
         Assert.assertEquals(Status.PROJECT_NOT_FOUND, proejctAuthFailRes.get(Constants.STATUS));
 
-        //project auth success
+        // project auth success
         ProcessInstance processInstance = getProcessInstance();
         putMsg(result, Status.SUCCESS, projectCode);
         ProcessDefinition processDefinition = getProcessDefinition();
         processDefinition.setProjectCode(projectCode);
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE)).thenReturn(result);
+        when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE)).thenReturn(result);
         when(processService.findProcessInstanceDetailById(processInstance.getId())).thenReturn(processInstance);
         when(processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
-            processInstance.getProcessDefinitionVersion())).thenReturn(processDefinition);
+                processInstance.getProcessDefinitionVersion())).thenReturn(processDefinition);
         Map<String, Object> successRes = processInstanceService.queryProcessInstanceById(loginUser, projectCode, 1);
         Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS));
 
-        //worker group null
+        // worker group null
         Map<String, Object> workerNullRes = processInstanceService.queryProcessInstanceById(loginUser, projectCode, 1);
         Assert.assertEquals(Status.SUCCESS, workerNullRes.get(Constants.STATUS));
 
-        //worker group exist
+        // worker group exist
         WorkerGroup workerGroup = getWorkGroup();
         Map<String, Object> workerExistRes = processInstanceService.queryProcessInstanceById(loginUser, projectCode, 1);
         Assert.assertEquals(Status.SUCCESS, workerExistRes.get(Constants.STATUS));
@@ -313,16 +327,17 @@ public class ProcessInstanceServiceTest {
         Map<String, Object> result = new HashMap<>();
         putMsg(result, Status.PROJECT_NOT_FOUND, projectCode);
 
-        //project auth fail
+        // project auth fail
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE)).thenReturn(result);
-        Map<String, Object> proejctAuthFailRes = processInstanceService.queryTaskListByProcessId(loginUser, projectCode, 1);
+        when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE)).thenReturn(result);
+        Map<String, Object> proejctAuthFailRes =
+                processInstanceService.queryTaskListByProcessId(loginUser, projectCode, 1);
         Assert.assertEquals(Status.PROJECT_NOT_FOUND, proejctAuthFailRes.get(Constants.STATUS));
 
-        //project auth success
+        // project auth success
         putMsg(result, Status.SUCCESS, projectCode);
         ProcessInstance processInstance = getProcessInstance();
-        processInstance.setState(ExecutionStatus.SUCCESS);
+        processInstance.setState(WorkflowExecutionStatus.SUCCESS);
         TaskInstance taskInstance = new TaskInstance();
         taskInstance.setTaskType("SHELL");
         List<TaskInstance> taskInstanceList = new ArrayList<>();
@@ -331,7 +346,7 @@ public class ProcessInstanceServiceTest {
         res.setCode(Status.SUCCESS.ordinal());
         res.setData("xxx");
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE)).thenReturn(result);
+        when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE)).thenReturn(result);
         when(processService.findProcessInstanceDetailById(processInstance.getId())).thenReturn(processInstance);
         when(processService.findValidTaskListByProcessId(processInstance.getId())).thenReturn(taskInstanceList);
         when(loggerService.queryLog(taskInstance.getId(), 0, 4098)).thenReturn(res);
@@ -341,13 +356,14 @@ public class ProcessInstanceServiceTest {
 
     @Test
     public void testParseLogForDependentResult() throws IOException {
-        String logString = "[INFO] 2019-03-19 17:11:08.475 org.apache.dolphinscheduler.server.worker.log.TaskLogger:[172]"
-            + " - [taskAppId=TASK_223_10739_452334] dependent item complete :|| 223-ALL-day-last1Day,SUCCESS\n"
-            + "[INFO] 2019-03-19 17:11:08.476 org.apache.dolphinscheduler.server.worker.runner.TaskScheduleThread:[172]"
-            + " - task : 223_10739_452334 exit status code : 0\n"
-            + "[root@node2 current]# ";
+        String logString =
+                "[INFO] 2019-03-19 17:11:08.475 org.apache.dolphinscheduler.server.worker.log.TaskLogger:[172]"
+                        + " - [taskAppId=TASK_223_10739_452334] dependent item complete :|| 223-ALL-day-last1Day,SUCCESS\n"
+                        + "[INFO] 2019-03-19 17:11:08.476 org.apache.dolphinscheduler.server.worker.runner.TaskScheduleThread:[172]"
+                        + " - task : 223_10739_452334 exit status code : 0\n"
+                        + "[root@node2 current]# ";
         Map<String, DependResult> resultMap =
-            processInstanceService.parseLogForDependentResult(logString);
+                processInstanceService.parseLogForDependentResult(logString);
         Assert.assertEquals(1, resultMap.size());
     }
 
@@ -359,21 +375,23 @@ public class ProcessInstanceServiceTest {
         Map<String, Object> result = new HashMap<>();
         putMsg(result, Status.PROJECT_NOT_FOUND, projectCode);
 
-        //project auth fail
+        // project auth fail
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE)).thenReturn(result);
-        Map<String, Object> proejctAuthFailRes = processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectCode, 1);
+        when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE)).thenReturn(result);
+        Map<String, Object> proejctAuthFailRes =
+                processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectCode, 1);
         Assert.assertEquals(Status.PROJECT_NOT_FOUND, proejctAuthFailRes.get(Constants.STATUS));
 
-        //task null
+        // task null
         putMsg(result, Status.SUCCESS, projectCode);
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE)).thenReturn(result);
+        when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE)).thenReturn(result);
         when(processService.findTaskInstanceById(1)).thenReturn(null);
-        Map<String, Object> taskNullRes = processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectCode, 1);
+        Map<String, Object> taskNullRes =
+                processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectCode, 1);
         Assert.assertEquals(Status.TASK_INSTANCE_NOT_EXISTS, taskNullRes.get(Constants.STATUS));
 
-        //task not sub process
+        // task not sub process
         TaskInstance taskInstance = getTaskInstance();
         taskInstance.setTaskType("HTTP");
         taskInstance.setProcessInstanceId(1);
@@ -382,24 +400,28 @@ public class ProcessInstanceServiceTest {
         TaskDefinition taskDefinition = new TaskDefinition();
         taskDefinition.setProjectCode(projectCode);
         when(taskDefinitionMapper.queryByCode(taskInstance.getTaskCode())).thenReturn(taskDefinition);
-        Map<String, Object> notSubprocessRes = processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectCode, 1);
+        Map<String, Object> notSubprocessRes =
+                processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectCode, 1);
         Assert.assertEquals(Status.TASK_INSTANCE_NOT_SUB_WORKFLOW_INSTANCE, notSubprocessRes.get(Constants.STATUS));
 
-        //sub process not exist
+        // sub process not exist
         TaskInstance subTask = getTaskInstance();
         subTask.setTaskType("SUB_PROCESS");
         subTask.setProcessInstanceId(1);
         putMsg(result, Status.SUCCESS, projectCode);
         when(processService.findTaskInstanceById(subTask.getId())).thenReturn(subTask);
         when(processService.findSubProcessInstance(subTask.getProcessInstanceId(), subTask.getId())).thenReturn(null);
-        Map<String, Object> subprocessNotExistRes = processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectCode, 1);
+        Map<String, Object> subprocessNotExistRes =
+                processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectCode, 1);
         Assert.assertEquals(Status.SUB_PROCESS_INSTANCE_NOT_EXIST, subprocessNotExistRes.get(Constants.STATUS));
 
-        //sub process exist
+        // sub process exist
         ProcessInstance processInstance = getProcessInstance();
         putMsg(result, Status.SUCCESS, projectCode);
-        when(processService.findSubProcessInstance(taskInstance.getProcessInstanceId(), taskInstance.getId())).thenReturn(processInstance);
-        Map<String, Object> subprocessExistRes = processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectCode, 1);
+        when(processService.findSubProcessInstance(taskInstance.getProcessInstanceId(), taskInstance.getId()))
+                .thenReturn(processInstance);
+        Map<String, Object> subprocessExistRes =
+                processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectCode, 1);
         Assert.assertEquals(Status.SUCCESS, subprocessExistRes.get(Constants.STATUS));
     }
 
@@ -411,33 +433,36 @@ public class ProcessInstanceServiceTest {
         Map<String, Object> result = new HashMap<>();
         putMsg(result, Status.PROJECT_NOT_FOUND, projectCode);
 
-        //project auth fail
+        // project auth fail
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(loginUser, project, projectCode,INSTANCE_UPDATE )).thenReturn(result);
+        when(projectService.checkProjectAndAuth(loginUser, project, projectCode, INSTANCE_UPDATE)).thenReturn(result);
         Map<String, Object> proejctAuthFailRes = processInstanceService.updateProcessInstance(loginUser, projectCode, 1,
-            shellJson, taskJson, "2020-02-21 00:00:00", true, "", "", 0, "");
+                shellJson, taskJson, "2020-02-21 00:00:00", true, "", "", 0, "");
         Assert.assertEquals(Status.PROJECT_NOT_FOUND, proejctAuthFailRes.get(Constants.STATUS));
 
-        //process instance null
+        // process instance null
         putMsg(result, Status.SUCCESS, projectCode);
         ProcessInstance processInstance = getProcessInstance();
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(loginUser, project, projectCode,INSTANCE_UPDATE )).thenReturn(result);
+        when(projectService.checkProjectAndAuth(loginUser, project, projectCode, INSTANCE_UPDATE)).thenReturn(result);
         when(processService.findProcessInstanceDetailById(1)).thenReturn(null);
-        Map<String, Object> processInstanceNullRes = processInstanceService.updateProcessInstance(loginUser, projectCode, 1,
-            shellJson, taskJson,"2020-02-21 00:00:00", true, "", "", 0, "");
+        Map<String, Object> processInstanceNullRes =
+                processInstanceService.updateProcessInstance(loginUser, projectCode, 1,
+                        shellJson, taskJson, "2020-02-21 00:00:00", true, "", "", 0, "");
         Assert.assertEquals(Status.PROCESS_INSTANCE_NOT_EXIST, processInstanceNullRes.get(Constants.STATUS));
 
-        //process instance not finish
+        // process instance not finish
         when(processService.findProcessInstanceDetailById(1)).thenReturn(processInstance);
-        processInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
+        processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION);
         putMsg(result, Status.SUCCESS, projectCode);
-        Map<String, Object> processInstanceNotFinishRes = processInstanceService.updateProcessInstance(loginUser, projectCode, 1,
-            shellJson, taskJson,"2020-02-21 00:00:00", true, "", "", 0, "");
-        Assert.assertEquals(Status.PROCESS_INSTANCE_STATE_OPERATION_ERROR, processInstanceNotFinishRes.get(Constants.STATUS));
-
-        //process instance finish
-        processInstance.setState(ExecutionStatus.SUCCESS);
+        Map<String, Object> processInstanceNotFinishRes =
+                processInstanceService.updateProcessInstance(loginUser, projectCode, 1,
+                        shellJson, taskJson, "2020-02-21 00:00:00", true, "", "", 0, "");
+        Assert.assertEquals(Status.PROCESS_INSTANCE_STATE_OPERATION_ERROR,
+                processInstanceNotFinishRes.get(Constants.STATUS));
+
+        // process instance finish
+        processInstance.setState(WorkflowExecutionStatus.SUCCESS);
         processInstance.setTimeout(3000);
         processInstance.setCommandType(CommandType.STOP);
         processInstance.setProcessDefinitionCode(46L);
@@ -457,17 +482,19 @@ public class ProcessInstanceServiceTest {
         when(processDefinitionService.checkProcessNodeList(taskRelationJson, taskDefinitionLogs)).thenReturn(result);
         putMsg(result, Status.SUCCESS, projectCode);
         when(taskPluginManager.checkTaskParameters(Mockito.any())).thenReturn(true);
-        Map<String, Object> processInstanceFinishRes = processInstanceService.updateProcessInstance(loginUser, projectCode, 1,
-            taskRelationJson, taskDefinitionJson,"2020-02-21 00:00:00", true, "", "", 0, "root");
+        Map<String, Object> processInstanceFinishRes =
+                processInstanceService.updateProcessInstance(loginUser, projectCode, 1,
+                        taskRelationJson, taskDefinitionJson, "2020-02-21 00:00:00", true, "", "", 0, "root");
         Assert.assertEquals(Status.SUCCESS, processInstanceFinishRes.get(Constants.STATUS));
 
-        //success
+        // success
         when(processDefineMapper.queryByCode(46L)).thenReturn(processDefinition);
         putMsg(result, Status.SUCCESS, projectCode);
 
-        when(processService.saveProcessDefine(loginUser, processDefinition, Boolean.FALSE, Boolean.FALSE)).thenReturn(1);
+        when(processService.saveProcessDefine(loginUser, processDefinition, Boolean.FALSE, Boolean.FALSE))
+                .thenReturn(1);
         Map<String, Object> successRes = processInstanceService.updateProcessInstance(loginUser, projectCode, 1,
-            taskRelationJson, taskDefinitionJson,"2020-02-21 00:00:00", Boolean.FALSE, "", "", 0, "root");
+                taskRelationJson, taskDefinitionJson, "2020-02-21 00:00:00", Boolean.FALSE, "", "", 0, "root");
         Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS));
     }
 
@@ -479,36 +506,40 @@ public class ProcessInstanceServiceTest {
         Map<String, Object> result = new HashMap<>();
         putMsg(result, Status.PROJECT_NOT_FOUND, projectCode);
 
-        //project auth fail
+        // project auth fail
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE)).thenReturn(result);
-        Map<String, Object> proejctAuthFailRes = processInstanceService.queryParentInstanceBySubId(loginUser, projectCode, 1);
+        when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE)).thenReturn(result);
+        Map<String, Object> proejctAuthFailRes =
+                processInstanceService.queryParentInstanceBySubId(loginUser, projectCode, 1);
         Assert.assertEquals(Status.PROJECT_NOT_FOUND, proejctAuthFailRes.get(Constants.STATUS));
 
-        //process instance null
+        // process instance null
         putMsg(result, Status.SUCCESS, projectCode);
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(loginUser, project, projectCode,WORKFLOW_INSTANCE)).thenReturn(result);
+        when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_INSTANCE)).thenReturn(result);
         when(processService.findProcessInstanceDetailById(1)).thenReturn(null);
-        Map<String, Object> processInstanceNullRes = processInstanceService.queryParentInstanceBySubId(loginUser, projectCode, 1);
+        Map<String, Object> processInstanceNullRes =
+                processInstanceService.queryParentInstanceBySubId(loginUser, projectCode, 1);
         Assert.assertEquals(Status.PROCESS_INSTANCE_NOT_EXIST, processInstanceNullRes.get(Constants.STATUS));
 
-        //not sub process
+        // not sub process
         ProcessInstance processInstance = getProcessInstance();
         processInstance.setIsSubProcess(Flag.NO);
         putMsg(result, Status.SUCCESS, projectCode);
         when(processService.findProcessInstanceDetailById(1)).thenReturn(processInstance);
-        Map<String, Object> notSubProcessRes = processInstanceService.queryParentInstanceBySubId(loginUser, projectCode, 1);
+        Map<String, Object> notSubProcessRes =
+                processInstanceService.queryParentInstanceBySubId(loginUser, projectCode, 1);
         Assert.assertEquals(Status.PROCESS_INSTANCE_NOT_SUB_PROCESS_INSTANCE, notSubProcessRes.get(Constants.STATUS));
 
-        //sub process
+        // sub process
         processInstance.setIsSubProcess(Flag.YES);
         putMsg(result, Status.SUCCESS, projectCode);
         when(processService.findParentProcessInstance(1)).thenReturn(null);
-        Map<String, Object> subProcessNullRes = processInstanceService.queryParentInstanceBySubId(loginUser, projectCode, 1);
+        Map<String, Object> subProcessNullRes =
+                processInstanceService.queryParentInstanceBySubId(loginUser, projectCode, 1);
         Assert.assertEquals(Status.SUB_PROCESS_INSTANCE_NOT_EXIST, subProcessNullRes.get(Constants.STATUS));
 
-        //success
+        // success
         putMsg(result, Status.SUCCESS, projectCode);
         when(processService.findParentProcessInstance(1)).thenReturn(processInstance);
         Map<String, Object> successRes = processInstanceService.queryParentInstanceBySubId(loginUser, projectCode, 1);
@@ -522,22 +553,22 @@ public class ProcessInstanceServiceTest {
         Project project = getProject(projectCode);
         Map<String, Object> result = new HashMap<>();
         putMsg(result, Status.PROJECT_NOT_FOUND, projectCode);
-        //process instance null
+        // process instance null
         putMsg(result, Status.SUCCESS, projectCode);
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(loginUser, project, projectCode,INSTANCE_DELETE)).thenReturn(result);
+        when(projectService.checkProjectAndAuth(loginUser, project, projectCode, INSTANCE_DELETE)).thenReturn(result);
         when(processService.findProcessInstanceDetailById(1)).thenReturn(null);
     }
 
     @Test
     public void testViewVariables() {
-        //process instance not null
+        // process instance not null
         ProcessInstance processInstance = getProcessInstance();
         processInstance.setCommandType(CommandType.SCHEDULER);
         processInstance.setScheduleTime(new Date());
         processInstance.setGlobalParams("");
         when(processInstanceMapper.queryDetailById(1)).thenReturn(processInstance);
-        Map<String, Object> successRes = processInstanceService.viewVariables(1L,1);
+        Map<String, Object> successRes = processInstanceService.viewVariables(1L, 1);
         Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS));
     }
 
@@ -545,13 +576,12 @@ public class ProcessInstanceServiceTest {
     public void testViewGantt() throws Exception {
         ProcessInstance processInstance = getProcessInstance();
         TaskInstance taskInstance = getTaskInstance();
-        taskInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
+        taskInstance.setState(TaskExecutionStatus.RUNNING_EXECUTION);
         taskInstance.setStartTime(new Date());
         when(processInstanceMapper.queryDetailById(1)).thenReturn(processInstance);
         when(processDefinitionLogMapper.queryByDefinitionCodeAndVersion(
-            processInstance.getProcessDefinitionCode(),
-            processInstance.getProcessDefinitionVersion()
-        )).thenReturn(new ProcessDefinitionLog());
+                processInstance.getProcessDefinitionCode(),
+                processInstance.getProcessDefinitionVersion())).thenReturn(new ProcessDefinitionLog());
         when(processInstanceMapper.queryDetailById(1)).thenReturn(processInstance);
         when(taskInstanceMapper.queryByInstanceIdAndName(Mockito.anyInt(), Mockito.any())).thenReturn(taskInstance);
         DAG<String, TaskNode, TaskNodeRelation> graph = new DAG<>();
@@ -560,7 +590,7 @@ public class ProcessInstanceServiceTest {
         }
 
         when(processService.genDagGraph(Mockito.any(ProcessDefinition.class)))
-            .thenReturn(graph);
+                .thenReturn(graph);
 
         Map<String, Object> successRes = processInstanceService.viewGantt(0L, 1);
         Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS));
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 0ce57acf13..c748838bfc 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
@@ -40,7 +40,7 @@ import org.apache.dolphinscheduler.dao.entity.User;
 import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
 import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
 import org.apache.dolphinscheduler.dao.mapper.TaskInstanceMapper;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.service.process.ProcessService;
 
 import java.text.MessageFormat;
@@ -97,22 +97,22 @@ public class TaskInstanceServiceTest {
         Map<String, Object> result = new HashMap<>();
         putMsg(result, Status.PROJECT_NOT_FOUND, projectCode);
 
-        //project auth fail
+        // project auth fail
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
         when(projectService.checkProjectAndAuth(loginUser, project, projectCode, TASK_INSTANCE)).thenReturn(result);
         Result projectAuthFailRes = taskInstanceService.queryTaskListPaging(loginUser,
-            projectCode,
-            0,
-            "",
-            "",
-            "test_user",
-            "2019-02-26 19:48:00",
-            "2019-02-26 19:48:22",
-            "",
-            null,
-            "",
-            1,
-            20);
+                projectCode,
+                0,
+                "",
+                "",
+                "test_user",
+                "2019-02-26 19:48:00",
+                "2019-02-26 19:48:22",
+                "",
+                null,
+                "",
+                1,
+                20);
         Assert.assertEquals(Status.PROJECT_NOT_FOUND.getCode(), (int) projectAuthFailRes.getCode());
 
         // data parameter check
@@ -120,21 +120,21 @@ public class TaskInstanceServiceTest {
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
         when(projectService.checkProjectAndAuth(loginUser, project, projectCode, TASK_INSTANCE)).thenReturn(result);
         Result dataParameterRes = taskInstanceService.queryTaskListPaging(loginUser,
-            projectCode,
-            1,
-            "",
-            "",
-            "test_user",
-            "20200101 00:00:00",
-            "2020-01-02 00:00:00",
-            "",
-            ExecutionStatus.SUCCESS,
-            "192.168.xx.xx",
-            1,
-            20);
+                projectCode,
+                1,
+                "",
+                "",
+                "test_user",
+                "20200101 00:00:00",
+                "2020-01-02 00:00:00",
+                "",
+                TaskExecutionStatus.SUCCESS,
+                "192.168.xx.xx",
+                1,
+                20);
         Assert.assertEquals(Status.REQUEST_PARAMS_NOT_VALID_ERROR.getCode(), (int) dataParameterRes.getCode());
 
-        //project
+        // project
         putMsg(result, Status.SUCCESS, projectCode);
         Date start = DateUtils.stringToDate("2020-01-01 00:00:00");
         Date end = DateUtils.stringToDate("2020-01-02 00:00:00");
@@ -148,46 +148,54 @@ public class TaskInstanceServiceTest {
         when(projectService.checkProjectAndAuth(loginUser, project, projectCode, TASK_INSTANCE)).thenReturn(result);
         when(usersService.queryUser(loginUser.getId())).thenReturn(loginUser);
         when(usersService.getUserIdByName(loginUser.getUserName())).thenReturn(loginUser.getId());
-        when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1), eq(""), eq(""), eq(""),
+        when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1),
+                eq(""), eq(""), eq(""),
                 eq(0), Mockito.any(), eq("192.168.xx.xx"), eq(start), eq(end))).thenReturn(pageReturn);
         when(usersService.queryUser(processInstance.getExecutorId())).thenReturn(loginUser);
-        when(processService.findProcessInstanceDetailById(taskInstance.getProcessInstanceId())).thenReturn(processInstance);
+        when(processService.findProcessInstanceDetailById(taskInstance.getProcessInstanceId()))
+                .thenReturn(processInstance);
 
         Result successRes = taskInstanceService.queryTaskListPaging(loginUser, projectCode, 1, "", "",
-                "test_user", "2020-01-01 00:00:00", "2020-01-02 00:00:00", "", ExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20);
-        Assert.assertEquals(Status.SUCCESS.getCode(), (int)successRes.getCode());
+                "test_user", "2020-01-01 00:00:00", "2020-01-02 00:00:00", "", TaskExecutionStatus.SUCCESS,
+                "192.168.xx.xx", 1, 20);
+        Assert.assertEquals(Status.SUCCESS.getCode(), (int) successRes.getCode());
 
-        //executor name empty
-        when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1), eq(""), eq(""), eq(""),
+        // executor name empty
+        when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1),
+                eq(""), eq(""), eq(""),
                 eq(0), Mockito.any(), eq("192.168.xx.xx"), eq(start), eq(end))).thenReturn(pageReturn);
         Result executorEmptyRes = taskInstanceService.queryTaskListPaging(loginUser, projectCode, 1, "", "",
-                "", "2020-01-01 00:00:00", "2020-01-02 00:00:00", "", ExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20);
-        Assert.assertEquals(Status.SUCCESS.getCode(), (int)executorEmptyRes.getCode());
+                "", "2020-01-01 00:00:00", "2020-01-02 00:00:00", "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", 1,
+                20);
+        Assert.assertEquals(Status.SUCCESS.getCode(), (int) executorEmptyRes.getCode());
 
-        //executor null
+        // executor null
         when(usersService.queryUser(loginUser.getId())).thenReturn(null);
         when(usersService.getUserIdByName(loginUser.getUserName())).thenReturn(-1);
         Result executorNullRes = taskInstanceService.queryTaskListPaging(loginUser, projectCode, 1, "", "",
-                "test_user", "2020-01-01 00:00:00", "2020-01-02 00:00:00", "", ExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20);
-        Assert.assertEquals(Status.SUCCESS.getCode(),(int)executorNullRes.getCode());
+                "test_user", "2020-01-01 00:00:00", "2020-01-02 00:00:00", "", TaskExecutionStatus.SUCCESS,
+                "192.168.xx.xx", 1, 20);
+        Assert.assertEquals(Status.SUCCESS.getCode(), (int) executorNullRes.getCode());
 
-        //start/end date null
-        when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1), eq(""), eq(""), eq(""),
+        // start/end date null
+        when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1),
+                eq(""), eq(""), eq(""),
                 eq(0), Mockito.any(), eq("192.168.xx.xx"), any(), any())).thenReturn(pageReturn);
         Result executorNullDateRes = taskInstanceService.queryTaskListPaging(loginUser, projectCode, 1, "", "",
-                "", null, null, "", ExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20);
-        Assert.assertEquals(Status.SUCCESS.getCode(),(int) executorNullDateRes.getCode());
+                "", null, null, "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20);
+        Assert.assertEquals(Status.SUCCESS.getCode(), (int) executorNullDateRes.getCode());
 
-        //start date error format
-        when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1), eq(""), eq(""), eq(""),
+        // start date error format
+        when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1),
+                eq(""), eq(""), eq(""),
                 eq(0), Mockito.any(), eq("192.168.xx.xx"), any(), any())).thenReturn(pageReturn);
 
         Result executorErrorStartDateRes = taskInstanceService.queryTaskListPaging(loginUser, projectCode, 1, "", "",
-                "", "error date", null, "", ExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20);
-        Assert.assertEquals(Status.REQUEST_PARAMS_NOT_VALID_ERROR.getCode(), (int)executorErrorStartDateRes.getCode());
+                "", "error date", null, "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20);
+        Assert.assertEquals(Status.REQUEST_PARAMS_NOT_VALID_ERROR.getCode(), (int) executorErrorStartDateRes.getCode());
         Result executorErrorEndDateRes = taskInstanceService.queryTaskListPaging(loginUser, projectCode, 1, "", "",
-                "", null, "error date", "", ExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20);
-        Assert.assertEquals(Status.REQUEST_PARAMS_NOT_VALID_ERROR.getCode(), (int)executorErrorEndDateRes.getCode());
+                "", null, "error date", "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20);
+        Assert.assertEquals(Status.REQUEST_PARAMS_NOT_VALID_ERROR.getCode(), (int) executorErrorEndDateRes.getCode());
     }
 
     /**
@@ -272,12 +280,12 @@ public class TaskInstanceServiceTest {
         // user auth failed
         Map<String, Object> mockFailure = new HashMap<>(5);
         putMsg(mockFailure, Status.USER_NO_OPERATION_PROJECT_PERM, user.getUserName(), projectCode);
-        when(projectService.checkProjectAndAuth(user, project, projectCode,FORCED_SUCCESS)).thenReturn(mockFailure);
+        when(projectService.checkProjectAndAuth(user, project, projectCode, FORCED_SUCCESS)).thenReturn(mockFailure);
         Map<String, Object> authFailRes = taskInstanceService.forceTaskSuccess(user, projectCode, taskId);
         Assert.assertNotSame(Status.SUCCESS, authFailRes.get(Constants.STATUS));
 
         // test task not found
-        when(projectService.checkProjectAndAuth(user, project, projectCode,FORCED_SUCCESS)).thenReturn(mockSuccess);
+        when(projectService.checkProjectAndAuth(user, project, projectCode, FORCED_SUCCESS)).thenReturn(mockSuccess);
         when(taskInstanceMapper.selectById(Mockito.anyInt())).thenReturn(null);
         TaskDefinition taskDefinition = new TaskDefinition();
         taskDefinition.setProjectCode(projectCode);
@@ -286,30 +294,30 @@ public class TaskInstanceServiceTest {
         Assert.assertEquals(Status.TASK_INSTANCE_NOT_FOUND, taskNotFoundRes.get(Constants.STATUS));
 
         // test task instance state error
-        task.setState(ExecutionStatus.SUCCESS);
+        task.setState(TaskExecutionStatus.SUCCESS);
         when(taskInstanceMapper.selectById(1)).thenReturn(task);
         Map<String, Object> result = new HashMap<>();
         putMsg(result, Status.SUCCESS, projectCode);
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(user, project, projectCode,FORCED_SUCCESS)).thenReturn(result);
+        when(projectService.checkProjectAndAuth(user, project, projectCode, FORCED_SUCCESS)).thenReturn(result);
         Map<String, Object> taskStateErrorRes = taskInstanceService.forceTaskSuccess(user, projectCode, taskId);
         Assert.assertEquals(Status.TASK_INSTANCE_STATE_OPERATION_ERROR, taskStateErrorRes.get(Constants.STATUS));
 
         // test error
-        task.setState(ExecutionStatus.FAILURE);
+        task.setState(TaskExecutionStatus.FAILURE);
         when(taskInstanceMapper.updateById(task)).thenReturn(0);
         putMsg(result, Status.SUCCESS, projectCode);
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(user, project, projectCode,FORCED_SUCCESS)).thenReturn(result);
+        when(projectService.checkProjectAndAuth(user, project, projectCode, FORCED_SUCCESS)).thenReturn(result);
         Map<String, Object> errorRes = taskInstanceService.forceTaskSuccess(user, projectCode, taskId);
         Assert.assertEquals(Status.FORCE_TASK_SUCCESS_ERROR, errorRes.get(Constants.STATUS));
 
         // test success
-        task.setState(ExecutionStatus.FAILURE);
+        task.setState(TaskExecutionStatus.FAILURE);
         when(taskInstanceMapper.updateById(task)).thenReturn(1);
         putMsg(result, Status.SUCCESS, projectCode);
         when(projectMapper.queryByCode(projectCode)).thenReturn(project);
-        when(projectService.checkProjectAndAuth(user, project, projectCode,FORCED_SUCCESS)).thenReturn(result);
+        when(projectService.checkProjectAndAuth(user, project, projectCode, FORCED_SUCCESS)).thenReturn(result);
         Map<String, Object> successRes = taskInstanceService.forceTaskSuccess(user, projectCode, taskId);
         Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS));
     }
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java
index 70f7748f96..9faf69c70a 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java
+++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java
@@ -17,10 +17,10 @@
 
 package org.apache.dolphinscheduler.common;
 
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.SystemUtils;
+import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 
 import java.time.Duration;
 import java.util.regex.Pattern;
@@ -49,7 +49,8 @@ public final class Constants {
     public static final String REGISTRY_DOLPHINSCHEDULER_LOCK_MASTERS = "/lock/masters";
     public static final String REGISTRY_DOLPHINSCHEDULER_LOCK_FAILOVER_MASTERS = "/lock/failover/masters";
     public static final String REGISTRY_DOLPHINSCHEDULER_LOCK_FAILOVER_WORKERS = "/lock/failover/workers";
-    public static final String REGISTRY_DOLPHINSCHEDULER_LOCK_FAILOVER_STARTUP_MASTERS = "/lock/failover/startup-masters";
+    public static final String REGISTRY_DOLPHINSCHEDULER_LOCK_FAILOVER_STARTUP_MASTERS =
+            "/lock/failover/startup-masters";
 
     public static final String FORMAT_SS = "%s%s";
     public static final String FORMAT_S_S = "%s/%s";
@@ -71,7 +72,6 @@ public final class Constants {
      */
     public static final String FS_DEFAULT_FS = "resource.hdfs.fs.defaultFS";
 
-
     /**
      * hadoop configuration
      */
@@ -84,7 +84,6 @@ public final class Constants {
      */
     public static final String YARN_RESOURCEMANAGER_HA_RM_IDS = "yarn.resourcemanager.ha.rm.ids";
 
-
     /**
      * yarn.application.status.address
      */
@@ -127,7 +126,8 @@ public final class Constants {
      */
     public static final String RESOURCE_VIEW_SUFFIXES = "resource.view.suffixs";
 
-    public static final String RESOURCE_VIEW_SUFFIXES_DEFAULT_VALUE = "txt,log,sh,bat,conf,cfg,py,java,sql,xml,hql,properties,json,yml,yaml,ini,js";
+    public static final String RESOURCE_VIEW_SUFFIXES_DEFAULT_VALUE =
+            "txt,log,sh,bat,conf,cfg,py,java,sql,xml,hql,properties,json,yml,yaml,ini,js";
 
     /**
      * development.state
@@ -224,7 +224,6 @@ public final class Constants {
      */
     public static final int HTTP_CONNECT_TIMEOUT = 60 * 1000;
 
-
     /**
      * http connect request time out
      */
@@ -270,13 +269,11 @@ public final class Constants {
      */
     public static final int READ_PERMISSION = 2;
 
-
     /**
      * write permission
      */
     public static final int WRITE_PERMISSION = 2 * 2;
 
-
     /**
      * execute permission
      */
@@ -292,7 +289,6 @@ public final class Constants {
      */
     public static final int DEFAULT_HASH_MAP_SIZE = 16;
 
-
     /**
      * all permissions
      */
@@ -583,7 +579,6 @@ public final class Constants {
     public static final String DEPENDENT_SPLIT = ":||";
     public static final long DEPENDENT_ALL_TASK_CODE = 0;
 
-
     /**
      * preview schedule execute count
      */
@@ -617,7 +612,8 @@ public final class Constants {
     /**
      * hadoop.security.authentication
      */
-    public static final String HADOOP_SECURITY_AUTHENTICATION_STARTUP_STATE = "hadoop.security.authentication.startup.state";
+    public static final String HADOOP_SECURITY_AUTHENTICATION_STARTUP_STATE =
+            "hadoop.security.authentication.startup.state";
 
     /**
      * com.amazonaws.services.s3.enableV4
@@ -642,23 +638,21 @@ public final class Constants {
      */
     public static final String TASK_LOG_INFO_FORMAT = "TaskLogInfo-%s";
 
-    public static final int[] NOT_TERMINATED_STATES = new int[] {
-        ExecutionStatus.SUBMITTED_SUCCESS.ordinal(),
-        ExecutionStatus.DISPATCH.ordinal(),
-        ExecutionStatus.RUNNING_EXECUTION.ordinal(),
-        ExecutionStatus.DELAY_EXECUTION.ordinal(),
-        ExecutionStatus.READY_PAUSE.ordinal(),
-        ExecutionStatus.READY_STOP.ordinal(),
-        ExecutionStatus.NEED_FAULT_TOLERANCE.ordinal(),
-        ExecutionStatus.WAITING_THREAD.ordinal(),
-        ExecutionStatus.WAITING_DEPEND.ordinal()
+    public static final int[] NOT_TERMINATED_STATES = new int[]{
+            WorkflowExecutionStatus.SUBMITTED_SUCCESS.ordinal(),
+            TaskExecutionStatus.DISPATCH.ordinal(),
+            WorkflowExecutionStatus.RUNNING_EXECUTION.ordinal(),
+            WorkflowExecutionStatus.DELAY_EXECUTION.ordinal(),
+            WorkflowExecutionStatus.READY_PAUSE.ordinal(),
+            WorkflowExecutionStatus.READY_STOP.ordinal(),
+            TaskExecutionStatus.NEED_FAULT_TOLERANCE.ordinal(),
     };
 
-    public static final int[] RUNNING_PROCESS_STATE = new int[] {
-        ExecutionStatus.RUNNING_EXECUTION.ordinal(),
-        ExecutionStatus.SUBMITTED_SUCCESS.ordinal(),
-        ExecutionStatus.DISPATCH.ordinal(),
-        ExecutionStatus.SERIAL_WAIT.ordinal()
+    public static final int[] RUNNING_PROCESS_STATE = new int[]{
+            TaskExecutionStatus.RUNNING_EXECUTION.ordinal(),
+            TaskExecutionStatus.SUBMITTED_SUCCESS.ordinal(),
+            TaskExecutionStatus.DISPATCH.ordinal(),
+            WorkflowExecutionStatus.SERIAL_WAIT.ordinal()
     };
 
     /**
@@ -686,7 +680,6 @@ public final class Constants {
      */
     public static final String PAGE_NUMBER = "pageNo";
 
-
     /**
      *
      */
@@ -741,7 +734,8 @@ public final class Constants {
     /**
      * dataSource sensitive param
      */
-    public static final String DATASOURCE_PASSWORD_REGEX = "(?<=((?i)password((\\\\\":\\\\\")|(=')))).*?(?=((\\\\\")|(')))";
+    public static final String DATASOURCE_PASSWORD_REGEX =
+            "(?<=((?i)password((\\\\\":\\\\\")|(=')))).*?(?=((\\\\\")|(')))";
 
     /**
      * default worker group
@@ -779,12 +773,14 @@ public final class Constants {
     /**
      * network interface preferred
      */
-    public static final String DOLPHIN_SCHEDULER_NETWORK_INTERFACE_PREFERRED = "dolphin.scheduler.network.interface.preferred";
+    public static final String DOLPHIN_SCHEDULER_NETWORK_INTERFACE_PREFERRED =
+            "dolphin.scheduler.network.interface.preferred";
 
     /**
      * network IP gets priority, default inner outer
      */
-    public static final String DOLPHIN_SCHEDULER_NETWORK_PRIORITY_STRATEGY = "dolphin.scheduler.network.priority.strategy";
+    public static final String DOLPHIN_SCHEDULER_NETWORK_PRIORITY_STRATEGY =
+            "dolphin.scheduler.network.priority.strategy";
 
     /**
      * exec shell scripts
@@ -796,7 +792,8 @@ public final class Constants {
      */
     public static final String PSTREE = "pstree";
 
-    public static final boolean KUBERNETES_MODE = !StringUtils.isEmpty(System.getenv("KUBERNETES_SERVICE_HOST")) && !StringUtils.isEmpty(System.getenv("KUBERNETES_SERVICE_PORT"));
+    public static final boolean KUBERNETES_MODE = !StringUtils.isEmpty(System.getenv("KUBERNETES_SERVICE_HOST"))
+            && !StringUtils.isEmpty(System.getenv("KUBERNETES_SERVICE_PORT"));
 
     /**
      * dry run flag
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskStateType.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskStateType.java
deleted file mode 100644
index 482cc658aa..0000000000
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskStateType.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.dolphinscheduler.common.enums;
-
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
-
-/**
- * type of task state
- */
-public enum TaskStateType {
-    /**
-     * 0 waiting running
-     * 1 running
-     * 2 finish
-     * 3 failed
-     * 4 success
-     */
-    WAITTING, RUNNING, FINISH, FAILED, SUCCESS;
-
-    /**
-     * convert task state to execute status integer array ;
-     *
-     * @param taskStateType task state type
-     * @return result of execution status
-     */
-    public static int[] convert2ExecutStatusIntArray(TaskStateType taskStateType) {
-
-        switch (taskStateType) {
-            case SUCCESS:
-                return new int[]{ExecutionStatus.SUCCESS.ordinal()};
-            case FAILED:
-                return new int[]{
-                        ExecutionStatus.FAILURE.ordinal(),
-                        ExecutionStatus.NEED_FAULT_TOLERANCE.ordinal()};
-            case FINISH:
-                return new int[]{
-                        ExecutionStatus.PAUSE.ordinal(),
-                        ExecutionStatus.STOP.ordinal()
-                };
-            case RUNNING:
-                return new int[]{ExecutionStatus.SUBMITTED_SUCCESS.ordinal(),
-                        ExecutionStatus.DISPATCH.ordinal(),
-                        ExecutionStatus.RUNNING_EXECUTION.ordinal(),
-                        ExecutionStatus.DELAY_EXECUTION.ordinal(),
-                        ExecutionStatus.READY_PAUSE.ordinal(),
-                        ExecutionStatus.READY_STOP.ordinal()};
-            case WAITTING:
-                return new int[]{
-                        ExecutionStatus.SUBMITTED_SUCCESS.ordinal()
-                };
-            default:
-                break;
-        }
-        return new int[0];
-    }
-
-}
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/WorkflowExecutionStatus.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/WorkflowExecutionStatus.java
new file mode 100644
index 0000000000..67be58c7e6
--- /dev/null
+++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/WorkflowExecutionStatus.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.common.enums;
+
+import com.baomidou.mybatisplus.annotation.EnumValue;
+import lombok.NonNull;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public enum WorkflowExecutionStatus {
+    // This class is split from <code>ExecutionStatus</code> #11339.
+    // In order to compatible with the old value, the code is not consecutive
+    SUBMITTED_SUCCESS(0, "submit success"),
+    RUNNING_EXECUTION(1, "running"),
+    READY_PAUSE(2, "ready pause"),
+    PAUSE(3, "pause"),
+    READY_STOP(4, "ready stop"),
+    STOP(5, "stop"),
+    FAILURE(6, "failure"),
+    SUCCESS(7, "success"),
+    DELAY_EXECUTION(12, "delay execution"),
+    SERIAL_WAIT(14, "serial wait"),
+    READY_BLOCK(15, "ready block"),
+    BLOCK(16, "block"),
+    ;
+
+    private static final Map<Integer, WorkflowExecutionStatus> CODE_MAP = new HashMap<>();
+    private static final int[] NEED_FAILOVER_STATES = new int[]{
+            SUBMITTED_SUCCESS.getCode(),
+            RUNNING_EXECUTION.getCode(),
+            DELAY_EXECUTION.getCode(),
+            READY_PAUSE.getCode(),
+            READY_STOP.getCode()
+    };
+
+    static {
+        for (WorkflowExecutionStatus executionStatus : WorkflowExecutionStatus.values()) {
+            CODE_MAP.put(executionStatus.getCode(), executionStatus);
+        }
+    }
+
+    /**
+     * Get <code>WorkflowExecutionStatus</code> by code, if the code is invalidated will throw {@link IllegalArgumentException}.
+     */
+    public static @NonNull WorkflowExecutionStatus of(int code) {
+        WorkflowExecutionStatus workflowExecutionStatus = CODE_MAP.get(code);
+        if (workflowExecutionStatus == null) {
+            throw new IllegalArgumentException(String.format("The workflow execution status code: %s is invalidated",
+                    code));
+        }
+        return workflowExecutionStatus;
+    }
+
+    public boolean isRunning() {
+        return this == RUNNING_EXECUTION;
+    }
+
+    public boolean isFinished() {
+        // todo: do we need to remove pause/block in finished judge?
+        return isSuccess() || isFailure() || isStop() || isPause() || isBlock();
+    }
+
+    /**
+     * status is success
+     *
+     * @return status
+     */
+    public boolean isSuccess() {
+        return this == SUCCESS;
+    }
+
+    public boolean isFailure() {
+        return this == FAILURE;
+    }
+
+    public boolean isPause() {
+        return this == PAUSE;
+    }
+
+    public boolean isReadyStop() {
+        return this == READY_STOP;
+    }
+
+    public boolean isStop() {
+        return this == STOP;
+    }
+
+    public boolean isBlock() {
+        return this == BLOCK;
+    }
+
+    public static int[] getNeedFailoverWorkflowInstanceState() {
+        return NEED_FAILOVER_STATES;
+    }
+
+    @EnumValue
+    private final int code;
+
+    private final String desc;
+
+    WorkflowExecutionStatus(int code, String desc) {
+        this.code = code;
+        this.desc = desc;
+    }
+
+    public int getCode() {
+        return code;
+    }
+
+    public String getDesc() {
+        return desc;
+    }
+
+    @Override
+    public String toString() {
+        return "WorkflowExecutionStatus{" + "code=" + code + ", desc='" + desc + '\'' + '}';
+    }
+}
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/HadoopUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/HadoopUtils.java
index b1ad942ac2..8d7e4c2a6b 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/HadoopUtils.java
+++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/HadoopUtils.java
@@ -26,7 +26,7 @@ import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.ResUploadType;
 import org.apache.dolphinscheduler.common.exception.BaseException;
 import org.apache.dolphinscheduler.common.storage.StorageOperate;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.spi.enums.ResourceType;
 
 import org.apache.commons.io.IOUtils;
@@ -74,13 +74,15 @@ public class HadoopUtils implements Closeable, StorageOperate {
     public static final String RM_HA_IDS = PropertyUtils.getString(Constants.YARN_RESOURCEMANAGER_HA_RM_IDS);
     public static final String APP_ADDRESS = PropertyUtils.getString(Constants.YARN_APPLICATION_STATUS_ADDRESS);
     public static final String JOB_HISTORY_ADDRESS = PropertyUtils.getString(Constants.YARN_JOB_HISTORY_STATUS_ADDRESS);
-    public static final int HADOOP_RESOURCE_MANAGER_HTTP_ADDRESS_PORT_VALUE = PropertyUtils.getInt(Constants.HADOOP_RESOURCE_MANAGER_HTTPADDRESS_PORT, 8088);
+    public static final int HADOOP_RESOURCE_MANAGER_HTTP_ADDRESS_PORT_VALUE =
+            PropertyUtils.getInt(Constants.HADOOP_RESOURCE_MANAGER_HTTPADDRESS_PORT, 8088);
     private static final String HADOOP_UTILS_KEY = "HADOOP_UTILS_KEY";
 
     private static final LoadingCache<String, HadoopUtils> cache = CacheBuilder
             .newBuilder()
             .expireAfterWrite(PropertyUtils.getInt(Constants.KERBEROS_EXPIRE_TIME, 2), TimeUnit.HOURS)
             .build(new CacheLoader<String, HadoopUtils>() {
+
                 @Override
                 public HadoopUtils load(String key) throws Exception {
                     return new HadoopUtils();
@@ -134,7 +136,7 @@ public class HadoopUtils implements Closeable, StorageOperate {
                 defaultFS = PropertyUtils.getString(Constants.FS_DEFAULT_FS);
             }
 
-            //first get key from core-site.xml hdfs-site.xml ,if null ,then try to get from properties file
+            // first get key from core-site.xml hdfs-site.xml ,if null ,then try to get from properties file
             // the default is the local file system
             if (StringUtils.isNotBlank(defaultFS)) {
                 Map<String, String> fsRelatedProps = PropertyUtils.getPrefixedProperties("fs.");
@@ -143,12 +145,12 @@ public class HadoopUtils implements Closeable, StorageOperate {
             } else {
                 logger.error("property:{} can not to be empty, please set!", Constants.FS_DEFAULT_FS);
                 throw new NullPointerException(
-                        String.format("property: %s can not to be empty, please set!", Constants.FS_DEFAULT_FS)
-                );
+                        String.format("property: %s can not to be empty, please set!", Constants.FS_DEFAULT_FS));
             }
 
             if (!defaultFS.startsWith("file")) {
-                logger.info("get property:{} -> {}, from core-site.xml hdfs-site.xml ", Constants.FS_DEFAULT_FS, defaultFS);
+                logger.info("get property:{} -> {}, from core-site.xml hdfs-site.xml ", Constants.FS_DEFAULT_FS,
+                        defaultFS);
             }
 
             if (StringUtils.isNotEmpty(hdfsUser)) {
@@ -203,7 +205,7 @@ public class HadoopUtils implements Closeable, StorageOperate {
     }
 
     public String getJobHistoryUrl(String applicationId) {
-        //eg:application_1587475402360_712719 -> job_1587475402360_712719
+        // eg:application_1587475402360_712719 -> job_1587475402360_712719
         String jobId = applicationId.replace("application", "job");
         return String.format(JOB_HISTORY_ADDRESS, jobId);
     }
@@ -251,7 +253,8 @@ public class HadoopUtils implements Closeable, StorageOperate {
     }
 
     @Override
-    public List<String> vimFile(String bucketName, String hdfsFilePath, int skipLineNums, int limit) throws IOException {
+    public List<String> vimFile(String bucketName, String hdfsFilePath, int skipLineNums,
+                                int limit) throws IOException {
         return catFile(hdfsFilePath, skipLineNums, limit);
     }
 
@@ -296,7 +299,8 @@ public class HadoopUtils implements Closeable, StorageOperate {
     }
 
     @Override
-    public void download(String bucketName, String srcHdfsFilePath, String dstFile, boolean deleteSource, boolean overwrite) throws IOException {
+    public void download(String bucketName, String srcHdfsFilePath, String dstFile, boolean deleteSource,
+                         boolean overwrite) throws IOException {
         copyHdfsToLocal(srcHdfsFilePath, dstFile, deleteSource, overwrite);
     }
 
@@ -326,7 +330,8 @@ public class HadoopUtils implements Closeable, StorageOperate {
      * @return if success or not
      * @throws IOException errors
      */
-    public boolean copyLocalToHdfs(String srcFile, String dstHdfsPath, boolean deleteSource, boolean overwrite) throws IOException {
+    public boolean copyLocalToHdfs(String srcFile, String dstHdfsPath, boolean deleteSource,
+                                   boolean overwrite) throws IOException {
         Path srcPath = new Path(srcFile);
         Path dstPath = new Path(dstHdfsPath);
 
@@ -336,7 +341,8 @@ public class HadoopUtils implements Closeable, StorageOperate {
     }
 
     @Override
-    public boolean upload(String buckName, String srcFile, String dstPath, boolean deleteSource, boolean overwrite) throws IOException {
+    public boolean upload(String buckName, String srcFile, String dstPath, boolean deleteSource,
+                          boolean overwrite) throws IOException {
         return copyLocalToHdfs(srcFile, dstPath, deleteSource, overwrite);
     }
 
@@ -344,13 +350,19 @@ public class HadoopUtils implements Closeable, StorageOperate {
      * copy hdfs file to local
      *
      * @param srcHdfsFilePath source hdfs file path
-     * @param dstFile         destination file
-     * @param deleteSource    delete source
-     * @param overwrite       overwrite
+     * 
+     * @param dstFile destination file
+     * 
+     * @param deleteSource delete source
+     * 
+     * @param overwrite overwrite
+     * 
      * @return result of copy hdfs file to local
+     * 
      * @throws IOException errors
      */
-    public boolean copyHdfsToLocal(String srcHdfsFilePath, String dstFile, boolean deleteSource, boolean overwrite) throws IOException {
+    public boolean copyHdfsToLocal(String srcHdfsFilePath, String dstFile, boolean deleteSource,
+                                   boolean overwrite) throws IOException {
         Path srcPath = new Path(srcHdfsFilePath);
         File dstPath = new File(dstFile);
 
@@ -442,7 +454,7 @@ public class HadoopUtils implements Closeable, StorageOperate {
      * @param applicationId application id
      * @return the return may be null or there may be other parse exceptions
      */
-    public ExecutionStatus getApplicationStatus(String applicationId) throws BaseException {
+    public TaskExecutionStatus getApplicationStatus(String applicationId) throws BaseException {
         if (StringUtils.isEmpty(applicationId)) {
             return null;
         }
@@ -451,51 +463,57 @@ public class HadoopUtils implements Closeable, StorageOperate {
         String applicationUrl = getApplicationUrl(applicationId);
         logger.debug("generate yarn application url, applicationUrl={}", applicationUrl);
 
-        String responseContent = Boolean.TRUE.equals(PropertyUtils.getBoolean(Constants.HADOOP_SECURITY_AUTHENTICATION_STARTUP_STATE, false)) ? KerberosHttpClient.get(applicationUrl) : HttpUtils.get(applicationUrl);
+        String responseContent = Boolean.TRUE
+                .equals(PropertyUtils.getBoolean(Constants.HADOOP_SECURITY_AUTHENTICATION_STARTUP_STATE, false))
+                        ? KerberosHttpClient.get(applicationUrl)
+                        : HttpUtils.get(applicationUrl);
         if (responseContent != null) {
             ObjectNode jsonObject = JSONUtils.parseObject(responseContent);
             if (!jsonObject.has("app")) {
-                return ExecutionStatus.FAILURE;
+                return TaskExecutionStatus.FAILURE;
             }
             result = jsonObject.path("app").path("finalStatus").asText();
 
         } else {
-            //may be in job history
+            // may be in job history
             String jobHistoryUrl = getJobHistoryUrl(applicationId);
             logger.debug("generate yarn job history application url, jobHistoryUrl={}", jobHistoryUrl);
-            responseContent = Boolean.TRUE.equals(PropertyUtils.getBoolean(Constants.HADOOP_SECURITY_AUTHENTICATION_STARTUP_STATE, false)) ? KerberosHttpClient.get(jobHistoryUrl) : HttpUtils.get(jobHistoryUrl);
+            responseContent = Boolean.TRUE
+                    .equals(PropertyUtils.getBoolean(Constants.HADOOP_SECURITY_AUTHENTICATION_STARTUP_STATE, false))
+                            ? KerberosHttpClient.get(jobHistoryUrl)
+                            : HttpUtils.get(jobHistoryUrl);
 
             if (null != responseContent) {
                 ObjectNode jsonObject = JSONUtils.parseObject(responseContent);
                 if (!jsonObject.has("job")) {
-                    return ExecutionStatus.FAILURE;
+                    return TaskExecutionStatus.FAILURE;
                 }
                 result = jsonObject.path("job").path("state").asText();
             } else {
-                return ExecutionStatus.FAILURE;
+                return TaskExecutionStatus.FAILURE;
             }
         }
 
         return getExecutionStatus(result);
     }
 
-    private ExecutionStatus getExecutionStatus(String result) {
+    private TaskExecutionStatus getExecutionStatus(String result) {
         switch (result) {
             case Constants.ACCEPTED:
-                return ExecutionStatus.SUBMITTED_SUCCESS;
+                return TaskExecutionStatus.SUBMITTED_SUCCESS;
             case Constants.SUCCEEDED:
             case Constants.ENDED:
-                return ExecutionStatus.SUCCESS;
+                return TaskExecutionStatus.SUCCESS;
             case Constants.NEW:
             case Constants.NEW_SAVING:
             case Constants.SUBMITTED:
             case Constants.FAILED:
-                return ExecutionStatus.FAILURE;
+                return TaskExecutionStatus.FAILURE;
             case Constants.KILLED:
-                return ExecutionStatus.KILL;
+                return TaskExecutionStatus.KILL;
             case Constants.RUNNING:
             default:
-                return ExecutionStatus.RUNNING_EXECUTION;
+                return TaskExecutionStatus.RUNNING_EXECUTION;
         }
     }
 
@@ -534,7 +552,6 @@ public class HadoopUtils implements Closeable, StorageOperate {
         return getHdfsDir(resourceType, tenantCode);
     }
 
-
     /**
      * hdfs resource dir
      *
@@ -615,7 +632,6 @@ public class HadoopUtils implements Closeable, StorageOperate {
      */
     public static String getAppAddress(String appAddress, String rmHa) {
 
-
         String[] split1 = appAddress.split(Constants.DOUBLE_SLASH);
 
         if (split1.length != 2) {
@@ -631,7 +647,7 @@ public class HadoopUtils implements Closeable, StorageOperate {
 
         String end = Constants.COLON + split2[1];
 
-        //get active ResourceManager
+        // get active ResourceManager
         String activeRM = YarnHAAdminUtils.getActiveRMName(start, rmHa);
 
         if (StringUtils.isEmpty(activeRM)) {
@@ -694,15 +710,18 @@ public class HadoopUtils implements Closeable, StorageOperate {
          */
         public static String getRMState(String url) {
 
-            String retStr = Boolean.TRUE.equals(PropertyUtils.getBoolean(Constants.HADOOP_SECURITY_AUTHENTICATION_STARTUP_STATE, false)) ? KerberosHttpClient.get(url) : HttpUtils.get(url);
+            String retStr = Boolean.TRUE
+                    .equals(PropertyUtils.getBoolean(Constants.HADOOP_SECURITY_AUTHENTICATION_STARTUP_STATE, false))
+                            ? KerberosHttpClient.get(url)
+                            : HttpUtils.get(url);
 
             if (StringUtils.isEmpty(retStr)) {
                 return null;
             }
-            //to json
+            // to json
             ObjectNode jsonObject = JSONUtils.parseObject(retStr);
 
-            //get ResourceManager state
+            // get ResourceManager state
             if (!jsonObject.has("clusterInfo")) {
                 return null;
             }
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/AlertDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/AlertDao.java
index 018cea4827..d8a6730fb6 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/AlertDao.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/AlertDao.java
@@ -109,7 +109,7 @@ public class AlertDao {
      * @param alert alert
      * @return sign's str
      */
-    private String generateSign (Alert alert) {
+    private String generateSign(Alert alert) {
         return Optional.of(alert)
                 .map(Alert::getContent)
                 .map(DigestUtils::sha1Hex)
@@ -144,12 +144,10 @@ public class AlertDao {
      * @param serverType serverType
      */
     public void sendServerStoppedAlert(int alertGroupId, String host, String serverType) {
-        ServerAlertContent serverStopAlertContent = ServerAlertContent.newBuilder().
-                type(serverType)
+        ServerAlertContent serverStopAlertContent = ServerAlertContent.newBuilder().type(serverType)
                 .host(host)
                 .event(AlertEvent.SERVER_DOWN)
-                .warningLevel(AlertWarnLevel.SERIOUS).
-                build();
+                .warningLevel(AlertWarnLevel.SERIOUS).build();
         String content = JSONUtils.toJsonString(Lists.newArrayList(serverStopAlertContent));
 
         Alert alert = new Alert();
@@ -164,7 +162,8 @@ public class AlertDao {
         alert.setSign(generateSign(alert));
         // we use this method to avoid insert duplicate alert(issue #5525)
         // we modified this method to optimize performance(issue #9174)
-        Date crashAlarmSuppressionStartTime = Date.from(LocalDateTime.now().plusMinutes(-crashAlarmSuppression).atZone(ZoneId.systemDefault()).toInstant());
+        Date crashAlarmSuppressionStartTime = Date.from(
+                LocalDateTime.now().plusMinutes(-crashAlarmSuppression).atZone(ZoneId.systemDefault()).toInstant());
         alertMapper.insertAlertWhenServerCrash(alert, crashAlarmSuppressionStartTime);
     }
 
@@ -178,7 +177,7 @@ public class AlertDao {
         int alertGroupId = processInstance.getWarningGroupId();
         Alert alert = new Alert();
         List<ProcessAlertContent> processAlertContentList = new ArrayList<>(1);
-        ProcessAlertContent processAlertContent = ProcessAlertContent.newBuilder()
+        ProcessAlertContent processAlertContent = ProcessAlertContent.builder()
                 .projectCode(projectUser.getProjectCode())
                 .projectName(projectUser.getProjectName())
                 .owner(projectUser.getUserName())
@@ -191,7 +190,7 @@ public class AlertDao {
                 .processStartTime(processInstance.getStartTime())
                 .processHost(processInstance.getHost())
                 .event(AlertEvent.TIME_OUT)
-                .warningLevel(AlertWarnLevel.MIDDLE)
+                .warnLevel(AlertWarnLevel.MIDDLE)
                 .build();
         processAlertContentList.add(processAlertContent);
         String content = JSONUtils.toJsonString(processAlertContentList);
@@ -221,10 +220,11 @@ public class AlertDao {
      * @param taskInstance taskInstance
      * @param projectUser projectUser
      */
-    public void sendTaskTimeoutAlert(ProcessInstance processInstance, TaskInstance taskInstance, ProjectUser projectUser) {
+    public void sendTaskTimeoutAlert(ProcessInstance processInstance, TaskInstance taskInstance,
+                                     ProjectUser projectUser) {
         Alert alert = new Alert();
         List<ProcessAlertContent> processAlertContentList = new ArrayList<>(1);
-        ProcessAlertContent processAlertContent = ProcessAlertContent.newBuilder()
+        ProcessAlertContent processAlertContent = ProcessAlertContent.builder()
                 .projectCode(projectUser.getProjectCode())
                 .projectName(projectUser.getProjectName())
                 .owner(projectUser.getUserName())
@@ -237,7 +237,7 @@ public class AlertDao {
                 .taskStartTime(taskInstance.getStartTime())
                 .taskHost(taskInstance.getHost())
                 .event(AlertEvent.TIME_OUT)
-                .warningLevel(AlertWarnLevel.MIDDLE)
+                .warnLevel(AlertWarnLevel.MIDDLE)
                 .build();
         processAlertContentList.add(processAlertContent);
         String content = JSONUtils.toJsonString(processAlertContentList);
@@ -254,13 +254,13 @@ public class AlertDao {
      */
     public List<Alert> listPendingAlerts() {
         LambdaQueryWrapper<Alert> wrapper = new QueryWrapper<>(new Alert()).lambda()
-            .eq(Alert::getAlertStatus, AlertStatus.WAIT_EXECUTION);
+                .eq(Alert::getAlertStatus, AlertStatus.WAIT_EXECUTION);
         return alertMapper.selectList(wrapper);
     }
 
     public List<Alert> listAlerts(int processInstanceId) {
         LambdaQueryWrapper<Alert> wrapper = new QueryWrapper<>(new Alert()).lambda()
-            .eq(Alert::getProcessInstanceId, processInstanceId);
+                .eq(Alert::getProcessInstanceId, processInstanceId);
         return alertMapper.selectList(wrapper);
     }
 
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ExecuteStatusCount.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ExecuteStatusCount.java
index 4452156b55..ffd45e49fa 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ExecuteStatusCount.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ExecuteStatusCount.java
@@ -17,45 +17,23 @@
 
 package org.apache.dolphinscheduler.dao.entity;
 
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
-
-/**
- * count execute state
- *
- */
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
+
+@Data
+@NoArgsConstructor
+@AllArgsConstructor
 public class ExecuteStatusCount {
 
     /**
      * execution state
      */
-    private ExecutionStatus state;
+    private TaskExecutionStatus state;
 
     /**
      * count for state
      */
     private int count;
-
-    public ExecutionStatus getExecutionStatus() {
-        return state;
-    }
-
-    public void setExecutionStatus(ExecutionStatus executionStatus) {
-        this.state = executionStatus;
-    }
-
-    public int getCount() {
-        return count;
-    }
-
-    public void setCount(int count) {
-        this.count = count;
-    }
-
-    @Override
-    public String toString() {
-        return "ExecuteStatusCount{"
-                + "state=" + state
-                + ", count=" + count
-                + '}';
-    }
 }
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessAlertContent.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessAlertContent.java
index fc7129a2f6..22dab95234 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessAlertContent.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessAlertContent.java
@@ -17,19 +17,23 @@
 
 package org.apache.dolphinscheduler.dao.entity;
 
-import org.apache.dolphinscheduler.common.enums.AlertEvent;
-import org.apache.dolphinscheduler.common.enums.AlertWarnLevel;
-import org.apache.dolphinscheduler.common.enums.CommandType;
-import org.apache.dolphinscheduler.common.enums.Flag;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
-
-import java.io.Serializable;
-import java.util.Date;
-
 import com.fasterxml.jackson.annotation.JsonInclude;
 import com.fasterxml.jackson.annotation.JsonInclude.Include;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+import org.apache.dolphinscheduler.common.enums.*;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 
+import java.io.Serializable;
+import java.util.Date;
+
+@Data
+@AllArgsConstructor
+@NoArgsConstructor
+@Builder
 @JsonInclude(Include.NON_NULL)
 public class ProcessAlertContent implements Serializable {
 
@@ -50,7 +54,7 @@ public class ProcessAlertContent implements Serializable {
     @JsonProperty("processType")
     private CommandType processType;
     @JsonProperty("processState")
-    private ExecutionStatus processState;
+    private WorkflowExecutionStatus processState;
     @JsonProperty("recovery")
     private Flag recovery;
     @JsonProperty("runTimes")
@@ -74,7 +78,7 @@ public class ProcessAlertContent implements Serializable {
     @JsonProperty("retryTimes")
     private Integer retryTimes;
     @JsonProperty("taskState")
-    private ExecutionStatus taskState;
+    private TaskExecutionStatus taskState;
     @JsonProperty("taskStartTime")
     private Date taskStartTime;
     @JsonProperty("taskEndTime")
@@ -84,193 +88,4 @@ public class ProcessAlertContent implements Serializable {
     @JsonProperty("logPath")
     private String logPath;
 
-    private ProcessAlertContent(Builder builder) {
-        this.projectId = builder.projectId;
-        this.projectCode = builder.projectCode;
-        this.projectName = builder.projectName;
-        this.owner = builder.owner;
-        this.processId = builder.processId;
-        this.processDefinitionCode = builder.processDefinitionCode;
-        this.processName = builder.processName;
-        this.processType = builder.processType;
-        this.recovery = builder.recovery;
-        this.processState = builder.processState;
-        this.runTimes = builder.runTimes;
-        this.processStartTime = builder.processStartTime;
-        this.processEndTime = builder.processEndTime;
-        this.processHost = builder.processHost;
-        this.taskCode = builder.taskCode;
-        this.taskName = builder.taskName;
-        this.event = builder.event;
-        this.warnLevel = builder.warnLevel;
-        this.taskType = builder.taskType;
-        this.taskState = builder.taskState;
-        this.taskStartTime = builder.taskStartTime;
-        this.taskEndTime = builder.taskEndTime;
-        this.taskHost = builder.taskHost;
-        this.logPath = builder.logPath;
-        this.retryTimes = builder.retryTimes;
-
-    }
-
-    public static Builder newBuilder() {
-        return new Builder();
-    }
-
-    public static class Builder {
-        private Integer projectId;
-        private Long projectCode;
-        private String projectName;
-        private String owner;
-        private Integer processId;
-        private Long processDefinitionCode;
-        private String processName;
-        private CommandType processType;
-        private Flag recovery;
-        private ExecutionStatus processState;
-        private Integer runTimes;
-        private Date processStartTime;
-        private Date processEndTime;
-        private String processHost;
-        private Long taskCode;
-        private String taskName;
-        private AlertEvent event;
-        private AlertWarnLevel warnLevel;
-        private String taskType;
-        private Integer retryTimes;
-        private ExecutionStatus taskState;
-        private Date taskStartTime;
-        private Date taskEndTime;
-        private String taskHost;
-        private String logPath;
-
-        public Builder projectId(Integer projectId) {
-            this.projectId = projectId;
-            return this;
-        }
-
-        public Builder projectCode(Long projectCode) {
-            this.projectCode = projectCode;
-            return this;
-        }
-
-        public Builder projectName(String projectName) {
-            this.projectName = projectName;
-            return this;
-        }
-
-        public Builder owner(String owner) {
-            this.owner = owner;
-            return this;
-        }
-
-        public Builder processId(Integer processId) {
-            this.processId = processId;
-            return this;
-        }
-
-        public Builder processDefinitionCode(Long processDefinitionCode) {
-            this.processDefinitionCode = processDefinitionCode;
-            return this;
-        }
-
-        public Builder processName(String processName) {
-            this.processName = processName;
-            return this;
-        }
-
-        public Builder processType(CommandType processType) {
-            this.processType = processType;
-            return this;
-        }
-
-        public Builder recovery(Flag recovery) {
-            this.recovery = recovery;
-            return this;
-        }
-
-        public Builder processState(ExecutionStatus processState) {
-            this.processState = processState;
-            return this;
-        }
-
-        public Builder runTimes(Integer runTimes) {
-            this.runTimes = runTimes;
-            return this;
-        }
-
-        public Builder processStartTime(Date processStartTime) {
-            this.processStartTime = processStartTime;
-            return this;
-        }
-
-        public Builder processEndTime(Date processEndTime) {
-            this.processEndTime = processEndTime;
-            return this;
-        }
-
-        public Builder processHost(String processHost) {
-            this.processHost = processHost;
-            return this;
-        }
-
-        public Builder taskCode(Long taskCode) {
-            this.taskCode = taskCode;
-            return this;
-        }
-
-        public Builder taskName(String taskName) {
-            this.taskName = taskName;
-            return this;
-        }
-
-        public Builder event(AlertEvent event) {
-            this.event = event;
-            return this;
-        }
-
-        public Builder warningLevel(AlertWarnLevel warnLevel) {
-            this.warnLevel = warnLevel;
-            return this;
-        }
-
-        public Builder taskType(String taskType) {
-            this.taskType = taskType;
-            return this;
-        }
-
-        public Builder retryTimes(Integer retryTimes) {
-            this.retryTimes = retryTimes;
-            return this;
-        }
-
-        public Builder taskState(ExecutionStatus taskState) {
-            this.taskState = taskState;
-            return this;
-        }
-
-        public Builder taskStartTime(Date taskStartTime) {
-            this.taskStartTime = taskStartTime;
-            return this;
-        }
-
-        public Builder taskEndTime(Date taskEndTime) {
-            this.taskEndTime = taskEndTime;
-            return this;
-        }
-
-        public Builder taskHost(String taskHost) {
-            this.taskHost = taskHost;
-            return this;
-        }
-
-        public Builder logPath(String logPath) {
-            this.logPath = logPath;
-            return this;
-        }
-
-        public ProcessAlertContent build() {
-            return new ProcessAlertContent(this);
-        }
-    }
 }
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java
index 054df92e4f..c9608416d1 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java
@@ -17,17 +17,19 @@
 
 package org.apache.dolphinscheduler.dao.entity;
 
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
 import org.apache.dolphinscheduler.common.enums.CommandType;
 import org.apache.dolphinscheduler.common.enums.FailureStrategy;
 import org.apache.dolphinscheduler.common.enums.Flag;
 import org.apache.dolphinscheduler.common.enums.Priority;
 import org.apache.dolphinscheduler.common.enums.TaskDependType;
 import org.apache.dolphinscheduler.common.enums.WarningType;
+import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 
 import java.util.Date;
-import java.util.Objects;
 
 import com.baomidou.mybatisplus.annotation.IdType;
 import com.baomidou.mybatisplus.annotation.TableField;
@@ -38,6 +40,8 @@ import com.google.common.base.Strings;
 /**
  * process instance
  */
+@NoArgsConstructor
+@Data
 @TableName("t_ds_process_instance")
 public class ProcessInstance {
 
@@ -60,7 +64,7 @@ public class ProcessInstance {
     /**
      * process state
      */
-    private ExecutionStatus state;
+    private WorkflowExecutionStatus state;
     /**
      * recovery flag for failover
      */
@@ -254,10 +258,6 @@ public class ProcessInstance {
     @TableField(exist = false)
     private boolean isBlocked;
 
-    public ProcessInstance() {
-
-    }
-
     /**
      * set the process name with process define version and timestamp
      *
@@ -272,266 +272,6 @@ public class ProcessInstance {
                 DateUtils.getCurrentTimeStamp());
     }
 
-    public String getVarPool() {
-        return varPool;
-    }
-
-    public void setVarPool(String varPool) {
-        this.varPool = varPool;
-    }
-
-    public ProcessDefinition getProcessDefinition() {
-        return processDefinition;
-    }
-
-    public void setProcessDefinition(ProcessDefinition processDefinition) {
-        this.processDefinition = processDefinition;
-    }
-
-    public int getId() {
-        return id;
-    }
-
-    public void setId(int id) {
-        this.id = id;
-    }
-
-    public ExecutionStatus getState() {
-        return state;
-    }
-
-    public void setState(ExecutionStatus state) {
-        this.state = state;
-    }
-
-    public Flag getRecovery() {
-        return recovery;
-    }
-
-    public void setRecovery(Flag recovery) {
-        this.recovery = recovery;
-    }
-
-    public Date getStartTime() {
-        return startTime;
-    }
-
-    public void setStartTime(Date startTime) {
-        this.startTime = startTime;
-    }
-
-    public Date getEndTime() {
-        return endTime;
-    }
-
-    public void setEndTime(Date endTime) {
-        this.endTime = endTime;
-    }
-
-    public int getRunTimes() {
-        return runTimes;
-    }
-
-    public void setRunTimes(int runTimes) {
-        this.runTimes = runTimes;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public String getHost() {
-        return host;
-    }
-
-    public void setHost(String host) {
-        this.host = host;
-    }
-
-    public CommandType getCommandType() {
-        return commandType;
-    }
-
-    public void setCommandType(CommandType commandType) {
-        this.commandType = commandType;
-    }
-
-    public String getCommandParam() {
-        return commandParam;
-    }
-
-    public void setCommandParam(String commandParam) {
-        this.commandParam = commandParam;
-    }
-
-    public TaskDependType getTaskDependType() {
-        return taskDependType;
-    }
-
-    public void setTaskDependType(TaskDependType taskDependType) {
-        this.taskDependType = taskDependType;
-    }
-
-    public int getMaxTryTimes() {
-        return maxTryTimes;
-    }
-
-    public void setMaxTryTimes(int maxTryTimes) {
-        this.maxTryTimes = maxTryTimes;
-    }
-
-    public FailureStrategy getFailureStrategy() {
-        return failureStrategy;
-    }
-
-    public void setFailureStrategy(FailureStrategy failureStrategy) {
-        this.failureStrategy = failureStrategy;
-    }
-
-    public boolean isProcessInstanceStop() {
-        return this.state.typeIsFinished();
-    }
-
-    public WarningType getWarningType() {
-        return warningType;
-    }
-
-    public void setWarningType(WarningType warningType) {
-        this.warningType = warningType;
-    }
-
-    public Integer getWarningGroupId() {
-        return warningGroupId;
-    }
-
-    public void setWarningGroupId(Integer warningGroupId) {
-        this.warningGroupId = warningGroupId;
-    }
-
-    public Date getScheduleTime() {
-        return scheduleTime;
-    }
-
-    public void setScheduleTime(Date scheduleTime) {
-        this.scheduleTime = scheduleTime;
-    }
-
-    public Date getCommandStartTime() {
-        return commandStartTime;
-    }
-
-    public void setCommandStartTime(Date commandStartTime) {
-        this.commandStartTime = commandStartTime;
-    }
-
-    public String getGlobalParams() {
-        return globalParams;
-    }
-
-    public void setGlobalParams(String globalParams) {
-        this.globalParams = globalParams;
-    }
-
-    public DagData getDagData() {
-        return dagData;
-    }
-
-    public void setDagData(DagData dagData) {
-        this.dagData = dagData;
-    }
-
-    public String getTenantCode() {
-        return tenantCode;
-    }
-
-    public void setTenantCode(String tenantCode) {
-        this.tenantCode = tenantCode;
-    }
-
-    public String getQueue() {
-        return queue;
-    }
-
-    public void setQueue(String queue) {
-        this.queue = queue;
-    }
-
-    public int getExecutorId() {
-        return executorId;
-    }
-
-    public void setExecutorId(int executorId) {
-        this.executorId = executorId;
-    }
-
-    public Flag getIsSubProcess() {
-        return isSubProcess;
-    }
-
-    public void setIsSubProcess(Flag isSubProcess) {
-        this.isSubProcess = isSubProcess;
-    }
-
-    public Priority getProcessInstancePriority() {
-        return processInstancePriority;
-    }
-
-    public void setProcessInstancePriority(Priority processInstancePriority) {
-        this.processInstancePriority = processInstancePriority;
-    }
-
-    public String getLocations() {
-        return locations;
-    }
-
-    public void setLocations(String locations) {
-        this.locations = locations;
-    }
-
-    public String getHistoryCmd() {
-        return historyCmd;
-    }
-
-    public void setHistoryCmd(String historyCmd) {
-        this.historyCmd = historyCmd;
-    }
-
-    public String getExecutorName() {
-        return executorName;
-    }
-
-    public void setExecutorName(String executorName) {
-        this.executorName = executorName;
-    }
-
-    public Long getEnvironmentCode() {
-        return this.environmentCode;
-    }
-
-    public void setEnvironmentCode(Long environmentCode) {
-        this.environmentCode = environmentCode;
-    }
-
-    public int getDryRun() {
-        return dryRun;
-    }
-
-    public void setDryRun(int dryRun) {
-        this.dryRun = dryRun;
-    }
-
-    public Date getRestartTime() {
-        return restartTime;
-    }
-
-    public void setRestartTime(Date restartTime) {
-        this.restartTime = restartTime;
-    }
-
     /**
      * add command to history
      *
@@ -570,177 +310,4 @@ public class ProcessInstance {
         return commandType;
     }
 
-    public String getDependenceScheduleTimes() {
-        return dependenceScheduleTimes;
-    }
-
-    public void setDependenceScheduleTimes(String dependenceScheduleTimes) {
-        this.dependenceScheduleTimes = dependenceScheduleTimes;
-    }
-
-    public String getDuration() {
-        return duration;
-    }
-
-    public void setDuration(String duration) {
-        this.duration = duration;
-    }
-
-    public String getWorkerGroup() {
-        return workerGroup;
-    }
-
-    public void setWorkerGroup(String workerGroup) {
-        this.workerGroup = workerGroup;
-    }
-
-    public int getTimeout() {
-        return timeout;
-    }
-
-    public void setTimeout(int timeout) {
-        this.timeout = timeout;
-    }
-
-    public int getTenantId() {
-        return this.tenantId;
-    }
-
-    public void setTenantId(int tenantId) {
-        this.tenantId = tenantId;
-    }
-
-    public Long getProcessDefinitionCode() {
-        return processDefinitionCode;
-    }
-
-    public void setProcessDefinitionCode(Long processDefinitionCode) {
-        this.processDefinitionCode = processDefinitionCode;
-    }
-
-    public int getProcessDefinitionVersion() {
-        return processDefinitionVersion;
-    }
-
-    public void setProcessDefinitionVersion(int processDefinitionVersion) {
-        this.processDefinitionVersion = processDefinitionVersion;
-    }
-
-    public boolean isBlocked() {
-        return isBlocked;
-    }
-
-    public void setBlocked(boolean blocked) {
-        isBlocked = blocked;
-    }
-
-    @Override
-    public String toString() {
-        return "ProcessInstance{"
-            + "id=" + id
-            + ", state=" + state
-            + ", recovery=" + recovery
-            + ", startTime=" + startTime
-            + ", endTime=" + endTime
-            + ", runTimes=" + runTimes
-            + ", name='" + name + '\''
-            + ", host='" + host + '\''
-            + ", processDefinition="
-            + processDefinition
-            + ", commandType="
-            + commandType
-            + ", commandParam='"
-            + commandParam
-            + '\''
-            + ", taskDependType="
-            + taskDependType
-            + ", maxTryTimes="
-            + maxTryTimes
-            + ", failureStrategy="
-            + failureStrategy
-            + ", warningType="
-            + warningType
-            + ", warningGroupId="
-            + warningGroupId
-            + ", scheduleTime="
-            + scheduleTime
-            + ", commandStartTime="
-            + commandStartTime
-            + ", globalParams='"
-            + globalParams
-            + '\''
-            + ", executorId="
-            + executorId
-            + ", tenantCode='"
-            + tenantCode
-            + '\''
-            + ", queue='"
-            + queue
-            + '\''
-            + ", isSubProcess="
-            + isSubProcess
-            + ", locations='"
-            + locations
-            + '\''
-            + ", historyCmd='"
-            + historyCmd
-            + '\''
-            + ", dependenceScheduleTimes='"
-            + dependenceScheduleTimes
-            + '\''
-            + ", duration="
-            + duration
-            + ", processInstancePriority="
-            + processInstancePriority
-            + ", workerGroup='"
-            + workerGroup
-            + '\''
-            + ", timeout="
-            + timeout
-            + ", tenantId="
-            + tenantId
-            + ", processDefinitionCode='"
-            + processDefinitionCode
-            + '\''
-            + ", processDefinitionVersion='"
-            + processDefinitionVersion
-            + '\''
-            + ", dryRun='"
-            + dryRun
-            + '\''
-            + '}'
-            + ", restartTime='"
-            + restartTime
-            + '\''
-            + ", isBlocked="
-            + isBlocked
-            + '}';
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-
-        ProcessInstance that = (ProcessInstance) o;
-
-        return id == that.id;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(id);
-    }
-
-    public int getNextProcessInstanceId() {
-        return nextProcessInstanceId;
-    }
-
-    public void setNextProcessInstanceId(int nextProcessInstanceId) {
-        this.nextProcessInstanceId = nextProcessInstanceId;
-    }
 }
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskAlertContent.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskAlertContent.java
index f8e361e9ed..fbd2f64a59 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskAlertContent.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskAlertContent.java
@@ -17,7 +17,10 @@
 
 package org.apache.dolphinscheduler.dao.entity;
 
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
 
 import java.io.Serializable;
 import java.util.Date;
@@ -25,9 +28,15 @@ import java.util.Date;
 import com.fasterxml.jackson.annotation.JsonInclude;
 import com.fasterxml.jackson.annotation.JsonInclude.Include;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 
+@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
 @JsonInclude(Include.NON_NULL)
 public class TaskAlertContent implements Serializable {
+
     @JsonProperty("taskInstanceId")
     private int taskInstanceId;
     @JsonProperty("taskName")
@@ -43,7 +52,7 @@ public class TaskAlertContent implements Serializable {
     @JsonProperty("processInstanceName")
     private String processInstanceName;
     @JsonProperty("state")
-    private ExecutionStatus state;
+    private TaskExecutionStatus state;
     @JsonProperty("startTime")
     private Date startTime;
     @JsonProperty("endTime")
@@ -53,101 +62,4 @@ public class TaskAlertContent implements Serializable {
     @JsonProperty("logPath")
     private String logPath;
 
-    private TaskAlertContent(Builder builder) {
-        this.taskInstanceId = builder.taskInstanceId;
-        this.taskName = builder.taskName;
-        this.taskType = builder.taskType;
-        this.processDefinitionId = builder.processDefinitionId;
-        this.processDefinitionName = builder.processDefinitionName;
-        this.processInstanceId = builder.processInstanceId;
-        this.processInstanceName = builder.processInstanceName;
-        this.state = builder.state;
-        this.startTime = builder.startTime;
-        this.endTime = builder.endTime;
-        this.host = builder.host;
-        this.logPath = builder.logPath;
-    }
-
-    public static Builder newBuilder() {
-        return new Builder();
-    }
-
-    public static class Builder {
-        private int taskInstanceId;
-        private String taskName;
-        private String taskType;
-        private int processDefinitionId;
-        private String processDefinitionName;
-        private int processInstanceId;
-        private String processInstanceName;
-        private ExecutionStatus state;
-        private Date startTime;
-        private Date endTime;
-        private String host;
-        private String logPath;
-
-        public Builder taskInstanceId(int taskInstanceId) {
-            this.taskInstanceId = taskInstanceId;
-            return this;
-        }
-
-        public Builder taskName(String taskName) {
-            this.taskName = taskName;
-            return this;
-        }
-
-        public Builder taskType(String taskType) {
-            this.taskType = taskType;
-            return this;
-        }
-
-        public Builder processDefinitionId(int processDefinitionId) {
-            this.processDefinitionId = processDefinitionId;
-            return this;
-        }
-
-        public Builder processDefinitionName(String processDefinitionName) {
-            this.processDefinitionName = processDefinitionName;
-            return this;
-        }
-
-        public Builder processInstanceId(int processInstanceId) {
-            this.processInstanceId = processInstanceId;
-            return this;
-        }
-
-        public Builder processInstanceName(String processInstanceName) {
-            this.processInstanceName = processInstanceName;
-            return this;
-        }
-
-        public Builder state(ExecutionStatus state) {
-            this.state = state;
-            return this;
-        }
-
-        public Builder startTime(Date startTime) {
-            this.startTime = startTime;
-            return this;
-        }
-
-        public Builder endTime(Date endTime) {
-            this.endTime = endTime;
-            return this;
-        }
-
-        public Builder host(String host) {
-            this.host = host;
-            return this;
-        }
-
-        public Builder logPath(String logPath) {
-            this.logPath = logPath;
-            return this;
-        }
-
-        public TaskAlertContent build() {
-            return new TaskAlertContent(this);
-        }
-    }
 }
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java
index fc66bfd4cd..b52528b62b 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java
@@ -29,7 +29,7 @@ import org.apache.dolphinscheduler.common.enums.Flag;
 import org.apache.dolphinscheduler.common.enums.Priority;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameters;
 import org.apache.dolphinscheduler.plugin.task.api.parameters.SwitchParameters;
 
@@ -43,9 +43,12 @@ import com.baomidou.mybatisplus.annotation.TableId;
 import com.baomidou.mybatisplus.annotation.TableName;
 import com.fasterxml.jackson.core.type.TypeReference;
 
+import lombok.Data;
+
 /**
  * task instance
  */
+@Data
 @TableName("t_ds_task_instance")
 public class TaskInstance implements Serializable {
 
@@ -60,7 +63,6 @@ public class TaskInstance implements Serializable {
      */
     private String name;
 
-
     /**
      * task type
      */
@@ -96,7 +98,7 @@ public class TaskInstance implements Serializable {
     /**
      * state
      */
-    private ExecutionStatus state;
+    private TaskExecutionStatus state;
 
     /**
      * task first submit time.
@@ -223,7 +225,6 @@ public class TaskInstance implements Serializable {
     @TableField(exist = false)
     private String dependentResult;
 
-
     /**
      * workerGroup
      */
@@ -255,7 +256,6 @@ public class TaskInstance implements Serializable {
     @TableField(exist = false)
     private String executorName;
 
-
     @TableField(exist = false)
     private Map<String, String> resources;
 
@@ -264,7 +264,6 @@ public class TaskInstance implements Serializable {
      */
     private int delayTime;
 
-
     /**
      * task params
      */
@@ -295,199 +294,13 @@ public class TaskInstance implements Serializable {
         this.executePath = executePath;
     }
 
-    public String getVarPool() {
-        return varPool;
-    }
-
-    public void setVarPool(String varPool) {
-        this.varPool = varPool;
-    }
-
-    public int getTaskGroupId() {
-        return taskGroupId;
-    }
-
-    public void setTaskGroupId(int taskGroupId) {
-        this.taskGroupId = taskGroupId;
-    }
-
-    public ProcessInstance getProcessInstance() {
-        return processInstance;
-    }
-
-    public void setProcessInstance(ProcessInstance processInstance) {
-        this.processInstance = processInstance;
-    }
-
-    public ProcessDefinition getProcessDefine() {
-        return processDefine;
-    }
-
-    public void setProcessDefine(ProcessDefinition processDefine) {
-        this.processDefine = processDefine;
-    }
-
-    public TaskDefinition getTaskDefine() {
-        return taskDefine;
-    }
-
-    public void setTaskDefine(TaskDefinition taskDefine) {
-        this.taskDefine = taskDefine;
-    }
-
-    public int getId() {
-        return id;
-    }
-
-    public void setId(int id) {
-        this.id = id;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public String getTaskType() {
-        return taskType;
-    }
-
-    public void setTaskType(String taskType) {
-        this.taskType = taskType;
-    }
-
-    public int getProcessInstanceId() {
-        return processInstanceId;
-    }
-
-    public void setProcessInstanceId(int processInstanceId) {
-        this.processInstanceId = processInstanceId;
-    }
-
-    public ExecutionStatus getState() {
-        return state;
-    }
-
-    public void setState(ExecutionStatus state) {
-        this.state = state;
-    }
-
-    public Date getFirstSubmitTime() {
-        return firstSubmitTime;
-    }
-
-    public void setFirstSubmitTime(Date firstSubmitTime) {
-        this.firstSubmitTime = firstSubmitTime;
-    }
-
-    public Date getSubmitTime() {
-        return submitTime;
-    }
-
-    public void setSubmitTime(Date submitTime) {
-        this.submitTime = submitTime;
-    }
-
-    public Date getStartTime() {
-        return startTime;
-    }
-
-    public void setStartTime(Date startTime) {
-        this.startTime = startTime;
-    }
-
-    public Date getEndTime() {
-        return endTime;
-    }
-
-    public void setEndTime(Date endTime) {
-        this.endTime = endTime;
-    }
-
-    public String getHost() {
-        return host;
-    }
-
-    public void setHost(String host) {
-        this.host = host;
-    }
-
-    public String getExecutePath() {
-        return executePath;
-    }
-
-    public void setExecutePath(String executePath) {
-        this.executePath = executePath;
-    }
-
-    public String getLogPath() {
-        return logPath;
-    }
-
-    public void setLogPath(String logPath) {
-        this.logPath = logPath;
-    }
-
-    public Flag getAlertFlag() {
-        return alertFlag;
-    }
-
-    public void setAlertFlag(Flag alertFlag) {
-        this.alertFlag = alertFlag;
-    }
-
-    public int getRetryTimes() {
-        return retryTimes;
-    }
-
-    public void setRetryTimes(int retryTimes) {
-        this.retryTimes = retryTimes;
-    }
-
-    public Boolean isTaskSuccess() {
-        return this.state == ExecutionStatus.SUCCESS;
-    }
-
-    public int getPid() {
-        return pid;
-    }
-
-    public void setPid(int pid) {
-        this.pid = pid;
-    }
-
-    public String getAppLink() {
-        return appLink;
-    }
-
-    public void setAppLink(String appLink) {
-        this.appLink = appLink;
-    }
-
-    public Long getEnvironmentCode() {
-        return this.environmentCode;
-    }
-
-    public void setEnvironmentCode(Long environmentCode) {
-        this.environmentCode = environmentCode;
-    }
-
-    public String getEnvironmentConfig() {
-        return this.environmentConfig;
-    }
-
-    public void setEnvironmentConfig(String environmentConfig) {
-        this.environmentConfig = environmentConfig;
-    }
-
     public DependentParameters getDependency() {
         if (this.dependency == null) {
-            Map<String, Object> taskParamsMap = JSONUtils.parseObject(this.getTaskParams(), new TypeReference<Map<String, Object>>() {
-            });
-            this.dependency = JSONUtils.parseObject((String) taskParamsMap.get(Constants.DEPENDENCE), DependentParameters.class);
+            Map<String, Object> taskParamsMap =
+                    JSONUtils.parseObject(this.getTaskParams(), new TypeReference<Map<String, Object>>() {
+                    });
+            this.dependency =
+                    JSONUtils.parseObject((String) taskParamsMap.get(Constants.DEPENDENCE), DependentParameters.class);
         }
         return this.dependency;
     }
@@ -498,98 +311,28 @@ public class TaskInstance implements Serializable {
 
     public SwitchParameters getSwitchDependency() {
         if (this.switchDependency == null) {
-            Map<String, Object> taskParamsMap = JSONUtils.parseObject(this.getTaskParams(), new TypeReference<Map<String, Object>>() {
-            });
-            this.switchDependency = JSONUtils.parseObject((String) taskParamsMap.get(Constants.SWITCH_RESULT), SwitchParameters.class);
+            Map<String, Object> taskParamsMap =
+                    JSONUtils.parseObject(this.getTaskParams(), new TypeReference<Map<String, Object>>() {
+                    });
+            this.switchDependency =
+                    JSONUtils.parseObject((String) taskParamsMap.get(Constants.SWITCH_RESULT), SwitchParameters.class);
         }
         return this.switchDependency;
     }
 
     public void setSwitchDependency(SwitchParameters switchDependency) {
-        Map<String, Object> taskParamsMap = JSONUtils.parseObject(this.getTaskParams(), new TypeReference<Map<String, Object>>() {
-        });
+        Map<String, Object> taskParamsMap =
+                JSONUtils.parseObject(this.getTaskParams(), new TypeReference<Map<String, Object>>() {
+                });
         taskParamsMap.put(Constants.SWITCH_RESULT, JSONUtils.toJsonString(switchDependency));
         this.setTaskParams(JSONUtils.toJsonString(taskParamsMap));
     }
 
-    public Flag getFlag() {
-        return flag;
-    }
-
-    public void setFlag(Flag flag) {
-        this.flag = flag;
-    }
-
-    public String getProcessInstanceName() {
-        return processInstanceName;
-    }
-
-    public void setProcessInstanceName(String processInstanceName) {
-        this.processInstanceName = processInstanceName;
-    }
-
-    public String getDuration() {
-        return duration;
-    }
-
-    public void setDuration(String duration) {
-        this.duration = duration;
-    }
-
-    public int getMaxRetryTimes() {
-        return maxRetryTimes;
-    }
-
-    public void setMaxRetryTimes(int maxRetryTimes) {
-        this.maxRetryTimes = maxRetryTimes;
-    }
-
-    public int getRetryInterval() {
-        return retryInterval;
-    }
-
-    public void setRetryInterval(int retryInterval) {
-        this.retryInterval = retryInterval;
-    }
-
-    public int getExecutorId() {
-        return executorId;
-    }
-
-    public void setExecutorId(int executorId) {
-        this.executorId = executorId;
-    }
-
-    public String getExecutorName() {
-        return executorName;
-    }
-
-    public void setExecutorName(String executorName) {
-        this.executorName = executorName;
-    }
-
-    public int getDryRun() {
-        return dryRun;
-    }
-
-    public void setDryRun(int dryRun) {
-        this.dryRun = dryRun;
-    }
-
     public boolean isTaskComplete() {
 
-        return this.getState().typeIsPause()
-                || this.getState().typeIsSuccess()
-                || this.getState().typeIsCancel()
-                || (this.getState().typeIsFailure() && !taskCanRetry());
-    }
-
-    public Map<String, String> getResources() {
-        return resources;
-    }
-
-    public void setResources(Map<String, String> resources) {
-        this.resources = resources;
+        return this.getState().isSuccess()
+                || this.getState().isKill()
+                || (this.getState().isFailure() && !taskCanRetry());
     }
 
     public boolean isSubProcess() {
@@ -622,10 +365,10 @@ public class TaskInstance implements Serializable {
         if (this.isSubProcess()) {
             return false;
         }
-        if (this.getState() == ExecutionStatus.NEED_FAULT_TOLERANCE) {
+        if (this.getState() == TaskExecutionStatus.NEED_FAULT_TOLERANCE) {
             return true;
         }
-        return this.getState() == ExecutionStatus.FAILURE && (this.getRetryTimes() < this.getMaxRetryTimes());
+        return this.getState() == TaskExecutionStatus.FAILURE && (this.getRetryTimes() < this.getMaxRetryTimes());
     }
 
     /**
@@ -634,7 +377,7 @@ public class TaskInstance implements Serializable {
      * @return Boolean
      */
     public boolean retryTaskIntervalOverTime() {
-        if (getState() != ExecutionStatus.FAILURE) {
+        if (getState() != TaskExecutionStatus.FAILURE) {
             return true;
         }
         if (getMaxRetryTimes() == 0 || getRetryInterval() == 0) {
@@ -646,137 +389,8 @@ public class TaskInstance implements Serializable {
         return getRetryInterval() * SEC_2_MINUTES_TIME_UNIT < failedTimeInterval;
     }
 
-    public Priority getTaskInstancePriority() {
-        return taskInstancePriority;
-    }
-
-    public void setTaskInstancePriority(Priority taskInstancePriority) {
-        this.taskInstancePriority = taskInstancePriority;
-    }
-
-    public Priority getProcessInstancePriority() {
-        return processInstancePriority;
-    }
-
-    public void setProcessInstancePriority(Priority processInstancePriority) {
-        this.processInstancePriority = processInstancePriority;
-    }
-
-    public String getWorkerGroup() {
-        return workerGroup;
-    }
-
-    public void setWorkerGroup(String workerGroup) {
-        this.workerGroup = workerGroup;
-    }
-
-    public String getDependentResult() {
-        return dependentResult;
-    }
-
-    public void setDependentResult(String dependentResult) {
-        this.dependentResult = dependentResult;
-    }
-
-    public int getDelayTime() {
-        return delayTime;
-    }
-
-    public void setDelayTime(int delayTime) {
-        this.delayTime = delayTime;
-    }
-
-    @Override
-    public String toString() {
-        return "TaskInstance{"
-                + "id=" + id
-                + ", name='" + name + '\''
-                + ", taskType='" + taskType + '\''
-                + ", processInstanceId=" + processInstanceId
-                + ", processInstanceName='" + processInstanceName + '\''
-                + ", state=" + state
-                + ", firstSubmitTime=" + firstSubmitTime
-                + ", submitTime=" + submitTime
-                + ", startTime=" + startTime
-                + ", endTime=" + endTime
-                + ", host='" + host + '\''
-                + ", executePath='" + executePath + '\''
-                + ", logPath='" + logPath + '\''
-                + ", retryTimes=" + retryTimes
-                + ", alertFlag=" + alertFlag
-                + ", processInstance=" + processInstance
-                + ", processDefine=" + processDefine
-                + ", pid=" + pid
-                + ", appLink='" + appLink + '\''
-                + ", flag=" + flag
-                + ", dependency='" + dependency + '\''
-                + ", duration=" + duration
-                + ", maxRetryTimes=" + maxRetryTimes
-                + ", retryInterval=" + retryInterval
-                + ", taskInstancePriority=" + taskInstancePriority
-                + ", processInstancePriority=" + processInstancePriority
-                + ", dependentResult='" + dependentResult + '\''
-                + ", workerGroup='" + workerGroup + '\''
-                + ", environmentCode=" + environmentCode
-                + ", environmentConfig='" + environmentConfig + '\''
-                + ", executorId=" + executorId
-                + ", executorName='" + executorName + '\''
-                + ", delayTime=" + delayTime
-                + ", dryRun=" + dryRun
-                + ", taskGroupId=" + taskGroupId
-                + ", taskGroupPriority=" + taskGroupPriority
-                + '}';
-    }
-
-    public long getTaskCode() {
-        return taskCode;
-    }
-
-    public void setTaskCode(long taskCode) {
-        this.taskCode = taskCode;
-    }
-
-    public int getTaskDefinitionVersion() {
-        return taskDefinitionVersion;
-    }
-
-    public void setTaskDefinitionVersion(int taskDefinitionVersion) {
-        this.taskDefinitionVersion = taskDefinitionVersion;
-    }
-
-    public String getTaskParams() {
-        return taskParams;
-    }
-
-    public void setTaskParams(String taskParams) {
-        this.taskParams = taskParams;
-    }
-
     public boolean isFirstRun() {
         return endTime == null;
     }
 
-    public int getTaskGroupPriority() {
-        return taskGroupPriority;
-    }
-
-    public void setTaskGroupPriority(int taskGroupPriority) {
-        this.taskGroupPriority = taskGroupPriority;
-    }
-
-    public Integer getCpuQuota() {
-        return cpuQuota == null ? -1 : cpuQuota;
-    }
-
-    public void setCpuQuota(Integer cpuQuota) {
-        this.cpuQuota = cpuQuota;
-    }
-
-    public Integer getMemoryMax() {
-        return memoryMax == null ? -1 : memoryMax;
-    }
-
-    public void setMemoryMax(Integer memoryMax) {
-        this.memoryMax = memoryMax;
-    }
 }
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.java
index 8c0abf3dce..32226e8b59 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.java
@@ -17,9 +17,9 @@
 
 package org.apache.dolphinscheduler.dao.mapper;
 
+import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
 import org.apache.dolphinscheduler.dao.entity.ExecuteStatusCount;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 
 import org.apache.ibatis.annotations.Param;
 
@@ -133,8 +133,8 @@ public interface ProcessInstanceMapper extends BaseMapper<ProcessInstance> {
      * @param destState   destState
      * @return update result
      */
-    int updateProcessInstanceByState(@Param("originState") ExecutionStatus originState,
-                                     @Param("destState") ExecutionStatus destState);
+    int updateProcessInstanceByState(@Param("originState") WorkflowExecutionStatus originState,
+                                     @Param("destState") WorkflowExecutionStatus destState);
 
     /**
      * update process instance by tenantId
@@ -167,9 +167,9 @@ public interface ProcessInstanceMapper extends BaseMapper<ProcessInstance> {
      * @return ExecuteStatusCount list
      */
     List<ExecuteStatusCount> countInstanceStateByProjectCodes(
-        @Param("startTime") Date startTime,
-        @Param("endTime") Date endTime,
-        @Param("projectCodes") Long[] projectCodes);
+                                                              @Param("startTime") Date startTime,
+                                                              @Param("endTime") Date endTime,
+                                                              @Param("projectCodes") Long[] projectCodes);
 
     /**
      * query process instance by processDefinitionCode
@@ -233,7 +233,7 @@ public interface ProcessInstanceMapper extends BaseMapper<ProcessInstance> {
     List<ProcessInstance> queryTopNProcessInstance(@Param("size") int size,
                                                    @Param("startTime") Date startTime,
                                                    @Param("endTime") Date endTime,
-                                                   @Param("status") ExecutionStatus status,
+                                                   @Param("status") WorkflowExecutionStatus status,
                                                    @Param("projectCode") long projectCode);
 
     /**
@@ -248,13 +248,16 @@ public interface ProcessInstanceMapper extends BaseMapper<ProcessInstance> {
                                                             @Param("states") int[] states);
 
     List<ProcessInstance> queryByProcessDefineCodeAndProcessDefinitionVersionAndStatusAndNextId(@Param("processDefinitionCode") Long processDefinitionCode,
-                                                                               @Param("processDefinitionVersion") int processDefinitionVersion,
-                                                                     @Param("states") int[] states, @Param("id") int id);
+                                                                                                @Param("processDefinitionVersion") int processDefinitionVersion,
+                                                                                                @Param("states") int[] states,
+                                                                                                @Param("id") int id);
 
     int updateGlobalParamsById(@Param("globalParams") String globalParams,
                                @Param("id") int id);
 
-    boolean updateNextProcessIdById(@Param("thisInstanceId") int thisInstanceId, @Param("runningInstanceId") int runningInstanceId);
+    boolean updateNextProcessIdById(@Param("thisInstanceId") int thisInstanceId,
+                                    @Param("runningInstanceId") int runningInstanceId);
 
-    ProcessInstance loadNextProcess4Serial(@Param("processDefinitionCode") Long processDefinitionCode, @Param("state") int state, @Param("id") int id);
+    ProcessInstance loadNextProcess4Serial(@Param("processDefinitionCode") Long processDefinitionCode,
+                                           @Param("state") int state, @Param("id") int id);
 }
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 f98996952d..17acff69d9 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
@@ -22,7 +22,7 @@ import com.baomidou.mybatisplus.core.metadata.IPage;
 import org.apache.dolphinscheduler.common.enums.Flag;
 import org.apache.dolphinscheduler.dao.entity.ExecuteStatusCount;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.ibatis.annotations.Param;
 
 import java.util.Date;
@@ -44,7 +44,7 @@ public interface TaskInstanceMapper extends BaseMapper<TaskInstance> {
 
     int setFailoverByHostAndStateArray(@Param("host") String host,
                                        @Param("states") int[] stateArray,
-                                       @Param("destStatus") ExecutionStatus destStatus);
+                                       @Param("destStatus") TaskExecutionStatus destStatus);
 
     TaskInstance queryByInstanceIdAndName(@Param("processInstanceId") int processInstanceId,
                                           @Param("name") String name);
@@ -82,7 +82,7 @@ public interface TaskInstanceMapper extends BaseMapper<TaskInstance> {
     List<ExecuteStatusCount> countTaskInstanceStateByProjectCodesAndStatesBySubmitTime(@Param("startTime") Date startTime,
                                                                                        @Param("endTime") Date endTime,
                                                                                        @Param("projectCodes") Long[] projectCodes,
-                                                                                       @Param("states") List<ExecutionStatus> states);
+                                                                                       @Param("states") List<TaskExecutionStatus> states);
 
     IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
                                                     @Param("projectCode") Long projectCode,
@@ -94,8 +94,8 @@ public interface TaskInstanceMapper extends BaseMapper<TaskInstance> {
                                                     @Param("states") int[] statusArray,
                                                     @Param("host") String host,
                                                     @Param("startTime") Date startTime,
-                                                    @Param("endTime") Date endTime
-    );
+                                                    @Param("endTime") Date endTime);
 
-    List<TaskInstance> loadAllInfosNoRelease(@Param("processInstanceId") int processInstanceId, @Param("status") int status);
+    List<TaskInstance> loadAllInfosNoRelease(@Param("processInstanceId") int processInstanceId,
+                                             @Param("status") int status);
 }
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 9aa69c333f..e2ddca73d9 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
@@ -50,7 +50,6 @@ import org.slf4j.LoggerFactory;
  */
 public class DagHelper {
 
-
     private static final Logger logger = LoggerFactory.getLogger(DagHelper.class);
 
     /**
@@ -85,8 +84,10 @@ public class DagHelper {
      * @param taskDependType taskDependType
      * @return task node list
      */
-    public static List<TaskNode> generateFlowNodeListByStartNode(List<TaskNode> taskNodeList, List<String> startNodeNameList,
-                                                                 List<String> recoveryNodeCodeList, TaskDependType taskDependType) {
+    public static List<TaskNode> generateFlowNodeListByStartNode(List<TaskNode> taskNodeList,
+                                                                 List<String> startNodeNameList,
+                                                                 List<String> recoveryNodeCodeList,
+                                                                 TaskDependType taskDependType) {
         List<TaskNode> destFlowNodeList = new ArrayList<>();
         List<String> startNodeList = startNodeNameList;
 
@@ -112,16 +113,16 @@ public class DagHelper {
                 List<TaskNode> childNodeList = new ArrayList<>();
                 if (startNode == null) {
                     logger.error("start node name [{}] is not in task node list [{}] ",
-                        startNodeCode,
-                            taskNodeList
-                    );
+                            startNodeCode,
+                            taskNodeList);
                     continue;
                 } else if (TaskDependType.TASK_POST == taskDependType) {
                     List<String> visitedNodeCodeList = new ArrayList<>();
                     childNodeList = getFlowNodeListPost(startNode, taskNodeList, visitedNodeCodeList);
                 } else if (TaskDependType.TASK_PRE == taskDependType) {
                     List<String> visitedNodeCodeList = new ArrayList<>();
-                    childNodeList = getFlowNodeListPre(startNode, recoveryNodeCodeList, taskNodeList, visitedNodeCodeList);
+                    childNodeList =
+                            getFlowNodeListPre(startNode, recoveryNodeCodeList, taskNodeList, visitedNodeCodeList);
                 } else {
                     childNodeList.add(startNode);
                 }
@@ -144,11 +145,13 @@ public class DagHelper {
      * @param taskNodeList taskNodeList
      * @return task node list
      */
-    private static List<TaskNode> getFlowNodeListPost(TaskNode startNode, List<TaskNode> taskNodeList, List<String> visitedNodeCodeList) {
+    private static List<TaskNode> getFlowNodeListPost(TaskNode startNode, List<TaskNode> taskNodeList,
+                                                      List<String> visitedNodeCodeList) {
         List<TaskNode> resultList = new ArrayList<>();
         for (TaskNode taskNode : taskNodeList) {
             List<String> depList = taskNode.getDepList();
-            if (null != depList && null != startNode && depList.contains(Long.toString(startNode.getCode())) && !visitedNodeCodeList.contains(Long.toString(taskNode.getCode()))) {
+            if (null != depList && null != startNode && depList.contains(Long.toString(startNode.getCode()))
+                    && !visitedNodeCodeList.contains(Long.toString(taskNode.getCode()))) {
                 resultList.addAll(getFlowNodeListPost(taskNode, taskNodeList, visitedNodeCodeList));
             }
         }
@@ -169,7 +172,8 @@ public class DagHelper {
      * @param taskNodeList taskNodeList
      * @return task node list
      */
-    private static List<TaskNode> getFlowNodeListPre(TaskNode startNode, List<String> recoveryNodeCodeList, List<TaskNode> taskNodeList, List<String> visitedNodeCodeList) {
+    private static List<TaskNode> getFlowNodeListPre(TaskNode startNode, List<String> recoveryNodeCodeList,
+                                                     List<TaskNode> taskNodeList, List<String> visitedNodeCodeList) {
 
         List<TaskNode> resultList = new ArrayList<>();
 
@@ -211,7 +215,8 @@ public class DagHelper {
                                              List<String> recoveryNodeCodeList,
                                              TaskDependType depNodeType) throws Exception {
 
-        List<TaskNode> destTaskNodeList = generateFlowNodeListByStartNode(totalTaskNodeList, startNodeNameList, recoveryNodeCodeList, depNodeType);
+        List<TaskNode> destTaskNodeList = generateFlowNodeListByStartNode(totalTaskNodeList, startNodeNameList,
+                recoveryNodeCodeList, depNodeType);
         if (destTaskNodeList.isEmpty()) {
             return null;
         }
@@ -334,8 +339,7 @@ public class DagHelper {
      * if all of the task dependence are skipped, skip it too.
      */
     private static boolean isTaskNodeNeedSkip(TaskNode taskNode,
-                                              Map<String, TaskNode> skipTaskNodeList
-    ) {
+                                              Map<String, TaskNode> skipTaskNodeList) {
         if (CollectionUtils.isEmpty(taskNode.getDepList())) {
             return false;
         }
@@ -367,10 +371,10 @@ public class DagHelper {
         ConditionsParameters conditionsParameters =
                 JSONUtils.parseObject(taskNode.getConditionResult(), ConditionsParameters.class);
         List<String> skipNodeList = new ArrayList<>();
-        if (taskInstance.getState().typeIsSuccess()) {
+        if (taskInstance.getState().isSuccess()) {
             conditionTaskList = conditionsParameters.getSuccessNode();
             skipNodeList = conditionsParameters.getFailedNode();
-        } else if (taskInstance.getState().typeIsFailure()) {
+        } else if (taskInstance.getState().isFailure()) {
             conditionTaskList = conditionsParameters.getFailedNode();
             skipNodeList = conditionsParameters.getSuccessNode();
         } else {
@@ -413,7 +417,8 @@ public class DagHelper {
                                                     Map<String, TaskInstance> completeTaskList,
                                                     DAG<String, TaskNode, TaskNodeRelation> dag) {
 
-        SwitchParameters switchParameters = completeTaskList.get(Long.toString(taskNode.getCode())).getSwitchDependency();
+        SwitchParameters switchParameters =
+                completeTaskList.get(Long.toString(taskNode.getCode())).getSwitchDependency();
         int resultConditionLocation = switchParameters.getResultConditionLocation();
         List<SwitchResultVo> conditionResultVoList = switchParameters.getDependTaskList();
         List<String> switchTaskList = conditionResultVoList.get(resultConditionLocation).getNextNode();
@@ -459,14 +464,14 @@ public class DagHelper {
 
         DAG<String, TaskNode, TaskNodeRelation> dag = new DAG<>();
 
-        //add vertex
+        // add vertex
         if (CollectionUtils.isNotEmpty(processDag.getNodes())) {
             for (TaskNode node : processDag.getNodes()) {
                 dag.addNode(Long.toString(node.getCode()), node);
             }
         }
 
-        //add edge
+        // add edge
         if (CollectionUtils.isNotEmpty(processDag.getEdges())) {
             for (TaskNodeRelation edge : processDag.getEdges()) {
                 dag.addEdge(edge.getStartNode(), edge.getEndNode());
@@ -526,7 +531,8 @@ public class DagHelper {
                     && taskNodeMap.containsKey(preTaskCode) && taskNodeMap.containsKey(postTaskCode)) {
                 TaskNode preNode = taskNodeMap.get(preTaskCode);
                 TaskNode postNode = taskNodeMap.get(postTaskCode);
-                taskNodeRelations.add(new TaskNodeRelation(Long.toString(preNode.getCode()), Long.toString(postNode.getCode())));
+                taskNodeRelations
+                        .add(new TaskNodeRelation(Long.toString(preNode.getCode()), Long.toString(postNode.getCode())));
             }
         }
         ProcessDag processDag = new ProcessDag();
@@ -539,8 +545,7 @@ public class DagHelper {
      * is there have conditions after the parent node
      */
     public static boolean haveConditionsAfterNode(String parentNodeCode,
-                                                  DAG<String, TaskNode, TaskNodeRelation> dag
-    ) {
+                                                  DAG<String, TaskNode, TaskNodeRelation> dag) {
         return haveSubAfterNode(parentNodeCode, dag, TaskConstants.TASK_TYPE_CONDITIONS);
     }
 
@@ -560,12 +565,11 @@ public class DagHelper {
         return false;
     }
 
-
     /**
      * is there have blocking node after the parent node
      */
     public static boolean haveBlockingAfterNode(String parentNodeCode,
-                                                DAG<String,TaskNode,TaskNodeRelation> dag) {
+                                                DAG<String, TaskNode, TaskNodeRelation> dag) {
         return haveSubAfterNode(parentNodeCode, dag, TaskConstants.TASK_TYPE_BLOCKING);
     }
 
@@ -573,7 +577,7 @@ public class DagHelper {
      * is there have all node after the parent node
      */
     public static boolean haveAllNodeAfterNode(String parentNodeCode,
-                                               DAG<String,TaskNode,TaskNodeRelation> dag) {
+                                               DAG<String, TaskNode, TaskNodeRelation> dag) {
         return haveSubAfterNode(parentNodeCode, dag, null);
     }
 
@@ -581,17 +585,17 @@ public class DagHelper {
      * Whether there is a specified type of child node after the parent node
      */
     public static boolean haveSubAfterNode(String parentNodeCode,
-                                           DAG<String,TaskNode,TaskNodeRelation> dag, String filterNodeType) {
+                                           DAG<String, TaskNode, TaskNodeRelation> dag, String filterNodeType) {
         Set<String> subsequentNodes = dag.getSubsequentNodes(parentNodeCode);
         if (CollectionUtils.isEmpty(subsequentNodes)) {
             return false;
         }
-        if (StringUtils.isBlank(filterNodeType)){
+        if (StringUtils.isBlank(filterNodeType)) {
             return true;
         }
         for (String nodeName : subsequentNodes) {
             TaskNode taskNode = dag.getNode(nodeName);
-            if (taskNode.getType().equalsIgnoreCase(filterNodeType)){
+            if (taskNode.getType().equalsIgnoreCase(filterNodeType)) {
                 return true;
             }
         }
diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapperTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapperTest.java
index c5f2d28ce1..6fccfa234b 100644
--- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapperTest.java
+++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapperTest.java
@@ -19,16 +19,17 @@ package org.apache.dolphinscheduler.dao.mapper;
 
 import org.apache.dolphinscheduler.common.enums.Flag;
 import org.apache.dolphinscheduler.common.enums.ReleaseState;
+import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
 import org.apache.dolphinscheduler.dao.BaseDaoTest;
 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.Project;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 
 import java.util.Date;
 import java.util.List;
 
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.junit.Assert;
 import org.junit.Test;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -57,7 +58,7 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
         processInstance.setProcessDefinitionCode(1L);
         processInstance.setStartTime(start);
         processInstance.setEndTime(end);
-        processInstance.setState(ExecutionStatus.SUCCESS);
+        processInstance.setState(WorkflowExecutionStatus.SUCCESS);
 
         processInstanceMapper.insert(processInstance);
         return processInstance;
@@ -69,14 +70,14 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
      * @return ProcessInstance
      */
     private ProcessInstance insertOne() {
-        //insertOne
+        // insertOne
         ProcessInstance processInstance = new ProcessInstance();
         Date start = new Date(2019 - 1900, 1 - 1, 1, 0, 10, 0);
         Date end = new Date(2019 - 1900, 1 - 1, 1, 1, 0, 0);
         processInstance.setProcessDefinitionCode(1L);
         processInstance.setStartTime(start);
         processInstance.setEndTime(end);
-        processInstance.setState(ExecutionStatus.SUBMITTED_SUCCESS);
+        processInstance.setState(WorkflowExecutionStatus.SUBMITTED_SUCCESS);
 
         processInstanceMapper.insert(processInstance);
         return processInstance;
@@ -87,9 +88,9 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
      */
     @Test
     public void testUpdate() {
-        //insertOne
+        // insertOne
         ProcessInstance processInstanceMap = insertOne();
-        //update
+        // update
         int update = processInstanceMapper.updateById(processInstanceMap);
         Assert.assertEquals(1, update);
         processInstanceMapper.deleteById(processInstanceMap.getId());
@@ -111,7 +112,7 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
     @Test
     public void testQuery() {
         ProcessInstance processInstance = insertOne();
-        //query
+        // query
         List<ProcessInstance> dataSources = processInstanceMapper.selectList(null);
         Assert.assertNotEquals(dataSources.size(), 0);
         processInstanceMapper.deleteById(processInstance.getId());
@@ -137,12 +138,12 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
     public void testQueryByHostAndStates() {
         ProcessInstance processInstance = insertOne();
         processInstance.setHost("192.168.2.155");
-        processInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
+        processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION);
         processInstanceMapper.updateById(processInstance);
 
         int[] stateArray = new int[]{
-                ExecutionStatus.RUNNING_EXECUTION.ordinal(),
-                ExecutionStatus.SUCCESS.ordinal()};
+                TaskExecutionStatus.RUNNING_EXECUTION.getCode(),
+                TaskExecutionStatus.SUCCESS.getCode()};
 
         List<ProcessInstance> processInstances = processInstanceMapper.queryByHostAndStatus(null, stateArray);
 
@@ -157,8 +158,8 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
     public void testQueryProcessInstanceListPaging() {
 
         int[] stateArray = new int[]{
-                ExecutionStatus.RUNNING_EXECUTION.ordinal(),
-                ExecutionStatus.SUCCESS.ordinal()};
+                WorkflowExecutionStatus.RUNNING_EXECUTION.getCode(),
+                WorkflowExecutionStatus.SUCCESS.getCode()};
 
         ProcessDefinition processDefinition = new ProcessDefinition();
         processDefinition.setCode(1L);
@@ -170,7 +171,7 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
 
         ProcessInstance processInstance = insertOne();
         processInstance.setProcessDefinitionCode(processDefinition.getCode());
-        processInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
+        processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION);
         processInstance.setIsSubProcess(Flag.NO);
         processInstance.setStartTime(new Date());
 
@@ -187,8 +188,7 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
                 stateArray,
                 processInstance.getHost(),
                 null,
-                null
-        );
+                null);
         Assert.assertNotEquals(processInstanceIPage.getTotal(), 0);
 
         processDefinitionMapper.deleteById(processDefinition.getId());
@@ -202,12 +202,12 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
     public void testSetFailoverByHostAndStateArray() {
 
         int[] stateArray = new int[]{
-                ExecutionStatus.RUNNING_EXECUTION.ordinal(),
-                ExecutionStatus.SUCCESS.ordinal()};
+                WorkflowExecutionStatus.RUNNING_EXECUTION.ordinal(),
+                WorkflowExecutionStatus.SUCCESS.ordinal()};
 
         ProcessInstance processInstance = insertOne();
 
-        processInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
+        processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION);
         processInstance.setHost("192.168.2.220");
         processInstanceMapper.updateById(processInstance);
         String host = processInstance.getHost();
@@ -227,14 +227,15 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
 
         ProcessInstance processInstance = insertOne();
 
-        processInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
+        processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION);
         processInstanceMapper.updateById(processInstance);
-        processInstanceMapper.updateProcessInstanceByState(ExecutionStatus.RUNNING_EXECUTION, ExecutionStatus.SUCCESS);
+        processInstanceMapper.updateProcessInstanceByState(WorkflowExecutionStatus.RUNNING_EXECUTION,
+                WorkflowExecutionStatus.SUCCESS);
 
         ProcessInstance processInstance1 = processInstanceMapper.selectById(processInstance.getId());
 
         processInstanceMapper.deleteById(processInstance.getId());
-        Assert.assertEquals(ExecutionStatus.SUCCESS, processInstance1.getState());
+        Assert.assertEquals(WorkflowExecutionStatus.SUCCESS, processInstance1.getState());
 
     }
 
@@ -264,7 +265,8 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
 
         Long[] projectCodes = new Long[]{processDefinition.getProjectCode()};
 
-        List<ExecuteStatusCount> executeStatusCounts = processInstanceMapper.countInstanceStateByProjectCodes(null, null, projectCodes);
+        List<ExecuteStatusCount> executeStatusCounts =
+                processInstanceMapper.countInstanceStateByProjectCodes(null, null, projectCodes);
 
         Assert.assertNotEquals(executeStatusCounts.size(), 0);
 
@@ -281,10 +283,12 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
         ProcessInstance processInstance = insertOne();
         ProcessInstance processInstance1 = insertOne();
 
-        List<ProcessInstance> processInstances = processInstanceMapper.queryByProcessDefineCode(processInstance.getProcessDefinitionCode(), 1);
+        List<ProcessInstance> processInstances =
+                processInstanceMapper.queryByProcessDefineCode(processInstance.getProcessDefinitionCode(), 1);
         Assert.assertEquals(1, processInstances.size());
 
-        processInstances = processInstanceMapper.queryByProcessDefineCode(processInstance.getProcessDefinitionCode(), 2);
+        processInstances =
+                processInstanceMapper.queryByProcessDefineCode(processInstance.getProcessDefinitionCode(), 2);
         Assert.assertEquals(2, processInstances.size());
 
         processInstanceMapper.deleteById(processInstance.getId());
@@ -300,7 +304,8 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
         processInstance.setScheduleTime(new Date());
         processInstanceMapper.updateById(processInstance);
 
-        ProcessInstance processInstance1 = processInstanceMapper.queryLastSchedulerProcess(processInstance.getProcessDefinitionCode(), null, null);
+        ProcessInstance processInstance1 =
+                processInstanceMapper.queryLastSchedulerProcess(processInstance.getProcessDefinitionCode(), null, null);
         Assert.assertNotEquals(processInstance1, null);
         processInstanceMapper.deleteById(processInstance.getId());
     }
@@ -311,14 +316,15 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
     @Test
     public void testQueryLastRunningProcess() {
         ProcessInstance processInstance = insertOne();
-        processInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
+        processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION);
         processInstanceMapper.updateById(processInstance);
 
         int[] stateArray = new int[]{
-                ExecutionStatus.RUNNING_EXECUTION.ordinal(),
-                ExecutionStatus.SUBMITTED_SUCCESS.ordinal()};
+                WorkflowExecutionStatus.RUNNING_EXECUTION.ordinal(),
+                WorkflowExecutionStatus.SUBMITTED_SUCCESS.ordinal()};
 
-        ProcessInstance processInstance1 = processInstanceMapper.queryLastRunningProcess(processInstance.getProcessDefinitionCode(), null, null, stateArray);
+        ProcessInstance processInstance1 = processInstanceMapper
+                .queryLastRunningProcess(processInstance.getProcessDefinitionCode(), null, null, stateArray);
 
         Assert.assertNotEquals(processInstance1, null);
         processInstanceMapper.deleteById(processInstance.getId());
@@ -334,13 +340,13 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
 
         Date start = new Date(2019 - 1900, 1 - 1, 01, 0, 0, 0);
         Date end = new Date(2019 - 1900, 1 - 1, 01, 5, 0, 0);
-        ProcessInstance processInstance1 = processInstanceMapper.queryLastManualProcess(processInstance.getProcessDefinitionCode(), start, end
-        );
+        ProcessInstance processInstance1 =
+                processInstanceMapper.queryLastManualProcess(processInstance.getProcessDefinitionCode(), start, end);
         Assert.assertEquals(processInstance1.getId(), processInstance.getId());
 
         start = new Date(2019 - 1900, 1 - 1, 01, 1, 0, 0);
-        processInstance1 = processInstanceMapper.queryLastManualProcess(processInstance.getProcessDefinitionCode(), start, end
-        );
+        processInstance1 =
+                processInstanceMapper.queryLastManualProcess(processInstance.getProcessDefinitionCode(), start, end);
         Assert.assertNull(processInstance1);
 
         processInstanceMapper.deleteById(processInstance.getId());
@@ -353,7 +359,8 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
     private boolean isSortedByDuration(List<ProcessInstance> processInstances) {
         for (int i = 1; i < processInstances.size(); i++) {
             long d1 = processInstances.get(i).getEndTime().getTime() - processInstances.get(i).getStartTime().getTime();
-            long d2 = processInstances.get(i - 1).getEndTime().getTime() - processInstances.get(i - 1).getStartTime().getTime();
+            long d2 = processInstances.get(i - 1).getEndTime().getTime()
+                    - processInstances.get(i - 1).getStartTime().getTime();
             if (d1 > d2) {
                 return false;
             }
@@ -377,11 +384,12 @@ public class ProcessInstanceMapperTest extends BaseDaoTest {
         ProcessInstance processInstance3 = insertOne(startTime3, endTime3);
         Date start = new Date(2020, 1, 1, 1, 1, 1);
         Date end = new Date(2021, 1, 1, 1, 1, 1);
-        List<ProcessInstance> processInstances = processInstanceMapper.queryTopNProcessInstance(2, start, end, ExecutionStatus.SUCCESS,0L);
+        List<ProcessInstance> processInstances =
+                processInstanceMapper.queryTopNProcessInstance(2, start, end, WorkflowExecutionStatus.SUCCESS, 0L);
         Assert.assertEquals(2, processInstances.size());
         Assert.assertTrue(isSortedByDuration(processInstances));
         for (ProcessInstance processInstance : processInstances) {
-            Assert.assertTrue(processInstance.getState().typeIsSuccess());
+            Assert.assertTrue(processInstance.getState().isSuccess());
         }
         processInstanceMapper.deleteById(processInstance1.getId());
         processInstanceMapper.deleteById(processInstance2.getId());
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 4bcdab8550..fe86cc7c0a 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
@@ -18,16 +18,17 @@
 package org.apache.dolphinscheduler.dao.mapper;
 
 import org.apache.dolphinscheduler.common.enums.Flag;
+import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
 import org.apache.dolphinscheduler.dao.BaseDaoTest;
 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.plugin.task.api.enums.ExecutionStatus;
 
 import java.util.Date;
 import java.util.List;
 
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -62,7 +63,7 @@ public class TaskInstanceMapperTest extends BaseDaoTest {
      * @return TaskInstance
      */
     private TaskInstance insertTaskInstance(int processInstanceId) {
-        //insertOne
+        // insertOne
         return insertTaskInstance(processInstanceId, "SHELL");
     }
 
@@ -75,12 +76,12 @@ public class TaskInstanceMapperTest extends BaseDaoTest {
         ProcessInstance processInstance = new ProcessInstance();
         processInstance.setId(1);
         processInstance.setName("taskName");
-        processInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
+        processInstance.setState(WorkflowExecutionStatus.RUNNING_EXECUTION);
         processInstance.setStartTime(new Date());
         processInstance.setEndTime(new Date());
         processInstance.setProcessDefinitionCode(1L);
         processInstanceMapper.insert(processInstance);
-        return processInstanceMapper.queryByProcessDefineCode(1L,1).get(0);
+        return processInstanceMapper.queryByProcessDefineCode(1L, 1).get(0);
     }
 
     /**
@@ -90,7 +91,7 @@ public class TaskInstanceMapperTest extends BaseDaoTest {
         TaskInstance taskInstance = new TaskInstance();
         taskInstance.setFlag(Flag.YES);
         taskInstance.setName("us task");
-        taskInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
+        taskInstance.setState(TaskExecutionStatus.RUNNING_EXECUTION);
         taskInstance.setStartTime(new Date());
         taskInstance.setEndTime(new Date());
         taskInstance.setProcessInstanceId(processInstanceId);
@@ -140,7 +141,7 @@ public class TaskInstanceMapperTest extends BaseDaoTest {
 
         // insert taskInstance
         TaskInstance taskInstance = insertTaskInstance(processInstance.getId());
-        //query
+        // query
         List<TaskInstance> taskInstances = taskInstanceMapper.selectList(null);
         taskInstanceMapper.deleteById(taskInstance.getId());
         Assert.assertNotEquals(taskInstances.size(), 0);
@@ -160,8 +161,7 @@ public class TaskInstanceMapperTest extends BaseDaoTest {
         taskInstanceMapper.updateById(task);
         List<Integer> taskInstances = taskInstanceMapper.queryTaskByProcessIdAndState(
                 task.getProcessInstanceId(),
-                ExecutionStatus.RUNNING_EXECUTION.ordinal()
-        );
+                TaskExecutionStatus.RUNNING_EXECUTION.getCode());
         taskInstanceMapper.deleteById(task.getId());
         Assert.assertNotEquals(taskInstances.size(), 0);
     }
@@ -184,8 +184,7 @@ public class TaskInstanceMapperTest extends BaseDaoTest {
 
         List<TaskInstance> taskInstances = taskInstanceMapper.findValidTaskListByProcessId(
                 task.getProcessInstanceId(),
-                Flag.YES
-        );
+                Flag.YES);
 
         task2.setFlag(Flag.NO);
         taskInstanceMapper.updateById(task2);
@@ -212,8 +211,7 @@ public class TaskInstanceMapperTest extends BaseDaoTest {
         taskInstanceMapper.updateById(task);
 
         List<TaskInstance> taskInstances = taskInstanceMapper.queryByHostAndStatus(
-                task.getHost(), new int[]{ExecutionStatus.RUNNING_EXECUTION.ordinal()}
-        );
+                task.getHost(), new int[]{TaskExecutionStatus.RUNNING_EXECUTION.getCode()});
         taskInstanceMapper.deleteById(task.getId());
         Assert.assertNotEquals(taskInstances.size(), 0);
     }
@@ -233,9 +231,8 @@ public class TaskInstanceMapperTest extends BaseDaoTest {
 
         int setResult = taskInstanceMapper.setFailoverByHostAndStateArray(
                 task.getHost(),
-                new int[]{ExecutionStatus.RUNNING_EXECUTION.ordinal()},
-                ExecutionStatus.NEED_FAULT_TOLERANCE
-        );
+                new int[]{TaskExecutionStatus.RUNNING_EXECUTION.getCode()},
+                TaskExecutionStatus.NEED_FAULT_TOLERANCE);
         taskInstanceMapper.deleteById(task.getId());
         Assert.assertNotEquals(setResult, 0);
     }
@@ -255,8 +252,7 @@ public class TaskInstanceMapperTest extends BaseDaoTest {
 
         TaskInstance taskInstance = taskInstanceMapper.queryByInstanceIdAndName(
                 task.getProcessInstanceId(),
-                task.getName()
-        );
+                task.getName());
         taskInstanceMapper.deleteById(task.getId());
         Assert.assertNotEquals(taskInstance, null);
     }
@@ -275,9 +271,8 @@ public class TaskInstanceMapperTest extends BaseDaoTest {
         taskInstanceMapper.updateById(task);
 
         TaskInstance taskInstance = taskInstanceMapper.queryByInstanceIdAndCode(
-            task.getProcessInstanceId(),
-            task.getTaskCode()
-        );
+                task.getProcessInstanceId(),
+                task.getTaskCode());
         taskInstanceMapper.deleteById(task.getId());
         Assert.assertNotEquals(taskInstance, null);
     }
@@ -302,12 +297,10 @@ public class TaskInstanceMapperTest extends BaseDaoTest {
 
         int countTask = taskInstanceMapper.countTask(
                 new Long[0],
-                new int[0]
-        );
+                new int[0]);
         int countTask2 = taskInstanceMapper.countTask(
                 new Long[]{definition.getProjectCode()},
-                new int[]{task.getId()}
-        );
+                new int[]{task.getId()});
         taskInstanceMapper.deleteById(task.getId());
         processDefinitionMapper.deleteById(definition.getId());
         Assert.assertEquals(countTask, 0);
@@ -336,8 +329,7 @@ public class TaskInstanceMapperTest extends BaseDaoTest {
 
         List<ExecuteStatusCount> count = taskInstanceMapper.countTaskInstanceStateByProjectCodes(
                 null, null,
-                new Long[]{definition.getProjectCode()}
-        );
+                new Long[]{definition.getProjectCode()});
 
         processDefinitionMapper.deleteById(definition.getId());
         taskInstanceMapper.deleteById(task.getId());
@@ -372,8 +364,7 @@ public class TaskInstanceMapperTest extends BaseDaoTest {
                 0,
                 new int[0],
                 "",
-                null, null
-        );
+                null, null);
         processInstanceMapper.deleteById(processInstance.getId());
         taskInstanceMapper.deleteById(task.getId());
         processDefinitionMapper.deleteById(definition.getId());
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 e0dafeb5bf..e03aa63cf3 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
@@ -29,7 +29,7 @@ import org.apache.dolphinscheduler.common.utils.JSONUtils;
 import org.apache.dolphinscheduler.dao.entity.ProcessData;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.plugin.task.api.model.SwitchResultVo;
 import org.apache.dolphinscheduler.plugin.task.api.parameters.SwitchParameters;
 
@@ -51,7 +51,7 @@ import com.fasterxml.jackson.core.JsonProcessingException;
 public class DagHelperTest {
 
     @Test
-    public void testHaveSubAfterNode(){
+    public void testHaveSubAfterNode() {
         String parentNodeCode = "5293789969856";
         List<TaskNodeRelation> taskNodeRelations = new ArrayList<>();
         TaskNodeRelation relation = new TaskNodeRelation();
@@ -86,7 +86,7 @@ public class DagHelperTest {
         ProcessDag processDag = new ProcessDag();
         processDag.setEdges(taskNodeRelations);
         processDag.setNodes(taskNodes);
-        DAG<String,TaskNode,TaskNodeRelation> dag = DagHelper.buildDagGraph(processDag);
+        DAG<String, TaskNode, TaskNodeRelation> dag = DagHelper.buildDagGraph(processDag);
         boolean canSubmit = DagHelper.haveAllNodeAfterNode(parentNodeCode, dag);
         Assert.assertTrue(canSubmit);
 
@@ -107,8 +107,8 @@ public class DagHelperTest {
      */
     @Test
     public void testTaskNodeCanSubmit() throws IOException {
-        //1->2->3->5->7
-        //4->3->6
+        // 1->2->3->5->7
+        // 4->3->6
         DAG<String, TaskNode, TaskNodeRelation> dag = generateDag();
         TaskNode taskNode3 = dag.getNode("3");
         Map<String, TaskInstance> completeTaskList = new HashMap<>();
@@ -151,14 +151,14 @@ public class DagHelperTest {
         Map<String, TaskNode> skipNodeList = new HashMap<>();
 
         Set<String> postNodes = null;
-        //complete : null
+        // complete : null
         // expect post: 1/4
         postNodes = DagHelper.parsePostNodes(null, skipNodeList, dag, completeTaskList);
         Assert.assertEquals(2, postNodes.size());
         Assert.assertTrue(postNodes.contains("1"));
         Assert.assertTrue(postNodes.contains("4"));
 
-        //complete : 1
+        // complete : 1
         // expect post: 2/4
         completeTaskList.put("1", new TaskInstance());
         postNodes = DagHelper.parsePostNodes(null, skipNodeList, dag, completeTaskList);
@@ -218,7 +218,7 @@ public class DagHelperTest {
         Map<String, TaskNode> skipNodeList = new HashMap<>();
         Set<String> postNodes = null;
         // dag: 1-2-3-5-7 4-3-6 2-8-5-7
-        // forbid:2 complete:1  post:4/8
+        // forbid:2 complete:1 post:4/8
         completeTaskList.put("1", new TaskInstance());
         TaskNode node2 = dag.getNode("2");
         node2.setRunFlag(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN);
@@ -227,7 +227,7 @@ public class DagHelperTest {
         Assert.assertTrue(postNodes.contains("4"));
         Assert.assertTrue(postNodes.contains("8"));
 
-        //forbid:2/4 complete:1 post:3/8
+        // forbid:2/4 complete:1 post:3/8
         TaskNode node4 = dag.getNode("4");
         node4.setRunFlag(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN);
         postNodes = DagHelper.parsePostNodes(null, skipNodeList, dag, completeTaskList);
@@ -235,7 +235,7 @@ public class DagHelperTest {
         Assert.assertTrue(postNodes.contains("3"));
         Assert.assertTrue(postNodes.contains("8"));
 
-        //forbid:2/4/5 complete:1/8 post:3
+        // forbid:2/4/5 complete:1/8 post:3
         completeTaskList.put("8", new TaskInstance());
         TaskNode node5 = dag.getNode("5");
         node5.setRunFlag(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN);
@@ -275,24 +275,24 @@ public class DagHelperTest {
                 "            }");
         completeTaskList.remove("3");
         TaskInstance taskInstance = new TaskInstance();
-        taskInstance.setState(ExecutionStatus.SUCCESS);
-        //complete 1/2/3/4 expect:8
+        taskInstance.setState(TaskExecutionStatus.SUCCESS);
+        // complete 1/2/3/4 expect:8
         completeTaskList.put("3", taskInstance);
         postNodes = DagHelper.parsePostNodes(null, skipNodeList, dag, completeTaskList);
         Assert.assertEquals(1, postNodes.size());
         Assert.assertTrue(postNodes.contains("8"));
 
-        //2.complete 1/2/3/4/8 expect:5 skip:6
+        // 2.complete 1/2/3/4/8 expect:5 skip:6
         completeTaskList.put("8", new TaskInstance());
         postNodes = DagHelper.parsePostNodes(null, skipNodeList, dag, completeTaskList);
         Assert.assertTrue(postNodes.contains("5"));
         Assert.assertEquals(1, skipNodeList.size());
         Assert.assertTrue(skipNodeList.containsKey("6"));
 
-        // 3.complete 1/2/3/4/5/8  expect post:7 skip:6
+        // 3.complete 1/2/3/4/5/8 expect post:7 skip:6
         skipNodeList.clear();
         TaskInstance taskInstance1 = new TaskInstance();
-        taskInstance.setState(ExecutionStatus.SUCCESS);
+        taskInstance.setState(TaskExecutionStatus.SUCCESS);
         completeTaskList.put("5", taskInstance1);
         postNodes = DagHelper.parsePostNodes(null, skipNodeList, dag, completeTaskList);
         Assert.assertEquals(1, postNodes.size());
@@ -310,7 +310,7 @@ public class DagHelperTest {
         Map<String, Object> taskParamsMap = new HashMap<>();
         taskParamsMap.put(Constants.SWITCH_RESULT, "");
         taskInstance.setTaskParams(JSONUtils.toJsonString(taskParamsMap));
-        taskInstance.setState(ExecutionStatus.FAILURE);
+        taskInstance.setState(TaskExecutionStatus.FAILURE);
         completeTaskList.put("3", taskInstance);
         postNodes = DagHelper.parsePostNodes(null, skipNodeList, dag, completeTaskList);
         Assert.assertEquals(1, postNodes.size());
@@ -526,15 +526,16 @@ public class DagHelperTest {
 
     @Test
     public void testBuildDagGraph() {
-        String shellJson = "{\"globalParams\":[],\"tasks\":[{\"type\":\"SHELL\",\"id\":\"tasks-9527\",\"name\":\"shell-1\","
-                +
-                "\"params\":{\"resourceList\":[],\"localParams\":[],\"rawScript\":\"#!/bin/bash\\necho \\\"shell-1\\\"\"},"
-                +
-                "\"description\":\"\",\"runFlag\":\"NORMAL\",\"dependence\":{},\"maxRetryTimes\":\"0\",\"retryInterval\":\"1\","
-                +
-                "\"timeout\":{\"strategy\":\"\",\"interval\":1,\"enable\":false},\"taskInstancePriority\":\"MEDIUM\","
-                +
-                "\"workerGroupId\":-1,\"preTasks\":[]}],\"tenantId\":1,\"timeout\":0}";
+        String shellJson =
+                "{\"globalParams\":[],\"tasks\":[{\"type\":\"SHELL\",\"id\":\"tasks-9527\",\"name\":\"shell-1\","
+                        +
+                        "\"params\":{\"resourceList\":[],\"localParams\":[],\"rawScript\":\"#!/bin/bash\\necho \\\"shell-1\\\"\"},"
+                        +
+                        "\"description\":\"\",\"runFlag\":\"NORMAL\",\"dependence\":{},\"maxRetryTimes\":\"0\",\"retryInterval\":\"1\","
+                        +
+                        "\"timeout\":{\"strategy\":\"\",\"interval\":1,\"enable\":false},\"taskInstancePriority\":\"MEDIUM\","
+                        +
+                        "\"workerGroupId\":-1,\"preTasks\":[]}],\"tenantId\":1,\"timeout\":0}";
 
         ProcessData processData = JSONUtils.parseObject(shellJson, ProcessData.class);
         assert processData != null;
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/builder/TaskExecutionContextBuilder.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/builder/TaskExecutionContextBuilder.java
index 975b7a64b4..65b3952efd 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/builder/TaskExecutionContextBuilder.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/builder/TaskExecutionContextBuilder.java
@@ -27,7 +27,7 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.plugin.task.api.DataQualityTaskExecutionContext;
 import org.apache.dolphinscheduler.plugin.task.api.K8sTaskExecutionContext;
 import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy;
 import org.apache.dolphinscheduler.plugin.task.api.model.Property;
 import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
@@ -43,7 +43,7 @@ public class TaskExecutionContextBuilder {
         return new TaskExecutionContextBuilder();
     }
 
-    private TaskExecutionContext taskExecutionContext =  new TaskExecutionContext();
+    private TaskExecutionContext taskExecutionContext = new TaskExecutionContext();
 
     /**
      * build taskInstance related info
@@ -65,7 +65,7 @@ public class TaskExecutionContextBuilder {
         taskExecutionContext.setDelayTime(taskInstance.getDelayTime());
         taskExecutionContext.setVarPool(taskInstance.getVarPool());
         taskExecutionContext.setDryRun(taskInstance.getDryRun());
-        taskExecutionContext.setCurrentExecutionStatus(ExecutionStatus.SUBMITTED_SUCCESS);
+        taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.SUBMITTED_SUCCESS);
         taskExecutionContext.setCpuQuota(taskInstance.getCpuQuota());
         taskExecutionContext.setMemoryMax(taskInstance.getMemoryMax());
         return this;
@@ -76,8 +76,9 @@ public class TaskExecutionContextBuilder {
         if (taskDefinition.getTimeoutFlag() == TimeoutFlag.OPEN) {
             taskExecutionContext.setTaskTimeoutStrategy(taskDefinition.getTimeoutNotifyStrategy());
             if (taskDefinition.getTimeoutNotifyStrategy() == TaskTimeoutStrategy.FAILED
-                || taskDefinition.getTimeoutNotifyStrategy() == TaskTimeoutStrategy.WARNFAILED) {
-                taskExecutionContext.setTaskTimeout(Math.min(taskDefinition.getTimeout() * SEC_2_MINUTES_TIME_UNIT, Integer.MAX_VALUE));
+                    || taskDefinition.getTimeoutNotifyStrategy() == TaskTimeoutStrategy.WARNFAILED) {
+                taskExecutionContext.setTaskTimeout(
+                        Math.min(taskDefinition.getTimeout() * SEC_2_MINUTES_TIME_UNIT, Integer.MAX_VALUE));
             }
         }
         taskExecutionContext.setTaskParams(taskDefinition.getTaskParams());
@@ -155,7 +156,6 @@ public class TaskExecutionContextBuilder {
         return this;
     }
 
-
     /**
      * create
      *
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/ProcessInstanceExecCacheManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/ProcessInstanceExecCacheManager.java
index 59e064105a..0eb0f09a73 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/ProcessInstanceExecCacheManager.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/ProcessInstanceExecCacheManager.java
@@ -17,12 +17,12 @@
 
 package org.apache.dolphinscheduler.server.master.cache;
 
-import lombok.NonNull;
-
 import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
 
 import java.util.Collection;
 
+import lombok.NonNull;
+
 /**
  * cache of process instance id and WorkflowExecuteThread
  */
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java
index 9c7d048ff8..a346142a74 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java
@@ -258,7 +258,7 @@ public class TaskPriorityQueueConsumer extends BaseDaemonThread {
      */
     public boolean taskInstanceIsFinalState(int taskInstanceId) {
         TaskInstance taskInstance = processService.findTaskInstanceById(taskInstanceId);
-        return taskInstance.getState().typeIsFinished();
+        return taskInstance.getState().isFinished();
     }
 
     /**
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/StateEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/StateEvent.java
index 68cd582994..f3a9a6579e 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/StateEvent.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/StateEvent.java
@@ -17,35 +17,32 @@
 
 package org.apache.dolphinscheduler.server.master.event;
 
+import lombok.NonNull;
 import org.apache.dolphinscheduler.common.enums.StateEventType;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 
 import io.netty.channel.Channel;
-import lombok.Data;
+
+import javax.annotation.Nullable;
 
 /**
  * state event
  */
-@Data
-public class StateEvent {
-
-    /**
-     * origin_pid-origin_task_id-process_instance_id-task_instance_id
-     */
-    private String key;
-
-    private StateEventType type;
+public interface StateEvent {
 
-    private ExecutionStatus executionStatus;
+    int getProcessInstanceId();
 
-    private int taskInstanceId;
+    int getTaskInstanceId();
 
-    private long taskCode;
+    @NonNull
+    StateEventType getType();
 
-    private int processInstanceId;
+    @Nullable
+    String getKey();
 
-    private String context;
+    @Nullable
+    Channel getChannel();
 
-    private Channel channel;
+    @Nullable
+    String getContext();
 
 }
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDelayEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDelayEventHandler.java
index ec3382aa3f..51a0830561 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDelayEventHandler.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDelayEventHandler.java
@@ -21,7 +21,6 @@ import org.apache.dolphinscheduler.common.enums.StateEventType;
 import org.apache.dolphinscheduler.common.enums.TaskEventType;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.dao.utils.TaskInstanceUtils;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.remote.command.TaskExecuteRunningAckMessage;
 import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
 import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent;
@@ -56,7 +55,7 @@ public class TaskDelayEventHandler implements TaskEventHandler {
         int processInstanceId = taskEvent.getProcessInstanceId();
 
         WorkflowExecuteRunnable workflowExecuteRunnable =
-            this.processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId);
+                this.processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId);
         if (workflowExecuteRunnable == null) {
             sendAckToWorker(taskEvent);
             throw new TaskEventHandleError("Cannot find related workflow instance from cache");
@@ -67,11 +66,11 @@ public class TaskDelayEventHandler implements TaskEventHandler {
             return;
         }
         TaskInstance taskInstance = taskInstanceOptional.get();
-        if (taskInstance.getState().typeIsFinished()) {
+        if (taskInstance.getState().isFinished()) {
             logger.warn(
-                "The current task status is: {}, will not handle the running event, this event is delay, will discard this event: {}",
-                taskInstance.getState(),
-                taskEvent);
+                    "The current task status is: {}, will not handle the running event, this event is delay, will discard this event: {}",
+                    taskInstance.getState(),
+                    taskEvent);
             sendAckToWorker(taskEvent);
             return;
         }
@@ -97,11 +96,12 @@ public class TaskDelayEventHandler implements TaskEventHandler {
             }
             throw new TaskEventHandleError("Handle task dispatch event error, update taskInstance to db failed", ex);
         }
-        StateEvent stateEvent = new StateEvent();
-        stateEvent.setProcessInstanceId(taskEvent.getProcessInstanceId());
-        stateEvent.setTaskInstanceId(taskEvent.getTaskInstanceId());
-        stateEvent.setExecutionStatus(taskEvent.getState());
-        stateEvent.setType(StateEventType.TASK_STATE_CHANGE);
+        TaskStateEvent stateEvent = TaskStateEvent.builder()
+                .processInstanceId(taskEvent.getProcessInstanceId())
+                .taskInstanceId(taskEvent.getTaskInstanceId())
+                .status(taskEvent.getState())
+                .type(StateEventType.TASK_STATE_CHANGE)
+                .build();
         workflowExecuteThreadPool.submitStateEvent(stateEvent);
 
     }
@@ -109,7 +109,7 @@ public class TaskDelayEventHandler implements TaskEventHandler {
     private void sendAckToWorker(TaskEvent taskEvent) {
         // If event handle success, send ack to worker to otherwise the worker will retry this event
         TaskExecuteRunningAckMessage taskExecuteRunningAckMessage =
-            new TaskExecuteRunningAckMessage(ExecutionStatus.SUCCESS, taskEvent.getTaskInstanceId());
+                new TaskExecuteRunningAckMessage(true, taskEvent.getTaskInstanceId());
         taskEvent.getChannel().writeAndFlush(taskExecuteRunningAckMessage.convert2Command());
     }
 
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDispatchEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDispatchEventHandler.java
index 9378f0c36e..83d11db80d 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDispatchEventHandler.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskDispatchEventHandler.java
@@ -20,7 +20,7 @@ package org.apache.dolphinscheduler.server.master.event;
 import org.apache.dolphinscheduler.common.enums.TaskEventType;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.dao.utils.TaskInstanceUtils;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
 import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent;
 import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
@@ -48,16 +48,16 @@ public class TaskDispatchEventHandler implements TaskEventHandler {
         int processInstanceId = taskEvent.getProcessInstanceId();
 
         WorkflowExecuteRunnable workflowExecuteRunnable =
-            this.processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId);
+                this.processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId);
         if (workflowExecuteRunnable == null) {
             throw new TaskEventHandleError("Cannot find related workflow instance from cache");
         }
         TaskInstance taskInstance = workflowExecuteRunnable.getTaskInstance(taskInstanceId)
-            .orElseThrow(() -> new TaskEventHandleError("Cannot find related taskInstance from cache"));
-        if (taskInstance.getState() != ExecutionStatus.SUBMITTED_SUCCESS) {
+                .orElseThrow(() -> new TaskEventHandleError("Cannot find related taskInstance from cache"));
+        if (taskInstance.getState() != TaskExecutionStatus.SUBMITTED_SUCCESS) {
             logger.warn(
-                "The current taskInstance status is not SUBMITTED_SUCCESS, so the dispatch event will be discarded, the current is a delay event, event: {}",
-                taskEvent);
+                    "The current taskInstance status is not SUBMITTED_SUCCESS, so the dispatch event will be discarded, the current is a delay event, event: {}",
+                    taskEvent);
             return;
         }
 
@@ -65,7 +65,7 @@ public class TaskDispatchEventHandler implements TaskEventHandler {
         TaskInstance oldTaskInstance = new TaskInstance();
         TaskInstanceUtils.copyTaskInstance(taskInstance, oldTaskInstance);
         // update the taskInstance status
-        taskInstance.setState(ExecutionStatus.DISPATCH);
+        taskInstance.setState(TaskExecutionStatus.DISPATCH);
         taskInstance.setHost(taskEvent.getWorkerAddress());
         try {
             if (!processService.updateTaskInstance(taskInstance)) {
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRejectByWorkerEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRejectByWorkerEventHandler.java
index ab63ada0ed..77ac5e52a2 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRejectByWorkerEventHandler.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRejectByWorkerEventHandler.java
@@ -19,7 +19,6 @@ package org.apache.dolphinscheduler.server.master.event;
 
 import org.apache.dolphinscheduler.common.enums.TaskEventType;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.remote.command.TaskRejectAckCommand;
 import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
 import org.apache.dolphinscheduler.server.master.config.MasterConfig;
@@ -44,19 +43,20 @@ public class TaskRejectByWorkerEventHandler implements TaskEventHandler {
         int processInstanceId = taskEvent.getProcessInstanceId();
 
         WorkflowExecuteRunnable workflowExecuteRunnable = this.processInstanceExecCacheManager.getByProcessInstanceId(
-            processInstanceId);
+                processInstanceId);
         if (workflowExecuteRunnable == null) {
             sendAckToWorker(taskEvent);
             throw new TaskEventHandleError(
-                "Handle task reject event error, cannot find related workflow instance from cache, will discard this event");
+                    "Handle task reject event error, cannot find related workflow instance from cache, will discard this event");
         }
         TaskInstance taskInstance = workflowExecuteRunnable.getTaskInstance(taskInstanceId).orElseThrow(() -> {
             sendAckToWorker(taskEvent);
             return new TaskEventHandleError(
-                "Handle task reject event error, cannot find the taskInstance from cache, will discord this event");
+                    "Handle task reject event error, cannot find the taskInstance from cache, will discord this event");
         });
         try {
-            // todo: If the worker submit multiple reject response to master, the task instance may be dispatch multiple,
+            // todo: If the worker submit multiple reject response to master, the task instance may be dispatch
+            // multiple,
             // we need to control the worker overload by master rather than worker
             // if the task resubmit and the worker failover, this task may be dispatch twice?
             // todo: we need to clear the taskInstance host and rollback the status to submit.
@@ -69,11 +69,11 @@ public class TaskRejectByWorkerEventHandler implements TaskEventHandler {
     }
 
     public void sendAckToWorker(TaskEvent taskEvent) {
-        TaskRejectAckCommand taskRejectAckMessage = new TaskRejectAckCommand(ExecutionStatus.SUCCESS,
-                                                                             taskEvent.getTaskInstanceId(),
-                                                                             masterConfig.getMasterAddress(),
-                                                                             taskEvent.getWorkerAddress(),
-                                                                             System.currentTimeMillis());
+        TaskRejectAckCommand taskRejectAckMessage = new TaskRejectAckCommand(true,
+                taskEvent.getTaskInstanceId(),
+                masterConfig.getMasterAddress(),
+                taskEvent.getWorkerAddress(),
+                System.currentTimeMillis());
         taskEvent.getChannel().writeAndFlush(taskRejectAckMessage.convert2Command());
     }
 
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskResultEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskResultEventHandler.java
index b50a81311d..5f36248d9e 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskResultEventHandler.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskResultEventHandler.java
@@ -21,7 +21,6 @@ import org.apache.dolphinscheduler.common.enums.StateEventType;
 import org.apache.dolphinscheduler.common.enums.TaskEventType;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.dao.utils.TaskInstanceUtils;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.remote.command.TaskExecuteAckCommand;
 import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
 import org.apache.dolphinscheduler.server.master.config.MasterConfig;
@@ -60,23 +59,23 @@ public class TaskResultEventHandler implements TaskEventHandler {
         int processInstanceId = taskEvent.getProcessInstanceId();
 
         WorkflowExecuteRunnable workflowExecuteRunnable = this.processInstanceExecCacheManager.getByProcessInstanceId(
-            processInstanceId);
+                processInstanceId);
         if (workflowExecuteRunnable == null) {
             sendAckToWorker(taskEvent);
             throw new TaskEventHandleError(
-                "Handle task result event error, cannot find related workflow instance from cache, will discard this event");
+                    "Handle task result event error, cannot find related workflow instance from cache, will discard this event");
         }
         Optional<TaskInstance> taskInstanceOptional = workflowExecuteRunnable.getTaskInstance(taskInstanceId);
         if (!taskInstanceOptional.isPresent()) {
             sendAckToWorker(taskEvent);
             throw new TaskEventHandleError(
-                "Handle task result event error, cannot find the taskInstance from cache, will discord this event");
+                    "Handle task result event error, cannot find the taskInstance from cache, will discord this event");
         }
         TaskInstance taskInstance = taskInstanceOptional.get();
-        if (taskInstance.getState().typeIsFinished()) {
+        if (taskInstance.getState().isFinished()) {
             sendAckToWorker(taskEvent);
             throw new TaskEventHandleError(
-                "Handle task result event error, the task instance is already finished, will discord this event");
+                    "Handle task result event error, the task instance is already finished, will discord this event");
         }
         dataQualityResultOperator.operateDqExecuteResult(taskEvent, taskInstance);
 
@@ -99,22 +98,23 @@ public class TaskResultEventHandler implements TaskEventHandler {
             TaskInstanceUtils.copyTaskInstance(oldTaskInstance, taskInstance);
             throw new TaskEventHandleError("Handle task result event error, save taskInstance to db error", ex);
         }
-        StateEvent stateEvent = new StateEvent();
-        stateEvent.setProcessInstanceId(taskEvent.getProcessInstanceId());
-        stateEvent.setTaskInstanceId(taskEvent.getTaskInstanceId());
-        stateEvent.setExecutionStatus(taskEvent.getState());
-        stateEvent.setType(StateEventType.TASK_STATE_CHANGE);
+        TaskStateEvent stateEvent = TaskStateEvent.builder()
+                .processInstanceId(taskEvent.getProcessInstanceId())
+                .taskInstanceId(taskEvent.getTaskInstanceId())
+                .status(taskEvent.getState())
+                .type(StateEventType.TASK_STATE_CHANGE)
+                .build();
         workflowExecuteThreadPool.submitStateEvent(stateEvent);
 
     }
 
     public void sendAckToWorker(TaskEvent taskEvent) {
         // we didn't set the receiver address, since the ack doen's need to retry
-        TaskExecuteAckCommand taskExecuteAckMessage = new TaskExecuteAckCommand(ExecutionStatus.SUCCESS,
-                                                                                taskEvent.getTaskInstanceId(),
-                                                                                masterConfig.getMasterAddress(),
-                                                                                taskEvent.getWorkerAddress(),
-                                                                                System.currentTimeMillis());
+        TaskExecuteAckCommand taskExecuteAckMessage = new TaskExecuteAckCommand(true,
+                taskEvent.getTaskInstanceId(),
+                masterConfig.getMasterAddress(),
+                taskEvent.getWorkerAddress(),
+                System.currentTimeMillis());
         taskEvent.getChannel().writeAndFlush(taskExecuteAckMessage.convert2Command());
     }
 
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRetryStateEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRetryStateEventHandler.java
index f6f7069ab3..1ac198497f 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRetryStateEventHandler.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRetryStateEventHandler.java
@@ -17,6 +17,7 @@
 
 package org.apache.dolphinscheduler.server.master.event;
 
+import com.google.auto.service.AutoService;
 import org.apache.dolphinscheduler.common.enums.StateEventType;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics;
@@ -24,19 +25,20 @@ import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
 
 import java.util.Map;
 
-import com.google.auto.service.AutoService;
-
 @AutoService(StateEventHandler.class)
 public class TaskRetryStateEventHandler implements StateEventHandler {
+
     @Override
-    public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable, StateEvent stateEvent)
-        throws StateEventHandleException {
+    public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable,
+                                    StateEvent stateEvent) throws StateEventHandleException {
+        TaskStateEvent taskStateEvent = (TaskStateEvent) stateEvent;
+
         TaskMetrics.incTaskInstanceByState("retry");
         Map<Long, TaskInstance> waitToRetryTaskInstanceMap = workflowExecuteRunnable.getWaitToRetryTaskInstanceMap();
-        TaskInstance taskInstance = waitToRetryTaskInstanceMap.get(stateEvent.getTaskCode());
+        TaskInstance taskInstance = waitToRetryTaskInstanceMap.get(taskStateEvent.getTaskCode());
         workflowExecuteRunnable.addTaskToStandByList(taskInstance);
         workflowExecuteRunnable.submitStandByTask();
-        waitToRetryTaskInstanceMap.remove(stateEvent.getTaskCode());
+        waitToRetryTaskInstanceMap.remove(taskStateEvent.getTaskCode());
         return true;
     }
 
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRunningEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRunningEventHandler.java
index a4de2d4b44..fe449a9f6f 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRunningEventHandler.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRunningEventHandler.java
@@ -21,23 +21,22 @@ import org.apache.dolphinscheduler.common.enums.StateEventType;
 import org.apache.dolphinscheduler.common.enums.TaskEventType;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.dao.utils.TaskInstanceUtils;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.remote.command.TaskExecuteRunningAckMessage;
 import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
 import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent;
 import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
 import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThreadPool;
 import org.apache.dolphinscheduler.service.process.ProcessService;
-
-import java.util.Optional;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Component;
 
+import java.util.Optional;
+
 @Component
 public class TaskRunningEventHandler implements TaskEventHandler {
+
     private final Logger logger = LoggerFactory.getLogger(TaskRunningEventHandler.class);
 
     @Autowired
@@ -55,23 +54,23 @@ public class TaskRunningEventHandler implements TaskEventHandler {
         int processInstanceId = taskEvent.getProcessInstanceId();
 
         WorkflowExecuteRunnable workflowExecuteRunnable =
-            this.processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId);
+                this.processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId);
         if (workflowExecuteRunnable == null) {
             sendAckToWorker(taskEvent);
             throw new TaskEventHandleError(
-                "Handle task running event error, cannot find related workflow instance from cache, will discard this event");
+                    "Handle task running event error, cannot find related workflow instance from cache, will discard this event");
         }
         Optional<TaskInstance> taskInstanceOptional = workflowExecuteRunnable.getTaskInstance(taskInstanceId);
         if (!taskInstanceOptional.isPresent()) {
             sendAckToWorker(taskEvent);
             throw new TaskEventHandleError(
-                "Handle running event error, cannot find the taskInstance from cache, will discord this event");
+                    "Handle running event error, cannot find the taskInstance from cache, will discord this event");
         }
         TaskInstance taskInstance = taskInstanceOptional.get();
-        if (taskInstance.getState().typeIsFinished()) {
+        if (taskInstance.getState().isFinished()) {
             sendAckToWorker(taskEvent);
             throw new TaskEventHandleError(
-                "Handle task running event error, this task instance is already finished, this event is delay, will discard this event");
+                    "Handle task running event error, this task instance is already finished, this event is delay, will discard this event");
         }
 
         TaskInstance oldTaskInstance = new TaskInstance();
@@ -96,18 +95,19 @@ public class TaskRunningEventHandler implements TaskEventHandler {
             throw new TaskEventHandleError("Handle task running event error, update taskInstance to db failed", ex);
         }
 
-        StateEvent stateEvent = new StateEvent();
-        stateEvent.setProcessInstanceId(taskEvent.getProcessInstanceId());
-        stateEvent.setTaskInstanceId(taskEvent.getTaskInstanceId());
-        stateEvent.setExecutionStatus(taskEvent.getState());
-        stateEvent.setType(StateEventType.TASK_STATE_CHANGE);
+        TaskStateEvent stateEvent = TaskStateEvent.builder()
+                .processInstanceId(taskEvent.getProcessInstanceId())
+                .taskInstanceId(taskEvent.getTaskInstanceId())
+                .status(taskEvent.getState())
+                .type(StateEventType.TASK_STATE_CHANGE)
+                .build();
         workflowExecuteThreadPool.submitStateEvent(stateEvent);
     }
 
     private void sendAckToWorker(TaskEvent taskEvent) {
         // If event handle success, send ack to worker to otherwise the worker will retry this event
         TaskExecuteRunningAckMessage taskExecuteRunningAckMessage =
-            new TaskExecuteRunningAckMessage(ExecutionStatus.SUCCESS, taskEvent.getTaskInstanceId());
+                new TaskExecuteRunningAckMessage(true, taskEvent.getTaskInstanceId());
         taskEvent.getChannel().writeAndFlush(taskExecuteRunningAckMessage.convert2Command());
     }
 
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/StateEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEvent.java
similarity index 74%
copy from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/StateEvent.java
copy to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEvent.java
index 68cd582994..1ad96b9346 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/StateEvent.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEvent.java
@@ -17,35 +17,36 @@
 
 package org.apache.dolphinscheduler.server.master.event;
 
-import org.apache.dolphinscheduler.common.enums.StateEventType;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
-
 import io.netty.channel.Channel;
+import lombok.AllArgsConstructor;
+import lombok.Builder;
 import lombok.Data;
+import lombok.NoArgsConstructor;
+import lombok.NonNull;
+import org.apache.dolphinscheduler.common.enums.StateEventType;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 
-/**
- * state event
- */
 @Data
-public class StateEvent {
-
-    /**
-     * origin_pid-origin_task_id-process_instance_id-task_instance_id
-     */
-    private String key;
-
-    private StateEventType type;
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
+public class TaskStateEvent implements StateEvent {
 
-    private ExecutionStatus executionStatus;
+    // todo: use wrapper type
+    private int processInstanceId;
 
     private int taskInstanceId;
 
     private long taskCode;
 
-    private int processInstanceId;
+    private TaskExecutionStatus status;
 
-    private String context;
+    private @NonNull StateEventType type;
+
+    private String key;
 
     private Channel channel;
 
+    private String context;
+
 }
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEventHandler.java
index 9854f95605..417d2ddec4 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEventHandler.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEventHandler.java
@@ -38,16 +38,17 @@ public class TaskStateEventHandler implements StateEventHandler {
     private static final Logger logger = LoggerFactory.getLogger(TaskStateEventHandler.class);
 
     @Override
-    public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable, StateEvent stateEvent)
-        throws StateEventHandleException, StateEventHandleError {
-        measureTaskState(stateEvent);
-        workflowExecuteRunnable.checkTaskInstanceByStateEvent(stateEvent);
+    public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable,
+                                    StateEvent stateEvent) throws StateEventHandleException, StateEventHandleError {
+        TaskStateEvent taskStateEvent = (TaskStateEvent) stateEvent;
+        measureTaskState(taskStateEvent);
+        workflowExecuteRunnable.checkTaskInstanceByStateEvent(taskStateEvent);
 
         Optional<TaskInstance> taskInstanceOptional =
-            workflowExecuteRunnable.getTaskInstance(stateEvent.getTaskInstanceId());
+                workflowExecuteRunnable.getTaskInstance(taskStateEvent.getTaskInstanceId());
 
         TaskInstance task = taskInstanceOptional.orElseThrow(() -> new StateEventHandleError(
-            "Cannot find task instance from taskMap by task instance id: " + stateEvent.getTaskInstanceId()));
+                "Cannot find task instance from taskMap by task instance id: " + taskStateEvent.getTaskInstanceId()));
 
         if (task.getState() == null) {
             throw new StateEventHandleError("Task state event handle error due to task state is null");
@@ -55,9 +56,9 @@ public class TaskStateEventHandler implements StateEventHandler {
 
         Map<Long, Integer> completeTaskMap = workflowExecuteRunnable.getCompleteTaskMap();
 
-        if (task.getState().typeIsFinished()) {
+        if (task.getState().isFinished()) {
             if (completeTaskMap.containsKey(task.getTaskCode())
-                && completeTaskMap.get(task.getTaskCode()) == task.getId()) {
+                    && completeTaskMap.get(task.getTaskCode()) == task.getId()) {
                 logger.warn("The task instance is already complete, stateEvent: {}", stateEvent);
                 return true;
             }
@@ -73,7 +74,7 @@ public class TaskStateEventHandler implements StateEventHandler {
             ITaskProcessor iTaskProcessor = activeTaskProcessMap.get(task.getTaskCode());
             iTaskProcessor.action(TaskAction.RUN);
 
-            if (iTaskProcessor.taskInstance().getState().typeIsFinished()) {
+            if (iTaskProcessor.taskInstance().getState().isFinished()) {
                 if (iTaskProcessor.taskInstance().getState() != task.getState()) {
                     task.setState(iTaskProcessor.taskInstance().getState());
                 }
@@ -82,7 +83,7 @@ public class TaskStateEventHandler implements StateEventHandler {
             return true;
         }
         throw new StateEventHandleException(
-            "Task state event handle error, due to the task is not in activeTaskProcessorMaps");
+                "Task state event handle error, due to the task is not in activeTaskProcessorMaps");
     }
 
     @Override
@@ -90,18 +91,18 @@ public class TaskStateEventHandler implements StateEventHandler {
         return StateEventType.TASK_STATE_CHANGE;
     }
 
-    private void measureTaskState(StateEvent taskStateEvent) {
-        if (taskStateEvent == null || taskStateEvent.getExecutionStatus() == null) {
+    private void measureTaskState(TaskStateEvent taskStateEvent) {
+        if (taskStateEvent == null || taskStateEvent.getStatus() == null) {
             // the event is broken
             logger.warn("The task event is broken..., taskEvent: {}", taskStateEvent);
             return;
         }
-        if (taskStateEvent.getExecutionStatus().typeIsFinished()) {
+        if (taskStateEvent.getStatus().isFinished()) {
             TaskMetrics.incTaskInstanceByState("finish");
         }
-        switch (taskStateEvent.getExecutionStatus()) {
-            case STOP:
-                TaskMetrics.incTaskInstanceByState("stop");
+        switch (taskStateEvent.getStatus()) {
+            case KILL:
+                TaskMetrics.incTaskInstanceByState("kill");
                 break;
             case SUCCESS:
                 TaskMetrics.incTaskInstanceByState("success");
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskTimeoutStateEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskTimeoutStateEventHandler.java
index c43c0bcbf2..22f3f9dd8f 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskTimeoutStateEventHandler.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskTimeoutStateEventHandler.java
@@ -17,6 +17,7 @@
 
 package org.apache.dolphinscheduler.server.master.event;
 
+import com.google.auto.service.AutoService;
 import org.apache.dolphinscheduler.common.enums.StateEventType;
 import org.apache.dolphinscheduler.common.enums.TimeoutFlag;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
@@ -28,17 +29,22 @@ import org.apache.dolphinscheduler.server.master.runner.task.TaskAction;
 
 import java.util.Map;
 
-import com.google.auto.service.AutoService;
-
 @AutoService(StateEventHandler.class)
 public class TaskTimeoutStateEventHandler implements StateEventHandler {
+
     @Override
-    public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable, StateEvent stateEvent)
-        throws StateEventHandleError {
+    public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable,
+                                    StateEvent stateEvent) throws StateEventHandleError {
+        TaskStateEvent taskStateEvent = (TaskStateEvent) stateEvent;
+
         TaskMetrics.incTaskInstanceByState("timeout");
-        workflowExecuteRunnable.checkTaskInstanceByStateEvent(stateEvent);
+        workflowExecuteRunnable.checkTaskInstanceByStateEvent(taskStateEvent);
 
-        TaskInstance taskInstance = workflowExecuteRunnable.getTaskInstance(stateEvent.getTaskInstanceId()).get();
+        TaskInstance taskInstance =
+                workflowExecuteRunnable.getTaskInstance(taskStateEvent.getTaskInstanceId()).orElseThrow(
+                        () -> new StateEventHandleError(String.format(
+                                "Cannot find the task instance from workflow execute runnable, taskInstanceId: %s",
+                                taskStateEvent.getTaskInstanceId())));
 
         if (TimeoutFlag.CLOSE == taskInstance.getTaskDefine().getTimeoutFlag()) {
             return true;
@@ -46,7 +52,7 @@ public class TaskTimeoutStateEventHandler implements StateEventHandler {
         TaskTimeoutStrategy taskTimeoutStrategy = taskInstance.getTaskDefine().getTimeoutNotifyStrategy();
         Map<Long, ITaskProcessor> activeTaskProcessMap = workflowExecuteRunnable.getActiveTaskProcessMap();
         if (TaskTimeoutStrategy.FAILED == taskTimeoutStrategy
-            || TaskTimeoutStrategy.WARNFAILED == taskTimeoutStrategy) {
+                || TaskTimeoutStrategy.WARNFAILED == taskTimeoutStrategy) {
             ITaskProcessor taskProcessor = activeTaskProcessMap.get(taskInstance.getTaskCode());
             taskProcessor.action(TaskAction.TIMEOUT);
         }
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStartEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStartEventHandler.java
index c598cb5a90..8c5e27c117 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStartEventHandler.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStartEventHandler.java
@@ -50,32 +50,30 @@ public class WorkflowStartEventHandler implements WorkflowEventHandler {
     private WorkflowEventQueue workflowEventQueue;
 
     @Override
-    public void handleWorkflowEvent(WorkflowEvent workflowEvent) throws WorkflowEventHandleError {
+    public void handleWorkflowEvent(final WorkflowEvent workflowEvent) throws WorkflowEventHandleError {
         logger.info("Handle workflow start event, begin to start a workflow, event: {}", workflowEvent);
-        WorkflowExecuteRunnable workflowExecuteRunnable =
-            processInstanceExecCacheManager.getByProcessInstanceId(workflowEvent.getWorkflowInstanceId());
+        WorkflowExecuteRunnable workflowExecuteRunnable = processInstanceExecCacheManager.getByProcessInstanceId(
+            workflowEvent.getWorkflowInstanceId());
         if (workflowExecuteRunnable == null) {
             throw new WorkflowEventHandleError(
                 "The workflow start event is invalid, cannot find the workflow instance from cache");
         }
-        ProcessInstance processInstance = workflowExecuteRunnable.getProcessInstance();
         ProcessInstanceMetrics.incProcessInstanceByState("submit");
-        CompletableFuture<WorkflowSubmitStatue> workflowSubmitFuture =
-            CompletableFuture.supplyAsync(workflowExecuteRunnable::call, workflowExecuteThreadPool);
-        workflowSubmitFuture.thenAccept(workflowSubmitStatue -> {
-            if (WorkflowSubmitStatue.SUCCESS == workflowSubmitStatue) {
-                // submit failed will resend the event to workflow event queue
-                logger.info("Success submit the workflow instance");
-                if (processInstance.getTimeout() > 0) {
-                    stateWheelExecuteThread.addProcess4TimeoutCheck(processInstance);
+        ProcessInstance processInstance = workflowExecuteRunnable.getProcessInstance();
+        CompletableFuture.supplyAsync(workflowExecuteRunnable::call, workflowExecuteThreadPool)
+            .thenAccept(workflowSubmitStatue -> {
+                if (WorkflowSubmitStatue.SUCCESS == workflowSubmitStatue) {
+                    // submit failed will resend the event to workflow event queue
+                    logger.info("Success submit the workflow instance");
+                    if (processInstance.getTimeout() > 0) {
+                        stateWheelExecuteThread.addProcess4TimeoutCheck(processInstance);
+                    }
+                } else {
+                    logger.error("Failed to submit the workflow instance, will resend the workflow start event: {}",
+                                 workflowEvent);
+                    workflowEventQueue.addEvent(workflowEvent);
                 }
-            } else {
-                logger.error("Failed to submit the workflow instance, will resend the workflow start event: {}",
-                             workflowEvent);
-                workflowEventQueue.addEvent(new WorkflowEvent(WorkflowEventType.START_WORKFLOW,
-                                                              processInstance.getId()));
-            }
-        });
+            });
     }
 
     @Override
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/StateEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStateEvent.java
similarity index 76%
copy from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/StateEvent.java
copy to dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStateEvent.java
index 68cd582994..3267108228 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/StateEvent.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStateEvent.java
@@ -17,35 +17,32 @@
 
 package org.apache.dolphinscheduler.server.master.event;
 
-import org.apache.dolphinscheduler.common.enums.StateEventType;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
-
 import io.netty.channel.Channel;
-import lombok.Data;
+import lombok.*;
+import org.apache.dolphinscheduler.common.enums.StateEventType;
+import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
 
-/**
- * state event
- */
 @Data
-public class StateEvent {
+@Builder
+@AllArgsConstructor
+@NoArgsConstructor
+public class WorkflowStateEvent implements StateEvent {
+
+    // todo: use wrapper type
+    private int processInstanceId;
 
     /**
-     * origin_pid-origin_task_id-process_instance_id-task_instance_id
+     * Some event may contains taskInstanceId
      */
-    private String key;
-
-    private StateEventType type;
-
-    private ExecutionStatus executionStatus;
-
     private int taskInstanceId;
 
-    private long taskCode;
+    private WorkflowExecutionStatus status;
 
-    private int processInstanceId;
+    private @NonNull StateEventType type;
 
-    private String context;
+    private String key;
 
     private Channel channel;
 
+    private String context;
 }
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStateEventHandler.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStateEventHandler.java
index a37b3023a3..c3f49111a7 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStateEventHandler.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStateEventHandler.java
@@ -17,51 +17,49 @@
 
 package org.apache.dolphinscheduler.server.master.event;
 
+import com.google.auto.service.AutoService;
 import org.apache.dolphinscheduler.common.enums.StateEventType;
 import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics;
 import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.auto.service.AutoService;
-
 @AutoService(StateEventHandler.class)
 public class WorkflowStateEventHandler implements StateEventHandler {
 
     private static final Logger logger = LoggerFactory.getLogger(WorkflowStateEventHandler.class);
 
     @Override
-    public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable, StateEvent stateEvent)
-        throws StateEventHandleException {
-        measureProcessState(stateEvent);
+    public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable,
+                                    StateEvent stateEvent) throws StateEventHandleException {
+        WorkflowStateEvent workflowStateEvent = (WorkflowStateEvent) stateEvent;
+        measureProcessState(workflowStateEvent);
         ProcessInstance processInstance = workflowExecuteRunnable.getProcessInstance();
         ProcessDefinition processDefinition = processInstance.getProcessDefinition();
 
         logger.info(
-            "Handle workflow instance state event, the current workflow instance state {} will be changed to {}",
-            processInstance.getState(), stateEvent.getExecutionStatus());
+                "Handle workflow instance state event, the current workflow instance state {} will be changed to {}",
+                processInstance.getState(), workflowStateEvent.getStatus());
 
-        if (stateEvent.getExecutionStatus() == ExecutionStatus.STOP) {
+        if (workflowStateEvent.getStatus().isStop()) {
             // serial wait execution type needs to wake up the waiting process
             if (processDefinition.getExecutionType().typeIsSerialWait() || processDefinition.getExecutionType()
-                                                                                            .typeIsSerialPriority()) {
+                    .typeIsSerialPriority()) {
                 workflowExecuteRunnable.endProcess();
                 return true;
             }
-            workflowExecuteRunnable.updateProcessInstanceState(stateEvent);
+            workflowExecuteRunnable.updateProcessInstanceState(workflowStateEvent);
             return true;
         }
         if (workflowExecuteRunnable.processComplementData()) {
             return true;
         }
-        if (stateEvent.getExecutionStatus().typeIsFinished()) {
+        if (workflowStateEvent.getStatus().isFinished()) {
             workflowExecuteRunnable.endProcess();
         }
-        if (processInstance.getState() == ExecutionStatus.READY_STOP) {
+        if (processInstance.getState().isReadyStop()) {
             workflowExecuteRunnable.killAllTasks();
         }
 
@@ -73,11 +71,11 @@ public class WorkflowStateEventHandler implements StateEventHandler {
         return StateEventType.PROCESS_STATE_CHANGE;
     }
 
-    private void measureProcessState(StateEvent processStateEvent) {
-        if (processStateEvent.getExecutionStatus().typeIsFinished()) {
+    private void measureProcessState(WorkflowStateEvent processStateEvent) {
+        if (processStateEvent.getStatus().isFinished()) {
             ProcessInstanceMetrics.incProcessInstanceByState("finish");
         }
-        switch (processStateEvent.getExecutionStatus()) {
+        switch (processStateEvent.getStatus()) {
             case STOP:
                 ProcessInstanceMetrics.incProcessInstanceByState("stop");
                 break;
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/StateEventProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/StateEventProcessor.java
index 18afd11ae5..9e859cd2e7 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/StateEventProcessor.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/StateEventProcessor.java
@@ -17,26 +17,25 @@
 
 package org.apache.dolphinscheduler.server.master.processor;
 
+import com.google.common.base.Preconditions;
+import io.netty.channel.Channel;
 import org.apache.dolphinscheduler.common.enums.StateEventType;
+import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
 import org.apache.dolphinscheduler.common.utils.LoggerUtils;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.remote.command.Command;
 import org.apache.dolphinscheduler.remote.command.CommandType;
-import org.apache.dolphinscheduler.remote.command.StateEventChangeCommand;
+import org.apache.dolphinscheduler.remote.command.WorkflowStateEventChangeCommand;
 import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
 import org.apache.dolphinscheduler.server.master.event.StateEvent;
+import org.apache.dolphinscheduler.server.master.event.TaskStateEvent;
+import org.apache.dolphinscheduler.server.master.event.WorkflowStateEvent;
 import org.apache.dolphinscheduler.server.master.processor.queue.StateEventResponseService;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Component;
 
-import com.google.common.base.Preconditions;
-
-import io.netty.channel.Channel;
-
 /**
  * handle state event received from master/api
  */
@@ -50,25 +49,21 @@ public class StateEventProcessor implements NettyRequestProcessor {
 
     @Override
     public void process(Channel channel, Command command) {
-        Preconditions.checkArgument(CommandType.STATE_EVENT_REQUEST == command.getType(), String.format("invalid command type: %s", command.getType()));
+        Preconditions.checkArgument(CommandType.STATE_EVENT_REQUEST == command.getType(),
+                String.format("invalid command type: %s", command.getType()));
 
-        StateEventChangeCommand stateEventChangeCommand = JSONUtils.parseObject(command.getBody(), StateEventChangeCommand.class);
-        StateEvent stateEvent = new StateEvent();
-        stateEvent.setKey(stateEventChangeCommand.getKey());
-        if (stateEventChangeCommand.getSourceProcessInstanceId() != stateEventChangeCommand.getDestProcessInstanceId()) {
-            stateEvent.setExecutionStatus(ExecutionStatus.RUNNING_EXECUTION);
+        WorkflowStateEventChangeCommand workflowStateEventChangeCommand =
+                JSONUtils.parseObject(command.getBody(), WorkflowStateEventChangeCommand.class);
+        StateEvent stateEvent;
+        if (workflowStateEventChangeCommand.getDestTaskInstanceId() == 0) {
+            stateEvent = createWorkflowStateEvent(workflowStateEventChangeCommand);
         } else {
-            stateEvent.setExecutionStatus(stateEventChangeCommand.getSourceStatus());
+            stateEvent = createTaskStateEvent(workflowStateEventChangeCommand);
         }
-        stateEvent.setProcessInstanceId(stateEventChangeCommand.getDestProcessInstanceId());
-        stateEvent.setTaskInstanceId(stateEventChangeCommand.getDestTaskInstanceId());
-        StateEventType
-            type = stateEvent.getTaskInstanceId() == 0 ? StateEventType.PROCESS_STATE_CHANGE : StateEventType.TASK_STATE_CHANGE;
-        stateEvent.setType(type);
 
         try {
             LoggerUtils.setWorkflowAndTaskInstanceIDMDC(stateEvent.getProcessInstanceId(),
-                stateEvent.getTaskInstanceId());
+                    stateEvent.getTaskInstanceId());
 
             logger.info("Received state change command, event: {}", stateEvent);
             stateEventResponseService.addStateChangeEvent(stateEvent);
@@ -78,4 +73,27 @@ public class StateEventProcessor implements NettyRequestProcessor {
 
     }
 
+    private TaskStateEvent createTaskStateEvent(WorkflowStateEventChangeCommand workflowStateEventChangeCommand) {
+        return TaskStateEvent.builder()
+                .processInstanceId(workflowStateEventChangeCommand.getDestProcessInstanceId())
+                .taskInstanceId(workflowStateEventChangeCommand.getDestTaskInstanceId())
+                .type(StateEventType.TASK_STATE_CHANGE)
+                .key(workflowStateEventChangeCommand.getKey())
+                .build();
+    }
+
+    private WorkflowStateEvent createWorkflowStateEvent(WorkflowStateEventChangeCommand workflowStateEventChangeCommand) {
+        WorkflowExecutionStatus workflowExecutionStatus = workflowStateEventChangeCommand.getSourceStatus();
+        if (workflowStateEventChangeCommand.getSourceProcessInstanceId() != workflowStateEventChangeCommand
+                .getDestProcessInstanceId()) {
+            workflowExecutionStatus = WorkflowExecutionStatus.RUNNING_EXECUTION;
+        }
+        return WorkflowStateEvent.builder()
+                .processInstanceId(workflowStateEventChangeCommand.getDestProcessInstanceId())
+                .type(StateEventType.PROCESS_STATE_CHANGE)
+                .status(workflowExecutionStatus)
+                .key(workflowStateEventChangeCommand.getKey())
+                .build();
+    }
+
 }
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskEventProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskEventProcessor.java
index 42d4ab4db7..22faf7f05a 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskEventProcessor.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskEventProcessor.java
@@ -17,7 +17,8 @@
 
 package org.apache.dolphinscheduler.server.master.processor;
 
-import org.apache.dolphinscheduler.server.master.event.StateEvent;
+import com.google.common.base.Preconditions;
+import io.netty.channel.Channel;
 import org.apache.dolphinscheduler.common.enums.StateEventType;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
 import org.apache.dolphinscheduler.common.utils.LoggerUtils;
@@ -25,17 +26,13 @@ import org.apache.dolphinscheduler.remote.command.Command;
 import org.apache.dolphinscheduler.remote.command.CommandType;
 import org.apache.dolphinscheduler.remote.command.TaskEventChangeCommand;
 import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
+import org.apache.dolphinscheduler.server.master.event.TaskStateEvent;
 import org.apache.dolphinscheduler.server.master.processor.queue.StateEventResponseService;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Component;
 
-import com.google.common.base.Preconditions;
-
-import io.netty.channel.Channel;
-
 /**
  * handle state event received from master/api
  */
@@ -50,17 +47,20 @@ public class TaskEventProcessor implements NettyRequestProcessor {
     @Override
     public void process(Channel channel, Command command) {
         Preconditions.checkArgument(CommandType.TASK_FORCE_STATE_EVENT_REQUEST == command.getType()
-                || CommandType.TASK_WAKEUP_EVENT_REQUEST == command.getType()
-            , String.format("invalid command type: %s", command.getType()));
+                || CommandType.TASK_WAKEUP_EVENT_REQUEST == command.getType(),
+                String.format("invalid command type: %s", command.getType()));
 
-        TaskEventChangeCommand taskEventChangeCommand = JSONUtils.parseObject(command.getBody(), TaskEventChangeCommand.class);
-        StateEvent stateEvent = new StateEvent();
-        stateEvent.setKey(taskEventChangeCommand.getKey());
-        stateEvent.setProcessInstanceId(taskEventChangeCommand.getProcessInstanceId());
-        stateEvent.setTaskInstanceId(taskEventChangeCommand.getTaskInstanceId());
-        stateEvent.setType(StateEventType.WAIT_TASK_GROUP);
+        TaskEventChangeCommand taskEventChangeCommand =
+                JSONUtils.parseObject(command.getBody(), TaskEventChangeCommand.class);
+        TaskStateEvent stateEvent = TaskStateEvent.builder()
+                .processInstanceId(taskEventChangeCommand.getProcessInstanceId())
+                .taskInstanceId(taskEventChangeCommand.getTaskInstanceId())
+                .key(taskEventChangeCommand.getKey())
+                .type(StateEventType.WAIT_TASK_GROUP)
+                .build();
         try {
-            LoggerUtils.setWorkflowAndTaskInstanceIDMDC(stateEvent.getProcessInstanceId(), stateEvent.getTaskInstanceId());
+            LoggerUtils.setWorkflowAndTaskInstanceIDMDC(stateEvent.getProcessInstanceId(),
+                    stateEvent.getTaskInstanceId());
             logger.info("Received task event change command, event: {}", stateEvent);
             stateEventResponseService.addEvent2WorkflowExecute(stateEvent);
         } finally {
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/StateEventResponseService.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/StateEventResponseService.java
index a09d661ebd..282219b8ec 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/StateEventResponseService.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/StateEventResponseService.java
@@ -17,30 +17,26 @@
 
 package org.apache.dolphinscheduler.server.master.processor.queue;
 
-import org.apache.dolphinscheduler.server.master.event.StateEvent;
+import io.netty.channel.Channel;
 import org.apache.dolphinscheduler.common.thread.BaseDaemonThread;
 import org.apache.dolphinscheduler.common.thread.Stopper;
 import org.apache.dolphinscheduler.common.utils.LoggerUtils;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.remote.command.StateEventResponseCommand;
 import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
+import org.apache.dolphinscheduler.server.master.event.StateEvent;
 import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
 import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThreadPool;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import javax.annotation.PostConstruct;
-import javax.annotation.PreDestroy;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Component;
 
-import io.netty.channel.Channel;
+import javax.annotation.PostConstruct;
+import javax.annotation.PreDestroy;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
 
 @Component
 public class StateEventResponseService {
@@ -80,7 +76,8 @@ public class StateEventResponseService {
             eventQueue.drainTo(remainEvents);
             for (StateEvent event : remainEvents) {
                 try {
-                    LoggerUtils.setWorkflowAndTaskInstanceIDMDC(event.getProcessInstanceId(), event.getTaskInstanceId());
+                    LoggerUtils.setWorkflowAndTaskInstanceIDMDC(event.getProcessInstanceId(),
+                            event.getTaskInstanceId());
                     this.persist(event);
 
                 } finally {
@@ -119,7 +116,8 @@ public class StateEventResponseService {
                 try {
                     // if not task , blocking here
                     StateEvent stateEvent = eventQueue.take();
-                    LoggerUtils.setWorkflowAndTaskInstanceIDMDC(stateEvent.getProcessInstanceId(), stateEvent.getTaskInstanceId());
+                    LoggerUtils.setWorkflowAndTaskInstanceIDMDC(stateEvent.getProcessInstanceId(),
+                            stateEvent.getTaskInstanceId());
                     persist(stateEvent);
                 } catch (InterruptedException e) {
                     logger.warn("State event loop service interrupted, will stop this loop", e);
@@ -133,10 +131,10 @@ public class StateEventResponseService {
         }
     }
 
-    private void writeResponse(StateEvent stateEvent, ExecutionStatus status) {
+    private void writeResponse(StateEvent stateEvent) {
         Channel channel = stateEvent.getChannel();
         if (channel != null) {
-            StateEventResponseCommand command = new StateEventResponseCommand(status, stateEvent.getKey());
+            StateEventResponseCommand command = new StateEventResponseCommand(stateEvent.getKey());
             channel.writeAndFlush(command.convert2Command());
         }
     }
@@ -145,12 +143,13 @@ public class StateEventResponseService {
         try {
             if (!this.processInstanceExecCacheManager.contains(stateEvent.getProcessInstanceId())) {
                 logger.warn("Persist event into workflow execute thread error, "
-                    + "cannot find the workflow instance from cache manager, event: {}", stateEvent);
-                writeResponse(stateEvent, ExecutionStatus.FAILURE);
+                        + "cannot find the workflow instance from cache manager, event: {}", stateEvent);
+                writeResponse(stateEvent);
                 return;
             }
 
-            WorkflowExecuteRunnable workflowExecuteThread = this.processInstanceExecCacheManager.getByProcessInstanceId(stateEvent.getProcessInstanceId());
+            WorkflowExecuteRunnable workflowExecuteThread =
+                    this.processInstanceExecCacheManager.getByProcessInstanceId(stateEvent.getProcessInstanceId());
             // We will refresh the task instance status first, if the refresh failed the event will not be removed
             switch (stateEvent.getType()) {
                 case TASK_STATE_CHANGE:
@@ -162,7 +161,8 @@ public class StateEventResponseService {
                 default:
             }
             workflowExecuteThreadPool.submitStateEvent(stateEvent);
-            writeResponse(stateEvent, ExecutionStatus.SUCCESS);
+            // this response is not needed.
+            writeResponse(stateEvent);
         } catch (Exception e) {
             logger.error("Persist event queue error, event: {}", stateEvent, e);
         }
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskEvent.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskEvent.java
index 1a367a96a2..9f6679c9d8 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskEvent.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskEvent.java
@@ -18,7 +18,7 @@
 package org.apache.dolphinscheduler.server.master.processor.queue;
 
 import org.apache.dolphinscheduler.common.enums.TaskEventType;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.remote.command.TaskExecuteResultCommand;
 import org.apache.dolphinscheduler.remote.command.TaskExecuteRunningCommand;
 import org.apache.dolphinscheduler.remote.command.TaskRejectCommand;
@@ -47,7 +47,7 @@ public class TaskEvent {
     /**
      * state
      */
-    private ExecutionStatus state;
+    private TaskExecutionStatus state;
 
     /**
      * start time
@@ -123,7 +123,7 @@ public class TaskEvent {
         TaskEvent event = new TaskEvent();
         event.setProcessInstanceId(command.getProcessInstanceId());
         event.setTaskInstanceId(command.getTaskInstanceId());
-        event.setState(ExecutionStatus.of(command.getStatus()));
+        event.setState(TaskExecutionStatus.of(command.getStatus()));
         event.setStartTime(command.getStartTime());
         event.setExecutePath(command.getExecutePath());
         event.setLogPath(command.getLogPath());
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterConnectionStateListener.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterConnectionStateListener.java
index bc1b217f9e..d0b86fc660 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterConnectionStateListener.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterConnectionStateListener.java
@@ -17,8 +17,6 @@
 
 package org.apache.dolphinscheduler.server.master.registry;
 
-import static com.google.common.base.Preconditions.checkNotNull;
-
 import org.apache.dolphinscheduler.registry.api.ConnectionListener;
 import org.apache.dolphinscheduler.registry.api.ConnectionState;
 import org.apache.dolphinscheduler.service.registry.RegistryClient;
@@ -26,6 +24,8 @@ import org.apache.dolphinscheduler.service.registry.RegistryClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import lombok.NonNull;
+
 public class MasterConnectionStateListener implements ConnectionListener {
 
     private static final Logger logger = LoggerFactory.getLogger(MasterConnectionStateListener.class);
@@ -33,9 +33,9 @@ public class MasterConnectionStateListener implements ConnectionListener {
     private final String masterNodePath;
     private final RegistryClient registryClient;
 
-    public MasterConnectionStateListener(String masterNodePath, RegistryClient registryClient) {
-        this.masterNodePath = checkNotNull(masterNodePath);
-        this.registryClient = checkNotNull(registryClient);
+    public MasterConnectionStateListener(@NonNull String masterNodePath, @NonNull RegistryClient registryClient) {
+        this.masterNodePath = masterNodePath;
+        this.registryClient = registryClient;
     }
 
     @Override
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistryClient.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistryClient.java
index aa536bae49..722a322288 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistryClient.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistryClient.java
@@ -219,10 +219,9 @@ public class MasterRegistryClient implements AutoCloseable {
 
     public void deregister() {
         try {
-            String address = getLocalAddress();
             String localNodePath = getCurrentNodePath();
             registryClient.remove(localNodePath);
-            logger.info("Master node : {} unRegistry to register center.", address);
+            logger.info("Master node : {} unRegistry to register center.", masterAddress);
             heartBeatExecutor.shutdown();
             logger.info("MasterServer heartbeat executor shutdown");
             registryClient.close();
@@ -235,15 +234,7 @@ public class MasterRegistryClient implements AutoCloseable {
      * get master path
      */
     private String getCurrentNodePath() {
-        String address = getLocalAddress();
-        return REGISTRY_DOLPHINSCHEDULER_MASTERS + "/" + address;
-    }
-
-    /**
-     * get local address
-     */
-    private String getLocalAddress() {
-        return NetUtils.getAddr(masterConfig.getListenPort());
+        return REGISTRY_DOLPHINSCHEDULER_MASTERS + "/" + masterAddress;
     }
 
 }
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StateWheelExecuteThread.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StateWheelExecuteThread.java
index c053cb238b..9ea9b6574b 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StateWheelExecuteThread.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StateWheelExecuteThread.java
@@ -17,9 +17,11 @@
 
 package org.apache.dolphinscheduler.server.master.runner;
 
+import lombok.NonNull;
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.StateEventType;
 import org.apache.dolphinscheduler.common.enums.TimeoutFlag;
+import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
 import org.apache.dolphinscheduler.common.thread.BaseDaemonThread;
 import org.apache.dolphinscheduler.common.thread.Stopper;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
@@ -27,23 +29,20 @@ import org.apache.dolphinscheduler.common.utils.LoggerUtils;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
 import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
 import org.apache.dolphinscheduler.server.master.config.MasterConfig;
-import org.apache.dolphinscheduler.server.master.event.StateEvent;
+import org.apache.dolphinscheduler.server.master.event.TaskStateEvent;
+import org.apache.dolphinscheduler.server.master.event.WorkflowStateEvent;
 import org.apache.dolphinscheduler.server.master.runner.task.TaskInstanceKey;
-
-import java.util.Optional;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import javax.annotation.PostConstruct;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Component;
 
-import lombok.NonNull;
+import javax.annotation.PostConstruct;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentLinkedQueue;
 
 /**
  * Check thread
@@ -137,10 +136,10 @@ public class StateWheelExecuteThread extends BaseDaemonThread {
             try {
                 LoggerUtils.setWorkflowInstanceIdMDC(processInstanceId);
                 WorkflowExecuteRunnable workflowExecuteThread = processInstanceExecCacheManager.getByProcessInstanceId(
-                    processInstanceId);
+                        processInstanceId);
                 if (workflowExecuteThread == null) {
                     logger.warn(
-                        "Check workflow timeout failed, can not find workflowExecuteThread from cache manager, will remove this workflowInstance from check list");
+                            "Check workflow timeout failed, can not find workflowExecuteThread from cache manager, will remove this workflowInstance from check list");
                     processInstanceTimeoutCheckList.remove(processInstanceId);
                     continue;
                 }
@@ -150,8 +149,8 @@ public class StateWheelExecuteThread extends BaseDaemonThread {
                     continue;
                 }
                 long timeRemain = DateUtils.getRemainTime(processInstance.getStartTime(),
-                                                          (long) processInstance.getTimeout()
-                                                              * Constants.SEC_2_MINUTES_TIME_UNIT);
+                        (long) processInstance.getTimeout()
+                                * Constants.SEC_2_MINUTES_TIME_UNIT);
                 if (timeRemain < 0) {
                     logger.info("Workflow instance timeout, adding timeout event");
                     addProcessTimeoutEvent(processInstance);
@@ -208,7 +207,7 @@ public class StateWheelExecuteThread extends BaseDaemonThread {
         }
         taskInstanceRetryCheckList.add(taskInstanceKey);
         logger.info("[WorkflowInstance-{}][TaskInstance-{}] Added task instance into retry check list",
-            processInstance.getId(), taskInstance.getId());
+                processInstance.getId(), taskInstance.getId());
     }
 
     public void removeTask4RetryCheck(@NonNull ProcessInstance processInstance, @NonNull TaskInstance taskInstance) {
@@ -246,26 +245,29 @@ public class StateWheelExecuteThread extends BaseDaemonThread {
                 LoggerUtils.setWorkflowInstanceIdMDC(processInstanceId);
                 long taskCode = taskInstanceKey.getTaskCode();
 
-                WorkflowExecuteRunnable workflowExecuteThread = processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId);
+                WorkflowExecuteRunnable workflowExecuteThread =
+                        processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId);
                 if (workflowExecuteThread == null) {
-                    logger.warn("Check task instance timeout failed, can not find workflowExecuteThread from cache manager, will remove this check task");
+                    logger.warn(
+                            "Check task instance timeout failed, can not find workflowExecuteThread from cache manager, will remove this check task");
                     taskInstanceTimeoutCheckList.remove(taskInstanceKey);
                     continue;
                 }
-                Optional<TaskInstance> taskInstanceOptional = workflowExecuteThread.getActiveTaskInstanceByTaskCode(taskCode);
+                Optional<TaskInstance> taskInstanceOptional =
+                        workflowExecuteThread.getActiveTaskInstanceByTaskCode(taskCode);
                 if (!taskInstanceOptional.isPresent()) {
                     logger.warn(
-                        "Check task instance timeout failed, can not get taskInstance from workflowExecuteThread, taskCode: {}"
-                            + "will remove this check task",
-                        taskCode);
+                            "Check task instance timeout failed, can not get taskInstance from workflowExecuteThread, taskCode: {}"
+                                    + "will remove this check task",
+                            taskCode);
                     taskInstanceTimeoutCheckList.remove(taskInstanceKey);
                     continue;
                 }
                 TaskInstance taskInstance = taskInstanceOptional.get();
                 if (TimeoutFlag.OPEN == taskInstance.getTaskDefine().getTimeoutFlag()) {
                     long timeRemain = DateUtils.getRemainTime(taskInstance.getStartTime(),
-                                                              (long) taskInstance.getTaskDefine().getTimeout()
-                                                                  * Constants.SEC_2_MINUTES_TIME_UNIT);
+                            (long) taskInstance.getTaskDefine().getTimeout()
+                                    * Constants.SEC_2_MINUTES_TIME_UNIT);
                     if (timeRemain < 0) {
                         logger.info("Task instance is timeout, adding task timeout event and remove the check");
                         addTaskTimeoutEvent(taskInstance);
@@ -291,21 +293,24 @@ public class StateWheelExecuteThread extends BaseDaemonThread {
             try {
                 LoggerUtils.setWorkflowInstanceIdMDC(processInstanceId);
 
-                WorkflowExecuteRunnable workflowExecuteThread = processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId);
+                WorkflowExecuteRunnable workflowExecuteThread =
+                        processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId);
 
                 if (workflowExecuteThread == null) {
                     logger.warn(
-                        "Task instance retry check failed, can not find workflowExecuteThread from cache manager, "
-                            + "will remove this check task");
+                            "Task instance retry check failed, can not find workflowExecuteThread from cache manager, "
+                                    + "will remove this check task");
                     taskInstanceRetryCheckList.remove(taskInstanceKey);
                     continue;
                 }
 
-                Optional<TaskInstance> taskInstanceOptional = workflowExecuteThread.getRetryTaskInstanceByTaskCode(taskCode);
+                Optional<TaskInstance> taskInstanceOptional =
+                        workflowExecuteThread.getRetryTaskInstanceByTaskCode(taskCode);
                 ProcessInstance processInstance = workflowExecuteThread.getProcessInstance();
 
-                if (processInstance.getState() == ExecutionStatus.READY_STOP) {
-                    logger.warn("The process instance is ready to stop, will send process stop event and remove the check task");
+                if (processInstance.getState().isReadyStop()) {
+                    logger.warn(
+                            "The process instance is ready to stop, will send process stop event and remove the check task");
                     addProcessStopEvent(processInstance);
                     taskInstanceRetryCheckList.remove(taskInstanceKey);
                     break;
@@ -313,22 +318,24 @@ public class StateWheelExecuteThread extends BaseDaemonThread {
 
                 if (!taskInstanceOptional.isPresent()) {
                     logger.warn(
-                        "Task instance retry check failed, can not find taskInstance from workflowExecuteThread, will remove this check");
+                            "Task instance retry check failed, can not find taskInstance from workflowExecuteThread, will remove this check");
                     taskInstanceRetryCheckList.remove(taskInstanceKey);
                     continue;
                 }
 
                 TaskInstance taskInstance = taskInstanceOptional.get();
-                // We check the status to avoid when we do worker failover we submit a failover task, this task may be resubmit by this
+                // We check the status to avoid when we do worker failover we submit a failover task, this task may be
+                // resubmit by this
                 // thread
-                if (taskInstance.getState() != ExecutionStatus.NEED_FAULT_TOLERANCE
-                    && taskInstance.retryTaskIntervalOverTime()) {
+                if (taskInstance.getState() != TaskExecutionStatus.NEED_FAULT_TOLERANCE
+                        && taskInstance.retryTaskIntervalOverTime()) {
                     // reset taskInstance endTime and state
-                    // todo relative funtion: TaskInstance.retryTaskIntervalOverTime, WorkflowExecuteThread.cloneRetryTaskInstance
+                    // todo relative funtion: TaskInstance.retryTaskIntervalOverTime,
+                    // WorkflowExecuteThread.cloneRetryTaskInstance
                     logger.info("[TaskInstance-{}]The task instance can retry, will retry this task instance",
-                                taskInstance.getId());
+                            taskInstance.getId());
                     taskInstance.setEndTime(null);
-                    taskInstance.setState(ExecutionStatus.SUBMITTED_SUCCESS);
+                    taskInstance.setState(TaskExecutionStatus.SUBMITTED_SUCCESS);
 
                     addTaskRetryEvent(taskInstance);
                     taskInstanceRetryCheckList.remove(taskInstanceKey);
@@ -351,21 +358,24 @@ public class StateWheelExecuteThread extends BaseDaemonThread {
 
             try {
                 LoggerUtils.setTaskInstanceIdMDC(processInstanceId);
-                WorkflowExecuteRunnable workflowExecuteThread = processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId);
+                WorkflowExecuteRunnable workflowExecuteThread =
+                        processInstanceExecCacheManager.getByProcessInstanceId(processInstanceId);
                 if (workflowExecuteThread == null) {
-                    logger.warn("Task instance state check failed, can not find workflowExecuteThread from cache manager, will remove this check task");
+                    logger.warn(
+                            "Task instance state check failed, can not find workflowExecuteThread from cache manager, will remove this check task");
                     taskInstanceStateCheckList.remove(taskInstanceKey);
                     continue;
                 }
-                Optional<TaskInstance> taskInstanceOptional = workflowExecuteThread.getActiveTaskInstanceByTaskCode(taskCode);
+                Optional<TaskInstance> taskInstanceOptional =
+                        workflowExecuteThread.getActiveTaskInstanceByTaskCode(taskCode);
                 if (!taskInstanceOptional.isPresent()) {
                     logger.warn(
-                        "Task instance state check failed, can not find taskInstance from workflowExecuteThread, will remove this check event");
+                            "Task instance state check failed, can not find taskInstance from workflowExecuteThread, will remove this check event");
                     taskInstanceStateCheckList.remove(taskInstanceKey);
                     continue;
                 }
                 TaskInstance taskInstance = taskInstanceOptional.get();
-                if (taskInstance.getState().typeIsFinished()) {
+                if (taskInstance.getState().isFinished()) {
                     continue;
                 }
                 addTaskStateChangeEvent(taskInstance);
@@ -378,46 +388,51 @@ public class StateWheelExecuteThread extends BaseDaemonThread {
     }
 
     private void addTaskStateChangeEvent(TaskInstance taskInstance) {
-        StateEvent stateEvent = new StateEvent();
-        stateEvent.setType(StateEventType.TASK_STATE_CHANGE);
-        stateEvent.setProcessInstanceId(taskInstance.getProcessInstanceId());
-        stateEvent.setTaskInstanceId(taskInstance.getId());
-        stateEvent.setTaskCode(taskInstance.getTaskCode());
-        stateEvent.setExecutionStatus(ExecutionStatus.RUNNING_EXECUTION);
+        TaskStateEvent stateEvent = TaskStateEvent.builder()
+                .processInstanceId(taskInstance.getProcessInstanceId())
+                .taskInstanceId(taskInstance.getId())
+                .taskCode(taskInstance.getTaskCode())
+                .type(StateEventType.TASK_STATE_CHANGE)
+                .status(TaskExecutionStatus.RUNNING_EXECUTION)
+                .build();
         workflowExecuteThreadPool.submitStateEvent(stateEvent);
     }
 
     private void addProcessStopEvent(ProcessInstance processInstance) {
-        StateEvent stateEvent = new StateEvent();
-        stateEvent.setType(StateEventType.PROCESS_STATE_CHANGE);
-        stateEvent.setProcessInstanceId(processInstance.getId());
-        stateEvent.setExecutionStatus(ExecutionStatus.STOP);
+        WorkflowStateEvent stateEvent = WorkflowStateEvent.builder()
+                .processInstanceId(processInstance.getId())
+                .type(StateEventType.PROCESS_STATE_CHANGE)
+                .status(WorkflowExecutionStatus.STOP)
+                .build();
         workflowExecuteThreadPool.submitStateEvent(stateEvent);
     }
 
     private void addTaskRetryEvent(TaskInstance taskInstance) {
-        StateEvent stateEvent = new StateEvent();
-        stateEvent.setType(StateEventType.TASK_RETRY);
-        stateEvent.setProcessInstanceId(taskInstance.getProcessInstanceId());
-        stateEvent.setTaskInstanceId(taskInstance.getId());
-        stateEvent.setTaskCode(taskInstance.getTaskCode());
-        stateEvent.setExecutionStatus(ExecutionStatus.RUNNING_EXECUTION);
+        TaskStateEvent stateEvent = TaskStateEvent.builder()
+                .processInstanceId(taskInstance.getProcessInstanceId())
+                .taskInstanceId(taskInstance.getId())
+                .taskCode(taskInstance.getTaskCode())
+                .status(TaskExecutionStatus.RUNNING_EXECUTION)
+                .type(StateEventType.TASK_RETRY)
+                .build();
         workflowExecuteThreadPool.submitStateEvent(stateEvent);
     }
 
     private void addTaskTimeoutEvent(TaskInstance taskInstance) {
-        StateEvent stateEvent = new StateEvent();
-        stateEvent.setType(StateEventType.TASK_TIMEOUT);
-        stateEvent.setProcessInstanceId(taskInstance.getProcessInstanceId());
-        stateEvent.setTaskInstanceId(taskInstance.getId());
-        stateEvent.setTaskCode(taskInstance.getTaskCode());
+        TaskStateEvent stateEvent = TaskStateEvent.builder()
+                .processInstanceId(taskInstance.getProcessInstanceId())
+                .taskInstanceId(taskInstance.getId())
+                .type(StateEventType.TASK_TIMEOUT)
+                .taskCode(taskInstance.getTaskCode())
+                .build();
         workflowExecuteThreadPool.submitStateEvent(stateEvent);
     }
 
     private void addProcessTimeoutEvent(ProcessInstance processInstance) {
-        StateEvent stateEvent = new StateEvent();
-        stateEvent.setType(StateEventType.PROCESS_TIMEOUT);
-        stateEvent.setProcessInstanceId(processInstance.getId());
+        WorkflowStateEvent stateEvent = WorkflowStateEvent.builder()
+                .processInstanceId(processInstance.getId())
+                .type(StateEventType.PROCESS_TIMEOUT)
+                .build();
         workflowExecuteThreadPool.submitStateEvent(stateEvent);
     }
 
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java
index 22e79daa6d..19b1ab5984 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java
@@ -31,13 +31,7 @@ import static org.apache.dolphinscheduler.plugin.task.api.enums.DataType.VARCHAR
 import static org.apache.dolphinscheduler.plugin.task.api.enums.Direct.IN;
 
 import org.apache.dolphinscheduler.common.Constants;
-import org.apache.dolphinscheduler.common.enums.CommandType;
-import org.apache.dolphinscheduler.common.enums.FailureStrategy;
-import org.apache.dolphinscheduler.common.enums.Flag;
-import org.apache.dolphinscheduler.common.enums.Priority;
-import org.apache.dolphinscheduler.common.enums.StateEventType;
-import org.apache.dolphinscheduler.common.enums.TaskDependType;
-import org.apache.dolphinscheduler.common.enums.TaskGroupQueueStatus;
+import org.apache.dolphinscheduler.common.enums.*;
 import org.apache.dolphinscheduler.common.graph.DAG;
 import org.apache.dolphinscheduler.common.model.TaskNode;
 import org.apache.dolphinscheduler.common.model.TaskNodeRelation;
@@ -60,17 +54,13 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.dao.utils.DagHelper;
 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.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.plugin.task.api.model.Property;
 import org.apache.dolphinscheduler.remote.command.HostUpdateCommand;
 import org.apache.dolphinscheduler.remote.utils.Host;
 import org.apache.dolphinscheduler.server.master.config.MasterConfig;
 import org.apache.dolphinscheduler.server.master.dispatch.executor.NettyExecutorManager;
-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.event.*;
 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.TaskAction;
@@ -137,7 +127,6 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
      */
     private String key;
 
-
     private WorkflowRunnableStatus workflowRunnableStatus = WorkflowRunnableStatus.CREATED;
 
     /**
@@ -225,13 +214,13 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
      * @param stateWheelExecuteThread stateWheelExecuteThread
      */
     public WorkflowExecuteRunnable(
-        @NonNull ProcessInstance processInstance,
-        @NonNull ProcessService processService,
-        @NonNull NettyExecutorManager nettyExecutorManager,
-        @NonNull ProcessAlertManager processAlertManager,
-        @NonNull MasterConfig masterConfig,
-        @NonNull StateWheelExecuteThread stateWheelExecuteThread,
-        @NonNull CuringParamsService curingParamsService) {
+                                   @NonNull ProcessInstance processInstance,
+                                   @NonNull ProcessService processService,
+                                   @NonNull NettyExecutorManager nettyExecutorManager,
+                                   @NonNull ProcessAlertManager processAlertManager,
+                                   @NonNull MasterConfig masterConfig,
+                                   @NonNull StateWheelExecuteThread stateWheelExecuteThread,
+                                   @NonNull CuringParamsService curingParamsService) {
         this.processService = processService;
         this.processInstance = processInstance;
         this.nettyExecutorManager = nettyExecutorManager;
@@ -255,8 +244,8 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
     public void handleEvents() {
         if (!isStart()) {
             logger.info(
-                "The workflow instance is not started, will not handle its state event, current state event size: {}",
-                stateEvents);
+                    "The workflow instance is not started, will not handle its state event, current state event size: {}",
+                    stateEvents);
             return;
         }
         StateEvent stateEvent = null;
@@ -264,14 +253,15 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
             try {
                 stateEvent = this.stateEvents.peek();
                 LoggerUtils.setWorkflowAndTaskInstanceIDMDC(stateEvent.getProcessInstanceId(),
-                    stateEvent.getTaskInstanceId());
+                        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"));
+                        StateEventHandlerManager.getStateEventHandler(stateEvent.getType())
+                                .orElseThrow(() -> new StateEventHandleError(
+                                        "Cannot find handler for the given state event"));
                 logger.info("Begin to handle state event, {}", stateEvent);
                 if (stateEventHandler.handleStateEvent(this, stateEvent)) {
                     this.stateEvents.remove(stateEvent);
@@ -282,14 +272,15 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
                 ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS);
             } catch (StateEventHandleException stateEventHandleException) {
                 logger.error("State event handle error, will retry this event: {}",
-                    stateEvent,
-                    stateEventHandleException);
+                        stateEvent,
+                        stateEventHandleException);
                 ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS);
             } 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 event handle error, get a unknown exception, will retry this event: {}",
-                    stateEvent,
-                    e);
+                        stateEvent,
+                        e);
                 ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS);
             } finally {
                 LoggerUtils.removeWorkflowAndTaskInstanceIdMDC();
@@ -304,9 +295,9 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
         }
 
         key = String.format("%d_%d_%d",
-            this.processDefinition.getCode(),
-            this.processDefinition.getVersion(),
-            this.processInstance.getId());
+                this.processDefinition.getCode(),
+                this.processDefinition.getVersion(),
+                this.processInstance.getId());
         return key;
     }
 
@@ -334,7 +325,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
             ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskInstance.getTaskCode());
             taskProcessor.action(TaskAction.DISPATCH);
             this.processService.updateTaskGroupQueueStatus(taskGroupQueue.getTaskId(),
-                TaskGroupQueueStatus.ACQUIRE_SUCCESS.getCode());
+                    TaskGroupQueueStatus.ACQUIRE_SUCCESS.getCode());
             return true;
         }
         if (taskGroupQueue.getInQueue() == Flag.YES.getCode()) {
@@ -368,7 +359,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
             stateWheelExecuteThread.removeTask4RetryCheck(processInstance, taskInstance);
             stateWheelExecuteThread.removeTask4StateCheck(processInstance, taskInstance);
 
-            if (taskInstance.getState().typeIsSuccess()) {
+            if (taskInstance.getState().isSuccess()) {
                 completeTaskMap.put(taskInstance.getTaskCode(), taskInstance.getId());
                 // todo: merge the last taskInstance
                 processInstance.setVarPool(taskInstance.getVarPool());
@@ -376,16 +367,16 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
                 if (!processInstance.isBlocked()) {
                     submitPostNode(Long.toString(taskInstance.getTaskCode()));
                 }
-            } else if (taskInstance.taskCanRetry() && processInstance.getState() != ExecutionStatus.READY_STOP) {
+            } else if (taskInstance.taskCanRetry() && processInstance.getState().isReadyStop()) {
                 // retry task
                 logger.info("Retry taskInstance taskInstance state: {}", taskInstance.getState());
                 retryTaskInstance(taskInstance);
-            } else if (taskInstance.getState().typeIsFailure()) {
+            } else if (taskInstance.getState().isFailure()) {
                 completeTaskMap.put(taskInstance.getTaskCode(), taskInstance.getId());
                 // There are child nodes and the failure policy is: CONTINUE
                 if (processInstance.getFailureStrategy() == FailureStrategy.CONTINUE && DagHelper.haveAllNodeAfterNode(
-                    Long.toString(taskInstance.getTaskCode()),
-                    dag)) {
+                        Long.toString(taskInstance.getTaskCode()),
+                        dag)) {
                     submitPostNode(Long.toString(taskInstance.getTaskCode()));
                 } else {
                     errorTaskMap.put(taskInstance.getTaskCode(), taskInstance.getId());
@@ -393,13 +384,13 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
                         killAllTasks();
                     }
                 }
-            } else if (taskInstance.getState().typeIsFinished()) {
+            } else if (taskInstance.getState().isFinished()) {
                 // todo: when the task instance type is pause, then it should not in completeTaskMap
                 completeTaskMap.put(taskInstance.getTaskCode(), taskInstance.getId());
             }
             logger.info("TaskInstance finished will try to update the workflow instance state, task code:{} state:{}",
-                        taskInstance.getTaskCode(),
-                        taskInstance.getState());
+                    taskInstance.getTaskCode(),
+                    taskInstance.getState());
             this.updateProcessInstanceState();
         } catch (Exception ex) {
             logger.error("Task finish failed, get a exception, will remove this taskInstance from completeTaskMap", ex);
@@ -420,14 +411,17 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
             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);
+                    TaskStateEvent nextEvent = TaskStateEvent.builder()
+                            .processInstanceId(processInstance.getId())
+                            .taskInstanceId(nextTaskInstance.getId())
+                            .type(StateEventType.WAIT_TASK_GROUP)
+                            .build();
                     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);
+                    ProcessInstance processInstance =
+                            this.processService.findProcessInstanceById(nextTaskInstance.getProcessInstanceId());
+                    this.processService.sendStartTask2Master(processInstance, nextTaskInstance.getId(),
+                            org.apache.dolphinscheduler.remote.command.CommandType.TASK_WAKEUP_EVENT_REQUEST);
                 }
             }
         }
@@ -445,14 +439,17 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
         TaskInstance newTaskInstance = cloneRetryTaskInstance(taskInstance);
         if (newTaskInstance == null) {
             logger.error("retry fail, new taskInstance is null, task code:{}, task id:{}",
-                taskInstance.getTaskCode(),
-                taskInstance.getId());
+                    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());
+            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 {
@@ -472,7 +469,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
         BeanUtils.copyProperties(newProcessInstance, processInstance);
 
         processDefinition = processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
-            processInstance.getProcessDefinitionVersion());
+                processInstance.getProcessDefinitionVersion());
         processInstance.setProcessDefinition(processDefinition);
     }
 
@@ -507,7 +504,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
     /**
      * check if task instance exist by state event
      */
-    public void checkTaskInstanceByStateEvent(StateEvent stateEvent) throws StateEventHandleError {
+    public void checkTaskInstanceByStateEvent(TaskStateEvent stateEvent) throws StateEventHandleError {
         if (stateEvent.getTaskInstanceId() == 0) {
             throw new StateEventHandleError("The taskInstanceId is 0");
         }
@@ -577,29 +574,30 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
 
         // when the serial complement is executed, the next complement instance is created,
         // and this method does not need to be executed when the parallel complement is used.
-        if (processInstance.getState() == ExecutionStatus.READY_STOP || !processInstance.getState().typeIsFinished()) {
+        if (processInstance.getState().isReadyStop() || !processInstance.getState().isFinished()) {
             return false;
         }
 
         Date scheduleDate = processInstance.getScheduleTime();
         if (scheduleDate == null) {
             scheduleDate = complementListDate.get(0);
-        } else if (processInstance.getState().typeIsFinished()) {
+        } else if (processInstance.getState().isFinished()) {
             endProcess();
             if (complementListDate.isEmpty()) {
                 logger.info("process complement end. process id:{}", processInstance.getId());
                 return true;
             }
             int index = complementListDate.indexOf(scheduleDate);
-            if (index >= complementListDate.size() - 1 || !processInstance.getState().typeIsSuccess()) {
+            if (index >= complementListDate.size() - 1 || !processInstance.getState().isSuccess()) {
                 logger.info("process complement end. process id:{}", processInstance.getId());
                 // complement data ends || no success
                 return true;
             }
-            logger.info("process complement continue. process id:{}, schedule time:{} complementListDate:{}", processInstance.getId(), processInstance.getScheduleTime(), complementListDate);
+            logger.info("process complement continue. process id:{}, schedule time:{} complementListDate:{}",
+                    processInstance.getId(), processInstance.getScheduleTime(), complementListDate);
             scheduleDate = complementListDate.get(index + 1);
         }
-        //the next process complement
+        // the next process complement
         int create = this.createComplementDataCommand(scheduleDate);
         if (create > 0) {
             logger.info("create complement data command successfully.");
@@ -619,12 +617,13 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
 
         if (cmdParam.containsKey(CMDPARAM_COMPLEMENT_DATA_SCHEDULE_DATE_LIST)) {
             cmdParam.replace(CMDPARAM_COMPLEMENT_DATA_SCHEDULE_DATE_LIST,
-                cmdParam.get(CMDPARAM_COMPLEMENT_DATA_SCHEDULE_DATE_LIST)
-                    .substring(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_SCHEDULE_DATE_LIST).indexOf(COMMA) + 1));
+                    cmdParam.get(CMDPARAM_COMPLEMENT_DATA_SCHEDULE_DATE_LIST)
+                            .substring(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_SCHEDULE_DATE_LIST).indexOf(COMMA) + 1));
         }
 
         if (cmdParam.containsKey(CMDPARAM_COMPLEMENT_DATA_START_DATE)) {
-            cmdParam.replace(CMDPARAM_COMPLEMENT_DATA_START_DATE, DateUtils.format(scheduleDate, YYYY_MM_DD_HH_MM_SS, null));
+            cmdParam.replace(CMDPARAM_COMPLEMENT_DATA_START_DATE,
+                    DateUtils.format(scheduleDate, YYYY_MM_DD_HH_MM_SS, null));
         }
         command.setCommandParam(JSONUtils.toJsonString(cmdParam));
         command.setTaskDependType(processInstance.getTaskDependType());
@@ -693,21 +692,18 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
     public void endProcess() {
         this.stateEvents.clear();
         if (processDefinition.getExecutionType().typeIsSerialWait() || processDefinition.getExecutionType()
-            .typeIsSerialPriority()) {
+                .typeIsSerialPriority()) {
             checkSerialProcess(processDefinition);
         }
-        if (processInstance.getState().typeIsWaitingThread()) {
-            processService.createRecoveryWaitingThreadCommand(null, processInstance);
-        }
         ProjectUser projectUser = processService.queryProjectWithUserByProcessInstanceId(processInstance.getId());
         if (processAlertManager.isNeedToSendWarning(processInstance)) {
             processAlertManager.sendAlertProcessInstance(processInstance, getValidTaskList(), projectUser);
         }
-        if (processInstance.getState().typeIsSuccess()) {
+        if (processInstance.getState().isSuccess()) {
             processAlertManager.closeAlert(processInstance);
         }
         if (checkTaskQueue()) {
-            //release task group
+            // release task group
             processService.releaseAllTaskGroup(processInstance.getId());
         }
     }
@@ -716,19 +712,21 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
         int nextInstanceId = processInstance.getNextProcessInstanceId();
         if (nextInstanceId == 0) {
             ProcessInstance nextProcessInstance =
-                this.processService.loadNextProcess4Serial(processInstance.getProcessDefinition().getCode(), ExecutionStatus.SERIAL_WAIT.getCode(), processInstance.getId());
+                    this.processService.loadNextProcess4Serial(processInstance.getProcessDefinition().getCode(),
+                            WorkflowExecutionStatus.SERIAL_WAIT.getCode(), processInstance.getId());
             if (nextProcessInstance == null) {
                 return;
             }
             ProcessInstance nextReadyStopProcessInstance =
-                this.processService.loadNextProcess4Serial(processInstance.getProcessDefinition().getCode(), ExecutionStatus.READY_STOP.getCode(), processInstance.getId());
+                    this.processService.loadNextProcess4Serial(processInstance.getProcessDefinition().getCode(),
+                            WorkflowExecutionStatus.READY_STOP.getCode(), processInstance.getId());
             if (processDefinition.getExecutionType().typeIsSerialPriority() && nextReadyStopProcessInstance != null) {
                 return;
             }
             nextInstanceId = nextProcessInstance.getId();
         }
         ProcessInstance nextProcessInstance = this.processService.findProcessInstanceById(nextInstanceId);
-        if (nextProcessInstance.getState().typeIsFinished() || nextProcessInstance.getState().typeIsRunning()) {
+        if (nextProcessInstance.getState().isFinished() || nextProcessInstance.getState().isRunning()) {
             return;
         }
         Map<String, Object> cmdParam = new HashMap<>();
@@ -748,13 +746,16 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
      * @throws Exception exception
      */
     private void buildFlowDag() throws Exception {
-        processDefinition = processService.findProcessDefinition(processInstance.getProcessDefinitionCode(), processInstance.getProcessDefinitionVersion());
+        processDefinition = processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
+                processInstance.getProcessDefinitionVersion());
         processInstance.setProcessDefinition(processDefinition);
 
         List<TaskInstance> recoverNodeList = getRecoverTaskInstanceList(processInstance.getCommandParam());
 
-        List<ProcessTaskRelation> processTaskRelations = processService.findRelationByCode(processDefinition.getCode(), processDefinition.getVersion());
-        List<TaskDefinitionLog> taskDefinitionLogs = processService.getTaskDefineLogListByRelation(processTaskRelations);
+        List<ProcessTaskRelation> processTaskRelations =
+                processService.findRelationByCode(processDefinition.getCode(), processDefinition.getVersion());
+        List<TaskDefinitionLog> taskDefinitionLogs =
+                processService.getTaskDefineLogListByRelation(processTaskRelations);
         List<TaskNode> taskNodeList = processService.transformTask(processTaskRelations, taskDefinitionLogs);
         forbiddenTaskMap.clear();
 
@@ -767,7 +768,8 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
         // generate process to get DAG info
         List<String> recoveryNodeCodeList = getRecoveryNodeCodeList(recoverNodeList);
         List<String> startNodeNameList = parseStartNodeName(processInstance.getCommandParam());
-        ProcessDag processDag = generateFlowDag(taskNodeList, startNodeNameList, recoveryNodeCodeList, processInstance.getTaskDependType());
+        ProcessDag processDag = generateFlowDag(taskNodeList, startNodeNameList, recoveryNodeCodeList,
+                processInstance.getTaskDependType());
         if (processDag == null) {
             logger.error("processDag is null");
             return;
@@ -790,27 +792,27 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
 
         if (!isNewProcessInstance()) {
             logger.info("The workflowInstance is not a newly running instance, runtimes: {}, recover flag: {}",
-                processInstance.getRunTimes(),
-                processInstance.getRecovery());
+                    processInstance.getRunTimes(),
+                    processInstance.getRecovery());
             List<TaskInstance> validTaskInstanceList =
-                processService.findValidTaskListByProcessId(processInstance.getId());
+                    processService.findValidTaskListByProcessId(processInstance.getId());
             for (TaskInstance task : validTaskInstanceList) {
                 try {
                     LoggerUtils.setWorkflowAndTaskInstanceIDMDC(task.getProcessInstanceId(), task.getId());
                     logger.info(
-                        "Check the taskInstance from a exist workflowInstance, existTaskInstanceCode: {}, taskInstanceStatus: {}",
-                        task.getTaskCode(),
-                        task.getState());
+                            "Check the taskInstance from a exist workflowInstance, existTaskInstanceCode: {}, taskInstanceStatus: {}",
+                            task.getTaskCode(),
+                            task.getState());
                     if (validTaskMap.containsKey(task.getTaskCode())) {
                         int oldTaskInstanceId = validTaskMap.get(task.getTaskCode());
                         TaskInstance oldTaskInstance = taskInstanceMap.get(oldTaskInstanceId);
-                        if (!oldTaskInstance.getState().typeIsFinished() && task.getState().typeIsFinished()) {
+                        if (!oldTaskInstance.getState().isFinished() && task.getState().isFinished()) {
                             task.setFlag(Flag.NO);
                             processService.updateTaskInstance(task);
                             continue;
                         }
                         logger.warn("have same taskCode taskInstance when init task queue, taskCode:{}",
-                            task.getTaskCode());
+                                task.getTaskCode());
                     }
 
                     validTaskMap.put(task.getTaskCode(), task.getId());
@@ -821,11 +823,11 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
                         continue;
                     }
                     if (task.isConditionsTask() || DagHelper.haveConditionsAfterNode(Long.toString(task.getTaskCode()),
-                        dag)) {
+                            dag)) {
                         continue;
                     }
                     if (task.taskCanRetry()) {
-                        if (task.getState() == ExecutionStatus.NEED_FAULT_TOLERANCE) {
+                        if (task.getState().isNeedFaultTolerance()) {
                             // tolerantTaskInstance add to standby list directly
                             TaskInstance tolerantTaskInstance = cloneTolerantTaskInstance(task);
                             addTaskToStandByList(tolerantTaskInstance);
@@ -834,7 +836,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
                         }
                         continue;
                     }
-                    if (task.getState().typeIsFailure()) {
+                    if (task.getState().isFailure()) {
                         errorTaskMap.put(task.getTaskCode(), task.getId());
                     }
                 } finally {
@@ -853,28 +855,31 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
 
                 Date start = null;
                 Date end = null;
-                if (cmdParam.containsKey(CMDPARAM_COMPLEMENT_DATA_START_DATE) && cmdParam.containsKey(CMDPARAM_COMPLEMENT_DATA_END_DATE)) {
+                if (cmdParam.containsKey(CMDPARAM_COMPLEMENT_DATA_START_DATE)
+                        && cmdParam.containsKey(CMDPARAM_COMPLEMENT_DATA_END_DATE)) {
                     start = DateUtils.stringToDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_START_DATE));
                     end = DateUtils.stringToDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_END_DATE));
                 }
-                List<Schedule> schedules = processService.queryReleaseSchedulerListByProcessDefinitionCode(processInstance.getProcessDefinitionCode());
                 if (complementListDate.isEmpty() && needComplementProcess()) {
                     if (start != null && end != null) {
+                        List<Schedule> schedules = processService.queryReleaseSchedulerListByProcessDefinitionCode(
+                                processInstance.getProcessDefinitionCode());
                         complementListDate = CronUtils.getSelfFireDateList(start, end, schedules);
                     }
                     if (cmdParam.containsKey(CMDPARAM_COMPLEMENT_DATA_SCHEDULE_DATE_LIST)) {
                         complementListDate = CronUtils.getSelfScheduleDateList(cmdParam);
                     }
-                    logger.info(" process definition code:{} complement data: {}", processInstance.getProcessDefinitionCode(), complementListDate);
+                    logger.info(" process definition code:{} complement data: {}",
+                            processInstance.getProcessDefinitionCode(), complementListDate);
 
                     if (!complementListDate.isEmpty() && Flag.NO == processInstance.getIsSubProcess()) {
                         processInstance.setScheduleTime(complementListDate.get(0));
                         String globalParams = curingParamsService.curingGlobalParams(processInstance.getId(),
-                            processDefinition.getGlobalParamMap(),
-                            processDefinition.getGlobalParamList(),
-                            CommandType.COMPLEMENT_DATA,
-                            processInstance.getScheduleTime(),
-                            cmdParam.get(Constants.SCHEDULE_TIMEZONE));
+                                processDefinition.getGlobalParamMap(),
+                                processDefinition.getGlobalParamList(),
+                                CommandType.COMPLEMENT_DATA,
+                                processInstance.getScheduleTime(),
+                                cmdParam.get(Constants.SCHEDULE_TIMEZONE));
                         processInstance.setGlobalParams(globalParams);
                         processService.updateProcessInstance(processInstance);
                     }
@@ -882,9 +887,9 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
             }
         }
         logger.info("Initialize task queue, dependFailedTaskMap: {}, completeTaskMap: {}, errorTaskMap: {}",
-                    dependFailedTaskMap,
-                    completeTaskMap,
-                    errorTaskMap);
+                dependFailedTaskMap,
+                completeTaskMap,
+                errorTaskMap);
     }
 
     /**
@@ -901,17 +906,18 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
             ITaskProcessor taskProcessor = TaskProcessorFactory.getTaskProcessor(taskInstance.getTaskType());
             taskProcessor.init(taskInstance, processInstance);
 
-            if (taskInstance.getState() == ExecutionStatus.RUNNING_EXECUTION && taskProcessor.getType().equalsIgnoreCase(Constants.COMMON_TASK_TYPE)) {
+            if (taskInstance.getState().isRunning()
+                    && taskProcessor.getType().equalsIgnoreCase(Constants.COMMON_TASK_TYPE)) {
                 notifyProcessHostUpdate(taskInstance);
             }
 
             boolean submit = taskProcessor.action(TaskAction.SUBMIT);
             if (!submit) {
                 logger.error("process id:{} name:{} submit standby task id:{} name:{} failed!",
-                             processInstance.getId(),
-                             processInstance.getName(),
-                             taskInstance.getId(),
-                             taskInstance.getName());
+                        processInstance.getId(),
+                        processInstance.getName(),
+                        taskInstance.getId(),
+                        taskInstance.getName());
                 return Optional.empty();
             }
 
@@ -937,12 +943,13 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
             int taskGroupId = taskInstance.getTaskGroupId();
             if (taskGroupId > 0) {
                 boolean acquireTaskGroup = processService.acquireTaskGroup(taskInstance.getId(),
-                                                                           taskInstance.getName(),
-                                                                           taskGroupId,
-                                                                           taskInstance.getProcessInstanceId(),
-                                                                           taskInstance.getTaskGroupPriority());
+                        taskInstance.getName(),
+                        taskGroupId,
+                        taskInstance.getProcessInstanceId(),
+                        taskInstance.getTaskGroupPriority());
                 if (!acquireTaskGroup) {
-                    logger.info("submit task name :{}, but the first time to try to acquire task group failed", taskInstance.getName());
+                    logger.info("submit task name :{}, but the first time to try to acquire task group failed",
+                            taskInstance.getName());
                     return Optional.of(taskInstance);
                 }
             }
@@ -950,10 +957,10 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
             boolean dispatchSuccess = taskProcessor.action(TaskAction.DISPATCH);
             if (!dispatchSuccess) {
                 logger.error("process id:{} name:{} dispatch standby task id:{} name:{} failed!",
-                    processInstance.getId(),
-                    processInstance.getName(),
-                    taskInstance.getId(),
-                    taskInstance.getName());
+                        processInstance.getId(),
+                        processInstance.getName(),
+                        taskInstance.getId(),
+                        taskInstance.getName());
                 return Optional.empty();
             }
             taskProcessor.action(TaskAction.RUN);
@@ -961,28 +968,30 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
             stateWheelExecuteThread.addTask4TimeoutCheck(processInstance, taskInstance);
             stateWheelExecuteThread.addTask4StateCheck(processInstance, taskInstance);
 
-            if (taskProcessor.taskInstance().getState().typeIsFinished()) {
+            if (taskProcessor.taskInstance().getState().isFinished()) {
                 if (processInstance.isBlocked()) {
-                    StateEvent processBlockEvent = new StateEvent();
-                    processBlockEvent.setProcessInstanceId(this.processInstance.getId());
-                    processBlockEvent.setTaskInstanceId(taskInstance.getId());
-                    processBlockEvent.setExecutionStatus(taskProcessor.taskInstance().getState());
-                    processBlockEvent.setType(StateEventType.PROCESS_BLOCKED);
+                    TaskStateEvent processBlockEvent = TaskStateEvent.builder()
+                            .processInstanceId(processInstance.getId())
+                            .taskInstanceId(taskInstance.getId())
+                            .status(taskProcessor.taskInstance().getState())
+                            .type(StateEventType.PROCESS_BLOCKED)
+                            .build();
                     this.stateEvents.add(processBlockEvent);
                 }
-                StateEvent taskStateChangeEvent = new StateEvent();
-                taskStateChangeEvent.setProcessInstanceId(this.processInstance.getId());
-                taskStateChangeEvent.setTaskInstanceId(taskInstance.getId());
-                taskStateChangeEvent.setExecutionStatus(taskProcessor.taskInstance().getState());
-                taskStateChangeEvent.setType(StateEventType.TASK_STATE_CHANGE);
+                TaskStateEvent taskStateChangeEvent = TaskStateEvent.builder()
+                        .processInstanceId(processInstance.getId())
+                        .taskInstanceId(taskInstance.getId())
+                        .status(taskProcessor.taskInstance().getState())
+                        .type(StateEventType.TASK_STATE_CHANGE)
+                        .build();
                 this.stateEvents.add(taskStateChangeEvent);
             }
             return Optional.of(taskInstance);
         } catch (Exception e) {
             logger.error("submit standby task error, taskCode: {}, taskInstanceId: {}",
-                         taskInstance.getTaskCode(),
-                         taskInstance.getId(),
-                         e);
+                    taskInstance.getTaskCode(),
+                    taskInstance.getId(),
+                    e);
             return Optional.empty();
         }
     }
@@ -1094,7 +1103,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
         // task name
         taskInstance.setName(taskNode.getName());
         // task instance state
-        taskInstance.setState(ExecutionStatus.SUBMITTED_SUCCESS);
+        taskInstance.setState(TaskExecutionStatus.SUBMITTED_SUCCESS);
         // process instance id
         taskInstance.setProcessInstanceId(processInstance.getId());
         // task instance type
@@ -1120,14 +1129,14 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
         // retry task instance interval
         taskInstance.setRetryInterval(taskNode.getRetryInterval());
 
-        //set task param
+        // set task param
         taskInstance.setTaskParams(taskNode.getTaskParams());
 
-        //set task group and priority
+        // set task group and priority
         taskInstance.setTaskGroupId(taskNode.getTaskGroupId());
         taskInstance.setTaskGroupPriority(taskNode.getTaskGroupPriority());
 
-        //set task cpu quota and max memory
+        // set task cpu quota and max memory
         taskInstance.setCpuQuota(taskNode.getCpuQuota());
         taskInstance.setMemoryMax(taskNode.getMemoryMax());
 
@@ -1140,10 +1149,13 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
 
         String processWorkerGroup = processInstance.getWorkerGroup();
         processWorkerGroup = StringUtils.isBlank(processWorkerGroup) ? DEFAULT_WORKER_GROUP : processWorkerGroup;
-        String taskWorkerGroup = StringUtils.isBlank(taskNode.getWorkerGroup()) ? processWorkerGroup : taskNode.getWorkerGroup();
+        String taskWorkerGroup =
+                StringUtils.isBlank(taskNode.getWorkerGroup()) ? processWorkerGroup : taskNode.getWorkerGroup();
 
-        Long processEnvironmentCode = Objects.isNull(processInstance.getEnvironmentCode()) ? -1 : processInstance.getEnvironmentCode();
-        Long taskEnvironmentCode = Objects.isNull(taskNode.getEnvironmentCode()) ? processEnvironmentCode : taskNode.getEnvironmentCode();
+        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);
@@ -1199,19 +1211,20 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
         return taskInstanceMap.values();
     }
 
-    private void setVarPoolValue(Map<String, Property> allProperty, Map<String, TaskInstance> allTaskInstance, TaskInstance preTaskInstance, Property thisProperty) {
-        //for this taskInstance all the param in this part is IN.
+    private void setVarPoolValue(Map<String, Property> allProperty, Map<String, TaskInstance> allTaskInstance,
+                                 TaskInstance preTaskInstance, Property thisProperty) {
+        // for this taskInstance all the param in this part is IN.
         thisProperty.setDirect(Direct.IN);
-        //get the pre taskInstance Property's name
+        // get the pre taskInstance Property's name
         String proName = thisProperty.getProp();
-        //if the Previous nodes have the Property of same name
+        // if the Previous nodes have the Property of same name
         if (allProperty.containsKey(proName)) {
-            //comparison the value of two Property
+            // comparison the value of two Property
             Property otherPro = allProperty.get(proName);
-            //if this property'value of loop is empty,use the other,whether the other's value is empty or not
+            // if this property'value of loop is empty,use the other,whether the other's value is empty or not
             if (StringUtils.isEmpty(thisProperty.getValue())) {
                 allProperty.put(proName, otherPro);
-                //if  property'value of loop is not empty,and the other's value is not empty too, use the earlier value
+                // if property'value of loop is not empty,and the other's value is not empty too, use the earlier value
             } else if (StringUtils.isNotEmpty(otherPro.getValue())) {
                 TaskInstance otherTask = allTaskInstance.get(proName);
                 if (otherTask.getEndTime().getTime() > preTaskInstance.getEndTime().getTime()) {
@@ -1241,8 +1254,8 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
             TaskInstance taskInstance = taskInstanceMap.get(taskInstanceId);
             if (taskInstance == null) {
                 logger.warn("Cannot find the taskInstance from taskInstanceMap, taskInstanceId: {}, taskConde: {}",
-                            taskInstanceId,
-                            taskConde);
+                        taskInstanceId,
+                        taskConde);
                 // This case will happen when we submit to db failed, then the taskInstanceId is 0
                 continue;
             }
@@ -1265,7 +1278,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
 
     private void submitPostNode(String parentNodeCode) throws StateEventHandleException {
         Set<String> submitTaskNodeList =
-            DagHelper.parsePostNodes(parentNodeCode, skipTaskNodeMap, dag, getCompleteTaskInstanceMap());
+                DagHelper.parsePostNodes(parentNodeCode, skipTaskNodeMap, dag, getCompleteTaskInstanceMap());
         List<TaskInstance> taskInstances = new ArrayList<>();
         for (String taskNode : submitTaskNodeList) {
             TaskNode taskNodeObject = dag.getNode(taskNode);
@@ -1277,7 +1290,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
             TaskInstance task = createTaskInstance(processInstance, taskNodeObject);
             taskInstances.add(task);
         }
-        //the end node of the branch of the dag
+        // the end node of the branch of the dag
         if (StringUtils.isNotEmpty(parentNodeCode) && dag.getEndNode().contains(parentNodeCode)) {
             TaskInstance endTaskInstance = taskInstanceMap.get(completeTaskMap.get(NumberUtils.toLong(parentNodeCode)));
             String taskInstanceVarPool = endTaskInstance.getVarPool();
@@ -1306,7 +1319,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
                 logger.info("task {} has already run success", task.getName());
                 continue;
             }
-            if (task.getState().typeIsPause() || task.getState().typeIsCancel()) {
+            if (task.getState().isKill()) {
                 logger.info("task {} stopped, the state is {}", task.getName(), task.getState());
                 continue;
             }
@@ -1340,8 +1353,8 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
                     return DependResult.WAITING;
                 }
                 Integer depsTaskId = completeTaskMap.get(despNodeTaskCode);
-                ExecutionStatus depTaskState = taskInstanceMap.get(depsTaskId).getState();
-                if (depTaskState.typeIsPause() || depTaskState.typeIsCancel()) {
+                TaskExecutionStatus depTaskState = taskInstanceMap.get(depsTaskId).getState();
+                if (depTaskState.isKill()) {
                     return DependResult.NON_EXEC;
                 }
                 // ignore task state if current task is block
@@ -1359,7 +1372,8 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
                 }
             }
         }
-        logger.info("taskCode: {} completeDependTaskList: {}", taskCode, Arrays.toString(completeTaskMap.keySet().toArray()));
+        logger.info("taskCode: {} completeDependTaskList: {}", taskCode,
+                Arrays.toString(completeTaskMap.keySet().toArray()));
         return DependResult.SUCCESS;
     }
 
@@ -1387,16 +1401,17 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
      */
     private boolean dependTaskSuccess(String dependNodeName, String nextNodeName) {
         if (dag.getNode(dependNodeName).isConditionsTask()) {
-            //condition task need check the branch to run
-            List<String> nextTaskList = DagHelper.parseConditionTask(dependNodeName, skipTaskNodeMap, dag, getCompleteTaskInstanceMap());
+            // condition task need check the branch to run
+            List<String> nextTaskList =
+                    DagHelper.parseConditionTask(dependNodeName, skipTaskNodeMap, dag, getCompleteTaskInstanceMap());
             if (!nextTaskList.contains(nextNodeName)) {
                 return false;
             }
         } else {
             long taskCode = Long.parseLong(dependNodeName);
             Integer taskInstanceId = completeTaskMap.get(taskCode);
-            ExecutionStatus depTaskState = taskInstanceMap.get(taskInstanceId).getState();
-            if (depTaskState.typeIsFailure()) {
+            TaskExecutionStatus depTaskState = taskInstanceMap.get(taskInstanceId).getState();
+            if (depTaskState.isFailure()) {
                 return false;
             }
         }
@@ -1409,7 +1424,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
      * @param state state
      * @return task instance list
      */
-    private List<TaskInstance> getCompleteTaskByState(ExecutionStatus state) {
+    private List<TaskInstance> getCompleteTaskByState(TaskExecutionStatus state) {
         List<TaskInstance> resultList = new ArrayList<>();
         for (Integer taskInstanceId : completeTaskMap.values()) {
             TaskInstance taskInstance = taskInstanceMap.get(taskInstanceId);
@@ -1426,13 +1441,14 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
      * @param state state
      * @return ExecutionStatus
      */
-    private ExecutionStatus runningState(ExecutionStatus state) {
-        if (state == ExecutionStatus.READY_STOP || state == ExecutionStatus.READY_PAUSE || state == ExecutionStatus.WAITING_THREAD || state == ExecutionStatus.READY_BLOCK ||
-            state == ExecutionStatus.DELAY_EXECUTION) {
+    private WorkflowExecutionStatus runningState(WorkflowExecutionStatus state) {
+        if (state == WorkflowExecutionStatus.READY_STOP || state == WorkflowExecutionStatus.READY_PAUSE
+                || state == WorkflowExecutionStatus.READY_BLOCK ||
+                state == WorkflowExecutionStatus.DELAY_EXECUTION) {
             // if the running task is not completed, the state remains unchanged
             return state;
         } else {
-            return ExecutionStatus.RUNNING_EXECUTION;
+            return WorkflowExecutionStatus.RUNNING_EXECUTION;
         }
     }
 
@@ -1464,22 +1480,13 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
                 return true;
             }
             if (processInstance.getFailureStrategy() == FailureStrategy.CONTINUE) {
-                return readyToSubmitTaskQueue.size() == 0 && activeTaskProcessorMaps.size() == 0 && waitToRetryTaskInstanceMap.size() == 0;
+                return readyToSubmitTaskQueue.size() == 0 && activeTaskProcessorMaps.size() == 0
+                        && waitToRetryTaskInstanceMap.size() == 0;
             }
         }
         return false;
     }
 
-    /**
-     * whether task for waiting thread
-     *
-     * @return Boolean whether has waiting thread task
-     */
-    private boolean hasWaitingThreadTask() {
-        List<TaskInstance> waitingList = getCompleteTaskByState(ExecutionStatus.WAITING_THREAD);
-        return CollectionUtils.isNotEmpty(waitingList);
-    }
-
     /**
      * prepare for pause
      * 1,failed retry task in the preparation queue , returns to failure directly
@@ -1488,16 +1495,15 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
      *
      * @return ExecutionStatus
      */
-    private ExecutionStatus processReadyPause() {
+    private WorkflowExecutionStatus processReadyPause() {
         if (hasRetryTaskInStandBy()) {
-            return ExecutionStatus.FAILURE;
+            return WorkflowExecutionStatus.FAILURE;
         }
 
-        List<TaskInstance> pauseList = getCompleteTaskByState(ExecutionStatus.PAUSE);
-        if (CollectionUtils.isNotEmpty(pauseList) || processInstance.isBlocked() || !isComplementEnd() || readyToSubmitTaskQueue.size() > 0) {
-            return ExecutionStatus.PAUSE;
+        if (processInstance.isBlocked() || !isComplementEnd() || readyToSubmitTaskQueue.size() > 0) {
+            return WorkflowExecutionStatus.PAUSE;
         } else {
-            return ExecutionStatus.SUCCESS;
+            return WorkflowExecutionStatus.SUCCESS;
         }
     }
 
@@ -1509,7 +1515,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
      *
      * @return ExecutionStatus
      */
-    private ExecutionStatus processReadyBlock() {
+    private WorkflowExecutionStatus processReadyBlock() {
         if (activeTaskProcessorMaps.size() > 0) {
             for (ITaskProcessor taskProcessor : activeTaskProcessorMaps.values()) {
                 if (!TASK_TYPE_BLOCKING.equals(taskProcessor.getType())) {
@@ -1518,11 +1524,11 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
             }
         }
         if (readyToSubmitTaskQueue.size() > 0) {
-            for (Iterator<TaskInstance> iter = readyToSubmitTaskQueue.iterator(); iter.hasNext(); ) {
-                iter.next().setState(ExecutionStatus.KILL);
+            for (Iterator<TaskInstance> iter = readyToSubmitTaskQueue.iterator(); iter.hasNext();) {
+                iter.next().setState(TaskExecutionStatus.KILL);
             }
         }
-        return ExecutionStatus.BLOCK;
+        return WorkflowExecutionStatus.BLOCK;
     }
 
     /**
@@ -1530,47 +1536,39 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
      *
      * @return process instance execution status
      */
-    private ExecutionStatus getProcessInstanceState(ProcessInstance instance) {
-        ExecutionStatus state = instance.getState();
+    private WorkflowExecutionStatus getProcessInstanceState(ProcessInstance instance) {
+        WorkflowExecutionStatus state = instance.getState();
 
         if (activeTaskProcessorMaps.size() > 0 || hasRetryTaskInStandBy()) {
             // active task and retry task exists
-            ExecutionStatus executionStatus = runningState(state);
+            WorkflowExecutionStatus executionStatus = runningState(state);
             logger.info("The workflowInstance has task running, the workflowInstance status is {}", executionStatus);
             return executionStatus;
         }
 
         // block
-        if (state == ExecutionStatus.READY_BLOCK) {
-            ExecutionStatus executionStatus = processReadyBlock();
+        if (state == WorkflowExecutionStatus.READY_BLOCK) {
+            WorkflowExecutionStatus executionStatus = processReadyBlock();
             logger.info("The workflowInstance is ready to block, the workflowInstance status is {}", executionStatus);
             return executionStatus;
         }
 
-        // waiting thread
-        if (hasWaitingThreadTask()) {
-            logger.info("The workflowInstance has waiting thread task, the workflow status is {}",
-                        ExecutionStatus.WAITING_THREAD);
-            return ExecutionStatus.WAITING_THREAD;
-        }
-
         // pause
-        if (state == ExecutionStatus.READY_PAUSE) {
-            ExecutionStatus executionStatus = processReadyPause();
+        if (state == WorkflowExecutionStatus.READY_PAUSE) {
+            WorkflowExecutionStatus executionStatus = processReadyPause();
             logger.info("The workflowInstance is ready to pause, the workflow status is {}", executionStatus);
             return executionStatus;
         }
 
         // stop
-        if (state == ExecutionStatus.READY_STOP) {
-            List<TaskInstance> stopList = getCompleteTaskByState(ExecutionStatus.STOP);
-            List<TaskInstance> killList = getCompleteTaskByState(ExecutionStatus.KILL);
-            List<TaskInstance> failList = getCompleteTaskByState(ExecutionStatus.FAILURE);
-            ExecutionStatus executionStatus;
-            if (CollectionUtils.isNotEmpty(stopList) || CollectionUtils.isNotEmpty(killList) || CollectionUtils.isNotEmpty(failList) || !isComplementEnd()) {
-                executionStatus = ExecutionStatus.STOP;
+        if (state == WorkflowExecutionStatus.READY_STOP) {
+            List<TaskInstance> killList = getCompleteTaskByState(TaskExecutionStatus.KILL);
+            List<TaskInstance> failList = getCompleteTaskByState(TaskExecutionStatus.FAILURE);
+            WorkflowExecutionStatus executionStatus;
+            if (CollectionUtils.isNotEmpty(killList) || CollectionUtils.isNotEmpty(failList) || !isComplementEnd()) {
+                executionStatus = WorkflowExecutionStatus.STOP;
             } else {
-                executionStatus = ExecutionStatus.SUCCESS;
+                executionStatus = WorkflowExecutionStatus.SUCCESS;
             }
             logger.info("The workflowInstance is ready to stop, the workflow status is {}", executionStatus);
             return executionStatus;
@@ -1578,22 +1576,22 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
 
         // process failure
         if (processFailed()) {
-            logger.info("The workflowInstance is failed, the workflow status is {}", ExecutionStatus.FAILURE);
-            return ExecutionStatus.FAILURE;
+            logger.info("The workflowInstance is failed, the workflow status is {}", WorkflowExecutionStatus.FAILURE);
+            return WorkflowExecutionStatus.FAILURE;
         }
 
         // success
-        if (state == ExecutionStatus.RUNNING_EXECUTION) {
-            List<TaskInstance> killTasks = getCompleteTaskByState(ExecutionStatus.KILL);
+        if (state == WorkflowExecutionStatus.RUNNING_EXECUTION) {
+            List<TaskInstance> killTasks = getCompleteTaskByState(TaskExecutionStatus.KILL);
             if (readyToSubmitTaskQueue.size() > 0 || waitToRetryTaskInstanceMap.size() > 0) {
-                //tasks currently pending submission, no retries, indicating that depend is waiting to complete
-                return ExecutionStatus.RUNNING_EXECUTION;
+                // tasks currently pending submission, no retries, indicating that depend is waiting to complete
+                return WorkflowExecutionStatus.RUNNING_EXECUTION;
             } else if (CollectionUtils.isNotEmpty(killTasks)) {
                 // tasks maybe killed manually
-                return ExecutionStatus.FAILURE;
+                return WorkflowExecutionStatus.FAILURE;
             } else {
-                //  if the waiting queue is empty and the status is in progress, then success
-                return ExecutionStatus.SUCCESS;
+                // if the waiting queue is empty and the status is in progress, then success
+                return WorkflowExecutionStatus.SUCCESS;
             }
         }
 
@@ -1620,43 +1618,44 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
      * after each batch of tasks is executed, the status of the process instance is updated
      */
     private void updateProcessInstanceState() throws StateEventHandleException {
-        ExecutionStatus state = getProcessInstanceState(processInstance);
+        WorkflowExecutionStatus state = getProcessInstanceState(processInstance);
         if (processInstance.getState() != state) {
             logger.info("Update workflowInstance states, origin state: {}, target state: {}",
-                        processInstance.getState(),
-                        state);
+                    processInstance.getState(),
+                    state);
             updateWorkflowInstanceStatesToDB(state);
 
-            StateEvent stateEvent = new StateEvent();
-            stateEvent.setExecutionStatus(processInstance.getState());
-            stateEvent.setProcessInstanceId(this.processInstance.getId());
-            stateEvent.setType(StateEventType.PROCESS_STATE_CHANGE);
+            WorkflowStateEvent stateEvent = WorkflowStateEvent.builder()
+                    .processInstanceId(processInstance.getId())
+                    .status(processInstance.getState())
+                    .type(StateEventType.PROCESS_STATE_CHANGE)
+                    .build();
             // replace with `stateEvents`, make sure `WorkflowExecuteThread` can be deleted to avoid memory leaks
             this.stateEvents.add(stateEvent);
         } else {
             logger.info("There is no need to update the workflow instance state, origin state: {}, target state: {}",
-                        processInstance.getState(),
-                        state);
+                    processInstance.getState(),
+                    state);
         }
     }
 
     /**
      * stateEvent's execution status as process instance state
      */
-    public void updateProcessInstanceState(StateEvent stateEvent) throws StateEventHandleException {
-        ExecutionStatus state = stateEvent.getExecutionStatus();
+    public void updateProcessInstanceState(WorkflowStateEvent stateEvent) throws StateEventHandleException {
+        WorkflowExecutionStatus state = stateEvent.getStatus();
         updateWorkflowInstanceStatesToDB(state);
     }
 
-    private void updateWorkflowInstanceStatesToDB(ExecutionStatus newStates) throws StateEventHandleException {
-        ExecutionStatus originStates = processInstance.getState();
+    private void updateWorkflowInstanceStatesToDB(WorkflowExecutionStatus newStates) throws StateEventHandleException {
+        WorkflowExecutionStatus originStates = processInstance.getState();
         if (originStates != newStates) {
             logger.info("Begin to update workflow instance state , state will change from {} to {}",
-                        originStates,
-                        newStates);
+                    originStates,
+                    newStates);
 
             processInstance.setState(newStates);
-            if (newStates.typeIsFinished()) {
+            if (newStates.isFinished()) {
                 processInstance.setEndTime(new Date());
             }
             try {
@@ -1691,9 +1690,9 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
             return;
         }
         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.incTaskInstanceByState("submit");
         readyToSubmitTaskQueue.put(taskInstance);
     }
@@ -1713,8 +1712,8 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
      * @return Boolean whether has retry task in standby
      */
     private boolean hasRetryTaskInStandBy() {
-        for (Iterator<TaskInstance> iter = readyToSubmitTaskQueue.iterator(); iter.hasNext(); ) {
-            if (iter.next().getState().typeIsFailure()) {
+        for (Iterator<TaskInstance> iter = readyToSubmitTaskQueue.iterator(); iter.hasNext();) {
+            if (iter.next().getState().isFailure()) {
                 return true;
             }
         }
@@ -1726,8 +1725,8 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
      */
     public void killAllTasks() {
         logger.info("kill called on process instance id: {}, num: {}",
-            processInstance.getId(),
-            activeTaskProcessorMaps.size());
+                processInstance.getId(),
+                activeTaskProcessorMaps.size());
 
         if (readyToSubmitTaskQueue.size() > 0) {
             readyToSubmitTaskQueue.clear();
@@ -1740,23 +1739,24 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
                 continue;
             }
             TaskInstance taskInstance = processService.findTaskInstanceById(taskInstanceId);
-            if (taskInstance == null || taskInstance.getState().typeIsFinished()) {
+            if (taskInstance == null || taskInstance.getState().isFinished()) {
                 continue;
             }
             taskProcessor.action(TaskAction.STOP);
-            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.taskInstance().getState());
-                this.addStateEvent(stateEvent);
+            if (taskProcessor.taskInstance().getState().isFinished()) {
+                TaskStateEvent taskStateEvent = TaskStateEvent.builder()
+                        .processInstanceId(processInstance.getId())
+                        .taskInstanceId(taskInstance.getId())
+                        .status(taskProcessor.taskInstance().getState())
+                        .type(StateEventType.TASK_STATE_CHANGE)
+                        .build();
+                this.addStateEvent(taskStateEvent);
             }
         }
     }
 
     public boolean workFlowFinish() {
-        return this.processInstance.getState().typeIsFinished();
+        return this.processInstance.getState().isFinished();
     }
 
     /**
@@ -1772,9 +1772,10 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
             // stop tasks which is retrying if forced success happens
             if (task.taskCanRetry()) {
                 TaskInstance retryTask = processService.findTaskInstanceById(task.getId());
-                if (retryTask != null && retryTask.getState().equals(ExecutionStatus.FORCED_SUCCESS)) {
+                if (retryTask != null && retryTask.getState().isForceSuccess()) {
                     task.setState(retryTask.getState());
-                    logger.info("task: {} has been forced success, put it into complete task list and stop retrying", task.getName());
+                    logger.info("task: {} has been forced success, put it into complete task list and stop retrying",
+                            task.getName());
                     removeTaskFromStandbyList(task);
                     completeTaskMap.put(task.getTaskCode(), task.getId());
                     taskInstanceMap.put(task.getId(), task);
@@ -1782,9 +1783,9 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
                     continue;
                 }
             }
-            //init varPool only this task is the first time running
+            // init varPool only this task is the first time running
             if (task.isFirstRun()) {
-                //get pre task ,get all the task varPool to this task
+                // get pre task ,get all the task varPool to this task
                 Set<String> preTask = dag.getPreviousNodes(Long.toString(task.getTaskCode()));
                 getPreVarPool(task, preTask);
             }
@@ -1796,18 +1797,18 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
                     // Remove and add to complete map and error map
                     if (!removeTaskFromStandbyList(task)) {
                         logger.error(
-                            "Task submit failed, remove from standby list failed, workflowInstanceId: {}, taskCode: {}",
-                            processInstance.getId(),
-                            task.getTaskCode());
+                                "Task submit failed, remove from standby list failed, workflowInstanceId: {}, taskCode: {}",
+                                processInstance.getId(),
+                                task.getTaskCode());
                     }
                     completeTaskMap.put(task.getTaskCode(), task.getId());
                     taskInstanceMap.put(task.getId(), task);
                     errorTaskMap.put(task.getTaskCode(), task.getId());
                     activeTaskProcessorMaps.remove(task.getTaskCode());
                     logger.error("Task submitted failed, workflowInstanceId: {}, taskInstanceId: {}, taskCode: {}",
-                                 task.getProcessInstanceId(),
-                                 task.getId(),
-                                 task.getTaskCode());
+                            task.getProcessInstanceId(),
+                            task.getId(),
+                            task.getTaskCode());
                 } else {
                     removeTaskFromStandbyList(task);
                 }
@@ -1815,11 +1816,13 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
                 // if the dependency fails, the current node is not submitted and the state changes to failure.
                 dependFailedTaskMap.put(task.getTaskCode(), task.getId());
                 removeTaskFromStandbyList(task);
-                logger.info("Task dependent result is failed, taskInstanceId:{} depend result : {}", task.getId(), dependResult);
+                logger.info("Task dependent result is failed, taskInstanceId:{} depend result : {}", task.getId(),
+                        dependResult);
             } else if (DependResult.NON_EXEC == dependResult) {
                 // for some reasons(depend task pause/stop) this task would not be submit
                 removeTaskFromStandbyList(task);
-                logger.info("Remove task due to depend result not executed, taskInstanceId:{} depend result : {}", task.getId(), dependResult);
+                logger.info("Remove task due to depend result not executed, taskInstanceId:{} depend result : {}",
+                        task.getId(), dependResult);
             }
         }
     }
@@ -1836,10 +1839,10 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
         // todo: Can we use a better way to set the recover taskInstanceId list? rather then use the cmdParam
         if (paramMap != null && paramMap.containsKey(CMD_PARAM_RECOVERY_START_NODE_STRING)) {
             List<Integer> startTaskInstanceIds = Arrays.stream(paramMap.get(CMD_PARAM_RECOVERY_START_NODE_STRING)
-                                                                   .split(COMMA))
-                .filter(StringUtils::isNotEmpty)
-                .map(Integer::valueOf)
-                .collect(Collectors.toList());
+                    .split(COMMA))
+                    .filter(StringUtils::isNotEmpty)
+                    .map(Integer::valueOf)
+                    .collect(Collectors.toList());
             if (CollectionUtils.isNotEmpty(startTaskInstanceIds)) {
                 return processService.findTaskInstanceByIdList(startTaskInstanceIds);
             }
@@ -1920,13 +1923,14 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
             return false;
         }
 
-        if (ExecutionStatus.RUNNING_EXECUTION == processInstance.getState() && processInstance.getRunTimes() == 1) {
+        if (WorkflowExecutionStatus.RUNNING_EXECUTION == processInstance.getState()
+                && processInstance.getRunTimes() == 1) {
             return true;
         }
         logger.info(
-            "The workflowInstance has been executed before, this execution is to reRun, processInstance status: {}, runTimes: {}",
-            processInstance.getState(),
-            processInstance.getRunTimes());
+                "The workflowInstance has been executed before, this execution is to reRun, processInstance status: {}, runTimes: {}",
+                processInstance.getState(),
+                processInstance.getRunTimes());
         return false;
     }
 
@@ -1969,14 +1973,14 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
         Map<String, String> globalMap = processDefinition.getGlobalParamMap();
         List<Property> globalParamList = processDefinition.getGlobalParamList();
         if (startParamMap.size() > 0 && globalMap != null) {
-            //start param to overwrite global param
+            // start param to overwrite global param
             for (Map.Entry<String, String> param : globalMap.entrySet()) {
                 String val = startParamMap.get(param.getKey());
                 if (val != null) {
                     param.setValue(val);
                 }
             }
-            //start param to create new global param if global not exist
+            // start param to create new global param if global not exist
             for (Map.Entry<String, String> startParam : startParamMap.entrySet()) {
                 if (!globalMap.containsKey(startParam.getKey())) {
                     globalMap.put(startParam.getKey(), startParam.getValue());
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThreadPool.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThreadPool.java
index ce66318967..5d0e237027 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThreadPool.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThreadPool.java
@@ -17,26 +17,23 @@
 
 package org.apache.dolphinscheduler.server.master.runner;
 
+import com.google.common.base.Strings;
+import lombok.NonNull;
 import org.apache.dolphinscheduler.common.enums.Flag;
-import org.apache.dolphinscheduler.server.master.event.StateEvent;
 import org.apache.dolphinscheduler.common.enums.StateEventType;
 import org.apache.dolphinscheduler.common.utils.LoggerUtils;
 import org.apache.dolphinscheduler.common.utils.NetUtils;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
-import org.apache.dolphinscheduler.remote.command.StateEventChangeCommand;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
+import org.apache.dolphinscheduler.remote.command.WorkflowStateEventChangeCommand;
 import org.apache.dolphinscheduler.remote.processor.StateEventCallbackService;
 import org.apache.dolphinscheduler.remote.utils.Host;
 import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
 import org.apache.dolphinscheduler.server.master.config.MasterConfig;
+import org.apache.dolphinscheduler.server.master.event.StateEvent;
+import org.apache.dolphinscheduler.server.master.event.TaskStateEvent;
 import org.apache.dolphinscheduler.service.process.ProcessService;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import javax.annotation.PostConstruct;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -45,9 +42,9 @@ import org.springframework.stereotype.Component;
 import org.springframework.util.concurrent.ListenableFuture;
 import org.springframework.util.concurrent.ListenableFutureCallback;
 
-import com.google.common.base.Strings;
-
-import lombok.NonNull;
+import javax.annotation.PostConstruct;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 /**
  * Used to execute {@link WorkflowExecuteRunnable}.
@@ -89,9 +86,11 @@ public class WorkflowExecuteThreadPool extends ThreadPoolTaskExecutor {
      * submit state event
      */
     public void submitStateEvent(StateEvent stateEvent) {
-        WorkflowExecuteRunnable workflowExecuteThread = processInstanceExecCacheManager.getByProcessInstanceId(stateEvent.getProcessInstanceId());
+        WorkflowExecuteRunnable workflowExecuteThread =
+                processInstanceExecCacheManager.getByProcessInstanceId(stateEvent.getProcessInstanceId());
         if (workflowExecuteThread == null) {
-            logger.warn("Submit state event error, cannot from workflowExecuteThread from cache manager, stateEvent:{}", stateEvent);
+            logger.warn("Submit state event error, cannot from workflowExecuteThread from cache manager, stateEvent:{}",
+                    stateEvent);
             return;
         }
         workflowExecuteThread.addStateEvent(stateEvent);
@@ -113,6 +112,7 @@ public class WorkflowExecuteThreadPool extends ThreadPoolTaskExecutor {
         int processInstanceId = workflowExecuteThread.getProcessInstance().getId();
         ListenableFuture<?> future = this.submitListenable(workflowExecuteThread::handleEvents);
         future.addCallback(new ListenableFutureCallback() {
+
             @Override
             public void onFailure(Throwable ex) {
                 LoggerUtils.setWorkflowInstanceIdMDC(processInstanceId);
@@ -129,7 +129,8 @@ public class WorkflowExecuteThreadPool extends ThreadPoolTaskExecutor {
                 try {
                     LoggerUtils.setWorkflowInstanceIdMDC(workflowExecuteThread.getProcessInstance().getId());
                     if (workflowExecuteThread.workFlowFinish()) {
-                        stateWheelExecuteThread.removeProcess4TimeoutCheck(workflowExecuteThread.getProcessInstance().getId());
+                        stateWheelExecuteThread
+                                .removeProcess4TimeoutCheck(workflowExecuteThread.getProcessInstance().getId());
                         processInstanceExecCacheManager.removeByProcessInstanceId(processInstanceId);
                         notifyProcessChanged(workflowExecuteThread.getProcessInstance());
                         logger.info("Workflow instance is finished.");
@@ -173,27 +174,30 @@ public class WorkflowExecuteThreadPool extends ThreadPoolTaskExecutor {
             logger.warn("The execute cache manager doesn't contains this workflow instance");
             return;
         }
-        StateEvent stateEvent = new StateEvent();
-        stateEvent.setTaskInstanceId(taskInstance.getId());
-        stateEvent.setType(StateEventType.TASK_STATE_CHANGE);
-        stateEvent.setProcessInstanceId(processInstance.getId());
-        stateEvent.setExecutionStatus(ExecutionStatus.RUNNING_EXECUTION);
+        TaskStateEvent stateEvent = TaskStateEvent.builder()
+                .processInstanceId(processInstance.getId())
+                .taskInstanceId(taskInstance.getId())
+                .type(StateEventType.TASK_STATE_CHANGE)
+                .status(TaskExecutionStatus.RUNNING_EXECUTION)
+                .build();
         this.submitStateEvent(stateEvent);
     }
 
     /**
      * notify process's master
      */
-    private void notifyProcess(ProcessInstance finishProcessInstance, ProcessInstance processInstance, TaskInstance taskInstance) {
+    private void notifyProcess(ProcessInstance finishProcessInstance, ProcessInstance processInstance,
+                               TaskInstance taskInstance) {
         String processInstanceHost = processInstance.getHost();
         if (Strings.isNullOrEmpty(processInstanceHost)) {
-            logger.error("process {} host is empty, cannot notify task {} now", processInstance.getId(), taskInstance.getId());
+            logger.error("process {} host is empty, cannot notify task {} now", processInstance.getId(),
+                    taskInstance.getId());
             return;
         }
-        StateEventChangeCommand stateEventChangeCommand = new StateEventChangeCommand(
-                finishProcessInstance.getId(), 0, finishProcessInstance.getState(), processInstance.getId(), taskInstance.getId()
-        );
+        WorkflowStateEventChangeCommand workflowStateEventChangeCommand = new WorkflowStateEventChangeCommand(
+                finishProcessInstance.getId(), 0, finishProcessInstance.getState(), processInstance.getId(),
+                taskInstance.getId());
         Host host = new Host(processInstanceHost);
-        stateEventCallbackService.sendResult(host, stateEventChangeCommand.convert2Command());
+        stateEventCallbackService.sendResult(host, workflowStateEventChangeCommand.convert2Command());
     }
 }
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseTaskProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseTaskProcessor.java
index 76b8388b38..d9f82a91dc 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseTaskProcessor.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseTaskProcessor.java
@@ -55,7 +55,7 @@ import org.apache.dolphinscheduler.plugin.task.api.K8sTaskExecutionContext;
 import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
 import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
 import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.plugin.task.api.enums.dp.ConnectorType;
 import org.apache.dolphinscheduler.plugin.task.api.enums.dp.ExecuteSqlType;
 import org.apache.dolphinscheduler.plugin.task.api.model.JdbcInfo;
@@ -102,7 +102,8 @@ import lombok.NonNull;
 
 public abstract class BaseTaskProcessor implements ITaskProcessor {
 
-    protected final Logger logger = LoggerFactory.getLogger(String.format(TaskConstants.TASK_LOG_LOGGER_NAME_FORMAT, getClass()));
+    protected final Logger logger =
+            LoggerFactory.getLogger(String.format(TaskConstants.TASK_LOG_LOGGER_NAME_FORMAT, getClass()));
 
     protected boolean killed = false;
 
@@ -141,7 +142,7 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
     }
 
     protected javax.sql.DataSource defaultDataSource =
-                        SpringApplicationContext.getBean(javax.sql.DataSource.class);
+            SpringApplicationContext.getBean(javax.sql.DataSource.class);
 
     /**
      * pause task, common tasks donot need this.
@@ -281,7 +282,7 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
 
         // verify tenant is null
         if (verifyTenantIsNull(tenant, taskInstance)) {
-            taskInstance.setState(ExecutionStatus.FAILURE);
+            taskInstance.setState(TaskExecutionStatus.FAILURE);
             processService.saveTaskInstance(taskInstance);
             return null;
         }
@@ -298,7 +299,7 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
         // TODO to be optimized
         DataQualityTaskExecutionContext dataQualityTaskExecutionContext = new DataQualityTaskExecutionContext();
         if (TASK_TYPE_DATA_QUALITY.equalsIgnoreCase(taskInstance.getTaskType())) {
-            setDataQualityTaskRelation(dataQualityTaskExecutionContext,taskInstance,tenant.getTenantCode());
+            setDataQualityTaskRelation(dataQualityTaskExecutionContext, taskInstance, tenant.getTenantCode());
         }
         K8sTaskExecutionContext k8sTaskExecutionContext = new K8sTaskExecutionContext();
         if (TASK_TYPE_K8S.equalsIgnoreCase(taskInstance.getTaskType())) {
@@ -307,8 +308,10 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
 
         Map<String, Property> businessParamsMap = curingParamsService.preBuildBusinessParams(processInstance);
 
-        AbstractParameters baseParam = taskPluginManager.getParameters(ParametersNode.builder().taskType(taskInstance.getTaskType()).taskParams(taskInstance.getTaskParams()).build());
-        Map<String, Property> propertyMap = curingParamsService.paramParsingPreparation(taskInstance, baseParam, processInstance);
+        AbstractParameters baseParam = taskPluginManager.getParameters(ParametersNode.builder()
+                .taskType(taskInstance.getTaskType()).taskParams(taskInstance.getTaskParams()).build());
+        Map<String, Property> propertyMap =
+                curingParamsService.paramParsingPreparation(taskInstance, baseParam, processInstance);
         return TaskExecutionContextBuilder.get()
                 .buildTaskInstanceRelatedInfo(taskInstance)
                 .buildTaskDefinitionRelatedInfo(taskInstance.getTaskDefine())
@@ -365,7 +368,8 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
         List<UdfFunc> udfFuncList = processService.queryUdfFunListByIds(map.keySet().toArray(new Integer[map.size()]));
 
         udfFuncList.forEach(udfFunc -> {
-            UdfFuncParameters udfFuncParameters = JSONUtils.parseObject(JSONUtils.toJsonString(udfFunc), UdfFuncParameters.class);
+            UdfFuncParameters udfFuncParameters =
+                    JSONUtils.parseObject(JSONUtils.toJsonString(udfFunc), UdfFuncParameters.class);
             udfFuncParameters.setDefaultFS(HadoopUtils.getInstance().getDefaultFS());
             String tenantCode = processService.queryTenantCodeByResName(udfFunc.getResourceName(), ResourceType.UDF);
             udfFuncParameters.setTenantCode(tenantCode);
@@ -379,19 +383,20 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
      * @param dataQualityTaskExecutionContext dataQualityTaskExecutionContext
      * @param taskInstance taskInstance
      */
-    private void setDataQualityTaskRelation(DataQualityTaskExecutionContext dataQualityTaskExecutionContext, TaskInstance taskInstance, String tenantCode) {
+    private void setDataQualityTaskRelation(DataQualityTaskExecutionContext dataQualityTaskExecutionContext,
+                                            TaskInstance taskInstance, String tenantCode) {
         DataQualityParameters dataQualityParameters =
                 JSONUtils.parseObject(taskInstance.getTaskParams(), DataQualityParameters.class);
         if (dataQualityParameters == null) {
             return;
         }
 
-        Map<String,String> config = dataQualityParameters.getRuleInputParameter();
+        Map<String, String> config = dataQualityParameters.getRuleInputParameter();
 
         int ruleId = dataQualityParameters.getRuleId();
         DqRule dqRule = processService.getDqRule(ruleId);
         if (dqRule == null) {
-            logger.error("can not get DqRule by id {}",ruleId);
+            logger.error("can not get DqRule by id {}", ruleId);
             return;
         }
 
@@ -401,7 +406,7 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
 
         List<DqRuleInputEntry> ruleInputEntryList = processService.getRuleInputEntry(ruleId);
         if (CollectionUtils.isEmpty(ruleInputEntryList)) {
-            logger.error("{} rule input entry list is empty ",ruleId);
+            logger.error("{} rule input entry list is empty ", ruleId);
             return;
         }
         List<DqRuleExecuteSql> executeSqlList = processService.getDqExecuteSql(ruleId);
@@ -412,9 +417,9 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
         // set the path used to store data quality task check error data
         dataQualityTaskExecutionContext.setHdfsPath(
                 PropertyUtils.getString(Constants.FS_DEFAULT_FS)
-                + PropertyUtils.getString(
-                        Constants.DATA_QUALITY_ERROR_OUTPUT_PATH,
-                        "/user/" + tenantCode + "/data_quality_error_data"));
+                        + PropertyUtils.getString(
+                                Constants.DATA_QUALITY_ERROR_OUTPUT_PATH,
+                                "/user/" + tenantCode + "/data_quality_error_data"));
 
         setSourceConfig(dataQualityTaskExecutionContext, config);
         setTargetConfig(dataQualityTaskExecutionContext, config);
@@ -460,7 +465,7 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
                     dqRuleExecuteSql.setIndex(1);
                     dqRuleExecuteSql.setSql(type.getExecuteSql());
                     dqRuleExecuteSql.setTableAlias(type.getOutputTable());
-                    executeSqlList.add(0,dqRuleExecuteSql);
+                    executeSqlList.add(0, dqRuleExecuteSql);
 
                     if (Boolean.TRUE.equals(type.getInnerSource())) {
                         dataQualityTaskExecutionContext.setComparisonNeedStatisticsValueTable(true);
@@ -480,17 +485,17 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
     public DataSource getDefaultDataSource() {
         DataSource dataSource = new DataSource();
 
-        HikariDataSource hikariDataSource = (HikariDataSource)defaultDataSource;
+        HikariDataSource hikariDataSource = (HikariDataSource) defaultDataSource;
         dataSource.setUserName(hikariDataSource.getUsername());
         JdbcInfo jdbcInfo = JdbcUrlParser.getJdbcInfo(hikariDataSource.getJdbcUrl());
         if (jdbcInfo != null) {
             Properties properties = new Properties();
-            properties.setProperty(USER,hikariDataSource.getUsername());
-            properties.setProperty(PASSWORD,hikariDataSource.getPassword());
+            properties.setProperty(USER, hikariDataSource.getUsername());
+            properties.setProperty(PASSWORD, hikariDataSource.getPassword());
             properties.setProperty(DATABASE, jdbcInfo.getDatabase());
-            properties.setProperty(ADDRESS,jdbcInfo.getAddress());
-            properties.setProperty(OTHER,jdbcInfo.getParams());
-            properties.setProperty(JDBC_URL,jdbcInfo.getAddress() + SINGLE_SLASH + jdbcInfo.getDatabase());
+            properties.setProperty(ADDRESS, jdbcInfo.getAddress());
+            properties.setProperty(OTHER, jdbcInfo.getParams());
+            properties.setProperty(JDBC_URL, jdbcInfo.getAddress() + SINGLE_SLASH + jdbcInfo.getDatabase());
             dataSource.setType(DbType.of(JdbcUrlParser.getDbType(jdbcInfo.getDriverName()).getCode()));
             dataSource.setConnectionParams(JSONUtils.toJsonString(properties));
         }
@@ -532,9 +537,11 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
      * @param dataQualityTaskExecutionContext
      * @param config
      */
-    private void setTargetConfig(DataQualityTaskExecutionContext dataQualityTaskExecutionContext, Map<String, String> config) {
+    private void setTargetConfig(DataQualityTaskExecutionContext dataQualityTaskExecutionContext,
+                                 Map<String, String> config) {
         if (StringUtils.isNotEmpty(config.get(TARGET_DATASOURCE_ID))) {
-            DataSource dataSource = processService.findDataSourceById(Integer.parseInt(config.get(TARGET_DATASOURCE_ID)));
+            DataSource dataSource =
+                    processService.findDataSourceById(Integer.parseInt(config.get(TARGET_DATASOURCE_ID)));
             if (dataSource != null) {
                 ConnectorType targetConnectorType = ConnectorType.of(
                         DbType.of(Integer.parseInt(config.get(TARGET_CONNECTOR_TYPE))).isHive() ? 1 : 0);
@@ -551,7 +558,8 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
      * @param dataQualityTaskExecutionContext
      * @param config
      */
-    private void setSourceConfig(DataQualityTaskExecutionContext dataQualityTaskExecutionContext, Map<String, String> config) {
+    private void setSourceConfig(DataQualityTaskExecutionContext dataQualityTaskExecutionContext,
+                                 Map<String, String> config) {
         if (StringUtils.isNotEmpty(config.get(SRC_DATASOURCE_ID))) {
             DataSource dataSource = processService.findDataSourceById(Integer.parseInt(config.get(SRC_DATASOURCE_ID)));
             if (dataSource != null) {
@@ -586,15 +594,18 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
      */
     protected Map<String, String> getResourceFullNames(TaskInstance taskInstance) {
         Map<String, String> resourcesMap = new HashMap<>();
-        AbstractParameters baseParam = taskPluginManager.getParameters(ParametersNode.builder().taskType(taskInstance.getTaskType()).taskParams(taskInstance.getTaskParams()).build());
+        AbstractParameters baseParam = taskPluginManager.getParameters(ParametersNode.builder()
+                .taskType(taskInstance.getTaskType()).taskParams(taskInstance.getTaskParams()).build());
         if (baseParam != null) {
             List<ResourceInfo> projectResourceFiles = baseParam.getResourceFilesList();
             if (CollectionUtils.isNotEmpty(projectResourceFiles)) {
 
                 // filter the resources that the resource id equals 0
-                Set<ResourceInfo> oldVersionResources = projectResourceFiles.stream().filter(t -> t.getId() == 0).collect(Collectors.toSet());
+                Set<ResourceInfo> oldVersionResources =
+                        projectResourceFiles.stream().filter(t -> t.getId() == 0).collect(Collectors.toSet());
                 if (CollectionUtils.isNotEmpty(oldVersionResources)) {
-                    oldVersionResources.forEach(t -> resourcesMap.put(t.getRes(), processService.queryTenantCodeByResName(t.getRes(), ResourceType.FILE)));
+                    oldVersionResources.forEach(t -> resourcesMap.put(t.getRes(),
+                            processService.queryTenantCodeByResName(t.getRes(), ResourceType.FILE)));
                 }
 
                 // get the resource id in order to get the resource names in batch
@@ -605,7 +616,8 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
                     Integer[] resourceIds = resourceIdsSet.toArray(new Integer[resourceIdsSet.size()]);
 
                     List<Resource> resources = processService.listResourceByIds(resourceIds);
-                    resources.forEach(t -> resourcesMap.put(t.getFullName(), processService.queryTenantCodeByResName(t.getFullName(), ResourceType.FILE)));
+                    resources.forEach(t -> resourcesMap.put(t.getFullName(),
+                            processService.queryTenantCodeByResName(t.getFullName(), ResourceType.FILE)));
                 }
             }
         }
@@ -619,8 +631,9 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
      * @param taskInstance taskInstance
      */
     private void setK8sTaskRelation(K8sTaskExecutionContext k8sTaskExecutionContext, TaskInstance taskInstance) {
-        K8sTaskParameters k8sTaskParameters = JSONUtils.parseObject(taskInstance.getTaskParams(), K8sTaskParameters.class);
-        Map<String,String> namespace = JSONUtils.toMap(k8sTaskParameters.getNamespace());
+        K8sTaskParameters k8sTaskParameters =
+                JSONUtils.parseObject(taskInstance.getTaskParams(), K8sTaskParameters.class);
+        Map<String, String> namespace = JSONUtils.toMap(k8sTaskParameters.getNamespace());
         String clusterName = namespace.get(CLUSTER);
         String configYaml = processService.findConfigYamlByName(clusterName);
         if (configYaml != null) {
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BlockingTaskProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BlockingTaskProcessor.java
index be7dbe103f..5148a9733f 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BlockingTaskProcessor.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BlockingTaskProcessor.java
@@ -21,11 +21,12 @@ import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYP
 
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.BlockingOpportunity;
+import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
 import org.apache.dolphinscheduler.common.utils.NetUtils;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.plugin.task.api.enums.DependResult;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.plugin.task.api.model.DependentItem;
 import org.apache.dolphinscheduler.plugin.task.api.model.DependentTaskModel;
 import org.apache.dolphinscheduler.plugin.task.api.parameters.BlockingParameters;
@@ -65,15 +66,16 @@ public class BlockingTaskProcessor extends BaseTaskProcessor {
     /**
      * complete task map
      */
-    private Map<Long, ExecutionStatus> completeTaskList = new ConcurrentHashMap<>();
+    private Map<Long, TaskExecutionStatus> completeTaskList = new ConcurrentHashMap<>();
 
     private void initTaskParameters() {
-        taskInstance.setLogPath(LogUtils.getTaskLogPath(taskInstance.getFirstSubmitTime(), processInstance.getProcessDefinitionCode(),
-            processInstance.getProcessDefinitionVersion(),
-            taskInstance.getProcessInstanceId(),
-            taskInstance.getId()));
+        taskInstance.setLogPath(
+                LogUtils.getTaskLogPath(taskInstance.getFirstSubmitTime(), processInstance.getProcessDefinitionCode(),
+                        processInstance.getProcessDefinitionVersion(),
+                        taskInstance.getProcessInstanceId(),
+                        taskInstance.getId()));
         this.taskInstance.setHost(NetUtils.getAddr(masterConfig.getListenPort()));
-        this.taskInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
+        this.taskInstance.setState(TaskExecutionStatus.RUNNING_EXECUTION);
         this.taskInstance.setStartTime(new Date());
         this.processService.saveTaskInstance(taskInstance);
         this.dependentParameters = taskInstance.getDependency();
@@ -82,7 +84,8 @@ public class BlockingTaskProcessor extends BaseTaskProcessor {
 
     @Override
     protected boolean pauseTask() {
-        taskInstance.setState(ExecutionStatus.PAUSE);
+        // todo: task cannot be pause
+        taskInstance.setState(TaskExecutionStatus.KILL);
         taskInstance.setEndTime(new Date());
         processService.saveTaskInstance(taskInstance);
         logger.info("blocking task has been paused");
@@ -91,7 +94,7 @@ public class BlockingTaskProcessor extends BaseTaskProcessor {
 
     @Override
     protected boolean killTask() {
-        taskInstance.setState(ExecutionStatus.KILL);
+        taskInstance.setState(TaskExecutionStatus.KILL);
         taskInstance.setEndTime(new Date());
         processService.saveTaskInstance(taskInstance);
         logger.info("blocking task has been killed");
@@ -105,7 +108,8 @@ public class BlockingTaskProcessor extends BaseTaskProcessor {
 
     @Override
     protected boolean submitTask() {
-        this.taskInstance = processService.submitTaskWithRetry(processInstance, taskInstance, maxRetryTimes, commitInterval);
+        this.taskInstance =
+                processService.submitTaskWithRetry(processInstance, taskInstance, maxRetryTimes, commitInterval);
         if (this.taskInstance == null) {
             return false;
         }
@@ -152,20 +156,21 @@ public class BlockingTaskProcessor extends BaseTaskProcessor {
             dependResult = DependResult.FAILED;
             return dependResult;
         }
-        ExecutionStatus executionStatus = completeTaskList.get(item.getDepTaskCode());
+        TaskExecutionStatus executionStatus = completeTaskList.get(item.getDepTaskCode());
         if (executionStatus != item.getStatus()) {
-            logger.info("depend item : {} expect status: {}, actual status: {}", item.getDepTaskCode(), item.getStatus(), executionStatus);
+            logger.info("depend item : {} expect status: {}, actual status: {}", item.getDepTaskCode(),
+                    item.getStatus(), executionStatus);
             dependResult = DependResult.FAILED;
         }
         logger.info("dependent item complete {} {},{}",
-            Constants.DEPENDENT_SPLIT, item.getDepTaskCode(), dependResult);
+                Constants.DEPENDENT_SPLIT, item.getDepTaskCode(), dependResult);
         return dependResult;
     }
 
     private void setConditionResult() {
 
         List<TaskInstance> taskInstances = processService
-            .findValidTaskListByProcessId(taskInstance.getProcessInstanceId());
+                .findValidTaskListByProcessId(taskInstance.getProcessInstanceId());
         for (TaskInstance task : taskInstances) {
             completeTaskList.putIfAbsent(task.getTaskCode(), task.getState());
         }
@@ -176,7 +181,8 @@ public class BlockingTaskProcessor extends BaseTaskProcessor {
             for (DependentItem item : dependentTaskModel.getDependItemList()) {
                 itemDependResult.add(getDependResultForItem(item));
             }
-            DependResult tempResult = DependentUtils.getDependResultForRelation(dependentTaskModel.getRelation(), itemDependResult);
+            DependResult tempResult =
+                    DependentUtils.getDependResultForRelation(dependentTaskModel.getRelation(), itemDependResult);
             tempResultList.add(tempResult);
         }
         conditionResult = DependentUtils.getDependResultForRelation(dependentParameters.getRelation(), tempResultList);
@@ -184,17 +190,17 @@ public class BlockingTaskProcessor extends BaseTaskProcessor {
     }
 
     private void endTask() {
-        ExecutionStatus status = ExecutionStatus.SUCCESS;
         DependResult expected = this.blockingParam.getBlockingOpportunity()
-            .equals(BlockingOpportunity.BLOCKING_ON_SUCCESS.getDesc())
-            ? DependResult.SUCCESS : DependResult.FAILED;
+                .equals(BlockingOpportunity.BLOCKING_ON_SUCCESS.getDesc())
+                        ? DependResult.SUCCESS
+                        : DependResult.FAILED;
         boolean isBlocked = (expected == this.conditionResult);
         logger.info("blocking opportunity: expected-->{}, actual-->{}", expected, this.conditionResult);
         processInstance.setBlocked(isBlocked);
         if (isBlocked) {
-            processInstance.setState(ExecutionStatus.READY_BLOCK);
+            processInstance.setState(WorkflowExecutionStatus.READY_BLOCK);
         }
-        taskInstance.setState(status);
+        taskInstance.setState(TaskExecutionStatus.SUCCESS);
         taskInstance.setEndTime(new Date());
         processService.updateTaskInstance(taskInstance);
         logger.info("blocking task execute complete, blocking:{}", isBlocked);
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessor.java
index f8cf041ae6..46e310f44c 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessor.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessor.java
@@ -19,7 +19,7 @@ package org.apache.dolphinscheduler.server.master.runner.task;
 
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.remote.command.TaskKillRequestCommand;
 import org.apache.dolphinscheduler.remote.utils.Host;
 import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext;
@@ -49,7 +49,8 @@ public class CommonTaskProcessor extends BaseTaskProcessor {
 
     @Override
     protected boolean submitTask() {
-        this.taskInstance = processService.submitTaskWithRetry(processInstance, taskInstance, maxRetryTimes, commitInterval);
+        this.taskInstance =
+                processService.submitTaskWithRetry(processInstance, taskInstance, maxRetryTimes, commitInterval);
 
         return this.taskInstance != null;
     }
@@ -92,21 +93,24 @@ public class CommonTaskProcessor extends BaseTaskProcessor {
             if (taskUpdateQueue == null) {
                 this.initQueue();
             }
-            if (taskInstance.getState().typeIsFinished()) {
-                logger.info("submit task , but task [{}] state [{}] is already  finished. ", taskInstance.getName(), taskInstance.getState());
+            if (taskInstance.getState().isFinished()) {
+                logger.info("submit task , but task [{}] state [{}] is already  finished. ", taskInstance.getName(),
+                        taskInstance.getState());
                 return true;
... 7159 lines suppressed ...