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

[dolphinscheduler] branch dev updated: [Feature-11223] support stream task (#11350)

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

caishunfeng 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 0464123c2b [Feature-11223] support stream task (#11350)
0464123c2b is described below

commit 0464123c2b8d507a16a966e2dbe8dbfe9bf781e2
Author: caishunfeng <ca...@gmail.com>
AuthorDate: Wed Aug 10 21:44:43 2022 +0800

    [Feature-11223] support stream task (#11350)
    
    * add task execute type
    
    * update task definition list paging
    
    * update task instance list paging
    
    * stream task start
    
    * [Feature][UI] Some changes to execute task.
        * Set the connection edge to dashed line.
        * Add FLINK_STREAM task.
    
    * add stream task
    
    * flink savepoint and cancel
    
    * fix query bug
    
    * add stream task definition
    
    * add task instance for stream task
    
    * delete stream task definition state
    
    * update api for stream task definition edit
    
    * modify search for stream task instance
    
    * add language
    
    * delete task type search for stream task definition
    
    * change task type search for stream task instance
    
    * add jump button
    
    * add savepoint
    
    * add down log for stream task instance
    
    * ui test
    
    * stream task start
    
    * run DAG
    
    * [Fix][UI] Fix the stream task edgs not to be dashed when filling back.
    
    * [Feature][UI] Remove some fields for FLINK_STREAM.
    
    * add start modal
    
    * add dryRun column for stream task instance
    
    * fix duration
    
    * fix pon
    
    * fix build error
    
    * Add success tip
    
    * add auto sync for stream task instance
    
    * remove forgien key for task instance
    
    * license header
    
    * UT fix
    
    * modify locales
    
    * recover common config
    
    * fix UT
    
    * add doc
    
    Co-authored-by: Amy <am...@163.com>
    Co-authored-by: devosend <de...@gmail.com>
---
 docs/docs/en/guide/project/task-definition.md      |  12 +-
 docs/docs/en/guide/project/task-instance.md        |  16 +-
 docs/docs/en/guide/project/workflow-definition.md  |   5 +
 docs/docs/zh/guide/project/task-definition.md      |  17 +-
 docs/docs/zh/guide/project/task-instance.md        |  12 +-
 docs/docs/zh/guide/project/workflow-definition.md  |   4 +
 .../new_ui/dev/project/batch-task-definition.png   | Bin 0 -> 72426 bytes
 .../img/new_ui/dev/project/batch-task-instance.png | Bin 0 -> 73007 bytes
 .../new_ui/dev/project/stream-task-definition.png  | Bin 0 -> 44125 bytes
 .../img/new_ui/dev/project/stream-task-execute.png | Bin 0 -> 66638 bytes
 .../new_ui/dev/project/stream-task-instance.png    | Bin 0 -> 55423 bytes
 docs/img/new_ui/dev/project/task-definition.jpg    | Bin 801293 -> 0 bytes
 docs/img/new_ui/dev/project/task-instance.png      | Bin 84196 -> 0 bytes
 .../workflow-definition-with-stream-task.png       | Bin 0 -> 35685 bytes
 .../api/controller/ExecutorController.java         |  47 ++
 .../api/controller/TaskDefinitionController.java   |   6 +-
 .../api/controller/TaskInstanceController.java     |  56 ++-
 .../apache/dolphinscheduler/api/enums/Status.java  |   4 +
 .../api/service/ExecutorService.java               |  18 +
 .../api/service/TaskDefinitionService.java         |   3 +
 .../api/service/TaskInstanceService.java           |  22 +
 .../api/service/impl/ExecutorServiceImpl.java      |  42 ++
 .../service/impl/TaskDefinitionServiceImpl.java    |  12 +-
 .../api/service/impl/TaskInstanceServiceImpl.java  |  79 +++-
 .../api/controller/TaskInstanceControllerTest.java |  16 +-
 .../api/service/TaskInstanceServiceTest.java       | 110 ++---
 .../common/enums/TaskExecuteType.java              |  38 +-
 .../dolphinscheduler/common/model/TaskNode.java    |  27 +-
 .../dolphinscheduler/common/utils/DateUtils.java   |  20 +-
 .../common/utils/DateUtilsTest.java                |  52 +--
 .../dao/entity/TaskDefinition.java                 |  18 +-
 .../dolphinscheduler/dao/entity/TaskInstance.java  |  22 +-
 .../dao/mapper/TaskDefinitionMapper.java           |   5 +-
 .../dao/mapper/TaskInstanceMapper.java             |  15 +
 .../dao/utils/TaskInstanceUtils.java               |   1 +
 .../dao/mapper/TaskDefinitionLogMapper.xml         |   6 +-
 .../dao/mapper/TaskDefinitionMapper.xml            |  12 +-
 .../dao/mapper/TaskInstanceMapper.xml              |  48 +-
 .../src/main/resources/sql/dolphinscheduler_h2.sql |   6 +-
 .../main/resources/sql/dolphinscheduler_mysql.sql  |   6 +-
 .../resources/sql/dolphinscheduler_postgresql.sql  |   6 +-
 .../3.1.0_schema/mysql/dolphinscheduler_ddl.sql    |  16 +-
 .../3.1.0_schema/mysql/dolphinscheduler_dml.sql    |  13 +-
 .../postgresql/dolphinscheduler_ddl.sql            |  49 +++
 .../postgresql/dolphinscheduler_dml.sql            |  13 +-
 .../dao/mapper/TaskInstanceMapperTest.java         |  25 +-
 .../cache/StreamTaskInstanceExecCacheManager.java  |  52 +++
 .../StreamTaskInstanceExecCacheManagerImpl.java    |  74 ++++
 .../server/master/event/TaskStateEventHandler.java |   2 +-
 .../server/master/metrics/TaskMetrics.java         |   3 +
 .../processor/TaskExecuteStartProcessor.java       |  76 ++++
 .../processor/queue/TaskExecuteThreadPool.java     |   9 +
 .../server/master/rpc/MasterRPCServer.java         |   5 +
 .../server/master/runner/EventExecuteService.java  |  27 +-
 .../master/runner/StreamTaskExecuteRunnable.java   | 481 +++++++++++++++++++++
 .../master/runner/StreamTaskExecuteThreadPool.java |  79 ++++
 .../master/runner/WorkflowExecuteRunnable.java     |  11 +-
 .../master/runner/task/BaseTaskProcessor.java      |   4 +-
 .../master/runner/task/TaskProcessorFactory.java   |   4 +
 .../remote/command/CommandType.java                |  15 +
 .../remote/command/TaskExecuteStartCommand.java    |  77 ++++
 .../command/TaskSavePointRequestCommand.java       |  53 ++-
 .../command/TaskSavePointResponseCommand.java      |  53 ++-
 .../remote/dto/TaskInstanceExecuteDto.java         |   4 +
 .../service/process/ProcessServiceImpl.java        |   1 +
 .../dolphinscheduler-task-all/pom.xml              |   6 +
 .../plugin/task/api/AbstractCommandExecutor.java   |  53 ---
 .../plugin/task/api/AbstractTask.java              |  63 ++-
 .../plugin/task/api/AbstractYarnTask.java          |  12 +-
 .../plugin/task/api/TaskConstants.java             |   6 +-
 .../plugin/task/api/stream/StreamTask.java         |  13 +-
 .../plugin/task/api/stream/StreamTaskChannel.java  |  32 +-
 .../plugin/task/api/AbstractTaskTest.java          |  40 +-
 .../plugin/task/chunjun/ChunJunTask.java           |   3 +-
 .../plugin/task/datax/DataxTask.java               |   3 +-
 .../dolphinscheduler/plugin/task/dvc/DvcTask.java  |   3 +-
 .../dolphinscheduler-task-flink-stream/pom.xml     |  48 ++
 .../plugin/task/flink/FlinkStreamParameters.java   |  14 +-
 .../plugin/task/flink/FlinkStreamTask.java}        |  62 ++-
 .../plugin/task/flink/FlinkStreamTaskChannel.java  |  59 +++
 .../task/flink/FlinkStreamTaskChannelFactory.java  |  44 +-
 .../plugin/task/flink/FlinkArgsUtilsTest.java      |  24 +-
 .../plugin/task/flink/FlinkParametersTest.java     |  57 +++
 .../plugin/task/flink/FlinkArgsUtils.java          |  39 +-
 .../plugin/task/flink/FlinkConstants.java          |   5 +
 .../plugin/task/flink/FlinkTask.java               |  57 ++-
 .../plugin/task/flink/FlinkArgsUtilsTest.java      |  12 +-
 .../plugin/task/jupyter/JupyterTask.java           |   2 +-
 .../plugin/task/mlflow/MlflowTask.java             |   3 +-
 .../plugin/task/python/PythonTask.java             |   2 +-
 .../plugin/task/seatunnel/SeatunnelTask.java       |   3 +-
 .../plugin/task/shell/ShellTask.java               |   3 +-
 dolphinscheduler-task-plugin/pom.xml               |   1 +
 dolphinscheduler-ui/src/common/common.ts           |  28 ++
 dolphinscheduler-ui/src/locales/en_US/project.ts   |  22 +-
 dolphinscheduler-ui/src/locales/zh_CN/project.ts   |  23 +-
 .../src/service/modules/task-definition/index.ts   |  27 +-
 .../src/service/modules/task-definition/types.ts   |  15 +-
 .../src/service/modules/task-instances/index.ts    |  14 +
 .../src/service/modules/task-instances/types.ts    |   2 +
 .../task/components/node/fields/use-flink.ts       |  15 +-
 .../projects/task/components/node/format-data.ts   |  10 +-
 .../projects/task/components/node/tasks/index.ts   |   4 +-
 .../task/components/node/tasks/use-flink-stream.ts |  88 ++++
 .../views/projects/task/components/node/types.ts   |   6 +-
 .../projects/task/components/node/use-task.ts      |   3 +
 .../src/views/projects/task/constants/task-type.ts |  16 +-
 .../task/definition/{index.tsx => batch-task.tsx}  |  18 +-
 .../task/definition/components/start-modal.tsx     | 267 ++++++++++++
 .../task/definition/components/use-start.tsx       | 143 ++++++
 .../src/views/projects/task/definition/index.tsx   | 208 +--------
 .../views/projects/task/definition/stream-task.tsx | 159 +++++++
 .../src/views/projects/task/definition/types.ts    |  14 +-
 .../projects/task/definition/use-stream-table.ts   | 202 +++++++++
 .../src/views/projects/task/definition/use-task.ts |  26 +-
 .../task/instance/{index.tsx => batch-task.tsx}    |  20 +-
 .../src/views/projects/task/instance/index.tsx     | 233 +---------
 .../task/instance/{index.tsx => stream-task.tsx}   |  65 +--
 .../projects/task/instance/use-stream-table.ts     | 357 +++++++++++++++
 .../workflow/components/dag/dag.module.scss        |   6 +-
 .../workflow/components/dag/use-canvas-init.ts     |  18 +-
 .../workflow/components/dag/use-cell-active.ts     |  16 +-
 .../workflow/components/dag/use-cell-update.ts     |   2 +-
 .../components/dag/use-custom-cell-builder.ts      |  38 +-
 .../projects/workflow/definition/tree/index.tsx    |   7 +-
 .../server/worker/processor/TaskKillProcessor.java |   2 +
 .../worker/processor/TaskSavePointProcessor.java   | 121 ++++++
 .../server/worker/rpc/WorkerRpcServer.java         |   5 +
 128 files changed, 3854 insertions(+), 924 deletions(-)

diff --git a/docs/docs/en/guide/project/task-definition.md b/docs/docs/en/guide/project/task-definition.md
index c30948987a..a08d3a3da8 100644
--- a/docs/docs/en/guide/project/task-definition.md
+++ b/docs/docs/en/guide/project/task-definition.md
@@ -1,13 +1,23 @@
 # Task Definition
 
+## Batch Task Definition
 Task definition allows to modify or operate tasks at the task level rather than modifying them in the workflow definition.
 We already have workflow level task editor in [workflow definition](workflow-definition.md) which you can click the specific
 workflow and then edit its task definition. It is depressing when you want to edit the task definition but do not remember
 which workflow it belongs to. So we decide to add `Task Definition` view under `Task` menu.
 
-![task-definition](../../../../img/new_ui/dev/project/task-definition.jpg)
+![task-definition](../../../../img/new_ui/dev/project/batch-task-definition.png)
 
 In this view, you can create, query, update, delete task definition by click the related button in `operation` column. The
 most exciting thing is you could query task by task name in the wildcard, and it is useful when you only remember the task
 name but forget which workflow it belongs to. It is also supported query by the task name alone with `Task Type` or
 `Workflow Name`
+
+## Stream Task Definition
+Stream task definitions are created in the workflow definition, and can be modified and executed.
+
+![task-definition](../../../../img/new_ui/dev/project/stream-task-definition.png)
+
+Click the execute button, check the execution parameters and click Confirm to submit the stream task.
+
+![task-definition](../../../../img/new_ui/dev/project/stream-task-execute.png)
\ No newline at end of file
diff --git a/docs/docs/en/guide/project/task-instance.md b/docs/docs/en/guide/project/task-instance.md
index 2d7651732c..0371ed9d79 100644
--- a/docs/docs/en/guide/project/task-instance.md
+++ b/docs/docs/en/guide/project/task-instance.md
@@ -1,13 +1,23 @@
 # Task Instance
 
-## Create Task Instance
+## Batch Task Instance
+### Create Task Instance
 
 Click `Project Management -> Workflow -> Task Instance` to enter the task instance page, as shown in the figure below, click the name of the workflow instance to jump to the DAG diagram of the workflow instance to view the task status.
 
-![task-instance](../../../../img/new_ui/dev/project/task-instance.png)
+![task-instance](../../../../img/new_ui/dev/project/batch-task-instance.png)
 
-## View Log
+### View Log
 
 Click the `View Log` button in the operation column to view the log of the task execution
 
 ![task-log](../../../../img/new_ui/dev/project/task-log.png)
+
+## Stream Task Instance
+
+-Switch to the stream task instance page, as shown in the following figure:
+
+![task-instance](../../../../img/new_ui/dev/project/stream-task-instance.png)
+
+- SavePoint: Click the `SavePoint` button in the operation column to do stream task savepoint.
+- Stop: Click the `Stop` button in the operation column to stop the stream task.
diff --git a/docs/docs/en/guide/project/workflow-definition.md b/docs/docs/en/guide/project/workflow-definition.md
index 443cedbed7..fe8e86dd3c 100644
--- a/docs/docs/en/guide/project/workflow-definition.md
+++ b/docs/docs/en/guide/project/workflow-definition.md
@@ -34,6 +34,11 @@ Click the plus sign on the right of the task node to connect the task; as shown
 
 ![workflow-dependent](../../../../img/new_ui/dev/project/workflow-dependent.png)
 
+### Dependencies with stream task
+If the DAG contains stream tasks, the relationship between stream tasks is displayed as a dotted line, and the execution of stream tasks will be skipped when the workflow instance is executed.
+
+  ![workflow-dependent](../../../../img/new_ui/dev/project/workflow-definition-with-stream-task.png)
+
 **Delete dependencies:** Using your mouse to select the connection line, and click the "Delete" icon in the upper right corner <img src= "../../../../img/delete.png" width="35"/>, delete dependencies between tasks.
 
 ![workflow-delete](../../../../img/new_ui/dev/project/workflow-delete.png)
diff --git a/docs/docs/zh/guide/project/task-definition.md b/docs/docs/zh/guide/project/task-definition.md
index 12b775ec49..6b1ffe55f7 100644
--- a/docs/docs/zh/guide/project/task-definition.md
+++ b/docs/docs/zh/guide/project/task-definition.md
@@ -1,9 +1,22 @@
 # 任务定义
 
-任务定义允许您在基于任务级别而不是在工作流中操作修改任务。再此之前,我们已经有了工作流级别的任务编辑器,你可以在[工作流定义](workflow-definition.md)
+## 批量任务定义
+批量任务定义允许您在基于任务级别而不是在工作流中操作修改任务。再此之前,我们已经有了工作流级别的任务编辑器,你可以在[工作流定义](workflow-definition.md)
 单击特定的工作流,然后编辑任务的定义。当您想编辑特定的任务定义但不记得它属于哪个工作流时,这是令人沮丧的。所以我们决定在 `任务` 菜单下添加 `任务定义` 视图。
 
-![task-definition](../../../../img/new_ui/dev/project/task-definition.jpg)
+![task-definition](../../../../img/new_ui/dev/project/batch-task-definition.png)
 
 在该视图中,您可以通过单击 `操作` 列中的相关按钮来进行创建、查询、更新、删除任务定义。最令人兴奋的是您可以通过通配符进行全部任务查询,当您只
 记得任务名称但忘记它属于哪个工作流时是非常有用的。也支持通过任务名称结合使用 `任务类型` 或 `工作流程名称` 进行查询。
+
+
+## 实时任务定义
+实时任务定义在工作流定义中创建,在任务定义页面可以进行修改和执行。
+
+![task-definition](../../../../img/new_ui/dev/project/stream-task-definition.png)
+
+点击实时任务执行,检查执行参数后点击确认,即可提交实时任务。
+
+![task-definition](../../../../img/new_ui/dev/project/stream-task-execute.png)
+
+
diff --git a/docs/docs/zh/guide/project/task-instance.md b/docs/docs/zh/guide/project/task-instance.md
index 205086d02a..b7a928b7e2 100644
--- a/docs/docs/zh/guide/project/task-instance.md
+++ b/docs/docs/zh/guide/project/task-instance.md
@@ -1,9 +1,19 @@
 # 任务实例
 
+## 批量任务实例
 - 点击项目管理->工作流->任务实例,进入任务实例页面,如下图所示,点击工作流实例名称,可跳转到工作流实例DAG图查看任务状态。
 
-![task-instance](../../../../img/new_ui/dev/project/task-instance.png)
+![task-instance](../../../../img/new_ui/dev/project/batch-task-instance.png)
 
 - 查看日志:点击操作列中的“查看日志”按钮,可以查看任务执行的日志情况。
 
 ![task-log](../../../../img/new_ui/dev/project/task-log.png)
+
+## 实时任务实例
+
+- 切换到实时任务实例页面,如下图所示:
+
+![task-instance](../../../../img/new_ui/dev/project/stream-task-instance.png)
+
+- SavePoint:点击操作列中的SavePoint按钮,可以进行实时任务的SavePoint。
+- Stop:点击操作列中的Stop按钮,可以停止该实时任务。
diff --git a/docs/docs/zh/guide/project/workflow-definition.md b/docs/docs/zh/guide/project/workflow-definition.md
index 65fbbdebf9..63c8e79d4b 100644
--- a/docs/docs/zh/guide/project/workflow-definition.md
+++ b/docs/docs/zh/guide/project/workflow-definition.md
@@ -24,6 +24,10 @@
 
   ![workflow-dependent](../../../../img/new_ui/dev/project/workflow-dependent.png)
 
+- **实时任务的依赖关系:** 若DAG中包含了实时任务的组件,则实时任务的关联关系显示为虚线,在执行工作流实例的时候会跳过实时任务的执行
+
+  ![workflow-dependent](../../../../img/new_ui/dev/project/workflow-definition-with-stream-task.png)
+
 - **删除依赖关系:** 点击右上角"箭头"图标<img src="../../../../img/arrow.png" width="35"/>,选中连接线,点击右上角"删除"图标<img src="../../../../img/delete.png" width="35"/>,删除任务间的依赖关系。
 
   ![workflow-delete](../../../../img/new_ui/dev/project/workflow-delete.png)
diff --git a/docs/img/new_ui/dev/project/batch-task-definition.png b/docs/img/new_ui/dev/project/batch-task-definition.png
new file mode 100644
index 0000000000..ca1608477e
Binary files /dev/null and b/docs/img/new_ui/dev/project/batch-task-definition.png differ
diff --git a/docs/img/new_ui/dev/project/batch-task-instance.png b/docs/img/new_ui/dev/project/batch-task-instance.png
new file mode 100644
index 0000000000..cdbbe3d38d
Binary files /dev/null and b/docs/img/new_ui/dev/project/batch-task-instance.png differ
diff --git a/docs/img/new_ui/dev/project/stream-task-definition.png b/docs/img/new_ui/dev/project/stream-task-definition.png
new file mode 100644
index 0000000000..b6c0629a45
Binary files /dev/null and b/docs/img/new_ui/dev/project/stream-task-definition.png differ
diff --git a/docs/img/new_ui/dev/project/stream-task-execute.png b/docs/img/new_ui/dev/project/stream-task-execute.png
new file mode 100644
index 0000000000..2b49cd8df7
Binary files /dev/null and b/docs/img/new_ui/dev/project/stream-task-execute.png differ
diff --git a/docs/img/new_ui/dev/project/stream-task-instance.png b/docs/img/new_ui/dev/project/stream-task-instance.png
new file mode 100644
index 0000000000..4b3693d209
Binary files /dev/null and b/docs/img/new_ui/dev/project/stream-task-instance.png differ
diff --git a/docs/img/new_ui/dev/project/task-definition.jpg b/docs/img/new_ui/dev/project/task-definition.jpg
deleted file mode 100644
index cd2db4e8ce..0000000000
Binary files a/docs/img/new_ui/dev/project/task-definition.jpg and /dev/null differ
diff --git a/docs/img/new_ui/dev/project/task-instance.png b/docs/img/new_ui/dev/project/task-instance.png
deleted file mode 100644
index 4026736573..0000000000
Binary files a/docs/img/new_ui/dev/project/task-instance.png and /dev/null differ
diff --git a/docs/img/new_ui/dev/project/workflow-definition-with-stream-task.png b/docs/img/new_ui/dev/project/workflow-definition-with-stream-task.png
new file mode 100644
index 0000000000..298df8b4ac
Binary files /dev/null and b/docs/img/new_ui/dev/project/workflow-definition-with-stream-task.png differ
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ExecutorController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ExecutorController.java
index aa48c67448..dfe9396089 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ExecutorController.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ExecutorController.java
@@ -380,4 +380,51 @@ public class ExecutorController extends BaseController {
         WorkflowExecuteDto workflowExecuteDto = execService.queryExecutingWorkflowByProcessInstanceId(processInstanceId);
         return Result.success(workflowExecuteDto);
     }
+
+    /**
+     * execute task instance
+     *
+     * @param loginUser login user
+     * @param projectCode project code
+     * @param code taskDefinitionCode
+     * @param version taskDefinitionVersion
+     * @param warningGroupId warning group id
+     * @param workerGroup worker group
+     * @return start task result code
+     */
+    @ApiOperation(value = "startTaskInstance", notes = "RUN_TASK_INSTANCE_NOTES")
+    @ApiImplicitParams({
+        @ApiImplicitParam(name = "version", value = "VERSION", dataType = "Int", example = "1"),
+        @ApiImplicitParam(name = "failureStrategy", value = "FAILURE_STRATEGY", required = true, dataType = "FailureStrategy"),
+        @ApiImplicitParam(name = "execType", value = "COMMAND_TYPE", dataType = "CommandType"),
+        @ApiImplicitParam(name = "warningType", value = "WARNING_TYPE", required = true, dataType = "WarningType"),
+        @ApiImplicitParam(name = "warningGroupId", value = "WARNING_GROUP_ID", dataType = "Int", example = "100"),
+        @ApiImplicitParam(name = "workerGroup", value = "WORKER_GROUP", dataType = "String", example = "default"),
+        @ApiImplicitParam(name = "environmentCode", value = "ENVIRONMENT_CODE", dataType = "Long", example = "-1"),
+        @ApiImplicitParam(name = "timeout", value = "TIMEOUT", dataType = "Int", example = "100"),
+        @ApiImplicitParam(name = "dryRun", value = "DRY_RUN", dataType = "Int", example = "0"),
+    })
+    @PostMapping(value = "/task-instance/{code}/start")
+    @ResponseStatus(HttpStatus.OK)
+    @ApiException(START_PROCESS_INSTANCE_ERROR)
+    @AccessLogAnnotation(ignoreRequestArgs = "loginUser")
+    public Result startStreamTaskInstance(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
+                                       @ApiParam(name = "projectCode", value = "PROJECT_CODE", required = true) @PathVariable long projectCode,
+                                       @ApiParam(name = "code", value = "TASK_CODE", required = true) @PathVariable long code,
+                                       @RequestParam(value = "version", required = true) int version,
+                                       @RequestParam(value = "warningGroupId", required = false, defaultValue = "0") Integer warningGroupId,
+                                       @RequestParam(value = "workerGroup", required = false, defaultValue = "default") String workerGroup,
+                                       @RequestParam(value = "environmentCode", required = false, defaultValue = "-1") Long environmentCode,
+                                       @RequestParam(value = "startParams", required = false) String startParams,
+                                       @RequestParam(value = "dryRun", defaultValue = "0", required = false) int dryRun) {
+
+        Map<String, String> startParamMap = null;
+        if (startParams != null) {
+            startParamMap = JSONUtils.toMap(startParams);
+        }
+
+        Map<String, Object> result = execService.execStreamTaskInstance(loginUser, projectCode, code, version,
+            warningGroupId, workerGroup, environmentCode, startParamMap, dryRun);
+        return returnDataList(result);
+    }
 }
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskDefinitionController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskDefinitionController.java
index a5d1b2f762..64269ddfa3 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskDefinitionController.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskDefinitionController.java
@@ -34,6 +34,7 @@ import org.apache.dolphinscheduler.api.service.TaskDefinitionService;
 import org.apache.dolphinscheduler.api.utils.Result;
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.ReleaseState;
+import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
 import org.apache.dolphinscheduler.common.utils.ParameterUtils;
 import org.apache.dolphinscheduler.dao.entity.User;
 
@@ -322,6 +323,7 @@ public class TaskDefinitionController extends BaseController {
      * @param searchWorkflowName searchWorkflowName
      * @param searchTaskName searchTaskName
      * @param taskType taskType
+     * @param taskExecuteType taskExecuteType
      * @param pageNo page number
      * @param pageSize page size
      * @return task definition page
@@ -332,6 +334,7 @@ public class TaskDefinitionController extends BaseController {
         @ApiImplicitParam(name = "searchWorkflowName", value = "SEARCH_WORKFLOW_NAME", required = false, type = "String"),
         @ApiImplicitParam(name = "searchTaskName", value = "SEARCH_TASK_NAME", required = false, type = "String"),
         @ApiImplicitParam(name = "taskType", value = "TASK_TYPE", required = false, dataType = "TaskType", example = "SHELL"),
+        @ApiImplicitParam(name = "taskExecuteType", value = "TASK_EXECUTE_TYPE", required = false, dataType = "TaskExecuteType", example = "STREAM"),
         @ApiImplicitParam(name = "pageNo", value = "PAGE_NO", required = true, dataType = "Int", example = "1"),
         @ApiImplicitParam(name = "pageSize", value = "PAGE_SIZE", required = true, dataType = "Int", example = "10")
     })
@@ -344,6 +347,7 @@ public class TaskDefinitionController extends BaseController {
                                                 @RequestParam(value = "searchWorkflowName", required = false) String searchWorkflowName,
                                                 @RequestParam(value = "searchTaskName", required = false) String searchTaskName,
                                                 @RequestParam(value = "taskType", required = false) String taskType,
+                                                @RequestParam(value = "taskExecuteType", required = false, defaultValue = "BATCH") TaskExecuteType taskExecuteType,
                                                 @RequestParam("pageNo") Integer pageNo,
                                                 @RequestParam("pageSize") Integer pageSize) {
         Result result = checkPageParams(pageNo, pageSize);
@@ -352,7 +356,7 @@ public class TaskDefinitionController extends BaseController {
         }
         searchWorkflowName = ParameterUtils.handleEscapes(searchWorkflowName);
         searchTaskName = ParameterUtils.handleEscapes(searchTaskName);
-        return taskDefinitionService.queryTaskDefinitionListPaging(loginUser, projectCode, searchWorkflowName, searchTaskName, taskType, pageNo, pageSize);
+        return taskDefinitionService.queryTaskDefinitionListPaging(loginUser, projectCode, searchWorkflowName, searchTaskName, taskType, taskExecuteType, pageNo, pageSize);
     }
 
     /**
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 411fd417e6..e7a707af28 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
@@ -19,12 +19,15 @@ package org.apache.dolphinscheduler.api.controller;
 
 import static org.apache.dolphinscheduler.api.enums.Status.FORCE_TASK_SUCCESS_ERROR;
 import static org.apache.dolphinscheduler.api.enums.Status.QUERY_TASK_LIST_PAGING_ERROR;
+import static org.apache.dolphinscheduler.api.enums.Status.TASK_SAVEPOINT_ERROR;
+import static org.apache.dolphinscheduler.api.enums.Status.TASK_STOP_ERROR;
 
 import org.apache.dolphinscheduler.api.aspect.AccessLogAnnotation;
 import org.apache.dolphinscheduler.api.exceptions.ApiException;
 import org.apache.dolphinscheduler.api.service.TaskInstanceService;
 import org.apache.dolphinscheduler.api.utils.Result;
 import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
 import org.apache.dolphinscheduler.common.utils.ParameterUtils;
 import org.apache.dolphinscheduler.dao.entity.User;
 
@@ -74,6 +77,7 @@ public class TaskInstanceController extends BaseController {
      * @param endTime end time
      * @param pageNo page number
      * @param pageSize page size
+     * @param taskExecuteType task execute type
      * @return task list page
      */
     @ApiOperation(value = "queryTaskListPaging", notes = "QUERY_TASK_INSTANCE_LIST_PAGING_NOTES")
@@ -87,8 +91,9 @@ public class TaskInstanceController extends BaseController {
             @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 = "taskExecuteType", value = "TASK_EXECUTE_TYPE", required = false, dataType = "TaskExecuteType", example = "STREAM"),
             @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 = "pageSize", value = "PAGE_SIZE", required = true, dataType = "Int", example = "20"),
     })
     @GetMapping()
     @ResponseStatus(HttpStatus.OK)
@@ -98,6 +103,7 @@ public class TaskInstanceController extends BaseController {
                                       @ApiParam(name = "projectCode", value = "PROJECT_CODE", required = true) @PathVariable long projectCode,
                                       @RequestParam(value = "processInstanceId", required = false, defaultValue = "0") Integer processInstanceId,
                                       @RequestParam(value = "processInstanceName", required = false) String processInstanceName,
+                                      @RequestParam(value = "processDefinitionName", required = false) String processDefinitionName,
                                       @RequestParam(value = "searchVal", required = false) String searchVal,
                                       @RequestParam(value = "taskName", required = false) String taskName,
                                       @RequestParam(value = "executorName", required = false) String executorName,
@@ -105,6 +111,7 @@ public class TaskInstanceController extends BaseController {
                                       @RequestParam(value = "host", required = false) String host,
                                       @RequestParam(value = "startDate", required = false) String startTime,
                                       @RequestParam(value = "endDate", required = false) String endTime,
+                                      @RequestParam(value = "taskExecuteType", required = false, defaultValue = "BATCH") TaskExecuteType taskExecuteType,
                                       @RequestParam("pageNo") Integer pageNo,
                                       @RequestParam("pageSize") Integer pageSize) {
         Result result = checkPageParams(pageNo, pageSize);
@@ -112,8 +119,8 @@ public class TaskInstanceController extends BaseController {
             return result;
         }
         searchVal = ParameterUtils.handleEscapes(searchVal);
-        result = taskInstanceService.queryTaskListPaging(loginUser, projectCode, processInstanceId, processInstanceName,
-                taskName, executorName, startTime, endTime, searchVal, stateType, host, pageNo, pageSize);
+        result = taskInstanceService.queryTaskListPaging(loginUser, projectCode, processInstanceId, processInstanceName, processDefinitionName,
+                taskName, executorName, startTime, endTime, searchVal, stateType, host, taskExecuteType, pageNo, pageSize);
         return result;
     }
 
@@ -140,4 +147,47 @@ public class TaskInstanceController extends BaseController {
         return returnDataList(result);
     }
 
+    /**
+     * task savepoint, for stream task
+     *
+     * @param loginUser login user
+     * @param projectCode project code
+     * @param id task instance id
+     * @return the result code and msg
+     */
+    @ApiOperation(value = "savepoint", notes = "TASK_SAVEPOINT")
+    @ApiImplicitParams({
+        @ApiImplicitParam(name = "id", value = "TASK_INSTANCE_ID", required = true, dataType = "Int", example = "12")
+    })
+    @PostMapping(value = "/{id}/savepoint")
+    @ResponseStatus(HttpStatus.OK)
+    @ApiException(TASK_SAVEPOINT_ERROR)
+    @AccessLogAnnotation(ignoreRequestArgs = "loginUser")
+    public Result<Object> taskSavePoint(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
+                                           @ApiParam(name = "projectCode", value = "PROJECT_CODE", required = true) @PathVariable long projectCode,
+                                           @PathVariable(value = "id") Integer id) {
+        return taskInstanceService.taskSavePoint(loginUser, projectCode, id);
+    }
+
+    /**
+     * task stop, for stream task
+     *
+     * @param loginUser login user
+     * @param projectCode project code
+     * @param id task instance id
+     * @return the result code and msg
+     */
+    @ApiOperation(value = "stop", notes = "TASK_STOP")
+    @ApiImplicitParams({
+        @ApiImplicitParam(name = "id", value = "TASK_INSTANCE_ID", required = true, dataType = "Int", example = "12")
+    })
+    @PostMapping(value = "/{id}/stop")
+    @ResponseStatus(HttpStatus.OK)
+    @ApiException(TASK_STOP_ERROR)
+    @AccessLogAnnotation(ignoreRequestArgs = "loginUser")
+    public Result<Object> stopTask(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
+                                        @ApiParam(name = "projectCode", value = "PROJECT_CODE", required = true) @PathVariable long projectCode,
+                                        @PathVariable(value = "id") Integer id) {
+        return taskInstanceService.stopTask(loginUser, projectCode, id);
+    }
 }
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java
index e54c23e906..53501a3c5f 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java
@@ -220,6 +220,8 @@ public enum Status {
     DELETE_PROCESS_DEFINITION_USE_BY_OTHER_FAIL(10193, "delete process definition fail, cause used by other tasks: {0}", "删除工作流定时失败,被其他任务引用:{0}"),
     DELETE_TASK_USE_BY_OTHER_FAIL(10194, "delete task {0} fail, cause used by other tasks: {1}", "删除任务 {0} 失败,被其他任务引用:{1}"),
     TASK_WITH_DEPENDENT_ERROR(10195, "task used in other tasks", "删除被其他任务引用"),
+    TASK_SAVEPOINT_ERROR(10196, "task savepoint error", "任务实例savepoint错误"),
+    TASK_STOP_ERROR(10197, "task stop error", "任务实例停止错误"),
 
     UDF_FUNCTION_NOT_EXIST(20001, "UDF function not found", "UDF函数不存在"),
     UDF_FUNCTION_EXISTS(20002, "UDF function already exists", "UDF函数已存在"),
@@ -305,6 +307,8 @@ public enum Status {
     NOT_SUPPORT_UPDATE_TASK_DEFINITION(50056, "task state does not support modification", "当前任务不支持修改"),
     NOT_SUPPORT_COPY_TASK_TYPE(50057, "task type [{0}] does not support copy", "不支持复制的任务类型[{0}]"),
     BATCH_EXECUTE_PROCESS_INSTANCE_ERROR(50058, "change process instance status error: {0}", "修改工作实例状态错误: {0}"),
+    START_TASK_INSTANCE_ERROR(50059, "start task instance error", "运行任务流实例错误"),
+
     HDFS_NOT_STARTUP(60001, "hdfs not startup", "hdfs未启用"),
     STORAGE_NOT_STARTUP(60002, "storage not startup", "存储未启用"),
     S3_CANNOT_RENAME(60003, "directory cannot be renamed", "S3无法重命名文件夹"),
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java
index 1aefd35471..8146f96e1c 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java
@@ -119,4 +119,22 @@ public interface ExecutorService {
      * @return
      */
     WorkflowExecuteDto queryExecutingWorkflowByProcessInstanceId(Integer processInstanceId);
+
+    /**
+     * execute stream task instance
+     *
+     * @param loginUser login user
+     * @param projectCode project code
+     * @param warningGroupId notify group id
+     * @param workerGroup worker group name
+     * @param environmentCode environment code
+     * @param startParams the global param values which pass to new process instance
+     * @return execute process instance code
+     */
+    Map<String, Object> execStreamTaskInstance(User loginUser, long projectCode,
+                                            long taskDefinitionCode, int taskDefinitionVersion,
+                                            int warningGroupId,
+                                            String workerGroup, Long environmentCode,
+                                            Map<String, String> startParams,
+                                            int dryRun);
 }
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskDefinitionService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskDefinitionService.java
index a715b68e97..0a84ec7553 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskDefinitionService.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskDefinitionService.java
@@ -19,6 +19,7 @@ package org.apache.dolphinscheduler.api.service;
 
 import org.apache.dolphinscheduler.api.utils.Result;
 import org.apache.dolphinscheduler.common.enums.ReleaseState;
+import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
 import org.apache.dolphinscheduler.dao.entity.User;
 
 import java.util.Map;
@@ -171,6 +172,7 @@ public interface TaskDefinitionService {
      * @param searchWorkflowName searchWorkflowName
      * @param searchTaskName searchTaskName
      * @param taskType taskType
+     * @param taskExecuteType taskExecuteType
      * @param pageNo page number
      * @param pageSize page size
      * @return task definition page
@@ -180,6 +182,7 @@ public interface TaskDefinitionService {
                                          String searchWorkflowName,
                                          String searchTaskName,
                                          String taskType,
+                                         TaskExecuteType taskExecuteType,
                                          Integer pageNo,
                                          Integer pageSize);
 
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 f532b7d046..b86d4d757e 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
@@ -18,6 +18,7 @@
 package org.apache.dolphinscheduler.api.service;
 
 import org.apache.dolphinscheduler.api.utils.Result;
+import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
 import org.apache.dolphinscheduler.dao.entity.User;
 import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 
@@ -40,6 +41,7 @@ public interface TaskInstanceService {
      * @param host host
      * @param startDate start time
      * @param endDate end time
+     * @param taskExecuteType task execute type
      * @param pageNo page number
      * @param pageSize page size
      * @return task list page
@@ -48,6 +50,7 @@ public interface TaskInstanceService {
                                long projectCode,
                                Integer processInstanceId,
                                String processInstanceName,
+                               String processDefinitionName,
                                String taskName,
                                String executorName,
                                String startDate,
@@ -55,6 +58,7 @@ public interface TaskInstanceService {
                                String searchVal,
                                TaskExecutionStatus stateType,
                                String host,
+                               TaskExecuteType taskExecuteType,
                                Integer pageNo,
                                Integer pageSize);
 
@@ -69,4 +73,22 @@ public interface TaskInstanceService {
     Map<String, Object> forceTaskSuccess(User loginUser,
                                          long projectCode,
                                          Integer taskInstanceId);
+
+    /**
+     * task savepoint
+     * @param loginUser
+     * @param projectCode
+     * @param taskInstanceId
+     * @return
+     */
+    Result taskSavePoint(User loginUser, long projectCode, Integer taskInstanceId);
+
+    /**
+     * stop task
+     * @param loginUser
+     * @param projectCode
+     * @param taskInstanceId
+     * @return
+     */
+    Result stopTask(User loginUser, long projectCode, Integer taskInstanceId);
 }
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 7cf164aaaa..4502fcd8ec 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
@@ -57,6 +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.remote.command.TaskExecuteStartCommand;
 import org.apache.dolphinscheduler.remote.command.WorkflowStateEventChangeCommand;
 import org.apache.dolphinscheduler.remote.command.WorkflowExecutingDataRequestCommand;
 import org.apache.dolphinscheduler.remote.command.WorkflowExecutingDataResponseCommand;
@@ -1013,4 +1014,45 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
                 JSONUtils.parseObject(command.getBody(), WorkflowExecutingDataResponseCommand.class);
         return responseCommand.getWorkflowExecuteDto();
     }
+
+    @Override
+    public Map<String, Object> execStreamTaskInstance(User loginUser, long projectCode, long taskDefinitionCode, int taskDefinitionVersion,
+                                                int warningGroupId, String workerGroup, Long environmentCode, Map<String, String> startParams, int dryRun) {
+        Project project = projectMapper.queryByCode(projectCode);
+        //check user access for project
+        Map<String, Object> result =
+            projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_START);
+        if (result.get(Constants.STATUS) != Status.SUCCESS) {
+            return result;
+        }
+
+        // check master exists
+        if (!checkMasterExists(result)) {
+            return result;
+        }
+
+        // todo dispatch improvement
+        List<Server> masterServerList = monitorService.getServerListFromRegistry(true);
+        Host host = new Host(masterServerList.get(0).getHost(), masterServerList.get(0).getPort());
+
+        TaskExecuteStartCommand taskExecuteStartCommand = new TaskExecuteStartCommand();
+        taskExecuteStartCommand.setExecutorId(loginUser.getId());
+        taskExecuteStartCommand.setExecutorName(loginUser.getUserName());
+        taskExecuteStartCommand.setProjectCode(projectCode);
+        taskExecuteStartCommand.setTaskDefinitionCode(taskDefinitionCode);
+        taskExecuteStartCommand.setTaskDefinitionVersion(taskDefinitionVersion);
+        taskExecuteStartCommand.setWorkerGroup(workerGroup);
+        taskExecuteStartCommand.setWarningGroupId(warningGroupId);
+        taskExecuteStartCommand.setEnvironmentCode(environmentCode);
+        taskExecuteStartCommand.setStartParams(startParams);
+        taskExecuteStartCommand.setDryRun(dryRun);
+
+        org.apache.dolphinscheduler.remote.command.Command response = stateEventCallbackService.sendSync(host, taskExecuteStartCommand.convert2Command());
+        if (response != null) {
+            putMsg(result, Status.SUCCESS);
+        } else {
+            putMsg(result, Status.START_TASK_INSTANCE_ERROR);
+        }
+        return result;
+    }
 }
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskDefinitionServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskDefinitionServiceImpl.java
index 338d6644e2..87ba643188 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskDefinitionServiceImpl.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskDefinitionServiceImpl.java
@@ -28,6 +28,7 @@ import org.apache.dolphinscheduler.common.enums.AuthorizationType;
 import org.apache.dolphinscheduler.common.enums.ConditionType;
 import org.apache.dolphinscheduler.common.enums.Flag;
 import org.apache.dolphinscheduler.common.enums.ReleaseState;
+import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
 import org.apache.dolphinscheduler.common.utils.CodeGenerateUtils;
 import org.apache.dolphinscheduler.common.utils.CodeGenerateUtils.CodeGenerateException;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
@@ -413,8 +414,11 @@ public class TaskDefinitionServiceImpl extends BaseServiceImpl implements TaskDe
             return null;
         }
         if (processService.isTaskOnline(taskCode) && taskDefinition.getFlag() == Flag.YES) {
-            putMsg(result, Status.NOT_SUPPORT_UPDATE_TASK_DEFINITION);
-            return null;
+            // if stream, can update task definition without online check
+            if (taskDefinition.getTaskExecuteType() != TaskExecuteType.STREAM) {
+                putMsg(result, Status.NOT_SUPPORT_UPDATE_TASK_DEFINITION);
+                return null;
+            }
         }
         TaskDefinitionLog taskDefinitionToUpdate = JSONUtils.parseObject(taskDefinitionJsonObj, TaskDefinitionLog.class);
         if (taskDefinition.equals(taskDefinitionToUpdate)) {
@@ -662,6 +666,7 @@ public class TaskDefinitionServiceImpl extends BaseServiceImpl implements TaskDe
                                                 String searchWorkflowName,
                                                 String searchTaskName,
                                                 String taskType,
+                                                TaskExecuteType taskExecuteType,
                                                 Integer pageNo,
                                                 Integer pageSize) {
         Result result = new Result();
@@ -673,9 +678,10 @@ public class TaskDefinitionServiceImpl extends BaseServiceImpl implements TaskDe
             putMsg(result, resultStatus);
             return result;
         }
+        taskType = taskType == null ? StringUtils.EMPTY : taskType;
         Page<TaskMainInfo> page = new Page<>(pageNo, pageSize);
         IPage<TaskMainInfo> taskMainInfoIPage = taskDefinitionMapper.queryDefineListPaging(page, projectCode, searchWorkflowName,
-                searchTaskName, taskType == null ? StringUtils.EMPTY : taskType);
+                searchTaskName, taskType, taskExecuteType);
         List<TaskMainInfo> records = taskMainInfoIPage.getRecords();
         if (!records.isEmpty()) {
             Map<Long, TaskMainInfo> taskMainInfoMap = new HashMap<>();
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 73a2e3f96a..67da7c95ed 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
@@ -28,6 +28,7 @@ import org.apache.dolphinscheduler.api.service.UsersService;
 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.TaskExecuteType;
 import org.apache.dolphinscheduler.common.utils.CollectionUtils;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
 import org.apache.dolphinscheduler.dao.entity.Project;
@@ -37,6 +38,10 @@ 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.remote.command.TaskKillRequestCommand;
+import org.apache.dolphinscheduler.remote.command.TaskSavePointRequestCommand;
+import org.apache.dolphinscheduler.remote.processor.StateEventCallbackService;
+import org.apache.dolphinscheduler.remote.utils.Host;
 import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.service.process.ProcessService;
 
@@ -81,6 +86,9 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
     @Autowired
     TaskDefinitionMapper taskDefinitionMapper;
 
+    @Autowired
+    private StateEventCallbackService stateEventCallbackService;
+
     /**
      * query task list by project, process instance, task name, task start time, task end time, task status, keyword paging
      *
@@ -102,6 +110,7 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
                                       long projectCode,
                                       Integer processInstanceId,
                                       String processInstanceName,
+                                      String processDefinitionName,
                                       String taskName,
                                       String executorName,
                                       String startDate,
@@ -109,6 +118,7 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
                                       String searchVal,
                                       TaskExecutionStatus stateType,
                                       String host,
+                                      TaskExecuteType taskExecuteType,
                                       Integer pageNo,
                                       Integer pageSize) {
         Result result = new Result();
@@ -136,9 +146,17 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
         Page<TaskInstance> page = new Page<>(pageNo, pageSize);
         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);
+        IPage<TaskInstance> taskInstanceIPage;
+        if (taskExecuteType == TaskExecuteType.STREAM) {
+            // stream task without process instance
+            taskInstanceIPage = taskInstanceMapper.queryStreamTaskInstanceListPaging(
+                page, project.getCode(), processDefinitionName, searchVal, taskName, executorId, statusArray, host, taskExecuteType, start, end
+            );
+        } else {
+            taskInstanceIPage = taskInstanceMapper.queryTaskInstanceListPaging(
+                page, project.getCode(), processInstanceId, processInstanceName, searchVal, taskName, executorId, statusArray, host, taskExecuteType, start, end
+            );
+        }
         Set<String> exclusionSet = new HashSet<>();
         exclusionSet.add(Constants.CLASS);
         exclusionSet.add("taskJson");
@@ -210,4 +228,59 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
         }
         return result;
     }
+
+    @Override
+    public Result taskSavePoint(User loginUser, long projectCode, Integer taskInstanceId) {
+        Result result = new Result();
+
+        Project project = projectMapper.queryByCode(projectCode);
+        //check user access for project
+        Map<String, Object> checkResult = projectService.checkProjectAndAuth(loginUser, project, projectCode,FORCED_SUCCESS);
+        Status status = (Status) checkResult.get(Constants.STATUS);
+        if (status != Status.SUCCESS) {
+            putMsg(result,status);
+            return result;
+        }
+
+        TaskInstance taskInstance = taskInstanceMapper.selectById(taskInstanceId);
+        if (taskInstance == null) {
+            putMsg(result, Status.TASK_INSTANCE_NOT_FOUND);
+            return result;
+        }
+
+        TaskSavePointRequestCommand command = new TaskSavePointRequestCommand(taskInstanceId);
+
+        Host host = new Host(taskInstance.getHost());
+        stateEventCallbackService.sendResult(host, command.convert2Command());
+        putMsg(result, Status.SUCCESS);
+
+        return result;
+    }
+
+    @Override
+    public Result stopTask(User loginUser, long projectCode, Integer taskInstanceId) {
+        Result result = new Result();
+
+        Project project = projectMapper.queryByCode(projectCode);
+        //check user access for project
+        Map<String, Object> checkResult = projectService.checkProjectAndAuth(loginUser, project, projectCode,FORCED_SUCCESS);
+        Status status = (Status) checkResult.get(Constants.STATUS);
+        if (status != Status.SUCCESS) {
+            putMsg(result,status);
+            return result;
+        }
+
+        TaskInstance taskInstance = taskInstanceMapper.selectById(taskInstanceId);
+        if (taskInstance == null) {
+            putMsg(result, Status.TASK_INSTANCE_NOT_FOUND);
+            return result;
+        }
+
+        TaskKillRequestCommand command = new TaskKillRequestCommand(taskInstanceId);
+        Host host = new Host(taskInstance.getHost());
+        stateEventCallbackService.sendResult(host, command.convert2Command());
+        putMsg(result, Status.SUCCESS);
+
+        return result;
+    }
 }
diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskInstanceControllerTest.java
index 852520f1f3..7090a2f372 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
@@ -31,6 +31,7 @@ import org.apache.dolphinscheduler.api.service.TaskInstanceService;
 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.TaskExecuteType;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.dao.entity.User;
@@ -69,11 +70,10 @@ 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(),
-                eq(""), Mockito.any(), eq("192.168.xx.xx"), any(), any())).thenReturn(result);
-        Result taskResult = taskInstanceController.queryTaskListPaging(null, 1L, 1, "", "",
-                "", "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", "2020-01-01 00:00:00", "2020-01-02 00:00:00",
-                pageNo, pageSize);
+        when(taskInstanceService.queryTaskListPaging(any(), eq(1L), eq(1), eq(""), eq(""), eq(""), eq(""), any(), any(),
+            eq(""), Mockito.any(), eq("192.168.xx.xx"), eq(TaskExecuteType.BATCH), any(), any())).thenReturn(result);
+        Result taskResult = taskInstanceController.queryTaskListPaging(null, 1L, 1, "", "", "",
+            "", "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", "2020-01-01 00:00:00", "2020-01-02 00:00:00", TaskExecuteType.BATCH, pageNo, pageSize);
         Assert.assertEquals(Integer.valueOf(Status.SUCCESS.getCode()), taskResult.getCode());
     }
 
@@ -91,9 +91,9 @@ public class TaskInstanceControllerTest extends AbstractControllerTest {
         MvcResult mvcResult = mockMvc.perform(post("/projects/{projectName}/task-instance/force-success", "cxc_1113")
                 .header(SESSION_ID, sessionId)
                 .params(paramsMap))
-                .andExpect(status().isOk())
-                .andExpect(content().contentType(MediaType.APPLICATION_JSON))
-                .andReturn();
+            .andExpect(status().isOk())
+            .andExpect(content().contentType(MediaType.APPLICATION_JSON))
+            .andReturn();
 
         Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class);
         Assert.assertEquals(Status.SUCCESS.getCode(), result.getCode().intValue());
diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java
index c748838bfc..e1e783c2ac 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
@@ -25,11 +25,13 @@ import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.when;
 
 import org.apache.dolphinscheduler.api.ApiApplicationServer;
+import org.apache.dolphinscheduler.api.dto.gantt.Task;
 import org.apache.dolphinscheduler.api.enums.Status;
 import org.apache.dolphinscheduler.api.service.impl.ProjectServiceImpl;
 import org.apache.dolphinscheduler.api.service.impl.TaskInstanceServiceImpl;
 import org.apache.dolphinscheduler.api.utils.Result;
 import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
 import org.apache.dolphinscheduler.common.enums.UserType;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
@@ -101,18 +103,20 @@ public class TaskInstanceServiceTest {
         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,
+            "",
+            TaskExecuteType.BATCH,
+            1,
+            20);
         Assert.assertEquals(Status.PROJECT_NOT_FOUND.getCode(), (int) projectAuthFailRes.getCode());
 
         // data parameter check
@@ -120,18 +124,20 @@ 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",
-                "",
-                TaskExecutionStatus.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",
+            TaskExecuteType.BATCH,
+            1,
+            20);
         Assert.assertEquals(Status.REQUEST_PARAMS_NOT_VALID_ERROR.getCode(), (int) dataParameterRes.getCode());
 
         // project
@@ -148,53 +154,47 @@ 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(""),
-                eq(0), Mockito.any(), eq("192.168.xx.xx"), eq(start), eq(end))).thenReturn(pageReturn);
+        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(TaskExecuteType.BATCH), eq(start), eq(end))).thenReturn(pageReturn);
         when(usersService.queryUser(processInstance.getExecutorId())).thenReturn(loginUser);
         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", "", TaskExecutionStatus.SUCCESS,
-                "192.168.xx.xx", 1, 20);
+        Result successRes = taskInstanceService.queryTaskListPaging(loginUser, projectCode, 1, "", "", "",
+            "test_user", "2020-01-01 00:00:00", "2020-01-02 00:00:00", "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", TaskExecuteType.BATCH, 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(""),
-                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", "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", 1,
-                20);
+        //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(TaskExecuteType.BATCH), 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", "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", TaskExecuteType.BATCH, 1, 20);
         Assert.assertEquals(Status.SUCCESS.getCode(), (int) executorEmptyRes.getCode());
 
         // 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", "", TaskExecutionStatus.SUCCESS,
-                "192.168.xx.xx", 1, 20);
+
+        Result executorNullRes = taskInstanceService.queryTaskListPaging(loginUser, projectCode, 1, "", "", "",
+            "test_user", "2020-01-01 00:00:00", "2020-01-02 00:00:00", "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", TaskExecuteType.BATCH, 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(""),
-                eq(0), Mockito.any(), eq("192.168.xx.xx"), any(), any())).thenReturn(pageReturn);
-        Result executorNullDateRes = taskInstanceService.queryTaskListPaging(loginUser, projectCode, 1, "", "",
-                "", null, null, "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20);
+        //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"), eq(TaskExecuteType.BATCH), any(), any())).thenReturn(pageReturn);
+        Result executorNullDateRes = taskInstanceService.queryTaskListPaging(loginUser, projectCode, 1, "", "", "",
+            "", null, null, "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", TaskExecuteType.BATCH, 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(""),
-                eq(0), Mockito.any(), eq("192.168.xx.xx"), any(), any())).thenReturn(pageReturn);
+        //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"), eq(TaskExecuteType.BATCH), any(), any())).thenReturn(pageReturn);
 
-        Result executorErrorStartDateRes = taskInstanceService.queryTaskListPaging(loginUser, projectCode, 1, "", "",
-                "", "error date", null, "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20);
+        Result executorErrorStartDateRes = taskInstanceService.queryTaskListPaging(loginUser, projectCode, 1, "", "", "",
+            "", "error date", null, "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", TaskExecuteType.BATCH, 1, 20);
         Assert.assertEquals(Status.REQUEST_PARAMS_NOT_VALID_ERROR.getCode(), (int) executorErrorStartDateRes.getCode());
-        Result executorErrorEndDateRes = taskInstanceService.queryTaskListPaging(loginUser, projectCode, 1, "", "",
-                "", null, "error date", "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20);
+        Result executorErrorEndDateRes = taskInstanceService.queryTaskListPaging(loginUser, projectCode, 1, "", "", "",
+            "", null, "error date", "", TaskExecutionStatus.SUCCESS, "192.168.xx.xx", TaskExecuteType.BATCH, 1, 20);
         Assert.assertEquals(Status.REQUEST_PARAMS_NOT_VALID_ERROR.getCode(), (int) executorErrorEndDateRes.getCode());
     }
 
@@ -253,6 +253,8 @@ public class TaskInstanceServiceTest {
         taskInstance.setStartTime(new Date());
         taskInstance.setEndTime(new Date());
         taskInstance.setExecutorId(-1);
+        taskInstance.setTaskExecuteType(TaskExecuteType.BATCH);
+        taskInstance.setState(TaskExecutionStatus.SUBMITTED_SUCCESS);
         return taskInstance;
     }
 
diff --git a/dolphinscheduler-ui/src/views/projects/task/definition/types.ts b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskExecuteType.java
similarity index 59%
copy from dolphinscheduler-ui/src/views/projects/task/definition/types.ts
copy to dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskExecuteType.java
index c15c840355..4694cec26d 100644
--- a/dolphinscheduler-ui/src/views/projects/task/definition/types.ts
+++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskExecuteType.java
@@ -15,13 +15,35 @@
  * limitations under the License.
  */
 
-export type { ITaskData, INodeData } from '../components/node/types'
-export type { ISingleSaveReq } from '@/service/modules/task-definition/types'
+package org.apache.dolphinscheduler.common.enums;
 
-interface IRecord {
-  processDefinitionCode: number
-  taskCode: number
-  taskName: string
-}
+import com.baomidou.mybatisplus.annotation.EnumValue;
+
+/**
+ * task execute type
+ */
+public enum TaskExecuteType {
+    /**
+     * 0 batch
+     * 1 stream
+     */
+    BATCH(0, "batch"),
+    STREAM(1, "stream");
+
+    TaskExecuteType(int code, String desc) {
+        this.code = code;
+        this.desc = desc;
+    }
 
-export { IRecord }
+    @EnumValue
+    private final int code;
+    private final String desc;
+
+    public int getCode() {
+        return code;
+    }
+
+    public String getDesc() {
+        return desc;
+    }
+}
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java
index b5f975b4d2..227c78b63f 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java
+++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java
@@ -24,6 +24,7 @@ import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYP
 import com.fasterxml.jackson.core.type.TypeReference;
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.Priority;
+import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
 import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy;
 import org.apache.dolphinscheduler.plugin.task.api.parameters.TaskTimeoutParameter;
 import org.apache.dolphinscheduler.common.utils.CollectionUtils;
@@ -188,6 +189,11 @@ public class TaskNode {
      */
     private Integer memoryMax;
 
+    /**
+     * task execute type
+     */
+    private TaskExecuteType taskExecuteType;
+
     public String getId() {
         return id;
     }
@@ -273,8 +279,11 @@ public class TaskNode {
     }
 
     public boolean isForbidden() {
-        return (!StringUtils.isEmpty(this.runFlag)
-                && this.runFlag.equals(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN));
+        // skip stream task when run DAG
+        if (taskExecuteType == TaskExecuteType.STREAM) {
+            return true;
+        }
+        return StringUtils.isNotEmpty(this.runFlag) && this.runFlag.equals(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN);
     }
 
     @Override
@@ -297,7 +306,8 @@ public class TaskNode {
                 && Objects.equals(workerGroup, taskNode.workerGroup)
                 && Objects.equals(environmentCode, taskNode.environmentCode)
                 && Objects.equals(conditionResult, taskNode.conditionResult)
-                && CollectionUtils.equalLists(depList, taskNode.depList);
+                && CollectionUtils.equalLists(depList, taskNode.depList)
+                && Objects.equals(taskExecuteType, taskNode.taskExecuteType);
     }
 
     @Override
@@ -464,7 +474,8 @@ public class TaskNode {
                 + ", workerGroup='" + workerGroup + '\''
                 + ", environmentCode=" + environmentCode
                 + ", timeout='" + timeout + '\''
-                + ", delayTime=" + delayTime
+                + ", delayTime=" + delayTime + '\''
+                + ", taskExecuteType=" + taskExecuteType
                 + '}';
     }
 
@@ -523,4 +534,12 @@ public class TaskNode {
     public void setMemoryMax(Integer memoryMax) {
         this.memoryMax = memoryMax;
     }
+
+    public TaskExecuteType getTaskExecuteType() {
+        return taskExecuteType;
+    }
+
+    public void setTaskExecuteType(TaskExecuteType taskExecuteType) {
+        this.taskExecuteType = taskExecuteType;
+    }
 }
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/DateUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/DateUtils.java
index 81af098f8b..f883d5ca68 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/DateUtils.java
+++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/DateUtils.java
@@ -340,17 +340,20 @@ public final class DateUtils {
     }
 
     /**
-     * format time to duration
+     * format time to duration, if end date is null, use current time as end time
      *
-     * @param d1 d1
-     * @param d2 d2
+     * @param start start
+     * @param end end
      * @return format time
      */
-    public static String format2Duration(Date d1, Date d2) {
-        if (d1 == null || d2 == null) {
+    public static String format2Duration(Date start, Date end) {
+        if (start == null) {
             return null;
         }
-        return format2Duration(differMs(d1, d2));
+        if (end == null) {
+            end = new Date();
+        }
+        return format2Duration(differMs(start, end));
     }
 
     /**
@@ -366,6 +369,11 @@ public final class DateUtils {
         long minutes = MILLISECONDS.toDurationMinutes(ms);
         long seconds = MILLISECONDS.toDurationSeconds(ms);
 
+        if (days == 0 && hours == 0 && minutes == 0 && seconds == 0) {
+            // if duration is zero, set 1s
+            seconds = 1;
+        }
+
         StringBuilder strBuilder = new StringBuilder();
         strBuilder = days > 0 ? strBuilder.append(days).append("d").append(" ") : strBuilder;
         strBuilder = hours > 0 ? strBuilder.append(hours).append("h").append(" ") : strBuilder;
diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/DateUtilsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/DateUtilsTest.java
index 2042fa2d80..0233fcbd91 100644
--- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/DateUtilsTest.java
+++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/DateUtilsTest.java
@@ -171,47 +171,49 @@ public class DateUtilsTest {
     public void testFormat2Duration() {
 
         // days hours minutes seconds
-        Date d1 = DateUtils.stringToDate("2020-01-20 11:00:00");
-        Date d2 = DateUtils.stringToDate("2020-01-21 12:10:10");
-        String duration = DateUtils.format2Duration(d2, d1);
+        Date start = DateUtils.stringToDate("2020-01-20 11:00:00");
+        Date end = DateUtils.stringToDate("2020-01-21 12:10:10");
+        String duration = DateUtils.format2Duration(start, end);
         Assert.assertEquals("1d 1h 10m 10s", duration);
 
         // hours minutes seconds
-        d1 = DateUtils.stringToDate("2020-01-20 11:00:00");
-        d2 = DateUtils.stringToDate("2020-01-20 12:10:10");
-        duration = DateUtils.format2Duration(d2, d1);
+        start = DateUtils.stringToDate("2020-01-20 11:00:00");
+        end = DateUtils.stringToDate("2020-01-20 12:10:10");
+        duration = DateUtils.format2Duration(start, end);
         Assert.assertEquals("1h 10m 10s", duration);
 
         // minutes seconds
-        d1 = DateUtils.stringToDate("2020-01-20 11:00:00");
-        d2 = DateUtils.stringToDate("2020-01-20 11:10:10");
-        duration = DateUtils.format2Duration(d2, d1);
+        start = DateUtils.stringToDate("2020-01-20 11:00:00");
+        end = DateUtils.stringToDate("2020-01-20 11:10:10");
+        duration = DateUtils.format2Duration(start, end);
         Assert.assertEquals("10m 10s", duration);
 
         // minutes seconds
-        d1 = DateUtils.stringToDate("2020-01-20 11:10:00");
-        d2 = DateUtils.stringToDate("2020-01-20 11:10:10");
-        duration = DateUtils.format2Duration(d2, d1);
+        start = DateUtils.stringToDate("2020-01-20 11:10:00");
+        end = DateUtils.stringToDate("2020-01-20 11:10:10");
+        duration = DateUtils.format2Duration(start, end);
         Assert.assertEquals("10s", duration);
 
-        d1 = DateUtils.stringToDate("2020-01-20 11:10:00");
-        d2 = DateUtils.stringToDate("2020-01-21 11:10:10");
-        duration = DateUtils.format2Duration(d2, d1);
+        start = DateUtils.stringToDate("2020-01-20 11:10:00");
+        end = DateUtils.stringToDate("2020-01-21 11:10:10");
+        duration = DateUtils.format2Duration(start, end);
         Assert.assertEquals("1d 10s", duration);
 
-        d1 = DateUtils.stringToDate("2020-01-20 11:10:00");
-        d2 = DateUtils.stringToDate("2020-01-20 16:10:10");
-        duration = DateUtils.format2Duration(d2, d1);
+        start = DateUtils.stringToDate("2020-01-20 11:10:00");
+        end = DateUtils.stringToDate("2020-01-20 16:10:10");
+        duration = DateUtils.format2Duration(start, end);
         Assert.assertEquals("5h 10s", duration);
 
-    }
+        // startTime = endTime, default 1s
+        start = DateUtils.stringToDate("2020-01-20 11:10:00");
+        end = DateUtils.stringToDate("2020-01-20 11:10:00");
+        duration = DateUtils.format2Duration(start, end);
+        Assert.assertEquals("1s", duration);
 
-    @Test
-    public void testNullDuration() {
-        // days hours minutes seconds
-        Date d1 = DateUtils.stringToDate("2020-01-20 11:00:00");
-        Date d2 = null;
-        Assert.assertNull(DateUtils.format2Duration(d1, d2));
+        // endTime is null, use current time
+        start = DateUtils.stringToDate("2020-01-20 11:10:00");
+        duration = DateUtils.format2Duration(start, null);
+        Assert.assertNotNull(duration);
     }
 
     @Test
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinition.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinition.java
index 61bbb99f9b..2db49f9e07 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinition.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinition.java
@@ -20,6 +20,7 @@ package org.apache.dolphinscheduler.dao.entity;
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.Flag;
 import org.apache.dolphinscheduler.common.enums.Priority;
+import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
 import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy;
 import org.apache.dolphinscheduler.common.enums.TimeoutFlag;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
@@ -211,6 +212,11 @@ public class TaskDefinition {
      */
     private Integer memoryMax;
 
+    /**
+     * task execute type
+     */
+    private TaskExecuteType taskExecuteType;
+
     public TaskDefinition() {
     }
 
@@ -483,6 +489,14 @@ public class TaskDefinition {
         this.memoryMax = memoryMax;
     }
 
+    public TaskExecuteType getTaskExecuteType() {
+        return taskExecuteType;
+    }
+
+    public void setTaskExecuteType(TaskExecuteType taskExecuteType) {
+        this.taskExecuteType = taskExecuteType;
+    }
+
     @Override
     public boolean equals(Object o) {
         if (o == null) {
@@ -509,7 +523,8 @@ public class TaskDefinition {
             && taskGroupId == that.taskGroupId
             && taskGroupPriority == that.taskGroupPriority
             && Objects.equals(cpuQuota, that.cpuQuota)
-            && Objects.equals(memoryMax, that.memoryMax);
+            && Objects.equals(memoryMax, that.memoryMax)
+            && Objects.equals(taskExecuteType, that.taskExecuteType);
     }
 
     @Override
@@ -543,6 +558,7 @@ public class TaskDefinition {
                 + ", resourceIds='" + resourceIds + '\''
                 + ", cpuQuota=" + cpuQuota
                 + ", memoryMax=" + memoryMax
+                + ", taskExecuteType=" + taskExecuteType
                 + ", createTime=" + createTime
                 + ", updateTime=" + updateTime
                 + '}';
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 b52528b62b..91d82df089 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
@@ -27,6 +27,7 @@ import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYP
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.Flag;
 import org.apache.dolphinscheduler.common.enums.Priority;
+import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
 import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
@@ -89,6 +90,12 @@ public class TaskInstance implements Serializable {
     @TableField(exist = false)
     private String processInstanceName;
 
+    /**
+     * process definition name
+     */
+    @TableField(exist = false)
+    private String processDefinitionName;
+
     /**
      * process instance name
      */
@@ -288,6 +295,11 @@ public class TaskInstance implements Serializable {
      */
     private Integer memoryMax;
 
+    /**
+     * task execute type
+     */
+    private TaskExecuteType taskExecuteType;
+
     public void init(String host, Date startTime, String executePath) {
         this.host = host;
         this.startTime = startTime;
@@ -355,6 +367,11 @@ public class TaskInstance implements Serializable {
         return TASK_TYPE_BLOCKING.equalsIgnoreCase(this.taskType);
     }
 
+    public boolean isFirstRun() {
+        return endTime == null;
+    }
+
+
     /**
      * determine if a task instance can retry
      * if subProcess,
@@ -388,9 +405,4 @@ public class TaskInstance implements Serializable {
         // task retry does not over time, return false
         return getRetryInterval() * SEC_2_MINUTES_TIME_UNIT < failedTimeInterval;
     }
-
-    public boolean isFirstRun() {
-        return endTime == null;
-    }
-
 }
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.java
index c426da6b04..4940b4e1d2 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.java
@@ -17,6 +17,7 @@
 
 package org.apache.dolphinscheduler.dao.mapper;
 
+import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
 import org.apache.dolphinscheduler.dao.entity.DefinitionGroupByUser;
 import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
 import org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog;
@@ -113,13 +114,15 @@ public interface TaskDefinitionMapper extends BaseMapper<TaskDefinition> {
      * @param searchWorkflowName searchWorkflowName
      * @param searchTaskName searchTaskName
      * @param taskType taskType
+     * @param taskExecuteType taskExecuteType
      * @return task main info IPage
      */
     IPage<TaskMainInfo> queryDefineListPaging(IPage<TaskMainInfo> page,
                                               @Param("projectCode") long projectCode,
                                               @Param("searchWorkflowName") String searchWorkflowName,
                                               @Param("searchTaskName") String searchTaskName,
-                                              @Param("taskType") String taskType);
+                                              @Param("taskType") String taskType,
+                                              @Param("taskExecuteType")TaskExecuteType taskExecuteType);
 
     /**
      * query task definition by code list
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 17acff69d9..ca68dc57e5 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
@@ -20,6 +20,7 @@ package org.apache.dolphinscheduler.dao.mapper;
 import com.baomidou.mybatisplus.core.mapper.BaseMapper;
 import com.baomidou.mybatisplus.core.metadata.IPage;
 import org.apache.dolphinscheduler.common.enums.Flag;
+import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
 import org.apache.dolphinscheduler.dao.entity.ExecuteStatusCount;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
@@ -93,6 +94,20 @@ public interface TaskInstanceMapper extends BaseMapper<TaskInstance> {
                                                     @Param("executorId") int executorId,
                                                     @Param("states") int[] statusArray,
                                                     @Param("host") String host,
+                                                    @Param("taskExecuteType") TaskExecuteType taskExecuteType,
+                                                    @Param("startTime") Date startTime,
+                                                    @Param("endTime") Date endTime
+    );
+
+    IPage<TaskInstance> queryStreamTaskInstanceListPaging(IPage<TaskInstance> page,
+                                                    @Param("projectCode") Long projectCode,
+                                                    @Param("processDefinitionName") String processDefinitionName,
+                                                    @Param("searchVal") String searchVal,
+                                                    @Param("taskName") String taskName,
+                                                    @Param("executorId") int executorId,
+                                                    @Param("states") int[] statusArray,
+                                                    @Param("host") String host,
+                                                    @Param("taskExecuteType") TaskExecuteType taskExecuteType,
                                                     @Param("startTime") Date startTime,
                                                     @Param("endTime") Date endTime);
 
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskInstanceUtils.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskInstanceUtils.java
index fa3bfec0ca..498abcb3a4 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskInstanceUtils.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskInstanceUtils.java
@@ -74,6 +74,7 @@ public class TaskInstanceUtils {
         target.setTaskGroupId(source.getTaskGroupId());
         target.setCpuQuota(source.getCpuQuota());
         target.setMemoryMax(source.getMemoryMax());
+        target.setTaskExecuteType(source.getTaskExecuteType());
     }
 
 }
diff --git a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionLogMapper.xml b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionLogMapper.xml
index 16dc698d26..d914f9db08 100644
--- a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionLogMapper.xml
+++ b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionLogMapper.xml
@@ -21,7 +21,7 @@
     <sql id="baseSql">
         id, code, name, version, description, project_code, user_id, task_type, task_params, flag, task_priority,
         worker_group, environment_code, fail_retry_times, fail_retry_interval, timeout_flag, timeout_notify_strategy, timeout, delay_time,
-        resource_ids, operator, operate_time, create_time, update_time, task_group_id, task_group_priority, cpu_quota, memory_max
+        resource_ids, operator, operate_time, create_time, update_time, task_group_id, task_group_priority, cpu_quota, memory_max, task_execute_type
     </sql>
     <select id="queryMaxVersionForDefinition" resultType="java.lang.Integer">
         select max(version)
@@ -52,7 +52,7 @@
         insert into t_ds_task_definition_log (code, name, version, description, project_code, user_id,
         task_type, task_params, flag, task_priority, worker_group, environment_code, fail_retry_times, fail_retry_interval,
         timeout_flag, timeout_notify_strategy, timeout, delay_time, resource_ids, operator, operate_time, create_time,
-        update_time, task_group_id, task_group_priority, cpu_quota, memory_max)
+        update_time, task_group_id, task_group_priority, cpu_quota, memory_max, task_execute_type)
         values
         <foreach collection="taskDefinitionLogs" item="taskDefinitionLog" separator=",">
             (#{taskDefinitionLog.code},#{taskDefinitionLog.name},#{taskDefinitionLog.version},#{taskDefinitionLog.description},
@@ -61,7 +61,7 @@
             #{taskDefinitionLog.failRetryTimes},#{taskDefinitionLog.failRetryInterval},#{taskDefinitionLog.timeoutFlag},#{taskDefinitionLog.timeoutNotifyStrategy},
             #{taskDefinitionLog.timeout},#{taskDefinitionLog.delayTime},#{taskDefinitionLog.resourceIds},#{taskDefinitionLog.operator},#{taskDefinitionLog.operateTime},
             #{taskDefinitionLog.createTime},#{taskDefinitionLog.updateTime}, #{taskDefinitionLog.taskGroupId},#{taskDefinitionLog.taskGroupPriority},
-            #{taskDefinitionLog.cpuQuota},#{taskDefinitionLog.memoryMax})
+            #{taskDefinitionLog.cpuQuota},#{taskDefinitionLog.memoryMax},#{taskDefinitionLog.taskExecuteType})
         </foreach>
     </insert>
     <delete id="deleteByCodeAndVersion">
diff --git a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.xml b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.xml
index 4417e6e463..a8a09f7271 100644
--- a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.xml
+++ b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.xml
@@ -21,14 +21,14 @@
     <sql id="baseSql">
         id, code, name, version, description, project_code, user_id, task_type, task_params, flag, task_priority,
         worker_group, environment_code, fail_retry_times, fail_retry_interval, timeout_flag, timeout_notify_strategy, timeout, delay_time,
-        resource_ids, create_time, update_time, task_group_id,task_group_priority, cpu_quota, memory_max
+        resource_ids, create_time, update_time, task_group_id,task_group_priority, cpu_quota, memory_max, task_execute_type
     </sql>
     <sql id="baseSqlV2">
         ${alias}.id, ${alias}.code, ${alias}.name, ${alias}.version, ${alias}.description, ${alias}.project_code, ${alias}.user_id,
         ${alias}.task_type, ${alias}.task_params, ${alias}.flag, ${alias}.task_priority, ${alias}.worker_group, ${alias}.environment_code,
         ${alias}.fail_retry_times, ${alias}.fail_retry_interval, ${alias}.timeout_flag, ${alias}.timeout_notify_strategy, ${alias}.timeout,
         ${alias}.delay_time, ${alias}.resource_ids, ${alias}.create_time, ${alias}.update_time, ${alias}.task_group_id,
-        ${alias}.task_group_priority, ${alias}.cpu_quota, ${alias}.memory_max
+        ${alias}.task_group_priority, ${alias}.cpu_quota, ${alias}.memory_max, ${alias}.task_execute_type
     </sql>
     <select id="queryByName" resultType="org.apache.dolphinscheduler.dao.entity.TaskDefinition">
         select
@@ -89,14 +89,15 @@
     <insert id="batchInsert">
         insert into t_ds_task_definition (code, name, version, description, project_code, user_id,
         task_type, task_params, flag, task_priority, worker_group, environment_code, fail_retry_times, fail_retry_interval,
-        timeout_flag, timeout_notify_strategy, timeout, delay_time, resource_ids, create_time, update_time,task_group_id)
+        timeout_flag, timeout_notify_strategy, timeout, delay_time, resource_ids, create_time, update_time,task_group_id, task_execute_type)
         values
         <foreach collection="taskDefinitions" item="taskDefinition" separator=",">
             (#{taskDefinition.code},#{taskDefinition.name},#{taskDefinition.version},#{taskDefinition.description},
             #{taskDefinition.projectCode},#{taskDefinition.userId},#{taskDefinition.taskType},#{taskDefinition.taskParams},#{taskDefinition.flag},
             #{taskDefinition.taskPriority},#{taskDefinition.workerGroup},#{taskDefinition.environmentCode},#{taskDefinition.failRetryTimes},
             #{taskDefinition.failRetryInterval},#{taskDefinition.timeoutFlag},#{taskDefinition.timeoutNotifyStrategy},#{taskDefinition.timeout},
-            #{taskDefinition.delayTime},#{taskDefinition.resourceIds},#{taskDefinition.createTime},#{taskDefinition.updateTime}, #{taskDefinition.taskGroupId})
+            #{taskDefinition.delayTime},#{taskDefinition.resourceIds},#{taskDefinition.createTime},#{taskDefinition.updateTime}, #{taskDefinition.taskGroupId},
+            #{taskDefinition.taskExecuteType})
         </foreach>
     </insert>
     <select id="queryDefineListPaging" resultType="org.apache.dolphinscheduler.dao.entity.TaskMainInfo">
@@ -111,6 +112,9 @@
         <if test="taskType != ''">
             and td.task_type = #{taskType}
         </if>
+        <if test="taskExecuteType != null">
+            and td.task_execute_type = #{taskExecuteType.code}
+        </if>
         <if test="searchWorkflowName != null and searchWorkflowName != ''">
             and pd.name like concat('%', #{searchWorkflowName}, '%')
         </if>
diff --git a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml
index 26fcd4b0f0..e09b8aa887 100644
--- a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml
+++ b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml
@@ -22,13 +22,13 @@
         id, name, task_type, process_instance_id, task_code, task_definition_version, state, submit_time,
         start_time, end_time, host, execute_path, log_path, alert_flag, retry_times, pid, app_link,
         flag, retry_interval, max_retry_times, task_instance_priority, worker_group,environment_code , executor_id,
-        first_submit_time, delay_time, task_params, var_pool, dry_run, task_group_id, cpu_quota, memory_max
+        first_submit_time, delay_time, task_params, var_pool, dry_run, task_group_id, cpu_quota, memory_max, task_execute_type
     </sql>
     <sql id="baseSqlV2">
         ${alias}.id, ${alias}.name, ${alias}.task_type, ${alias}.task_code, ${alias}.task_definition_version, ${alias}.process_instance_id, ${alias}.state, ${alias}.submit_time,
         ${alias}.start_time, ${alias}.end_time, ${alias}.host, ${alias}.execute_path, ${alias}.log_path, ${alias}.alert_flag, ${alias}.retry_times, ${alias}.pid, ${alias}.app_link,
         ${alias}.flag, ${alias}.retry_interval, ${alias}.max_retry_times, ${alias}.task_instance_priority, ${alias}.worker_group,${alias}.environment_code , ${alias}.executor_id,
-        ${alias}.first_submit_time, ${alias}.delay_time, ${alias}.task_params, ${alias}.var_pool, ${alias}.dry_run, ${alias}.task_group_id
+        ${alias}.first_submit_time, ${alias}.delay_time, ${alias}.task_params, ${alias}.var_pool, ${alias}.dry_run, ${alias}.task_group_id, ${alias}.task_execute_type
     </sql>
     <update id="setFailoverByHostAndStateArray">
         update t_ds_task_instance
@@ -184,6 +184,9 @@
         <if test="host != null and host != ''">
             and instance.host like concat('%', #{host}, '%')
         </if>
+        <if test="taskExecuteType != null">
+            and instance.task_execute_type = #{taskExecuteType.code}
+        </if>
         <if test="executorId != 0">
             and instance.executor_id = #{executorId}
         </if>
@@ -192,6 +195,47 @@
         </if>
         order by instance.start_time desc
     </select>
+    <select id="queryStreamTaskInstanceListPaging" resultType="org.apache.dolphinscheduler.dao.entity.TaskInstance">
+        select
+        <include refid="baseSqlV2">
+            <property name="alias" value="instance"/>
+        </include>
+        ,
+        process.name as process_definition_name
+        from t_ds_task_instance instance
+        left join t_ds_task_definition_log define on define.code=instance.task_code and define.version=instance.task_definition_version
+        left join t_ds_process_task_relation relation on relation.post_task_code = define.code and relation.post_task_version = define.version
+        left join t_ds_process_definition process on process.code=relation.process_definition_code and process.version = relation.process_definition_version
+        where define.project_code = #{projectCode}
+        <if test="startTime != null">
+            and instance.start_time > #{startTime} and instance.start_time <![CDATA[ <=]]> #{endTime}
+        </if>
+        <if test="searchVal != null and searchVal != ''">
+            and instance.name like concat('%', #{searchVal}, '%')
+        </if>
+        <if test="taskName != null and taskName != ''">
+            and instance.name=#{taskName}
+        </if>
+        <if test="states != null and states.length != 0">
+            and instance.state in
+            <foreach collection="states" index="index" item="i" open="(" separator="," close=")">
+                #{i}
+            </foreach>
+        </if>
+        <if test="host != null and host != ''">
+            and instance.host like concat('%', #{host}, '%')
+        </if>
+        <if test="taskExecuteType != null">
+            and instance.task_execute_type = #{taskExecuteType.code}
+        </if>
+        <if test="executorId != 0">
+            and instance.executor_id = #{executorId}
+        </if>
+        <if test="processDefinitionName != null and processDefinitionName != ''">
+            and process.name like concat('%', #{processDefinitionName}, '%')
+        </if>
+        order by instance.start_time desc
+    </select>
     <select id="loadAllInfosNoRelease" resultType="org.apache.dolphinscheduler.dao.entity.TaskInstance">
         select
         <include refid="baseSqlV2">
diff --git a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_h2.sql b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_h2.sql
index 18e6efb61c..47bff2c0ef 100644
--- a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_h2.sql
+++ b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_h2.sql
@@ -473,6 +473,7 @@ CREATE TABLE t_ds_task_definition
     project_code            bigint(20) NOT NULL,
     user_id                 int(11) DEFAULT NULL,
     task_type               varchar(50) NOT NULL,
+    task_execute_type       int(11) DEFAULT '0',
     task_params             longtext,
     flag                    tinyint(2) DEFAULT NULL,
     task_priority           tinyint(4) DEFAULT NULL,
@@ -508,6 +509,7 @@ CREATE TABLE t_ds_task_definition_log
     project_code            bigint(20) NOT NULL,
     user_id                 int(11) DEFAULT NULL,
     task_type               varchar(50) NOT NULL,
+    task_execute_type       int(11) DEFAULT '0',
     task_params             text,
     flag                    tinyint(2) DEFAULT NULL,
     task_priority           tinyint(4) DEFAULT NULL,
@@ -839,6 +841,7 @@ CREATE TABLE t_ds_task_instance
     id                      int(11) NOT NULL AUTO_INCREMENT,
     name                    varchar(255) DEFAULT NULL,
     task_type               varchar(50) NOT NULL,
+    task_execute_type       int(11) DEFAULT '0',
     task_code               bigint(20) NOT NULL,
     task_definition_version int(11) DEFAULT NULL,
     process_instance_id     int(11) DEFAULT NULL,
@@ -869,8 +872,7 @@ CREATE TABLE t_ds_task_instance
     dry_run                 int NULL DEFAULT 0,
     cpu_quota               int(11) DEFAULT '-1' NOT NULL,
     memory_max              int(11) DEFAULT '-1' NOT NULL,
-    PRIMARY KEY (id),
-    FOREIGN KEY (process_instance_id) REFERENCES t_ds_process_instance (id) ON DELETE CASCADE
+    PRIMARY KEY (id)
 );
 
 -- ----------------------------
diff --git a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql
index 714d32a121..c2b676fb5c 100644
--- a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql
+++ b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql
@@ -473,6 +473,7 @@ CREATE TABLE `t_ds_task_definition` (
   `project_code` bigint(20) NOT NULL COMMENT 'project code',
   `user_id` int(11) DEFAULT NULL COMMENT 'task definition creator id',
   `task_type` varchar(50) NOT NULL COMMENT 'task type',
+  `task_execute_type` int(11) DEFAULT '0' COMMENT 'task execute type: 0-batch, 1-stream',
   `task_params` longtext COMMENT 'job custom parameters',
   `flag` tinyint(2) DEFAULT NULL COMMENT '0 not available, 1 available',
   `task_priority` tinyint(4) DEFAULT NULL COMMENT 'job priority',
@@ -507,6 +508,7 @@ CREATE TABLE `t_ds_task_definition_log` (
   `project_code` bigint(20) NOT NULL COMMENT 'project code',
   `user_id` int(11) DEFAULT NULL COMMENT 'task definition creator id',
   `task_type` varchar(50) NOT NULL COMMENT 'task type',
+  `task_execute_type` int(11) DEFAULT '0' COMMENT 'task execute type: 0-batch, 1-stream',
   `task_params` longtext COMMENT 'job custom parameters',
   `flag` tinyint(2) DEFAULT NULL COMMENT '0 not available, 1 available',
   `task_priority` tinyint(4) DEFAULT NULL COMMENT 'job priority',
@@ -834,6 +836,7 @@ CREATE TABLE `t_ds_task_instance` (
   `id` int(11) NOT NULL AUTO_INCREMENT COMMENT 'key',
   `name` varchar(255) DEFAULT NULL COMMENT 'task name',
   `task_type` varchar(50) NOT NULL COMMENT 'task type',
+  `task_execute_type` int(11) DEFAULT '0' COMMENT 'task execute type: 0-batch, 1-stream',
   `task_code` bigint(20) NOT NULL COMMENT 'task definition code',
   `task_definition_version` int(11) DEFAULT '0' COMMENT 'task definition version',
   `process_instance_id` int(11) DEFAULT NULL COMMENT 'process instance id',
@@ -866,8 +869,7 @@ CREATE TABLE `t_ds_task_instance` (
   `memory_max` int(11) DEFAULT '-1' NOT NULL COMMENT 'MemoryMax(MB): -1:Infinity',
   PRIMARY KEY (`id`),
   KEY `process_instance_id` (`process_instance_id`) USING BTREE,
-  KEY `idx_code_version` (`task_code`, `task_definition_version`) USING BTREE,
-  CONSTRAINT `foreign_key_instance_id` FOREIGN KEY (`process_instance_id`) REFERENCES `t_ds_process_instance` (`id`) ON DELETE CASCADE
+  KEY `idx_code_version` (`task_code`, `task_definition_version`) USING BTREE
 ) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8;
 
 -- ----------------------------
diff --git a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql
index 27b5e80d89..51f19fa8e9 100644
--- a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql
+++ b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql
@@ -390,6 +390,7 @@ CREATE TABLE t_ds_task_definition (
   project_code bigint DEFAULT NULL ,
   user_id int DEFAULT NULL ,
   task_type varchar(50) DEFAULT NULL ,
+  task_execute_type int DEFAULT '0',
   task_params text ,
   flag int DEFAULT NULL ,
   task_priority int DEFAULT NULL ,
@@ -427,6 +428,7 @@ CREATE TABLE t_ds_task_definition_log (
   project_code bigint DEFAULT NULL ,
   user_id int DEFAULT NULL ,
   task_type varchar(50) DEFAULT NULL ,
+  task_execute_type int(11) DEFAULT '0',
   task_params text ,
   flag int DEFAULT NULL ,
   task_priority int DEFAULT NULL ,
@@ -739,6 +741,7 @@ CREATE TABLE t_ds_task_instance (
   id int NOT NULL  ,
   name varchar(255) DEFAULT NULL ,
   task_type varchar(50) DEFAULT NULL ,
+  task_execute_type int DEFAULT '0',
   task_code bigint NOT NULL,
   task_definition_version int DEFAULT NULL ,
   process_instance_id int DEFAULT NULL ,
@@ -769,8 +772,7 @@ CREATE TABLE t_ds_task_instance (
   dry_run int DEFAULT '0' ,
   cpu_quota int DEFAULT '-1' NOT NULL,
   memory_max int DEFAULT '-1' NOT NULL,
-  PRIMARY KEY (id),
-  CONSTRAINT foreign_key_instance_id FOREIGN KEY(process_instance_id) REFERENCES t_ds_process_instance(id) ON DELETE CASCADE
+  PRIMARY KEY (id)
 ) ;
 
 create index idx_task_instance_code_version on t_ds_task_instance (task_code, task_definition_version);
diff --git a/dolphinscheduler-ui/src/views/projects/task/definition/types.ts b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.1.0_schema/mysql/dolphinscheduler_ddl.sql
similarity index 59%
copy from dolphinscheduler-ui/src/views/projects/task/definition/types.ts
copy to dolphinscheduler-dao/src/main/resources/sql/upgrade/3.1.0_schema/mysql/dolphinscheduler_ddl.sql
index c15c840355..db2ead4b76 100644
--- a/dolphinscheduler-ui/src/views/projects/task/definition/types.ts
+++ b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.1.0_schema/mysql/dolphinscheduler_ddl.sql
@@ -13,15 +13,9 @@
  * 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.
- */
+*/
 
-export type { ITaskData, INodeData } from '../components/node/types'
-export type { ISingleSaveReq } from '@/service/modules/task-definition/types'
-
-interface IRecord {
-  processDefinitionCode: number
-  taskCode: number
-  taskName: string
-}
-
-export { IRecord }
+ALTER TABLE `t_ds_task_definition` ADD COLUMN `task_execute_type` int(11) DEFAULT '0' COMMENT 'task execute type: 0-batch, 1-stream' AFTER `task_type`;
+ALTER TABLE `t_ds_task_definition_log` ADD COLUMN `task_execute_type` int(11) DEFAULT '0' COMMENT 'task execute type: 0-batch, 1-stream' AFTER `task_type`;
+ALTER TABLE `t_ds_task_instance` ADD COLUMN `task_execute_type` int(11) DEFAULT '0' COMMENT 'task execute type: 0-batch, 1-stream' AFTER `task_type`;
+ALTER TABLE `t_ds_task_instance` DROP FOREIGN KEY foreign_key_instance_id;
diff --git a/dolphinscheduler-ui/src/views/projects/task/definition/types.ts b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.1.0_schema/mysql/dolphinscheduler_dml.sql
similarity index 75%
copy from dolphinscheduler-ui/src/views/projects/task/definition/types.ts
copy to dolphinscheduler-dao/src/main/resources/sql/upgrade/3.1.0_schema/mysql/dolphinscheduler_dml.sql
index c15c840355..4a14f326b9 100644
--- a/dolphinscheduler-ui/src/views/projects/task/definition/types.ts
+++ b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.1.0_schema/mysql/dolphinscheduler_dml.sql
@@ -13,15 +13,4 @@
  * 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.
- */
-
-export type { ITaskData, INodeData } from '../components/node/types'
-export type { ISingleSaveReq } from '@/service/modules/task-definition/types'
-
-interface IRecord {
-  processDefinitionCode: number
-  taskCode: number
-  taskName: string
-}
-
-export { IRecord }
+*/
diff --git a/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.1.0_schema/postgresql/dolphinscheduler_ddl.sql b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.1.0_schema/postgresql/dolphinscheduler_ddl.sql
new file mode 100644
index 0000000000..c677ae58b3
--- /dev/null
+++ b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.1.0_schema/postgresql/dolphinscheduler_ddl.sql
@@ -0,0 +1,49 @@
+/*
+ * 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.
+*/
+
+delimiter d//
+CREATE OR REPLACE FUNCTION public.dolphin_update_metadata(
+    )
+    RETURNS character varying
+    LANGUAGE 'plpgsql'
+    COST 100
+    VOLATILE PARALLEL UNSAFE
+AS $BODY$
+DECLARE
+v_schema varchar;
+BEGIN
+    ---get schema name
+    v_schema =current_schema();
+
+--- add column
+EXECUTE 'ALTER TABLE ' || quote_ident(v_schema) ||'.t_ds_task_definition ADD COLUMN IF NOT EXISTS task_execute_type int DEFAULT ''0''  ';
+EXECUTE 'ALTER TABLE ' || quote_ident(v_schema) ||'.t_ds_task_definition_log ADD COLUMN IF NOT EXISTS task_execute_type int DEFAULT ''0''  ';
+EXECUTE 'ALTER TABLE ' || quote_ident(v_schema) ||'.t_ds_task_instance ADD COLUMN IF NOT EXISTS task_execute_type int DEFAULT ''0''  ';
+EXECUTE 'ALTER TABLE ' || quote_ident(v_schema) ||'.t_ds_task_instance ADD COLUMN IF NOT EXISTS task_execute_type int DEFAULT ''0''  ';
+EXECUTE 'ALTER TABLE ' || quote_ident(v_schema) ||'.t_ds_task_instance DROP CONSTRAINT foreign_key_instance_id';
+
+
+return 'Success!';
+exception when others then
+        ---Raise EXCEPTION '(%)',SQLERRM;
+        return SQLERRM;
+END;
+$BODY$;
+
+select dolphin_update_metadata();
+
+d//
\ No newline at end of file
diff --git a/dolphinscheduler-ui/src/views/projects/task/definition/types.ts b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.1.0_schema/postgresql/dolphinscheduler_dml.sql
similarity index 75%
copy from dolphinscheduler-ui/src/views/projects/task/definition/types.ts
copy to dolphinscheduler-dao/src/main/resources/sql/upgrade/3.1.0_schema/postgresql/dolphinscheduler_dml.sql
index c15c840355..4a14f326b9 100644
--- a/dolphinscheduler-ui/src/views/projects/task/definition/types.ts
+++ b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.1.0_schema/postgresql/dolphinscheduler_dml.sql
@@ -13,15 +13,4 @@
  * 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.
- */
-
-export type { ITaskData, INodeData } from '../components/node/types'
-export type { ISingleSaveReq } from '@/service/modules/task-definition/types'
-
-interface IRecord {
-  processDefinitionCode: number
-  taskCode: number
-  taskName: string
-}
-
-export { IRecord }
+*/
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 fe86cc7c0a..fa5a256caa 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,6 +18,7 @@
 package org.apache.dolphinscheduler.dao.mapper;
 
 import org.apache.dolphinscheduler.common.enums.Flag;
+import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
 import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
 import org.apache.dolphinscheduler.dao.BaseDaoTest;
 import org.apache.dolphinscheduler.dao.entity.ExecuteStatusCount;
@@ -189,7 +190,7 @@ public class TaskInstanceMapperTest extends BaseDaoTest {
         task2.setFlag(Flag.NO);
         taskInstanceMapper.updateById(task2);
         List<TaskInstance> taskInstances1 = taskInstanceMapper.findValidTaskListByProcessId(task.getProcessInstanceId(),
-                Flag.NO);
+            Flag.NO);
 
         taskInstanceMapper.deleteById(task2.getId());
         taskInstanceMapper.deleteById(task.getId());
@@ -355,16 +356,18 @@ public class TaskInstanceMapperTest extends BaseDaoTest {
 
         Page<TaskInstance> page = new Page(1, 3);
         IPage<TaskInstance> taskInstanceIPage = taskInstanceMapper.queryTaskInstanceListPaging(
-                page,
-                definition.getProjectCode(),
-                task.getProcessInstanceId(),
-                "",
-                "",
-                "",
-                0,
-                new int[0],
-                "",
-                null, null);
+            page,
+            definition.getProjectCode(),
+            task.getProcessInstanceId(),
+            "",
+            "",
+            "",
+            0,
+            new int[0],
+            "",
+            TaskExecuteType.BATCH,
+            null, null
+        );
         processInstanceMapper.deleteById(processInstance.getId());
         taskInstanceMapper.deleteById(task.getId());
         processDefinitionMapper.deleteById(definition.getId());
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/StreamTaskInstanceExecCacheManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/StreamTaskInstanceExecCacheManager.java
new file mode 100644
index 0000000000..681d5129bc
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/StreamTaskInstanceExecCacheManager.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.server.master.cache;
+
+import org.apache.dolphinscheduler.server.master.runner.StreamTaskExecuteRunnable;
+import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
+
+import java.util.Collection;
+
+import lombok.NonNull;
+
+/**
+ * cache of stream task instance
+ */
+public interface StreamTaskInstanceExecCacheManager {
+
+    StreamTaskExecuteRunnable getByTaskInstanceId(int taskInstanceId);
+
+    boolean contains(int taskInstanceId);
+
+    void removeByTaskInstanceId(int taskInstanceId);
+
+    /**
+     * cache
+     *
+     * @param taskInstanceId     taskInstanceId
+     * @param streamTaskExecuteRunnable if it is null, will not be cached
+     */
+    void cache(int taskInstanceId, @NonNull StreamTaskExecuteRunnable streamTaskExecuteRunnable);
+
+    /**
+     * get all streamTaskExecuteRunnable from cache
+     *
+     * @return all streamTaskExecuteRunnable in cache
+     */
+    Collection<StreamTaskExecuteRunnable> getAll();
+}
\ No newline at end of file
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/impl/StreamTaskInstanceExecCacheManagerImpl.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/impl/StreamTaskInstanceExecCacheManagerImpl.java
new file mode 100644
index 0000000000..1033ab35e5
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/impl/StreamTaskInstanceExecCacheManagerImpl.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.server.master.cache.impl;
+
+import org.apache.dolphinscheduler.server.master.cache.StreamTaskInstanceExecCacheManager;
+import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics;
+import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics;
+import org.apache.dolphinscheduler.server.master.runner.StreamTaskExecuteRunnable;
+import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
+
+import java.util.Collection;
+import java.util.concurrent.ConcurrentHashMap;
+
+import javax.annotation.PostConstruct;
+
+import org.springframework.stereotype.Component;
+
+import com.google.common.collect.ImmutableList;
+
+import lombok.NonNull;
+
+/**
+ * cache of process instance id and WorkflowExecuteThread
+ */
+@Component
+public class StreamTaskInstanceExecCacheManagerImpl implements StreamTaskInstanceExecCacheManager {
+
+    private final ConcurrentHashMap<Integer, StreamTaskExecuteRunnable> streamTaskInstanceExecMaps = new ConcurrentHashMap<>();
+
+    @PostConstruct
+    public void registerMetrics() {
+        TaskMetrics.registerTaskPrepared(streamTaskInstanceExecMaps::size);
+    }
+
+    @Override
+    public StreamTaskExecuteRunnable getByTaskInstanceId(int taskInstanceId) {
+        return streamTaskInstanceExecMaps.get(taskInstanceId);
+    }
+
+    @Override
+    public boolean contains(int taskInstanceId) {
+        return streamTaskInstanceExecMaps.containsKey(taskInstanceId);
+    }
+
+    @Override
+    public void removeByTaskInstanceId(int taskInstanceId) {
+        streamTaskInstanceExecMaps.remove(taskInstanceId);
+    }
+
+    @Override
+    public void cache(int taskInstanceId, @NonNull StreamTaskExecuteRunnable streamTaskExecuteRunnable) {
+        streamTaskInstanceExecMaps.put(taskInstanceId, streamTaskExecuteRunnable);
+    }
+
+    @Override
+    public Collection<StreamTaskExecuteRunnable> getAll() {
+        return ImmutableList.copyOf(streamTaskInstanceExecMaps.values());
+    }
+}
\ No newline at end of file
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 417d2ddec4..07e6ab546c 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
@@ -102,7 +102,7 @@ public class TaskStateEventHandler implements StateEventHandler {
         }
         switch (taskStateEvent.getStatus()) {
             case KILL:
-                TaskMetrics.incTaskInstanceByState("kill");
+                TaskMetrics.incTaskInstanceByState("stop");
                 break;
             case SUCCESS:
                 TaskMetrics.incTaskInstanceByState("success");
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/TaskMetrics.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/TaskMetrics.java
index 2b71343a45..6f9634fa5d 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/TaskMetrics.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/TaskMetrics.java
@@ -84,6 +84,9 @@ public class TaskMetrics {
     }
 
     public void incTaskInstanceByState(final String state) {
+        if (taskInstanceCounters.get(state) == null) {
+            return;
+        }
         taskInstanceCounters.get(state).increment();
     }
 
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteStartProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteStartProcessor.java
new file mode 100644
index 0000000000..7e1a32d490
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteStartProcessor.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.server.master.processor;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
+import org.apache.dolphinscheduler.remote.command.Command;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+import org.apache.dolphinscheduler.remote.command.TaskExecuteStartCommand;
+import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
+import org.apache.dolphinscheduler.server.master.cache.StreamTaskInstanceExecCacheManager;
+import org.apache.dolphinscheduler.server.master.runner.StreamTaskExecuteRunnable;
+import org.apache.dolphinscheduler.server.master.runner.StreamTaskExecuteThreadPool;
+import org.apache.dolphinscheduler.service.process.ProcessService;
+
+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;
+
+import org.apache.directory.api.util.Strings;
+
+/**
+ * task execute start processor, from api to master
+ */
+@Component
+public class TaskExecuteStartProcessor implements NettyRequestProcessor {
+
+    private final Logger logger = LoggerFactory.getLogger(TaskExecuteStartProcessor.class);
+
+    @Autowired
+    private StreamTaskExecuteThreadPool streamTaskExecuteThreadPool;
+
+    @Autowired
+    private ProcessService processService;
+
+    @Override
+    public void process(Channel channel, Command command) {
+        Preconditions.checkArgument(CommandType.TASK_EXECUTE_START == command.getType(), String.format("invalid command type : %s", command.getType()));
+        TaskExecuteStartCommand taskExecuteStartCommand = JSONUtils.parseObject(command.getBody(), TaskExecuteStartCommand.class);
+        logger.info("taskExecuteStartCommand: {}", taskExecuteStartCommand);
+
+        TaskDefinition taskDefinition = processService.findTaskDefinition(taskExecuteStartCommand.getTaskDefinitionCode(), taskExecuteStartCommand.getTaskDefinitionVersion());
+        if (taskDefinition == null) {
+            logger.error("Task definition can not be found, taskDefinitionCode:{}, taskDefinitionVersion:{}", taskExecuteStartCommand.getTaskDefinitionCode(), taskExecuteStartCommand.getTaskDefinitionVersion());
+            return;
+        }
+        streamTaskExecuteThreadPool.execute(new StreamTaskExecuteRunnable(taskDefinition, taskExecuteStartCommand));
+
+        // response
+        Command response = new Command(command.getOpaque());
+        response.setType(CommandType.TASK_EXECUTE_START);
+        response.setBody(Strings.EMPTY_BYTES);
+        channel.writeAndFlush(response);
+    }
+
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskExecuteThreadPool.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskExecuteThreadPool.java
index 8d5f2fd3db..80d2f12d04 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskExecuteThreadPool.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskExecuteThreadPool.java
@@ -19,6 +19,7 @@ package org.apache.dolphinscheduler.server.master.processor.queue;
 
 import org.apache.dolphinscheduler.common.enums.TaskEventType;
 import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
+import org.apache.dolphinscheduler.server.master.cache.StreamTaskInstanceExecCacheManager;
 import org.apache.dolphinscheduler.server.master.config.MasterConfig;
 import org.apache.dolphinscheduler.server.master.event.TaskEventHandler;
 
@@ -53,6 +54,9 @@ public class TaskExecuteThreadPool extends ThreadPoolTaskExecutor {
     @Autowired
     private List<TaskEventHandler> taskEventHandlerList;
 
+    @Autowired
+    private StreamTaskInstanceExecCacheManager streamTaskInstanceExecCacheManager;
+
     private Map<TaskEventType, TaskEventHandler> taskEventHandlerMap = new HashMap<>();
 
     /**
@@ -71,6 +75,11 @@ public class TaskExecuteThreadPool extends ThreadPoolTaskExecutor {
     }
 
     public void submitTaskEvent(TaskEvent taskEvent) {
+        // stream task event handle
+        if (taskEvent.getProcessInstanceId() == 0 && streamTaskInstanceExecCacheManager.contains(taskEvent.getTaskInstanceId())) {
+            streamTaskInstanceExecCacheManager.getByTaskInstanceId(taskEvent.getTaskInstanceId()).addTaskEvent(taskEvent);
+            return;
+        }
         if (!processInstanceExecCacheManager.contains(taskEvent.getProcessInstanceId())) {
             logger.warn("Cannot find workflowExecuteThread from cacheManager, event: {}", taskEvent);
             return;
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterRPCServer.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterRPCServer.java
index b6b86979de..c2a9de0446 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterRPCServer.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterRPCServer.java
@@ -27,6 +27,7 @@ import org.apache.dolphinscheduler.server.master.processor.StateEventProcessor;
 import org.apache.dolphinscheduler.server.master.processor.TaskEventProcessor;
 import org.apache.dolphinscheduler.server.master.processor.TaskExecuteResponseProcessor;
 import org.apache.dolphinscheduler.server.master.processor.TaskExecuteRunningProcessor;
+import org.apache.dolphinscheduler.server.master.processor.TaskExecuteStartProcessor;
 import org.apache.dolphinscheduler.server.master.processor.TaskKillResponseProcessor;
 import org.apache.dolphinscheduler.server.master.processor.TaskRecallProcessor;
 import org.apache.dolphinscheduler.server.master.processor.WorkflowExecutingDataRequestProcessor;
@@ -78,6 +79,9 @@ public class MasterRPCServer implements AutoCloseable {
     @Autowired
     private WorkflowExecutingDataRequestProcessor workflowExecutingDataRequestProcessor;
 
+    @Autowired
+    private TaskExecuteStartProcessor taskExecuteStartProcessor;
+
     @PostConstruct
     private void init() {
         // init remoting server
@@ -93,6 +97,7 @@ public class MasterRPCServer implements AutoCloseable {
         this.nettyRemotingServer.registerProcessor(CommandType.CACHE_EXPIRE, cacheProcessor);
         this.nettyRemotingServer.registerProcessor(CommandType.TASK_REJECT, taskRecallProcessor);
         this.nettyRemotingServer.registerProcessor(CommandType.WORKFLOW_EXECUTING_DATA_REQUEST, workflowExecutingDataRequestProcessor);
+        this.nettyRemotingServer.registerProcessor(CommandType.TASK_EXECUTE_START, taskExecuteStartProcessor);
 
         // logger server
         this.nettyRemotingServer.registerProcessor(CommandType.GET_LOG_BYTES_REQUEST, loggerRequestProcessor);
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/EventExecuteService.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/EventExecuteService.java
index 3ce7d4e240..24a23485ff 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/EventExecuteService.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/EventExecuteService.java
@@ -22,6 +22,7 @@ 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.server.master.cache.ProcessInstanceExecCacheManager;
+import org.apache.dolphinscheduler.server.master.cache.StreamTaskInstanceExecCacheManager;
 
 import java.util.concurrent.TimeUnit;
 
@@ -38,12 +39,15 @@ public class EventExecuteService extends BaseDaemonThread {
     @Autowired
     private ProcessInstanceExecCacheManager processInstanceExecCacheManager;
 
-    /**
-     * workflow exec service
-     */
+    @Autowired
+    private StreamTaskInstanceExecCacheManager streamTaskInstanceExecCacheManager;
+
     @Autowired
     private WorkflowExecuteThreadPool workflowExecuteThreadPool;
 
+    @Autowired
+    private StreamTaskExecuteThreadPool streamTaskExecuteThreadPool;
+
     protected EventExecuteService() {
         super("EventServiceStarted");
     }
@@ -59,7 +63,8 @@ public class EventExecuteService extends BaseDaemonThread {
     public void run() {
         while (Stopper.isRunning()) {
             try {
-                eventHandler();
+                workflowEventHandler();
+                streamTaskEventHandler();
                 TimeUnit.MILLISECONDS.sleep(Constants.SLEEP_TIME_MILLIS_SHORT);
             } catch (InterruptedException interruptedException) {
                 logger.warn("Master event service interrupted, will exit this loop", interruptedException);
@@ -71,7 +76,7 @@ public class EventExecuteService extends BaseDaemonThread {
         }
     }
 
-    private void eventHandler() {
+    private void workflowEventHandler() {
         for (WorkflowExecuteRunnable workflowExecuteThread : this.processInstanceExecCacheManager.getAll()) {
             try {
                 LoggerUtils.setWorkflowInstanceIdMDC(workflowExecuteThread.getProcessInstance().getId());
@@ -82,4 +87,16 @@ public class EventExecuteService extends BaseDaemonThread {
             }
         }
     }
+
+    private void streamTaskEventHandler() {
+        for (StreamTaskExecuteRunnable streamTaskExecuteRunnable : streamTaskInstanceExecCacheManager.getAll()) {
+            try {
+                LoggerUtils.setTaskInstanceIdMDC(streamTaskExecuteRunnable.getTaskInstance().getId());
+                streamTaskExecuteThreadPool.executeEvent(streamTaskExecuteRunnable);
+
+            } finally {
+                LoggerUtils.removeWorkflowInstanceIdMDC();
+            }
+        }
+    }
 }
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java
new file mode 100644
index 0000000000..77b499d557
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java
@@ -0,0 +1,481 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.server.master.runner;
+
+import static org.apache.dolphinscheduler.common.Constants.DEFAULT_WORKER_GROUP;
+
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.Flag;
+import org.apache.dolphinscheduler.common.enums.Priority;
+import org.apache.dolphinscheduler.common.thread.ThreadUtils;
+import org.apache.dolphinscheduler.common.utils.LoggerUtils;
+import org.apache.dolphinscheduler.dao.entity.Environment;
+import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
+import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelation;
+import org.apache.dolphinscheduler.dao.entity.Resource;
+import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.dao.entity.Tenant;
+import org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationMapper;
+import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
+import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
+import org.apache.dolphinscheduler.remote.command.TaskDispatchCommand;
+import org.apache.dolphinscheduler.remote.command.TaskExecuteRunningAckMessage;
+import org.apache.dolphinscheduler.remote.command.TaskExecuteStartCommand;
+import org.apache.dolphinscheduler.server.master.builder.TaskExecutionContextBuilder;
+import org.apache.dolphinscheduler.server.master.cache.StreamTaskInstanceExecCacheManager;
+import org.apache.dolphinscheduler.server.master.config.MasterConfig;
+import org.apache.dolphinscheduler.server.master.dispatch.ExecutorDispatcher;
+import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext;
+import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType;
+import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException;
+import org.apache.dolphinscheduler.server.master.event.StateEventHandleError;
+import org.apache.dolphinscheduler.server.master.event.StateEventHandleException;
+import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics;
+import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
+import org.apache.dolphinscheduler.service.task.TaskPluginManager;
+import org.apache.dolphinscheduler.spi.enums.ResourceType;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import lombok.NonNull;
+
+/**
+ * stream task execute
+ */
+public class StreamTaskExecuteRunnable implements Runnable {
+
+    private static final Logger logger = LoggerFactory.getLogger(StreamTaskExecuteRunnable.class);
+
+    protected MasterConfig masterConfig;
+
+    protected ProcessService processService;
+
+    protected ExecutorDispatcher dispatcher;
+
+    protected ProcessTaskRelationMapper processTaskRelationMapper;
+
+    protected TaskPluginManager taskPluginManager;
+
+    private StreamTaskInstanceExecCacheManager streamTaskInstanceExecCacheManager;
+
+    protected TaskDefinition taskDefinition;
+
+    protected TaskInstance taskInstance;
+
+    protected ProcessDefinition processDefinition;
+
+    protected TaskExecuteStartCommand taskExecuteStartCommand;
+
+    /**
+     * task event queue
+     */
+    private final ConcurrentLinkedQueue<TaskEvent> taskEvents = new ConcurrentLinkedQueue<>();
+
+    private TaskRunnableStatus taskRunnableStatus = TaskRunnableStatus.CREATED;
+
+    public StreamTaskExecuteRunnable(TaskDefinition taskDefinition, TaskExecuteStartCommand taskExecuteStartCommand) {
+        this.processService = SpringApplicationContext.getBean(ProcessService.class);
+        this.masterConfig = SpringApplicationContext.getBean(MasterConfig.class);
+        this.dispatcher = SpringApplicationContext.getBean(ExecutorDispatcher.class);
+        this.taskPluginManager = SpringApplicationContext.getBean(TaskPluginManager.class);
+        this.processTaskRelationMapper = SpringApplicationContext.getBean(ProcessTaskRelationMapper.class);
+        this.streamTaskInstanceExecCacheManager = SpringApplicationContext.getBean(StreamTaskInstanceExecCacheManager.class);
+        this.taskDefinition = taskDefinition;
+        this.taskExecuteStartCommand = taskExecuteStartCommand;
+    }
+
+    public TaskInstance getTaskInstance() {
+        return taskInstance;
+    }
+
+    @Override
+    public void run() {
+        // submit task
+        processService.updateTaskDefinitionResources(taskDefinition);
+        taskInstance = newTaskInstance(taskDefinition);
+        processService.saveTaskInstance(taskInstance);
+
+        // add cache
+        streamTaskInstanceExecCacheManager.cache(taskInstance.getId(), this);
+
+        List<ProcessTaskRelation> processTaskRelationList = processTaskRelationMapper.queryByTaskCode(taskDefinition.getCode());
+        long processDefinitionCode = processTaskRelationList.get(0).getProcessDefinitionCode();
+        int processDefinitionVersion = processTaskRelationList.get(0).getProcessDefinitionVersion();
+        processDefinition = processService.findProcessDefinition(processDefinitionCode, processDefinitionVersion);
+
+        // dispatch task
+        TaskExecutionContext taskExecutionContext = getTaskExecutionContext(taskInstance);
+        if (taskExecutionContext == null) {
+            taskInstance.setState(TaskExecutionStatus.FAILURE);
+            processService.saveTaskInstance(taskInstance);
+            return;
+        }
+
+        TaskDispatchCommand dispatchCommand = new TaskDispatchCommand(taskExecutionContext,
+            masterConfig.getMasterAddress(),
+            taskExecutionContext.getHost(),
+            System.currentTimeMillis());
+
+        ExecutionContext executionContext = new ExecutionContext(dispatchCommand.convert2Command(), ExecutorType.WORKER, taskExecutionContext.getWorkerGroup(), taskInstance);
+        Boolean dispatchSuccess = false;
+        try {
+            dispatchSuccess = dispatcher.dispatch(executionContext);
+        } catch (ExecuteException e) {
+            logger.error("Master dispatch task to worker error, taskInstanceId: {}, worker: {}",
+                taskInstance.getId(),
+                executionContext.getHost(),
+                e);
+        }
+        if (!dispatchSuccess) {
+            logger.info("Master failed to dispatch task to worker, taskInstanceId: {}, worker: {}",
+                taskInstance.getId(),
+                executionContext.getHost());
+
+            // set task instance fail
+            taskInstance.setState(TaskExecutionStatus.FAILURE);
+            processService.saveTaskInstance(taskInstance);
+            return;
+        }
+
+        // set started flag
+        taskRunnableStatus = TaskRunnableStatus.STARTED;
+
+        logger.info("Master success dispatch task to worker, taskInstanceId: {}, worker: {}",
+            taskInstance.getId(),
+            executionContext.getHost());
+    }
+
+    public boolean isStart() {
+        return TaskRunnableStatus.STARTED == taskRunnableStatus;
+    }
+
+    public boolean addTaskEvent(TaskEvent taskEvent) {
+        if (taskInstance.getId() != taskEvent.getTaskInstanceId()) {
+            logger.info("state event would be abounded, taskInstanceId:{}, eventType:{}, state:{}", taskEvent.getTaskInstanceId(), taskEvent.getEvent(), taskEvent.getState());
+            return false;
+        }
+        taskEvents.add(taskEvent);
+        return true;
+    }
+
+    public int eventSize() {
+        return this.taskEvents.size();
+    }
+
+    /**
+     * handle event
+     */
+    public void handleEvents() {
+        if (!isStart()) {
+            logger.info(
+                "The stream task instance is not started, will not handle its state event, current state event size: {}",
+                taskEvents.size());
+            return;
+        }
+        TaskEvent taskEvent = null;
+        while (!this.taskEvents.isEmpty()) {
+            try {
+                taskEvent = this.taskEvents.peek();
+                LoggerUtils.setTaskInstanceIdMDC(taskEvent.getTaskInstanceId());
+
+                logger.info("Begin to handle state event, {}", taskEvent);
+                if (this.handleTaskEvent(taskEvent)) {
+                    this.taskEvents.remove(taskEvent);
+                }
+            } catch (StateEventHandleError stateEventHandleError) {
+                logger.error("State event handle error, will remove this event: {}", taskEvent, stateEventHandleError);
+                this.taskEvents.remove(taskEvent);
+                ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS);
+            } catch (StateEventHandleException stateEventHandleException) {
+                logger.error("State event handle error, will retry this event: {}",
+                    taskEvent,
+                    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.
+                logger.error("State event handle error, get a unknown exception, will retry this event: {}",
+                    taskEvent,
+                    e);
+                ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS);
+            } finally {
+                LoggerUtils.removeWorkflowAndTaskInstanceIdMDC();
+            }
+        }
+    }
+
+    public TaskInstance newTaskInstance(TaskDefinition taskDefinition) {
+        TaskInstance taskInstance = new TaskInstance();
+        taskInstance.setTaskCode(taskDefinition.getCode());
+        taskInstance.setTaskDefinitionVersion(taskDefinition.getVersion());
+        taskInstance.setName(taskDefinition.getName());
+        // task instance state
+        taskInstance.setState(TaskExecutionStatus.SUBMITTED_SUCCESS);
+        // set process instance id to 0
+        taskInstance.setProcessInstanceId(0);
+        // task instance type
+        taskInstance.setTaskType(taskDefinition.getTaskType().toUpperCase());
+        // task instance whether alert
+        taskInstance.setAlertFlag(Flag.NO);
+
+        // task instance start time
+        taskInstance.setStartTime(null);
+
+        // task instance flag
+        taskInstance.setFlag(Flag.YES);
+
+        // task instance current retry times
+        taskInstance.setRetryTimes(0);
+        taskInstance.setMaxRetryTimes(taskDefinition.getFailRetryTimes());
+        taskInstance.setRetryInterval(taskDefinition.getFailRetryInterval());
+
+        //set task param
+        taskInstance.setTaskParams(taskDefinition.getTaskParams());
+
+        //set task group and priority
+        taskInstance.setTaskGroupId(taskDefinition.getTaskGroupId());
+        taskInstance.setTaskGroupPriority(taskDefinition.getTaskGroupPriority());
+
+        //set task cpu quota and max memory
+        taskInstance.setCpuQuota(taskDefinition.getCpuQuota());
+        taskInstance.setMemoryMax(taskDefinition.getMemoryMax());
+
+        // task instance priority
+        taskInstance.setTaskInstancePriority(Priority.MEDIUM);
+        if (taskDefinition.getTaskPriority() != null) {
+            taskInstance.setTaskInstancePriority(taskDefinition.getTaskPriority());
+        }
+
+        // delay execution time
+        taskInstance.setDelayTime(taskDefinition.getDelayTime());
+
+        // task dry run flag
+        taskInstance.setDryRun(taskExecuteStartCommand.getDryRun());
+
+        taskInstance.setWorkerGroup(StringUtils.isBlank(taskDefinition.getWorkerGroup()) ? DEFAULT_WORKER_GROUP : taskDefinition.getWorkerGroup());
+        taskInstance.setEnvironmentCode(taskDefinition.getEnvironmentCode() == 0 ? -1 : taskDefinition.getEnvironmentCode());
+
+        if (!taskInstance.getEnvironmentCode().equals(-1L)) {
+            Environment environment = processService.findEnvironmentByCode(taskInstance.getEnvironmentCode());
+            if (Objects.nonNull(environment) && StringUtils.isNotEmpty(environment.getConfig())) {
+                taskInstance.setEnvironmentConfig(environment.getConfig());
+            }
+        }
+
+        if (taskInstance.getSubmitTime() == null) {
+            taskInstance.setSubmitTime(new Date());
+        }
+        if (taskInstance.getFirstSubmitTime() == null) {
+            taskInstance.setFirstSubmitTime(taskInstance.getSubmitTime());
+        }
+
+        taskInstance.setTaskExecuteType(taskDefinition.getTaskExecuteType());
+        taskInstance.setExecutorId(taskExecuteStartCommand.getExecutorId());
+        taskInstance.setExecutorName(taskExecuteStartCommand.getExecutorName());
+
+        return taskInstance;
+    }
+
+    /**
+     * get TaskExecutionContext
+     *
+     * @param taskInstance taskInstance
+     * @return TaskExecutionContext
+     */
+    protected TaskExecutionContext getTaskExecutionContext(TaskInstance taskInstance) {
+        int userId = taskDefinition == null ? 0 : taskDefinition.getUserId();
+        Tenant tenant = processService.getTenantForProcess(processDefinition.getTenantId(), userId);
+
+        // verify tenant is null
+        if (tenant == null) {
+            logger.error("tenant not exists,task instance id : {}", taskInstance.getId());
+            return null;
+        }
+
+        taskInstance.setResources(getResourceFullNames(taskInstance));
+
+        TaskChannel taskChannel = taskPluginManager.getTaskChannel(taskInstance.getTaskType());
+        ResourceParametersHelper resources = taskChannel.getResources(taskInstance.getTaskParams());
+
+        AbstractParameters baseParam = taskPluginManager.getParameters(ParametersNode.builder().taskType(taskInstance.getTaskType()).taskParams(taskInstance.getTaskParams()).build());
+        Map<String, Property> propertyMap = paramParsingPreparation(taskInstance, baseParam);
+        TaskExecutionContext taskExecutionContext = TaskExecutionContextBuilder.get()
+            .buildTaskInstanceRelatedInfo(taskInstance)
+            .buildTaskDefinitionRelatedInfo(taskDefinition)
+            .buildResourceParametersInfo(resources)
+            .buildBusinessParamsMap(new HashMap<>())
+            .buildParamInfo(propertyMap)
+            .create();
+
+        taskExecutionContext.setTenantCode(tenant.getTenantCode());
+        taskExecutionContext.setProjectCode(processDefinition.getProjectCode());
+        taskExecutionContext.setProcessDefineCode(processDefinition.getCode());
+        taskExecutionContext.setProcessDefineVersion(processDefinition.getVersion());
+        // process instance id default 0
+        taskExecutionContext.setProcessInstanceId(0);
+
+        return taskExecutionContext;
+    }
+
+    /**
+     * get resource map key is full name and value is tenantCode
+     */
+    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());
+        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());
+                if (CollectionUtils.isNotEmpty(oldVersionResources)) {
+                    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
+                Stream<Integer> resourceIdStream = projectResourceFiles.stream().map(ResourceInfo::getId);
+                Set<Integer> resourceIdsSet = resourceIdStream.collect(Collectors.toSet());
+
+                if (CollectionUtils.isNotEmpty(resourceIdsSet)) {
+                    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)));
+                }
+            }
+        }
+
+        return resourcesMap;
+    }
+
+    protected boolean handleTaskEvent(TaskEvent taskEvent)  throws StateEventHandleException, StateEventHandleError {
+        measureTaskState(taskEvent);
+
+        if (taskInstance.getState() == null) {
+            throw new StateEventHandleError("Task state event handle error due to task state is null");
+        }
+
+        taskInstance.setStartTime(taskEvent.getStartTime());
+        taskInstance.setHost(taskEvent.getWorkerAddress());
+        taskInstance.setLogPath(taskEvent.getLogPath());
+        taskInstance.setExecutePath(taskEvent.getExecutePath());
+        taskInstance.setPid(taskEvent.getProcessId());
+        taskInstance.setAppLink(taskEvent.getAppIds());
+        taskInstance.setState(taskEvent.getState());
+        taskInstance.setEndTime(taskEvent.getEndTime());
+        taskInstance.setVarPool(taskEvent.getVarPool());
+        processService.changeOutParam(taskInstance);
+        processService.updateTaskInstance(taskInstance);
+
+        // send ack
+        sendAckToWorker(taskEvent);
+
+        if (taskInstance.getState().isFinished()) {
+            streamTaskInstanceExecCacheManager.removeByTaskInstanceId(taskInstance.getId());
+            logger.info("The stream task instance is finish, taskInstanceId:{}, state:{}", taskInstance.getId(), taskEvent.getState());
+        }
+
+        return true;
+    }
+
+    private void measureTaskState(TaskEvent taskEvent) {
+        if (taskEvent == null || taskEvent.getState() == null) {
+            // the event is broken
+            logger.warn("The task event is broken..., taskEvent: {}", taskEvent);
+            return;
+        }
+        if (taskEvent.getState().isFinished()) {
+            TaskMetrics.incTaskInstanceByState("finish");
+        }
+        switch (taskEvent.getState()) {
+            case KILL:
+                TaskMetrics.incTaskInstanceByState("stop");
+                break;
+            case SUCCESS:
+                TaskMetrics.incTaskInstanceByState("success");
+                break;
+            case FAILURE:
+                TaskMetrics.incTaskInstanceByState("fail");
+                break;
+            default:
+                break;
+        }
+    }
+
+    public Map<String, Property> paramParsingPreparation(@NonNull TaskInstance taskInstance, @NonNull AbstractParameters parameters) {
+        // assign value to definedParams here
+        Map<String,String> globalParamsMap = taskExecuteStartCommand.getStartParams();
+        Map<String, Property> globalParams = ParamUtils.getUserDefParamsMap(globalParamsMap);
+
+        // combining local and global parameters
+        Map<String, Property> localParams = parameters.getInputLocalParametersMap();
+
+        //stream pass params
+        parameters.setVarPool(taskInstance.getVarPool());
+        Map<String, Property> varParams = parameters.getVarPoolMap();
+
+        if (globalParams.isEmpty() && localParams.isEmpty() && varParams.isEmpty()) {
+            return null;
+        }
+
+        if (varParams.size() != 0) {
+            globalParams.putAll(varParams);
+        }
+        if (localParams.size() != 0) {
+            globalParams.putAll(localParams);
+        }
+
+        return globalParams;
+    }
+
+    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(true, taskEvent.getTaskInstanceId());
+        taskEvent.getChannel().writeAndFlush(taskExecuteRunningAckMessage.convert2Command());
+    }
+
+    private enum TaskRunnableStatus {
+        CREATED, STARTED,
+        ;
+    }
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteThreadPool.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteThreadPool.java
new file mode 100644
index 0000000000..ba845b7d15
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteThreadPool.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.server.master.runner;
+
+import org.apache.dolphinscheduler.common.utils.LoggerUtils;
+import org.apache.dolphinscheduler.remote.processor.StateEventCallbackService;
+import org.apache.dolphinscheduler.server.master.config.MasterConfig;
+import org.apache.dolphinscheduler.service.process.ProcessService;
+
+import javax.annotation.PostConstruct;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.scheduling.concurrent.ThreadPoolTaskExecutor;
+import org.springframework.stereotype.Component;
+import org.springframework.util.concurrent.ListenableFuture;
+import org.springframework.util.concurrent.ListenableFutureCallback;
+
+/**
+ * Used to execute {@link StreamTaskExecuteRunnable}.
+ */
+@Component
+public class StreamTaskExecuteThreadPool extends ThreadPoolTaskExecutor {
+
+    private static final Logger logger = LoggerFactory.getLogger(StreamTaskExecuteThreadPool.class);
+
+    @Autowired
+    private MasterConfig masterConfig;
+
+    @PostConstruct
+    private void init() {
+        this.setDaemon(true);
+        this.setThreadNamePrefix("StreamTaskExecuteThread-");
+        this.setMaxPoolSize(masterConfig.getExecThreads());
+        this.setCorePoolSize(masterConfig.getExecThreads());
+    }
+
+    /**
+     * Handle the events belong to the given stream task
+     */
+    public void executeEvent(final StreamTaskExecuteRunnable streamTaskExecuteRunnable) {
+        if (!streamTaskExecuteRunnable.isStart() || streamTaskExecuteRunnable.eventSize() == 0) {
+            return;
+        }
+        int taskInstanceId = streamTaskExecuteRunnable.getTaskInstance().getId();
+        ListenableFuture<?> future = this.submitListenable(streamTaskExecuteRunnable::handleEvents);
+        future.addCallback(new ListenableFutureCallback() {
+            @Override
+            public void onFailure(Throwable ex) {
+                LoggerUtils.setTaskInstanceIdMDC(taskInstanceId);
+                logger.error("Stream task instance events handle failed", ex);
+                LoggerUtils.removeTaskInstanceIdMDC();
+            }
+
+            @Override
+            public void onSuccess(Object result) {
+                LoggerUtils.setTaskInstanceIdMDC(taskInstanceId);
+                logger.info("Stream task instance is finished.");
+                LoggerUtils.removeTaskInstanceIdMDC();
+            }
+        });
+    }
+}
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 19b1ab5984..09a19f6d63 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,7 +31,15 @@ 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.*;
+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.TaskExecuteType;
+import org.apache.dolphinscheduler.common.enums.TaskGroupQueueStatus;
+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;
@@ -1173,6 +1181,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatue> {
         }
         // delay execution time
         taskInstance.setDelayTime(taskNode.getDelayTime());
+        taskInstance.setTaskExecuteType(taskNode.getTaskExecuteType());
         return taskInstance;
     }
 
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 d9f82a91dc..112e4916a3 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
@@ -325,7 +325,7 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
                 .create();
     }
 
-    private void setTaskResourceInfo(ResourceParametersHelper resourceParametersHelper) {
+    public void setTaskResourceInfo(ResourceParametersHelper resourceParametersHelper) {
         if (Objects.isNull(resourceParametersHelper)) {
             return;
         }
@@ -592,7 +592,7 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
     /**
      * get resource map key is full name and value is tenantCode
      */
-    protected Map<String, String> getResourceFullNames(TaskInstance taskInstance) {
+    public 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());
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/TaskProcessorFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/TaskProcessorFactory.java
index 4c732a0325..9595c46dd8 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/TaskProcessorFactory.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/TaskProcessorFactory.java
@@ -18,6 +18,7 @@
 package org.apache.dolphinscheduler.server.master.runner.task;
 
 import static org.apache.dolphinscheduler.common.Constants.COMMON_TASK_TYPE;
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE_STREAM;
 
 import org.apache.commons.lang3.StringUtils;
 
@@ -32,6 +33,9 @@ import org.slf4j.LoggerFactory;
 
 import lombok.experimental.UtilityClass;
 
+import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+
 /**
  * the factory to create task processor
  */
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java
index 040d82d974..6e26973a76 100644
--- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java
@@ -63,6 +63,11 @@ public enum CommandType {
     WORKER_REQUEST,
     MASTER_RESPONSE,
 
+    /**
+     * task execute start, from api to master
+     */
+    TASK_EXECUTE_START,
+
     /**
      * dispatch task request
      */
@@ -96,6 +101,16 @@ public enum CommandType {
 
     TASK_REJECT_ACK,
 
+    /**
+     * task savepoint, for stream task
+     */
+    TASK_SAVEPOINT_REQUEST,
+
+    /**
+     * task savepoint ack, for stream task
+     */
+    TASK_SAVEPOINT_RESPONSE,
+
     HEART_BEAT,
 
     PING,
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskExecuteStartCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskExecuteStartCommand.java
new file mode 100644
index 0000000000..80f63652eb
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskExecuteStartCommand.java
@@ -0,0 +1,77 @@
+/*
+ * 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.remote.command;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+
+import java.util.Map;
+
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import lombok.NoArgsConstructor;
+import lombok.ToString;
+
+/**
+ * Task execute start message, from api to master.
+ */
+@Data
+@NoArgsConstructor
+@ToString(callSuper = true)
+@EqualsAndHashCode(callSuper = true)
+public class TaskExecuteStartCommand extends BaseCommand {
+
+    private int executorId;
+
+    private String executorName;
+
+    private long projectCode;
+
+    private long taskDefinitionCode;
+
+    private int taskDefinitionVersion;
+
+    private int warningGroupId;
+
+    private String workerGroup;
+
+    private Long environmentCode;
+
+    private Map<String, String> startParams;
+
+    private int tenantId;
+
+    private int dryRun;
+
+    public TaskExecuteStartCommand(String messageSenderAddress, String messageReceiverAddress, long messageSendTime) {
+        super(messageSenderAddress, messageReceiverAddress, messageSendTime);
+    }
+
+    /**
+     * package request command
+     *
+     * @return command
+     */
+    public Command convert2Command() {
+        Command command = new Command();
+        command.setType(CommandType.TASK_EXECUTE_START);
+        byte[] body = JSONUtils.toJsonByteArray(this);
+        command.setBody(body);
+        return command;
+    }
+
+}
diff --git a/dolphinscheduler-ui/src/service/modules/task-instances/index.ts b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskSavePointRequestCommand.java
similarity index 50%
copy from dolphinscheduler-ui/src/service/modules/task-instances/index.ts
copy to dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskSavePointRequestCommand.java
index 94d8b1635c..8a1dfae6da 100644
--- a/dolphinscheduler-ui/src/service/modules/task-instances/index.ts
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskSavePointRequestCommand.java
@@ -15,28 +15,39 @@
  * limitations under the License.
  */
 
-import { axios } from '@/service/service'
-import utils from '@/utils'
-import { ProjectCodeReq, IdReq, TaskListReq } from './types'
+package org.apache.dolphinscheduler.remote.command;
 
-export function queryTaskListPaging(
-  params: TaskListReq,
-  projectCode: ProjectCodeReq
-): any {
-  return axios({
-    url: `/projects/${projectCode.projectCode}/task-instances`,
-    method: 'get',
-    params
-  })
-}
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
 
-export function forceSuccess(taskId: IdReq, projectCode: ProjectCodeReq): any {
-  return axios({
-    url: `/projects/${projectCode.projectCode}/task-instances/${taskId.id}/force-success`,
-    method: 'post'
-  })
-}
+import java.io.Serializable;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+/**
+ *  task savepoint request command
+ */
+@Data
+@NoArgsConstructor
+@AllArgsConstructor
+public class TaskSavePointRequestCommand implements Serializable {
+
+    /**
+     *  task id
+     */
+    private int taskInstanceId;
 
-export function downloadLog(id: number): void {
-  utils.downloadFile('log/download-log', { taskInstanceId: id })
+    /**
+     *  package request command
+     *
+     * @return command
+     */
+    public Command convert2Command() {
+        Command command = new Command();
+        command.setType(CommandType.TASK_SAVEPOINT_REQUEST);
+        byte[] body = JSONUtils.toJsonByteArray(this);
+        command.setBody(body);
+        return command;
+    }
 }
diff --git a/dolphinscheduler-ui/src/service/modules/task-instances/index.ts b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskSavePointResponseCommand.java
similarity index 50%
copy from dolphinscheduler-ui/src/service/modules/task-instances/index.ts
copy to dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskSavePointResponseCommand.java
index 94d8b1635c..c126a45055 100644
--- a/dolphinscheduler-ui/src/service/modules/task-instances/index.ts
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskSavePointResponseCommand.java
@@ -15,28 +15,39 @@
  * limitations under the License.
  */
 
-import { axios } from '@/service/service'
-import utils from '@/utils'
-import { ProjectCodeReq, IdReq, TaskListReq } from './types'
+package org.apache.dolphinscheduler.remote.command;
 
-export function queryTaskListPaging(
-  params: TaskListReq,
-  projectCode: ProjectCodeReq
-): any {
-  return axios({
-    url: `/projects/${projectCode.projectCode}/task-instances`,
-    method: 'get',
-    params
-  })
-}
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.NoArgsConstructor;
 
-export function forceSuccess(taskId: IdReq, projectCode: ProjectCodeReq): any {
-  return axios({
-    url: `/projects/${projectCode.projectCode}/task-instances/${taskId.id}/force-success`,
-    method: 'post'
-  })
-}
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+
+import java.io.Serializable;
+
+/**
+ * task savepoint response command
+ */
+@Data
+@NoArgsConstructor
+@AllArgsConstructor
+public class TaskSavePointResponseCommand implements Serializable {
+
+    /**
+     * taskInstanceId
+     */
+    private int taskInstanceId;
 
-export function downloadLog(id: number): void {
-  utils.downloadFile('log/download-log', { taskInstanceId: id })
+    /**
+     * package request command
+     *
+     * @return command
+     */
+    public Command convert2Command() {
+        Command command = new Command();
+        command.setType(CommandType.TASK_SAVEPOINT_RESPONSE);
+        byte[] body = JSONUtils.toJsonByteArray(this);
+        command.setBody(body);
+        return command;
+    }
 }
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/dto/TaskInstanceExecuteDto.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/dto/TaskInstanceExecuteDto.java
index d674195125..1779da3270 100644
--- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/dto/TaskInstanceExecuteDto.java
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/dto/TaskInstanceExecuteDto.java
@@ -24,6 +24,8 @@ import java.util.Date;
 import java.util.Map;
 
 import lombok.Data;
+
+import org.apache.dolphinscheduler.common.enums.TaskExecuteType;
 import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 
 @Data
@@ -106,4 +108,6 @@ public class TaskInstanceExecuteDto {
     private Integer cpuQuota;
 
     private Integer memoryMax;
+
+    private TaskExecuteType taskExecuteType;
 }
diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java
index 94618cbb42..f655d9b744 100644
--- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java
@@ -2849,6 +2849,7 @@ public class ProcessServiceImpl implements ProcessService {
                 taskNode.setTaskGroupPriority(taskDefinitionLog.getTaskGroupPriority());
                 taskNode.setCpuQuota(taskDefinitionLog.getCpuQuota());
                 taskNode.setMemoryMax(taskDefinitionLog.getMemoryMax());
+                taskNode.setTaskExecuteType(taskDefinitionLog.getTaskExecuteType());
                 taskNodeList.add(taskNode);
             }
         }
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-all/pom.xml b/dolphinscheduler-task-plugin/dolphinscheduler-task-all/pom.xml
index aebaa5e07f..84ad0169ac 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-all/pom.xml
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-all/pom.xml
@@ -58,6 +58,12 @@
             <version>${project.version}</version>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.dolphinscheduler</groupId>
+            <artifactId>dolphinscheduler-task-flink-stream</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
         <dependency>
             <groupId>org.apache.dolphinscheduler</groupId>
             <artifactId>dolphinscheduler-task-http</artifactId>
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractCommandExecutor.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractCommandExecutor.java
index aa169676ed..6209ee45c6 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractCommandExecutor.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractCommandExecutor.java
@@ -56,10 +56,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
  * abstract command executor
  */
 public abstract class AbstractCommandExecutor {
-    /**
-     * rules for extracting application ID
-     */
-    protected static final Pattern APPLICATION_REGEX = Pattern.compile(TaskConstants.APPLICATION_REGEX);
     
     /**
      * rules for extracting Var Pool
@@ -229,9 +225,6 @@ public abstract class AbstractCommandExecutor {
 
         // if SHELL task exit
         if (status) {
-            // set appIds
-            List<String> appIds = getAppIds(taskRequest.getLogPath());
-            result.setAppIds(String.join(TaskConstants.COMMA, appIds));
 
             // SHELL task state
             result.setExitStatusCode(process.exitValue());
@@ -397,38 +390,6 @@ public abstract class AbstractCommandExecutor {
         parseProcessOutputExecutorService.shutdown();
     }
 
-    /**
-     * get app links
-     *
-     * @param logPath log path
-     * @return app id list
-     */
-    private List<String> getAppIds(String logPath) {
-        List<String> appIds = new ArrayList<>();
-
-        File file = new File(logPath);
-        if (!file.exists()) {
-            return appIds;
-        }
-
-        /*
-         * analysis log?get submited yarn application id
-         */
-        try (BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(logPath), StandardCharsets.UTF_8))) {
-            String line;
-            while ((line = br.readLine()) != null) {
-                String appId = findAppId(line);
-                if (StringUtils.isNotEmpty(appId) && !appIds.contains(appId)) {
-                    logger.info("find app id: {}", appId);
-                    appIds.add(appId);
-                }
-            }
-        } catch (Exception e) {
-            logger.error(String.format("read file: %s failed : ", logPath), e);
-        }
-        return appIds;
-    }
-
     /**
      * find var pool
      * @param line
@@ -442,20 +403,6 @@ public abstract class AbstractCommandExecutor {
         return null;
     }
 
-    /**
-     * find app id
-     *
-     * @param line line
-     * @return appid
-     */
-    private String findAppId(String line) {
-        Matcher matcher = APPLICATION_REGEX.matcher(line);
-        if (matcher.find()) {
-            return matcher.group();
-        }
-        return null;
-    }
-
     /**
      * get remain time(s)
      *
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractTask.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractTask.java
index 70e9822ed9..80ee36b9d1 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractTask.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractTask.java
@@ -20,12 +20,31 @@ package org.apache.dolphinscheduler.plugin.task.api;
 import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
 import org.apache.dolphinscheduler.plugin.task.api.model.TaskAlertInfo;
 import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 /**
  * executive task
  */
 public abstract class AbstractTask {
 
+    /**
+     * rules for extracting application ID
+     */
+    protected static final Pattern YARN_APPLICATION_REGEX = Pattern.compile(TaskConstants.YARN_APPLICATION_REGEX);
+
     /**
      * varPool string
      */
@@ -52,7 +71,7 @@ public abstract class AbstractTask {
     protected String appIds;
 
     /**
-     * cancel
+     * cancel flag
      */
     protected volatile boolean cancel = false;
 
@@ -101,6 +120,48 @@ public abstract class AbstractTask {
         this.cancel = status;
     }
 
+    /**
+     * get application ids
+     * @return
+     * @throws IOException
+     */
+    public Set<String> getApplicationIds() throws IOException {
+        Set<String> appIds = new HashSet<>();
+
+        File file = new File(taskRequest.getLogPath());
+        if (!file.exists()) {
+            return appIds;
+        }
+
+        /*
+         * analysis log? get submitted yarn application id
+         */
+        try (BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(taskRequest.getLogPath()), StandardCharsets.UTF_8))) {
+            String line;
+            while ((line = br.readLine()) != null) {
+                String appId = findAppId(line);
+                if (StringUtils.isNotEmpty(appId)) {
+                    appIds.add(appId);
+                }
+            }
+        }
+        return appIds;
+    }
+
+    /**
+     * find app id
+     *
+     * @param line line
+     * @return appid
+     */
+    protected String findAppId(String line) {
+        Matcher matcher = YARN_APPLICATION_REGEX.matcher(line);
+        if (matcher.find()) {
+            return matcher.group();
+        }
+        return null;
+    }
+
     public void setVarPool(String varPool) {
         this.varPool = varPool;
     }
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractYarnTask.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractYarnTask.java
index aff6d253ab..4f1913ba3d 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractYarnTask.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractYarnTask.java
@@ -20,6 +20,10 @@ package org.apache.dolphinscheduler.plugin.task.api;
 import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
 import org.apache.dolphinscheduler.plugin.task.api.model.TaskResponse;
 
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
 /**
  * abstract yarn task
  */
@@ -29,6 +33,11 @@ public abstract class AbstractYarnTask extends AbstractTaskExecutor {
      */
     private ShellCommandExecutor shellCommandExecutor;
 
+    /**
+     * rules for extracting application ID
+     */
+    protected static final Pattern YARN_APPLICATION_REGEX = Pattern.compile(TaskConstants.YARN_APPLICATION_REGEX);
+
     /**
      * Abstract Yarn Task
      *
@@ -47,7 +56,8 @@ public abstract class AbstractYarnTask extends AbstractTaskExecutor {
             // SHELL task exit code
             TaskResponse response = shellCommandExecutor.run(buildCommand());
             setExitStatusCode(response.getExitStatusCode());
-            setAppIds(response.getAppIds());
+            // set appIds
+            setAppIds(String.join(TaskConstants.COMMA, getApplicationIds()));
             setProcessId(response.getProcessId());
         } catch (Exception e) {
             logger.error("yarn process failure", e);
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskConstants.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskConstants.java
index 56d2ca467f..dfea163536 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskConstants.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskConstants.java
@@ -27,7 +27,9 @@ public class TaskConstants {
         throw new IllegalStateException("Utility class");
     }
 
-    public static final String APPLICATION_REGEX = "application_\\d+_\\d+";
+    public static final String YARN_APPLICATION_REGEX = "application_\\d+_\\d+";
+
+    public static final String FLINK_APPLICATION_REGEX = "JobID \\w+";
 
     public static final String SETVALUE_REGEX = "[\\$#]\\{setValue\\(([^)]*)\\)}";
 
@@ -396,6 +398,8 @@ public class TaskConstants {
 
     public static final String TASK_TYPE_BLOCKING = "BLOCKING";
 
+    public static final String TASK_TYPE_STREAM = "STREAM";
+
     public static final List<String> COMPLEX_TASK_TYPES = Arrays.asList(new String[]{TASK_TYPE_CONDITIONS, TASK_TYPE_SWITCH, TASK_TYPE_SUB_PROCESS, TASK_TYPE_DEPENDENT});
 
     /**
diff --git a/dolphinscheduler-ui/src/views/projects/task/definition/types.ts b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/stream/StreamTask.java
similarity index 75%
copy from dolphinscheduler-ui/src/views/projects/task/definition/types.ts
copy to dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/stream/StreamTask.java
index c15c840355..d554278516 100644
--- a/dolphinscheduler-ui/src/views/projects/task/definition/types.ts
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/stream/StreamTask.java
@@ -15,13 +15,10 @@
  * limitations under the License.
  */
 
-export type { ITaskData, INodeData } from '../components/node/types'
-export type { ISingleSaveReq } from '@/service/modules/task-definition/types'
+package org.apache.dolphinscheduler.plugin.task.api.stream;
 
-interface IRecord {
-  processDefinitionCode: number
-  taskCode: number
-  taskName: string
-}
+import java.io.IOException;
 
-export { IRecord }
+public interface StreamTask {
+    public void savePoint() throws Exception;
+}
diff --git a/dolphinscheduler-ui/src/service/modules/task-instances/index.ts b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/stream/StreamTaskChannel.java
similarity index 52%
copy from dolphinscheduler-ui/src/service/modules/task-instances/index.ts
copy to dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/stream/StreamTaskChannel.java
index 94d8b1635c..a3bf9e5e23 100644
--- a/dolphinscheduler-ui/src/service/modules/task-instances/index.ts
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/stream/StreamTaskChannel.java
@@ -15,28 +15,18 @@
  * limitations under the License.
  */
 
-import { axios } from '@/service/service'
-import utils from '@/utils'
-import { ProjectCodeReq, IdReq, TaskListReq } from './types'
+package org.apache.dolphinscheduler.plugin.task.api.stream;
 
-export function queryTaskListPaging(
-  params: TaskListReq,
-  projectCode: ProjectCodeReq
-): any {
-  return axios({
-    url: `/projects/${projectCode.projectCode}/task-instances`,
-    method: 'get',
-    params
-  })
-}
+import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
+import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
 
-export function forceSuccess(taskId: IdReq, projectCode: ProjectCodeReq): any {
-  return axios({
-    url: `/projects/${projectCode.projectCode}/task-instances/${taskId.id}/force-success`,
-    method: 'post'
-  })
-}
+public interface StreamTaskChannel extends TaskChannel {
+
+    AbstractTask pauseTask(TaskExecutionContext taskExecutionContext);
 
-export function downloadLog(id: number): void {
-  utils.downloadFile('log/download-log', { taskInstanceId: id })
+    AbstractTask recoverTask(TaskExecutionContext taskExecutionContext);
 }
diff --git a/dolphinscheduler-ui/src/service/modules/task-instances/index.ts b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/test/java/org/apache/dolphinscheduler/plugin/task/api/AbstractTaskTest.java
similarity index 53%
copy from dolphinscheduler-ui/src/service/modules/task-instances/index.ts
copy to dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/test/java/org/apache/dolphinscheduler/plugin/task/api/AbstractTaskTest.java
index 94d8b1635c..0027698e4a 100644
--- a/dolphinscheduler-ui/src/service/modules/task-instances/index.ts
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/test/java/org/apache/dolphinscheduler/plugin/task/api/AbstractTaskTest.java
@@ -15,28 +15,26 @@
  * limitations under the License.
  */
 
-import { axios } from '@/service/service'
-import utils from '@/utils'
-import { ProjectCodeReq, IdReq, TaskListReq } from './types'
+package org.apache.dolphinscheduler.plugin.task.api;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
-export function queryTaskListPaging(
-  params: TaskListReq,
-  projectCode: ProjectCodeReq
-): any {
-  return axios({
-    url: `/projects/${projectCode.projectCode}/task-instances`,
-    method: 'get',
-    params
-  })
-}
+import org.junit.Assert;
+import org.junit.Test;
 
-export function forceSuccess(taskId: IdReq, projectCode: ProjectCodeReq): any {
-  return axios({
-    url: `/projects/${projectCode.projectCode}/task-instances/${taskId.id}/force-success`,
-    method: 'post'
-  })
-}
+public class AbstractTaskTest {
+
+    @Test
+    public void testFindFlinkJobId() {
+        String jobId = "cca7bc1061d61cf15238e92312c2fc20";
+        Pattern FLINK_APPLICATION_REGEX = Pattern.compile(TaskConstants.FLINK_APPLICATION_REGEX);
+        Matcher matcher = FLINK_APPLICATION_REGEX.matcher("Job has been submitted with JobID " + jobId);
+        String str = null;
+        if (matcher.find()) {
+            str = matcher.group();
+        }
+        Assert.assertNotNull(str);
+        Assert.assertEquals(jobId, str.substring(6));
+    }
 
-export function downloadLog(id: number): void {
-  utils.downloadFile('log/download-log', { taskInstanceId: id })
 }
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-chunjun/src/main/java/org/apache/dolphinscheduler/plugin/task/chunjun/ChunJunTask.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-chunjun/src/main/java/org/apache/dolphinscheduler/plugin/task/chunjun/ChunJunTask.java
index 92a9912eb1..a416d0c02c 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-chunjun/src/main/java/org/apache/dolphinscheduler/plugin/task/chunjun/ChunJunTask.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-chunjun/src/main/java/org/apache/dolphinscheduler/plugin/task/chunjun/ChunJunTask.java
@@ -22,6 +22,7 @@ import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.RWXR_XR_
 
 import org.apache.dolphinscheduler.plugin.task.api.AbstractTaskExecutor;
 import org.apache.dolphinscheduler.plugin.task.api.ShellCommandExecutor;
+import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
 import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
 import org.apache.dolphinscheduler.plugin.task.api.model.Property;
 import org.apache.dolphinscheduler.plugin.task.api.model.TaskResponse;
@@ -112,7 +113,7 @@ public class ChunJunTask extends AbstractTaskExecutor {
             TaskResponse commandExecuteResult = shellCommandExecutor.run(shellCommandFilePath);
 
             setExitStatusCode(commandExecuteResult.getExitStatusCode());
-            setAppIds(commandExecuteResult.getAppIds());
+            setAppIds(String.join(TaskConstants.COMMA, getApplicationIds()));
             setProcessId(commandExecuteResult.getProcessId());
         } catch (Exception e) {
             logger.error("chunjun task failed.", e);
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-datax/src/main/java/org/apache/dolphinscheduler/plugin/task/datax/DataxTask.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-datax/src/main/java/org/apache/dolphinscheduler/plugin/task/datax/DataxTask.java
index b8801becbd..43a6a7e696 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-datax/src/main/java/org/apache/dolphinscheduler/plugin/task/datax/DataxTask.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-datax/src/main/java/org/apache/dolphinscheduler/plugin/task/datax/DataxTask.java
@@ -25,6 +25,7 @@ import org.apache.dolphinscheduler.plugin.datasource.api.plugin.DataSourceClient
 import org.apache.dolphinscheduler.plugin.datasource.api.utils.DataSourceUtils;
 import org.apache.dolphinscheduler.plugin.task.api.AbstractTaskExecutor;
 import org.apache.dolphinscheduler.plugin.task.api.ShellCommandExecutor;
+import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
 import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
 import org.apache.dolphinscheduler.plugin.task.api.model.Property;
 import org.apache.dolphinscheduler.plugin.task.api.model.TaskResponse;
@@ -160,7 +161,7 @@ public class DataxTask extends AbstractTaskExecutor {
             TaskResponse commandExecuteResult = shellCommandExecutor.run(shellCommandFilePath);
 
             setExitStatusCode(commandExecuteResult.getExitStatusCode());
-            setAppIds(commandExecuteResult.getAppIds());
+            setAppIds(String.join(TaskConstants.COMMA, getApplicationIds()));
             setProcessId(commandExecuteResult.getProcessId());
         } catch (Exception e) {
             logger.error("datax task error", e);
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-dvc/src/main/java/org/apache/dolphinscheduler/plugin/task/dvc/DvcTask.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-dvc/src/main/java/org/apache/dolphinscheduler/plugin/task/dvc/DvcTask.java
index 24bd015dc9..8176999e4c 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-dvc/src/main/java/org/apache/dolphinscheduler/plugin/task/dvc/DvcTask.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-dvc/src/main/java/org/apache/dolphinscheduler/plugin/task/dvc/DvcTask.java
@@ -21,6 +21,7 @@ import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.EXIT_COD
 
 import org.apache.dolphinscheduler.plugin.task.api.AbstractTaskExecutor;
 import org.apache.dolphinscheduler.plugin.task.api.ShellCommandExecutor;
+import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
 import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
 import org.apache.dolphinscheduler.plugin.task.api.model.TaskResponse;
 import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
@@ -79,7 +80,7 @@ public class DvcTask extends AbstractTaskExecutor {
             String command = buildCommand();
             TaskResponse commandExecuteResult = shellCommandExecutor.run(command);
             setExitStatusCode(commandExecuteResult.getExitStatusCode());
-            setAppIds(commandExecuteResult.getAppIds());
+            setAppIds(String.join(TaskConstants.COMMA, getApplicationIds()));
             setProcessId(commandExecuteResult.getProcessId());
             parameters.dealOutParam(shellCommandExecutor.getVarPool());
         } catch (Exception e) {
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/pom.xml b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/pom.xml
new file mode 100644
index 0000000000..e2dd5753f8
--- /dev/null
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/pom.xml
@@ -0,0 +1,48 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>dolphinscheduler-task-plugin</artifactId>
+        <groupId>org.apache.dolphinscheduler</groupId>
+        <version>dev-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>dolphinscheduler-task-flink-stream</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.dolphinscheduler</groupId>
+            <artifactId>dolphinscheduler-spi</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.dolphinscheduler</groupId>
+            <artifactId>dolphinscheduler-task-api</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.dolphinscheduler</groupId>
+            <artifactId>dolphinscheduler-task-flink</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file
diff --git a/dolphinscheduler-ui/src/views/projects/task/definition/types.ts b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkStreamParameters.java
similarity index 75%
copy from dolphinscheduler-ui/src/views/projects/task/definition/types.ts
copy to dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkStreamParameters.java
index c15c840355..54bac1ba95 100644
--- a/dolphinscheduler-ui/src/views/projects/task/definition/types.ts
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkStreamParameters.java
@@ -15,13 +15,11 @@
  * limitations under the License.
  */
 
-export type { ITaskData, INodeData } from '../components/node/types'
-export type { ISingleSaveReq } from '@/service/modules/task-definition/types'
+package org.apache.dolphinscheduler.plugin.task.flink;
 
-interface IRecord {
-  processDefinitionCode: number
-  taskCode: number
-  taskName: string
-}
+/**
+ * flink stream parameters
+ */
+public class FlinkStreamParameters extends FlinkParameters {
 
-export { IRecord }
+}
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkTask.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkStreamTask.java
similarity index 57%
copy from dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkTask.java
copy to dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkStreamTask.java
index 5a78eed1bd..a6915ea182 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkTask.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkStreamTask.java
@@ -17,35 +17,36 @@
 
 package org.apache.dolphinscheduler.plugin.task.flink;
 
-import org.apache.dolphinscheduler.plugin.task.api.AbstractYarnTask;
+import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
 import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
-import org.apache.dolphinscheduler.plugin.task.api.model.Property;
 import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
 import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
-import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
 import org.apache.dolphinscheduler.plugin.task.api.parser.ParameterUtils;
-import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
+import org.apache.dolphinscheduler.plugin.task.api.stream.StreamTask;
 import org.apache.dolphinscheduler.spi.utils.JSONUtils;
-import org.apache.dolphinscheduler.spi.utils.StringUtils;
 
-import java.util.ArrayList;
-import java.util.HashMap;
+import org.apache.commons.collections4.CollectionUtils;
+
+import java.io.IOException;
+import java.util.Collections;
 import java.util.List;
-import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
-public class FlinkTask extends AbstractYarnTask {
+public class FlinkStreamTask extends FlinkTask implements StreamTask {
 
     /**
      * flink parameters
      */
-    private FlinkParameters flinkParameters;
+    private FlinkStreamParameters flinkParameters;
 
     /**
      * taskExecutionContext
      */
     private TaskExecutionContext taskExecutionContext;
 
-    public FlinkTask(TaskExecutionContext taskExecutionContext) {
+    public FlinkStreamTask(TaskExecutionContext taskExecutionContext) {
         super(taskExecutionContext);
         this.taskExecutionContext = taskExecutionContext;
     }
@@ -54,7 +55,7 @@ public class FlinkTask extends AbstractYarnTask {
     public void init() {
         logger.info("flink task params {}", taskExecutionContext.getTaskParams());
 
-        flinkParameters = JSONUtils.parseObject(taskExecutionContext.getTaskParams(), FlinkParameters.class);
+        flinkParameters = JSONUtils.parseObject(taskExecutionContext.getTaskParams(), FlinkStreamParameters.class);
 
         if (flinkParameters == null || !flinkParameters.checkParameters()) {
             throw new RuntimeException("flink task params is not valid");
@@ -73,7 +74,7 @@ public class FlinkTask extends AbstractYarnTask {
     @Override
     protected String buildCommand() {
         // flink run/run-application [OPTIONS] <jar-file> <arguments>
-        List<String> args = FlinkArgsUtils.buildCommandLine(taskExecutionContext, flinkParameters);
+        List<String> args = FlinkArgsUtils.buildRunCommandLine(taskExecutionContext, flinkParameters);
 
         String command = ParameterUtils
                 .convertParameterPlaceholders(String.join(" ", args), taskExecutionContext.getDefinedParams());
@@ -94,4 +95,39 @@ public class FlinkTask extends AbstractYarnTask {
     public AbstractParameters getParameters() {
         return flinkParameters;
     }
+
+    @Override
+    public void cancelApplication(boolean status) throws Exception {
+        Set<String> appIds = getApplicationIds();
+        if (CollectionUtils.isEmpty(appIds)) {
+            logger.error("can not get appId, taskInstanceId:{}", taskExecutionContext.getTaskInstanceId());
+            return;
+        }
+        taskExecutionContext.setAppIds(String.join(TaskConstants.COMMA, appIds));
+        List<String> args = FlinkArgsUtils.buildCancelCommandLine(taskExecutionContext);
+
+        logger.info("cancel application args:{}", args);
+
+        ProcessBuilder processBuilder = new ProcessBuilder();
+        processBuilder.command(args);
+        processBuilder.start();
+        super.cancelApplication(status);
+    }
+
+    @Override
+    public void savePoint() throws Exception {
+        Set<String> appIds = getApplicationIds();
+        if (CollectionUtils.isEmpty(appIds)) {
+            logger.warn("can not get appId, taskInstanceId:{}", taskExecutionContext.getTaskInstanceId());
+            return;
+        }
+
+        taskExecutionContext.setAppIds(String.join(TaskConstants.COMMA, appIds));
+        List<String> args = FlinkArgsUtils.buildSavePointCommandLine(taskExecutionContext);
+        logger.info("savepoint args:{}", args);
+
+        ProcessBuilder processBuilder = new ProcessBuilder();
+        processBuilder.command(args);
+        processBuilder.start();
+    }
 }
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkStreamTaskChannel.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkStreamTaskChannel.java
new file mode 100644
index 0000000000..5a86dbe3f2
--- /dev/null
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkStreamTaskChannel.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.plugin.task.flink;
+
+import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
+import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
+import org.apache.dolphinscheduler.plugin.task.api.stream.StreamTaskChannel;
+import org.apache.dolphinscheduler.spi.utils.JSONUtils;
+
+public class FlinkStreamTaskChannel implements StreamTaskChannel {
+    @Override
+    public void cancelApplication(boolean status) {
+
+    }
+
+    @Override
+    public FlinkStreamTask createTask(TaskExecutionContext taskRequest) {
+        return new FlinkStreamTask(taskRequest);
+    }
+
+    @Override
+    public AbstractParameters parseParameters(ParametersNode parametersNode) {
+        return JSONUtils.parseObject(parametersNode.getTaskParams(), FlinkStreamParameters.class);
+    }
+
+    @Override
+    public ResourceParametersHelper getResources(String parameters) {
+        return null;
+    }
+
+    @Override
+    public AbstractTask pauseTask(TaskExecutionContext taskExecutionContext) {
+        return null;
+    }
+
+    @Override
+    public AbstractTask recoverTask(TaskExecutionContext taskExecutionContext) {
+        return null;
+    }
+}
diff --git a/dolphinscheduler-ui/src/service/modules/task-instances/index.ts b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkStreamTaskChannelFactory.java
similarity index 53%
copy from dolphinscheduler-ui/src/service/modules/task-instances/index.ts
copy to dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkStreamTaskChannelFactory.java
index 94d8b1635c..f90a1e47b7 100644
--- a/dolphinscheduler-ui/src/service/modules/task-instances/index.ts
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkStreamTaskChannelFactory.java
@@ -15,28 +15,30 @@
  * limitations under the License.
  */
 
-import { axios } from '@/service/service'
-import utils from '@/utils'
-import { ProjectCodeReq, IdReq, TaskListReq } from './types'
+package org.apache.dolphinscheduler.plugin.task.flink;
 
-export function queryTaskListPaging(
-  params: TaskListReq,
-  projectCode: ProjectCodeReq
-): any {
-  return axios({
-    url: `/projects/${projectCode.projectCode}/task-instances`,
-    method: 'get',
-    params
-  })
-}
+import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
+import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
+import org.apache.dolphinscheduler.spi.params.base.PluginParams;
 
-export function forceSuccess(taskId: IdReq, projectCode: ProjectCodeReq): any {
-  return axios({
-    url: `/projects/${projectCode.projectCode}/task-instances/${taskId.id}/force-success`,
-    method: 'post'
-  })
-}
+import java.util.List;
+
+import com.google.auto.service.AutoService;
+
+@AutoService(TaskChannelFactory.class)
+public class FlinkStreamTaskChannelFactory implements TaskChannelFactory {
+    @Override
+    public TaskChannel create() {
+        return new FlinkStreamTaskChannel();
+    }
+
+    @Override
+    public String getName() {
+        return "FLINK_STREAM";
+    }
 
-export function downloadLog(id: number): void {
-  utils.downloadFile('log/download-log', { taskInstanceId: id })
+    @Override
+    public List<PluginParams> getParams() {
+        return null;
+    }
 }
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtilsTest.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtilsTest.java
similarity index 79%
copy from dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtilsTest.java
copy to dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtilsTest.java
index b87960fe96..cbd9852e2e 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtilsTest.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtilsTest.java
@@ -30,8 +30,8 @@ public class FlinkArgsUtilsTest {
         return String.join(" ", stringList);
     }
 
-    private FlinkParameters buildTestFlinkParametersWithDeployMode(FlinkDeployMode flinkDeployMode) {
-        FlinkParameters flinkParameters = new FlinkParameters();
+    private FlinkStreamParameters buildTestFlinkParametersWithDeployMode(FlinkDeployMode flinkDeployMode) {
+        FlinkStreamParameters flinkParameters = new FlinkStreamParameters();
         flinkParameters.setProgramType(ProgramType.SCALA);
         flinkParameters.setDeployMode(flinkDeployMode);
         flinkParameters.setParallelism(4);
@@ -57,8 +57,8 @@ public class FlinkArgsUtilsTest {
 
     @Test
     public void testRunJarInApplicationMode() throws Exception {
-        FlinkParameters flinkParameters = buildTestFlinkParametersWithDeployMode(FlinkDeployMode.APPLICATION);
-        List<String> commandLine = FlinkArgsUtils.buildCommandLine(buildTestTaskExecutionContext(), flinkParameters);
+        FlinkStreamParameters flinkParameters = buildTestFlinkParametersWithDeployMode(FlinkDeployMode.APPLICATION);
+        List<String> commandLine = FlinkArgsUtils.buildRunCommandLine(buildTestTaskExecutionContext(), flinkParameters);
 
         Assert.assertEquals(
                 "flink run-application -t yarn-application -ys 4 -ynm demo-app-name -yjm 1024m -ytm 1024m -p 4 -sae -c org.example.Main /opt/job.jar",
@@ -67,23 +67,23 @@ public class FlinkArgsUtilsTest {
 
     @Test
     public void testRunJarInClusterMode() throws Exception {
-        FlinkParameters flinkParameters = buildTestFlinkParametersWithDeployMode(FlinkDeployMode.CLUSTER);
+        FlinkStreamParameters flinkParameters = buildTestFlinkParametersWithDeployMode(FlinkDeployMode.CLUSTER);
         flinkParameters.setFlinkVersion("1.11");
-        List<String> commandLine1 = FlinkArgsUtils.buildCommandLine(buildTestTaskExecutionContext(), flinkParameters);
+        List<String> commandLine1 = FlinkArgsUtils.buildRunCommandLine(buildTestTaskExecutionContext(), flinkParameters);
 
         Assert.assertEquals(
                 "flink run -m yarn-cluster -ys 4 -ynm demo-app-name -yjm 1024m -ytm 1024m -p 4 -sae -c org.example.Main /opt/job.jar",
                 joinStringListWithSpace(commandLine1));
 
         flinkParameters.setFlinkVersion("<1.10");
-        List<String> commandLine2 = FlinkArgsUtils.buildCommandLine(buildTestTaskExecutionContext(), flinkParameters);
+        List<String> commandLine2 = FlinkArgsUtils.buildRunCommandLine(buildTestTaskExecutionContext(), flinkParameters);
 
         Assert.assertEquals(
                 "flink run -m yarn-cluster -ys 4 -ynm demo-app-name -yjm 1024m -ytm 1024m -p 4 -sae -c org.example.Main /opt/job.jar",
                 joinStringListWithSpace(commandLine2));
 
         flinkParameters.setFlinkVersion(">=1.12");
-        List<String> commandLine3 = FlinkArgsUtils.buildCommandLine(buildTestTaskExecutionContext(), flinkParameters);
+        List<String> commandLine3 = FlinkArgsUtils.buildRunCommandLine(buildTestTaskExecutionContext(), flinkParameters);
 
         Assert.assertEquals(
                 "flink run -t yarn-per-job -ys 4 -ynm demo-app-name -yjm 1024m -ytm 1024m -p 4 -sae -c org.example.Main /opt/job.jar",
@@ -92,8 +92,8 @@ public class FlinkArgsUtilsTest {
 
     @Test
     public void testRunJarInLocalMode() throws Exception {
-        FlinkParameters flinkParameters = buildTestFlinkParametersWithDeployMode(FlinkDeployMode.LOCAL);
-        List<String> commandLine = FlinkArgsUtils.buildCommandLine(buildTestTaskExecutionContext(), flinkParameters);
+        FlinkStreamParameters flinkParameters = buildTestFlinkParametersWithDeployMode(FlinkDeployMode.LOCAL);
+        List<String> commandLine = FlinkArgsUtils.buildRunCommandLine(buildTestTaskExecutionContext(), flinkParameters);
 
         Assert.assertEquals(
                 "flink run -p 4 -sae -c org.example.Main /opt/job.jar",
@@ -102,9 +102,9 @@ public class FlinkArgsUtilsTest {
 
     @Test
     public void testRunSql() throws Exception {
-        FlinkParameters flinkParameters = buildTestFlinkParametersWithDeployMode(FlinkDeployMode.CLUSTER);
+        FlinkStreamParameters flinkParameters = buildTestFlinkParametersWithDeployMode(FlinkDeployMode.CLUSTER);
         flinkParameters.setProgramType(ProgramType.SQL);
-        List<String> commandLine = FlinkArgsUtils.buildCommandLine(buildTestTaskExecutionContext(), flinkParameters);
+        List<String> commandLine = FlinkArgsUtils.buildRunCommandLine(buildTestTaskExecutionContext(), flinkParameters);
 
         Assert.assertEquals("sql-client.sh -i /tmp/execution/app-id_init.sql -f /tmp/execution/app-id_node.sql",
                 joinStringListWithSpace(commandLine));
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkParametersTest.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkParametersTest.java
new file mode 100644
index 0000000000..e998540316
--- /dev/null
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkParametersTest.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.plugin.task.flink;
+
+import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class FlinkParametersTest {
+    @Test
+    public void getResourceFilesList() {
+        FlinkStreamParameters flinkParameters = new FlinkStreamParameters();
+        Assert.assertTrue(flinkParameters.getResourceFilesList().isEmpty());
+
+        ResourceInfo mainResource = new ResourceInfo();
+        mainResource.setRes("testFlinkMain-1.0.0-SNAPSHOT.jar");
+        flinkParameters.setMainJar(mainResource);
+
+        List<ResourceInfo> resourceInfos = new LinkedList<>();
+        ResourceInfo resourceInfo1 = new ResourceInfo();
+        resourceInfo1.setRes("testFlinkParameters1.jar");
+        resourceInfos.add(resourceInfo1);
+
+        flinkParameters.setResourceList(resourceInfos);
+        List<ResourceInfo> resourceFilesList = flinkParameters.getResourceFilesList();
+        Assert.assertNotNull(resourceFilesList);
+        Assert.assertEquals(2, resourceFilesList.size());
+
+        ResourceInfo resourceInfo2 = new ResourceInfo();
+        resourceInfo2.setRes("testFlinkParameters2.jar");
+        resourceInfos.add(resourceInfo2);
+
+        flinkParameters.setResourceList(resourceInfos);
+        resourceFilesList = flinkParameters.getResourceFilesList();
+        Assert.assertNotNull(resourceFilesList);
+        Assert.assertEquals(3, resourceFilesList.size());
+    }
+}
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtils.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtils.java
index 53f94ea0d1..403a9bc632 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtils.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtils.java
@@ -51,26 +51,51 @@ public class FlinkArgsUtils {
     public static final FlinkDeployMode DEFAULT_DEPLOY_MODE = FlinkDeployMode.CLUSTER;
 
     /**
-     * build flink command line
+     * build flink run command line
      *
      * @param param flink parameters
      * @return argument list
      */
-    public static List<String> buildCommandLine(TaskExecutionContext taskExecutionContext, FlinkParameters param) {
+    public static List<String> buildRunCommandLine(TaskExecutionContext taskExecutionContext, FlinkParameters param) {
         switch (param.getProgramType()) {
             case SQL:
-                return buildCommandLineForSql(taskExecutionContext, param);
+                return buildRunCommandLineForSql(taskExecutionContext, param);
             default:
-                return buildCommandLineForOthers(taskExecutionContext, param);
+                return buildRunCommandLineForOthers(taskExecutionContext, param);
         }
     }
 
     /**
-     * build flink command line for SQL
+     * build flink cancel command line
+     * @param taskExecutionContext
+     * @return
+     */
+    public static List<String> buildCancelCommandLine(TaskExecutionContext taskExecutionContext) {
+        List<String> args = new ArrayList<>();
+        args.add(FlinkConstants.FLINK_COMMAND);
+        args.add(FlinkConstants.FLINK_CANCEL);
+        args.add(taskExecutionContext.getAppIds());
+        return args;
+    }
+
+    /**
+     * build flink savepoint command line, the savepoint folder should be set in flink conf
+     * @return
+     */
+    public static List<String> buildSavePointCommandLine(TaskExecutionContext taskExecutionContext) {
+        List<String> args = new ArrayList<>();
+        args.add(FlinkConstants.FLINK_COMMAND);
+        args.add(FlinkConstants.FLINK_SAVEPOINT);
+        args.add(taskExecutionContext.getAppIds());
+        return args;
+    }
+
+    /**
+     * build flink run command line for SQL
      *
      * @return argument list
      */
-    private static List<String> buildCommandLineForSql(TaskExecutionContext taskExecutionContext, FlinkParameters flinkParameters) {
+    private static List<String> buildRunCommandLineForSql(TaskExecutionContext taskExecutionContext, FlinkParameters flinkParameters) {
         List<String> args = new ArrayList<>();
 
         args.add(FlinkConstants.FLINK_SQL_COMMAND);
@@ -150,7 +175,7 @@ public class FlinkArgsUtils {
         return initOptions;
     }
 
-    private static List<String> buildCommandLineForOthers(TaskExecutionContext taskExecutionContext, FlinkParameters flinkParameters) {
+    private static List<String> buildRunCommandLineForOthers(TaskExecutionContext taskExecutionContext, FlinkParameters flinkParameters) {
         List<String> args = new ArrayList<>();
 
         args.add(FlinkConstants.FLINK_COMMAND);
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkConstants.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkConstants.java
index 91e4efc125..e448596370 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkConstants.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkConstants.java
@@ -55,6 +55,11 @@ public class FlinkConstants {
     public static final String FLINK_PARALLELISM = "-p";
     public static final String FLINK_SHUTDOWN_ON_ATTACHED_EXIT = "-sae";
     public static final String FLINK_PYTHON = "-py";
+    public static final String FLINK_SAVEPOINT = "savepoint";
+    public static final String FLINK_METRICS = "metrics";
+    public static final String FLINK_OVERVIEW = "overview";
+    public static final String FLINK_JOBS = "jobs";
+    public static final String FLINK_CANCEL = "cancel";
     // For Flink SQL
     public static final String FLINK_FORMAT_EXECUTION_TARGET = "set execution.target=%s";
     public static final String FLINK_FORMAT_YARN_APPLICATION_NAME = "set yarn.application.name=%s";
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkTask.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkTask.java
index 5a78eed1bd..b7dc41fe01 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkTask.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkTask.java
@@ -18,6 +18,7 @@
 package org.apache.dolphinscheduler.plugin.task.flink;
 
 import org.apache.dolphinscheduler.plugin.task.api.AbstractYarnTask;
+import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
 import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
 import org.apache.dolphinscheduler.plugin.task.api.model.Property;
 import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
@@ -28,10 +29,20 @@ import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
 import org.apache.dolphinscheduler.spi.utils.JSONUtils;
 import org.apache.dolphinscheduler.spi.utils.StringUtils;
 
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 public class FlinkTask extends AbstractYarnTask {
 
@@ -45,6 +56,11 @@ public class FlinkTask extends AbstractYarnTask {
      */
     private TaskExecutionContext taskExecutionContext;
 
+    /**
+     * rules for flink application ID
+     */
+    protected static final Pattern FLINK_APPLICATION_REGEX = Pattern.compile(TaskConstants.FLINK_APPLICATION_REGEX);
+
     public FlinkTask(TaskExecutionContext taskExecutionContext) {
         super(taskExecutionContext);
         this.taskExecutionContext = taskExecutionContext;
@@ -73,7 +89,7 @@ public class FlinkTask extends AbstractYarnTask {
     @Override
     protected String buildCommand() {
         // flink run/run-application [OPTIONS] <jar-file> <arguments>
-        List<String> args = FlinkArgsUtils.buildCommandLine(taskExecutionContext, flinkParameters);
+        List<String> args = FlinkArgsUtils.buildRunCommandLine(taskExecutionContext, flinkParameters);
 
         String command = ParameterUtils
                 .convertParameterPlaceholders(String.join(" ", args), taskExecutionContext.getDefinedParams());
@@ -94,4 +110,43 @@ public class FlinkTask extends AbstractYarnTask {
     public AbstractParameters getParameters() {
         return flinkParameters;
     }
+
+    @Override
+    public Set<String> getApplicationIds() throws IOException {
+        Set<String> appIds = new HashSet<>();
+
+        File file = new File(taskRequest.getLogPath());
+        if (!file.exists()) {
+            return appIds;
+        }
+
+        /*
+         * analysis log? get submitted yarn application id
+         */
+        try (BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(taskRequest.getLogPath()), StandardCharsets.UTF_8))) {
+            String line;
+            while ((line = br.readLine()) != null) {
+                String appId = findAppId(line);
+                if (StringUtils.isNotEmpty(appId)) {
+                    appIds.add(appId);
+                }
+            }
+        }
+        return appIds;
+    }
+
+    /**
+     * find app id
+     *
+     * @param line line
+     * @return appid
+     */
+    protected String findAppId(String line) {
+        Matcher matcher = FLINK_APPLICATION_REGEX.matcher(line);
+        if (matcher.find()) {
+            String str = matcher.group();
+            return str.substring(6);
+        }
+        return null;
+    }
 }
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtilsTest.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtilsTest.java
index b87960fe96..f71afb4d0e 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtilsTest.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtilsTest.java
@@ -58,7 +58,7 @@ public class FlinkArgsUtilsTest {
     @Test
     public void testRunJarInApplicationMode() throws Exception {
         FlinkParameters flinkParameters = buildTestFlinkParametersWithDeployMode(FlinkDeployMode.APPLICATION);
-        List<String> commandLine = FlinkArgsUtils.buildCommandLine(buildTestTaskExecutionContext(), flinkParameters);
+        List<String> commandLine = FlinkArgsUtils.buildRunCommandLine(buildTestTaskExecutionContext(), flinkParameters);
 
         Assert.assertEquals(
                 "flink run-application -t yarn-application -ys 4 -ynm demo-app-name -yjm 1024m -ytm 1024m -p 4 -sae -c org.example.Main /opt/job.jar",
@@ -69,21 +69,21 @@ public class FlinkArgsUtilsTest {
     public void testRunJarInClusterMode() throws Exception {
         FlinkParameters flinkParameters = buildTestFlinkParametersWithDeployMode(FlinkDeployMode.CLUSTER);
         flinkParameters.setFlinkVersion("1.11");
-        List<String> commandLine1 = FlinkArgsUtils.buildCommandLine(buildTestTaskExecutionContext(), flinkParameters);
+        List<String> commandLine1 = FlinkArgsUtils.buildRunCommandLine(buildTestTaskExecutionContext(), flinkParameters);
 
         Assert.assertEquals(
                 "flink run -m yarn-cluster -ys 4 -ynm demo-app-name -yjm 1024m -ytm 1024m -p 4 -sae -c org.example.Main /opt/job.jar",
                 joinStringListWithSpace(commandLine1));
 
         flinkParameters.setFlinkVersion("<1.10");
-        List<String> commandLine2 = FlinkArgsUtils.buildCommandLine(buildTestTaskExecutionContext(), flinkParameters);
+        List<String> commandLine2 = FlinkArgsUtils.buildRunCommandLine(buildTestTaskExecutionContext(), flinkParameters);
 
         Assert.assertEquals(
                 "flink run -m yarn-cluster -ys 4 -ynm demo-app-name -yjm 1024m -ytm 1024m -p 4 -sae -c org.example.Main /opt/job.jar",
                 joinStringListWithSpace(commandLine2));
 
         flinkParameters.setFlinkVersion(">=1.12");
-        List<String> commandLine3 = FlinkArgsUtils.buildCommandLine(buildTestTaskExecutionContext(), flinkParameters);
+        List<String> commandLine3 = FlinkArgsUtils.buildRunCommandLine(buildTestTaskExecutionContext(), flinkParameters);
 
         Assert.assertEquals(
                 "flink run -t yarn-per-job -ys 4 -ynm demo-app-name -yjm 1024m -ytm 1024m -p 4 -sae -c org.example.Main /opt/job.jar",
@@ -93,7 +93,7 @@ public class FlinkArgsUtilsTest {
     @Test
     public void testRunJarInLocalMode() throws Exception {
         FlinkParameters flinkParameters = buildTestFlinkParametersWithDeployMode(FlinkDeployMode.LOCAL);
-        List<String> commandLine = FlinkArgsUtils.buildCommandLine(buildTestTaskExecutionContext(), flinkParameters);
+        List<String> commandLine = FlinkArgsUtils.buildRunCommandLine(buildTestTaskExecutionContext(), flinkParameters);
 
         Assert.assertEquals(
                 "flink run -p 4 -sae -c org.example.Main /opt/job.jar",
@@ -104,7 +104,7 @@ public class FlinkArgsUtilsTest {
     public void testRunSql() throws Exception {
         FlinkParameters flinkParameters = buildTestFlinkParametersWithDeployMode(FlinkDeployMode.CLUSTER);
         flinkParameters.setProgramType(ProgramType.SQL);
-        List<String> commandLine = FlinkArgsUtils.buildCommandLine(buildTestTaskExecutionContext(), flinkParameters);
+        List<String> commandLine = FlinkArgsUtils.buildRunCommandLine(buildTestTaskExecutionContext(), flinkParameters);
 
         Assert.assertEquals("sql-client.sh -i /tmp/execution/app-id_init.sql -f /tmp/execution/app-id_node.sql",
                 joinStringListWithSpace(commandLine));
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-jupyter/src/main/java/org/apache/dolphinscheduler/plugin/task/jupyter/JupyterTask.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-jupyter/src/main/java/org/apache/dolphinscheduler/plugin/task/jupyter/JupyterTask.java
index ea3bf750c6..17f4946e19 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-jupyter/src/main/java/org/apache/dolphinscheduler/plugin/task/jupyter/JupyterTask.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-jupyter/src/main/java/org/apache/dolphinscheduler/plugin/task/jupyter/JupyterTask.java
@@ -83,7 +83,7 @@ public class JupyterTask extends AbstractTaskExecutor {
             // SHELL task exit code
             TaskResponse response = shellCommandExecutor.run(buildCommand());
             setExitStatusCode(response.getExitStatusCode());
-            setAppIds(response.getAppIds());
+            setAppIds(String.join(TaskConstants.COMMA, getApplicationIds()));
             setProcessId(response.getProcessId());
         } catch (Exception e) {
             logger.error("jupyter task execution failure", e);
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-mlflow/src/main/java/org/apache/dolphinscheduler/plugin/task/mlflow/MlflowTask.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-mlflow/src/main/java/org/apache/dolphinscheduler/plugin/task/mlflow/MlflowTask.java
index 0f3df4f357..6e49da322e 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-mlflow/src/main/java/org/apache/dolphinscheduler/plugin/task/mlflow/MlflowTask.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-mlflow/src/main/java/org/apache/dolphinscheduler/plugin/task/mlflow/MlflowTask.java
@@ -22,6 +22,7 @@ import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.EXIT_COD
 import org.apache.dolphinscheduler.common.thread.ThreadUtils;
 import org.apache.dolphinscheduler.plugin.task.api.AbstractTaskExecutor;
 import org.apache.dolphinscheduler.plugin.task.api.ShellCommandExecutor;
+import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
 import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
 import org.apache.dolphinscheduler.plugin.task.api.model.Property;
 import org.apache.dolphinscheduler.plugin.task.api.model.TaskResponse;
@@ -91,7 +92,7 @@ public class MlflowTask extends AbstractTaskExecutor {
                 exitCode = commandExecuteResult.getExitStatusCode();
             }
             setExitStatusCode(exitCode);
-            setAppIds(commandExecuteResult.getAppIds());
+            setAppIds(String.join(TaskConstants.COMMA, getApplicationIds()));
             setProcessId(commandExecuteResult.getProcessId());
             mlflowParameters.dealOutParam(shellCommandExecutor.getVarPool());
         } catch (Exception e) {
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-python/src/main/java/org/apache/dolphinscheduler/plugin/task/python/PythonTask.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-python/src/main/java/org/apache/dolphinscheduler/plugin/task/python/PythonTask.java
index f7f94736dc..75dae471e4 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-python/src/main/java/org/apache/dolphinscheduler/plugin/task/python/PythonTask.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-python/src/main/java/org/apache/dolphinscheduler/plugin/task/python/PythonTask.java
@@ -113,7 +113,7 @@ public class PythonTask extends AbstractTaskExecutor {
 
             TaskResponse taskResponse = shellCommandExecutor.run(command);
             setExitStatusCode(taskResponse.getExitStatusCode());
-            setAppIds(taskResponse.getAppIds());
+            setAppIds(String.join(TaskConstants.COMMA, getApplicationIds()));
             setProcessId(taskResponse.getProcessId());
             setVarPool(shellCommandExecutor.getVarPool());
         } catch (Exception e) {
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-seatunnel/src/main/java/org/apache/dolphinscheduler/plugin/task/seatunnel/SeatunnelTask.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-seatunnel/src/main/java/org/apache/dolphinscheduler/plugin/task/seatunnel/SeatunnelTask.java
index a077aa1c7c..b44bc6eeac 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-seatunnel/src/main/java/org/apache/dolphinscheduler/plugin/task/seatunnel/SeatunnelTask.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-seatunnel/src/main/java/org/apache/dolphinscheduler/plugin/task/seatunnel/SeatunnelTask.java
@@ -22,6 +22,7 @@ import static org.apache.dolphinscheduler.plugin.task.seatunnel.Constants.CONFIG
 
 import org.apache.dolphinscheduler.plugin.task.api.AbstractTaskExecutor;
 import org.apache.dolphinscheduler.plugin.task.api.ShellCommandExecutor;
+import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
 import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
 import org.apache.dolphinscheduler.plugin.task.api.model.Property;
 import org.apache.dolphinscheduler.plugin.task.api.model.TaskResponse;
@@ -90,7 +91,7 @@ public class SeatunnelTask extends AbstractTaskExecutor {
             String command = buildCommand();
             TaskResponse commandExecuteResult = shellCommandExecutor.run(command);
             setExitStatusCode(commandExecuteResult.getExitStatusCode());
-            setAppIds(commandExecuteResult.getAppIds());
+            setAppIds(String.join(TaskConstants.COMMA, getApplicationIds()));
             setProcessId(commandExecuteResult.getProcessId());
             seatunnelParameters.dealOutParam(shellCommandExecutor.getVarPool());
         } catch (Exception e) {
diff --git a/dolphinscheduler-task-plugin/dolphinscheduler-task-shell/src/main/java/org/apache/dolphinscheduler/plugin/task/shell/ShellTask.java b/dolphinscheduler-task-plugin/dolphinscheduler-task-shell/src/main/java/org/apache/dolphinscheduler/plugin/task/shell/ShellTask.java
index b5529bc8eb..f333e617b5 100644
--- a/dolphinscheduler-task-plugin/dolphinscheduler-task-shell/src/main/java/org/apache/dolphinscheduler/plugin/task/shell/ShellTask.java
+++ b/dolphinscheduler-task-plugin/dolphinscheduler-task-shell/src/main/java/org/apache/dolphinscheduler/plugin/task/shell/ShellTask.java
@@ -22,6 +22,7 @@ import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.RWXR_XR_
 
 import org.apache.dolphinscheduler.plugin.task.api.AbstractTaskExecutor;
 import org.apache.dolphinscheduler.plugin.task.api.ShellCommandExecutor;
+import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
 import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
 import org.apache.dolphinscheduler.plugin.task.api.model.Property;
 import org.apache.dolphinscheduler.plugin.task.api.model.TaskResponse;
@@ -95,7 +96,7 @@ public class ShellTask extends AbstractTaskExecutor {
             String command = buildCommand();
             TaskResponse commandExecuteResult = shellCommandExecutor.run(command);
             setExitStatusCode(commandExecuteResult.getExitStatusCode());
-            setAppIds(commandExecuteResult.getAppIds());
+            setAppIds(String.join(TaskConstants.COMMA, getApplicationIds()));
             setProcessId(commandExecuteResult.getProcessId());
             shellParameters.dealOutParam(shellCommandExecutor.getVarPool());
         } catch (Exception e) {
diff --git a/dolphinscheduler-task-plugin/pom.xml b/dolphinscheduler-task-plugin/pom.xml
index 3764a730d9..a432fc3fd1 100644
--- a/dolphinscheduler-task-plugin/pom.xml
+++ b/dolphinscheduler-task-plugin/pom.xml
@@ -59,6 +59,7 @@
         <module>dolphinscheduler-task-dinky</module>
         <module>dolphinscheduler-task-sagemaker</module>
         <module>dolphinscheduler-task-chunjun</module>
+        <module>dolphinscheduler-task-flink-stream</module>
     </modules>
 
     <dependencyManagement>
diff --git a/dolphinscheduler-ui/src/common/common.ts b/dolphinscheduler-ui/src/common/common.ts
index 2f5ed4d709..1c2fb478ff 100644
--- a/dolphinscheduler-ui/src/common/common.ts
+++ b/dolphinscheduler-ui/src/common/common.ts
@@ -139,6 +139,34 @@ export const stateType = (t: any) => [
   }))
 ]
 
+/**
+ * Stream task state
+ */
+const streamTaskState = [
+  'SUBMITTED_SUCCESS',
+  'RUNNING_EXECUTION',
+  'FAILURE',
+  'STOP',
+  'KILL',
+  'SUCCESS'
+]
+
+/**
+ * Stream task State code table
+ */
+export const streamStateType = (t: any) => [
+  {
+    value: '',
+    label: `${t('project.workflow.all_status')}`
+  },
+  ...Object.entries(tasksState(t))
+    .filter(([key]) => streamTaskState.includes(key))
+    .map(([key, item]) => ({
+      value: key,
+      label: item.desc
+    }))
+]
+
 /**
  * Task status
  * @id id
diff --git a/dolphinscheduler-ui/src/locales/en_US/project.ts b/dolphinscheduler-ui/src/locales/en_US/project.ts
index 599366ff91..1c142bf1db 100644
--- a/dolphinscheduler-ui/src/locales/en_US/project.ts
+++ b/dolphinscheduler-ui/src/locales/en_US/project.ts
@@ -240,7 +240,27 @@ export default {
     forced_success: 'Forced Success',
     view_log: 'View Log',
     download_log: 'Download Log',
-    refresh: 'Refresh'
+    refresh: 'Refresh',
+    app_id: 'Application ID',
+    execute: 'Execute',
+    stop: 'Stop',
+    batch_task: 'Batch Task',
+    stream_task: 'Stream Task',
+    jump_tip: 'Jump, not open yet!',
+    savepoint: 'Savepoint',
+    success: 'Success',
+    none_send: 'None',
+    success_send: 'Success',
+    failure_send: 'Failure',
+    all_send: 'All',
+    set_parameters_before_starting: 'Please set the parameters before starting',
+    notification_strategy: 'Notification Strategy',
+    worker_group: 'Worker group',
+    environment_name: 'Environment Name',
+    alarm_group: 'Alarm group',
+    startup_parameter: 'Startup Parameter',
+    whether_dry_run: 'Whether Dry-Run',
+    please_choose: 'Please Choose'
   },
   dag: {
     create: 'Create Workflow',
diff --git a/dolphinscheduler-ui/src/locales/zh_CN/project.ts b/dolphinscheduler-ui/src/locales/zh_CN/project.ts
index 5658bb827e..f2857b6ef7 100644
--- a/dolphinscheduler-ui/src/locales/zh_CN/project.ts
+++ b/dolphinscheduler-ui/src/locales/zh_CN/project.ts
@@ -209,6 +209,7 @@ export default {
     offline: '已下线',
     task_name: '任务名称',
     task_type: '任务类型',
+    task_instance: '任务实例',
     create_task: '创建任务',
     workflow_instance: '工作流实例',
     workflow_name: '工作流名称',
@@ -240,7 +241,27 @@ export default {
     forced_success: '强制成功',
     view_log: '查看日志',
     download_log: '下载日志',
-    refresh: '刷新'
+    refresh: '刷新',
+    app_id: 'Application ID',
+    execute: '执行',
+    stop: '停止',
+    batch_task: '批量任务',
+    stream_task: '实时任务',
+    jump_tip: '跳转,暂未开放!',
+    savepoint: 'Savepoint',
+    success: '成功',
+    none_send: '都不发',
+    success_send: '成功发',
+    failure_send: '失败发',
+    all_send: '成功或失败都发',
+    set_parameters_before_starting: '启动前请先设置参数',
+    notification_strategy: '通知策略',
+    worker_group: 'Worker分组',
+    environment_name: '环境名称',
+    alarm_group: '告警组',
+    startup_parameter: '启动参数',
+    whether_dry_run: '是否空跑',
+    please_choose: '请选择'
   },
   dag: {
     create: '创建工作流',
diff --git a/dolphinscheduler-ui/src/service/modules/task-definition/index.ts b/dolphinscheduler-ui/src/service/modules/task-definition/index.ts
index 70429138d3..65856a93dd 100644
--- a/dolphinscheduler-ui/src/service/modules/task-definition/index.ts
+++ b/dolphinscheduler-ui/src/service/modules/task-definition/index.ts
@@ -25,7 +25,8 @@ import {
   TaskDefinitionJsonObjReq,
   ReleaseStateReq,
   VersionReq,
-  ISingleSaveReq
+  ISingleSaveReq,
+  TaskDefinitionReq
 } from './types'
 
 export function queryTaskDefinitionListPaging(
@@ -70,10 +71,10 @@ export function queryTaskDefinitionByCode(
   })
 }
 
-export function update(
-  data: TaskDefinitionJsonObjReq,
-  code: CodeReq,
-  projectCode: ProjectCodeReq
+export function updateTask(
+  projectCode: number,
+  code: number,
+  data: TaskDefinitionJsonObjReq
 ): any {
   return axios({
     url: `/projects/${projectCode}/task-definition/${code}`,
@@ -94,8 +95,8 @@ export function deleteTaskDefinition(
 
 export function releaseTaskDefinition(
   data: ReleaseStateReq,
-  code: CodeReq,
-  projectCode: ProjectCodeReq
+  code: number,
+  projectCode: number
 ): any {
   return axios({
     url: `/projects/${projectCode}/task-definition/${code}/release`,
@@ -157,3 +158,15 @@ export function updateWithUpstream(
     data
   })
 }
+
+export function startTaskDefinition(
+  projectCode: number,
+  code: number,
+  data: TaskDefinitionReq
+) {
+  return axios({
+    url: `projects/${projectCode}/executors/task-instance/${code}/start`,
+    method: 'post',
+    data
+  })
+}
diff --git a/dolphinscheduler-ui/src/service/modules/task-definition/types.ts b/dolphinscheduler-ui/src/service/modules/task-definition/types.ts
index 78482510a6..3f90b42762 100644
--- a/dolphinscheduler-ui/src/service/modules/task-definition/types.ts
+++ b/dolphinscheduler-ui/src/service/modules/task-definition/types.ts
@@ -31,6 +31,7 @@ interface ProjectCodeReq {
 interface TaskDefinitionListReq extends ListReq {
   taskType?: string
   userId?: number
+  taskExecuteType?: 'BATCH' | 'STREAM'
 }
 
 interface TaskDefinitionJsonReq {
@@ -43,6 +44,7 @@ interface CodeReq {
 
 interface TaskDefinitionJsonObjReq {
   taskDefinitionJsonObj: string
+  taskExecuteType?: string
 }
 
 interface ReleaseStateReq {
@@ -128,6 +130,16 @@ interface ISingleSaveReq {
   taskDefinitionJsonObj: string
 }
 
+interface TaskDefinitionReq {
+  version: number
+  warningType: string
+  warningGroupId: number
+  workerGroup?: string
+  environmentCode?: number
+  startParams?: string
+  dryRun?: number
+}
+
 export {
   PageReq,
   ListReq,
@@ -142,5 +154,6 @@ export {
   TaskDefinitionRes,
   TaskDefinitionVersionItem,
   TaskDefinitionVersionRes,
-  ISingleSaveReq
+  ISingleSaveReq,
+  TaskDefinitionReq
 }
diff --git a/dolphinscheduler-ui/src/service/modules/task-instances/index.ts b/dolphinscheduler-ui/src/service/modules/task-instances/index.ts
index 94d8b1635c..4de389f131 100644
--- a/dolphinscheduler-ui/src/service/modules/task-instances/index.ts
+++ b/dolphinscheduler-ui/src/service/modules/task-instances/index.ts
@@ -40,3 +40,17 @@ export function forceSuccess(taskId: IdReq, projectCode: ProjectCodeReq): any {
 export function downloadLog(id: number): void {
   utils.downloadFile('log/download-log', { taskInstanceId: id })
 }
+
+export function streamTaskStop(projectCode: number, taskId: number): any {
+  return axios({
+    url: `projects/${projectCode}/task-instances/${taskId}/stop`,
+    method: 'post'
+  })
+}
+
+export function savePoint(projectCode: number, taskId: number): any {
+  return axios({
+    url: `projects/${projectCode}/task-instances/${taskId}/savepoint`,
+    method: 'post'
+  })
+}
diff --git a/dolphinscheduler-ui/src/service/modules/task-instances/types.ts b/dolphinscheduler-ui/src/service/modules/task-instances/types.ts
index 4c5b67f50c..744894473e 100644
--- a/dolphinscheduler-ui/src/service/modules/task-instances/types.ts
+++ b/dolphinscheduler-ui/src/service/modules/task-instances/types.ts
@@ -31,10 +31,12 @@ interface TaskListReq {
   host?: string
   processInstanceId?: number
   processInstanceName?: string
+  processDefinitionName?: string
   searchVal?: string
   startDate?: string
   stateType?: string
   taskName?: string
+  taskExecuteType?: 'BATCH' | 'STREAM'
 }
 
 interface Dependency {
diff --git a/dolphinscheduler-ui/src/views/projects/task/components/node/fields/use-flink.ts b/dolphinscheduler-ui/src/views/projects/task/components/node/fields/use-flink.ts
index 7b61477bc5..89035f0441 100644
--- a/dolphinscheduler-ui/src/views/projects/task/components/node/fields/use-flink.ts
+++ b/dolphinscheduler-ui/src/views/projects/task/components/node/fields/use-flink.ts
@@ -39,9 +39,7 @@ export function useFlink(model: { [field: string]: any }): IJsonItem[] {
     model.flinkVersion === '<1.10' && model.deployMode !== 'local' ? 12 : 0
   )
 
-  const deployModeSpan = computed(() =>
-    model.deployMode !== 'local' ? 12 : 0
-  )
+  const deployModeSpan = computed(() => (model.deployMode !== 'local' ? 12 : 0))
 
   const appNameSpan = computed(() => (model.deployMode !== 'local' ? 24 : 0))
 
@@ -56,7 +54,7 @@ export function useFlink(model: { [field: string]: any }): IJsonItem[] {
           label: 'local',
           value: 'local'
         }
-      ];
+      ]
     } else {
       return [
         {
@@ -71,14 +69,17 @@ export function useFlink(model: { [field: string]: any }): IJsonItem[] {
           label: 'local',
           value: 'local'
         }
-      ];
+      ]
     }
   })
 
   watch(
     () => model.flinkVersion,
     () => {
-      if (model.flinkVersion === '<1.10' && model.deployMode === 'application') {
+      if (
+        model.flinkVersion === '<1.10' &&
+        model.deployMode === 'application'
+      ) {
         model.deployMode = 'cluster'
       }
     }
@@ -92,7 +93,7 @@ export function useFlink(model: { [field: string]: any }): IJsonItem[] {
     {
       type: 'select',
       field: 'programType',
-      span: 12,
+      span: 24,
       name: t('project.node.program_type'),
       options: PROGRAM_TYPES,
       props: {
diff --git a/dolphinscheduler-ui/src/views/projects/task/components/node/format-data.ts b/dolphinscheduler-ui/src/views/projects/task/components/node/format-data.ts
index bc39ce97b1..be2e1d0e17 100644
--- a/dolphinscheduler-ui/src/views/projects/task/components/node/format-data.ts
+++ b/dolphinscheduler-ui/src/views/projects/task/components/node/format-data.ts
@@ -36,9 +36,8 @@ export function formatParams(data: INodeData): {
     taskParams.processDefinitionCode = data.processDefinitionCode
   }
   if (
-    data.taskType === 'SPARK' ||
-    data.taskType === 'MR' ||
-    data.taskType === 'FLINK'
+    data.taskType &&
+    ['SPARK', 'MR', 'FLINK', 'FLINK_STREAM'].includes(data.taskType)
   ) {
     taskParams.programType = data.programType
     taskParams.mainClass = data.mainClass
@@ -60,7 +59,7 @@ export function formatParams(data: INodeData): {
     taskParams.executorCores = data.executorCores
   }
 
-  if (data.taskType === 'FLINK') {
+  if (data.taskType === 'FLINK' || data.taskType === 'FLINK_STREAM') {
     taskParams.flinkVersion = data.flinkVersion
     taskParams.jobManagerMemory = data.jobManagerMemory
     taskParams.taskManagerMemory = data.taskManagerMemory
@@ -456,7 +455,8 @@ export function formatParams(data: INodeData): {
       timeoutNotifyStrategy: data.timeoutFlag ? timeoutNotifyStrategy : '',
       workerGroup: data.workerGroup,
       cpuQuota: data.cpuQuota || -1,
-      memoryMax: data.memoryMax || -1
+      memoryMax: data.memoryMax || -1,
+      taskExecuteType: data.taskExecuteType
     }
   } as {
     processDefinitionCode: string
diff --git a/dolphinscheduler-ui/src/views/projects/task/components/node/tasks/index.ts b/dolphinscheduler-ui/src/views/projects/task/components/node/tasks/index.ts
index 81eaa8c023..f2b0d148bb 100644
--- a/dolphinscheduler-ui/src/views/projects/task/components/node/tasks/index.ts
+++ b/dolphinscheduler-ui/src/views/projects/task/components/node/tasks/index.ts
@@ -16,6 +16,7 @@
  */
 
 import { useFlink } from './use-flink'
+import { useFlinkStream } from './use-flink-stream'
 import { useShell } from './use-shell'
 import { useSubProcess } from './use-sub-process'
 import { usePigeon } from './use-pigeon'
@@ -70,5 +71,6 @@ export default {
   DVC: useDvc,
   DINKY: useDinky,
   SAGEMAKER: userSagemaker,
-  CHUNJUN: useChunjun
+  CHUNJUN: useChunjun,
+  FLINK_STREAM: useFlinkStream
 }
diff --git a/dolphinscheduler-ui/src/views/projects/task/components/node/tasks/use-flink-stream.ts b/dolphinscheduler-ui/src/views/projects/task/components/node/tasks/use-flink-stream.ts
new file mode 100644
index 0000000000..33883e46fa
--- /dev/null
+++ b/dolphinscheduler-ui/src/views/projects/task/components/node/tasks/use-flink-stream.ts
@@ -0,0 +1,88 @@
+/*
+ * 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.
+ */
+
+import { reactive } from 'vue'
+import * as Fields from '../fields/index'
+import type { IJsonItem, INodeData, ITaskData } from '../types'
+
+export function useFlinkStream({
+  projectCode,
+  from = 0,
+  readonly,
+  data
+}: {
+  projectCode: number
+  from?: number
+  readonly?: boolean
+  data?: ITaskData
+}) {
+  const model = reactive<INodeData>({
+    taskType: 'FLINK_STREAM',
+    name: '',
+    flag: 'YES',
+    description: '',
+    timeoutFlag: false,
+    localParams: [],
+    environmentCode: null,
+    failRetryInterval: 1,
+    failRetryTimes: 0,
+    workerGroup: 'default',
+    delayTime: 0,
+    timeout: 30,
+    programType: 'SCALA',
+    deployMode: 'cluster',
+    initScript: '',
+    rawScript: '',
+    flinkVersion: '<1.10',
+    jobManagerMemory: '1G',
+    taskManagerMemory: '2G',
+    slot: 1,
+    taskManager: 2,
+    parallelism: 1,
+    timeoutNotifyStrategy: ['WARN']
+  })
+
+  let extra: IJsonItem[] = []
+  if (from === 1) {
+    extra = [
+      Fields.useTaskType(model, readonly),
+      Fields.useProcessName({
+        model,
+        projectCode,
+        isCreate: !data?.id,
+        from,
+        processName: data?.processName
+      })
+    ]
+  }
+
+  return {
+    json: [
+      Fields.useName(from),
+      ...extra,
+      Fields.useRunFlag(),
+      Fields.useDescription(),
+      Fields.useTaskPriority(),
+      Fields.useWorkerGroup(),
+      Fields.useEnvironmentName(model, !data?.id),
+      Fields.useDelayTime(model),
+      ...Fields.useFlink(model),
+      Fields.usePreTasks()
+    ] as IJsonItem[],
+    model
+  }
+}
diff --git a/dolphinscheduler-ui/src/views/projects/task/components/node/types.ts b/dolphinscheduler-ui/src/views/projects/task/components/node/types.ts
index 912f4b2132..bda369f6fd 100644
--- a/dolphinscheduler-ui/src/views/projects/task/components/node/types.ts
+++ b/dolphinscheduler-ui/src/views/projects/task/components/node/types.ts
@@ -17,7 +17,10 @@
 
 import { VNode } from 'vue'
 import type { SelectOption } from 'naive-ui'
-import type { TaskType } from '@/views/projects/task/constants/task-type'
+import type {
+  TaskExecuteType,
+  TaskType
+} from '@/views/projects/task/constants/task-type'
 import type { IDataBase } from '@/service/modules/data-source/types'
 import type {
   IFormItem,
@@ -415,6 +418,7 @@ interface INodeData
   udfs?: string[]
   customConfig?: boolean
   mapping_columns?: object[]
+  taskExecuteType?: TaskExecuteType
 }
 
 interface ITaskData
diff --git a/dolphinscheduler-ui/src/views/projects/task/components/node/use-task.ts b/dolphinscheduler-ui/src/views/projects/task/components/node/use-task.ts
index 4a23a2a6a8..721fcc7818 100644
--- a/dolphinscheduler-ui/src/views/projects/task/components/node/use-task.ts
+++ b/dolphinscheduler-ui/src/views/projects/task/components/node/use-task.ts
@@ -18,6 +18,7 @@ import { ref, Ref, unref } from 'vue'
 import nodes from './tasks'
 import getElementByJson from '@/components/form/get-elements-by-json'
 import { useTaskNodeStore } from '@/store/project/task-node'
+import { TASK_TYPES_MAP } from '../../constants/task-type'
 import type {
   IFormItem,
   IJsonItem,
@@ -66,6 +67,8 @@ export function useTask({
   jsonRef.value = json
   model.preTasks = taskStore.getPreTasks
   model.name = taskStore.getName
+  model.taskExecuteType =
+    TASK_TYPES_MAP[data.taskType || 'SHELL'].taskExecuteType || 'BATCH'
 
   const getElements = () => {
     const { rules, elements } = getElementByJson(jsonRef.value, model)
diff --git a/dolphinscheduler-ui/src/views/projects/task/constants/task-type.ts b/dolphinscheduler-ui/src/views/projects/task/constants/task-type.ts
index 5f2e4c95f4..3b42fc6fed 100644
--- a/dolphinscheduler-ui/src/views/projects/task/constants/task-type.ts
+++ b/dolphinscheduler-ui/src/views/projects/task/constants/task-type.ts
@@ -42,6 +42,9 @@ export type TaskType =
   | 'DINKY'
   | 'SAGEMAKER'
   | 'CHUNJUN'
+  | 'FLINK_STREAM'
+
+export type TaskExecuteType = 'STREAM' | 'BATCH'
 
 export const TASK_TYPES_MAP = {
   SHELL: {
@@ -138,5 +141,16 @@ export const TASK_TYPES_MAP = {
   CHUNJUN: {
     alias: 'CHUNJUN',
     helperLinkDisable: true
+  },
+  FLINK_STREAM: {
+    alias: 'FLINK_STREAM',
+    helperLinkDisable: true,
+    taskExecuteType: 'STREAM'
+  }
+} as {
+  [key in TaskType]: {
+    alias: string
+    helperLinkDisable?: boolean
+    taskExecuteType?: TaskExecuteType
   }
-} as { [key in TaskType]: { alias: string; helperLinkDisable?: boolean } }
+}
diff --git a/dolphinscheduler-ui/src/views/projects/task/definition/index.tsx b/dolphinscheduler-ui/src/views/projects/task/definition/batch-task.tsx
similarity index 95%
copy from dolphinscheduler-ui/src/views/projects/task/definition/index.tsx
copy to dolphinscheduler-ui/src/views/projects/task/definition/batch-task.tsx
index 5ddeccee14..4e58bd19cf 100644
--- a/dolphinscheduler-ui/src/views/projects/task/definition/index.tsx
+++ b/dolphinscheduler-ui/src/views/projects/task/definition/batch-task.tsx
@@ -15,7 +15,13 @@
  * limitations under the License.
  */
 
-import { defineComponent, getCurrentInstance, onMounted, toRefs, watch } from 'vue'
+import {
+  defineComponent,
+  getCurrentInstance,
+  onMounted,
+  toRefs,
+  watch
+} from 'vue'
 import { useRoute } from 'vue-router'
 import {
   NButton,
@@ -39,8 +45,8 @@ import TaskModal from '@/views/projects/task/components/node/detail-modal'
 import styles from './index.module.scss'
 import type { INodeData } from './types'
 
-const TaskDefinition = defineComponent({
-  name: 'task-definition',
+const BatchTaskDefinition = defineComponent({
+  name: 'batch-task-definition',
   setup() {
     const route = useRoute()
     const projectCode = Number(route.params.projectCode)
@@ -138,14 +144,14 @@ const TaskDefinition = defineComponent({
             </div>
             <NSpace justify='end'>
               <NInput
-                  allowInput={this.trim}
+                allowInput={this.trim}
                 size='small'
                 clearable
                 v-model={[this.searchTaskName, 'value']}
                 placeholder={t('project.task.task_name')}
               />
               <NInput
-                  allowInput={this.trim}
+                allowInput={this.trim}
                 size='small'
                 clearable
                 v-model={[this.searchWorkflowName, 'value']}
@@ -220,4 +226,4 @@ const TaskDefinition = defineComponent({
   }
 })
 
-export default TaskDefinition
+export default BatchTaskDefinition
diff --git a/dolphinscheduler-ui/src/views/projects/task/definition/components/start-modal.tsx b/dolphinscheduler-ui/src/views/projects/task/definition/components/start-modal.tsx
new file mode 100644
index 0000000000..8e3a1cd060
--- /dev/null
+++ b/dolphinscheduler-ui/src/views/projects/task/definition/components/start-modal.tsx
@@ -0,0 +1,267 @@
+/*
+ * 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.
+ */
+
+import {
+  defineComponent,
+  PropType,
+  toRefs,
+  onMounted,
+  watch,
+  getCurrentInstance
+} from 'vue'
+import { useI18n } from 'vue-i18n'
+import Modal from '@/components/modal'
+import { useStart } from './use-start'
+import {
+  NForm,
+  NFormItem,
+  NButton,
+  NIcon,
+  NInput,
+  NSpace,
+  NSelect,
+  NSwitch
+} from 'naive-ui'
+import { DeleteOutlined, PlusCircleOutlined } from '@vicons/antd'
+import styles from '../index.module.scss'
+
+const props = {
+  row: {
+    type: Object,
+    default: {}
+  },
+  show: {
+    type: Boolean as PropType<boolean>,
+    default: false
+  },
+  taskCode: {
+    type: String
+  }
+}
+
+export default defineComponent({
+  name: 'task-definition-start',
+  props,
+  emits: ['update:show', 'update:row', 'updateList'],
+  setup(props, ctx) {
+    const { t } = useI18n()
+    const {
+      variables,
+      handleStartDefinition,
+      getWorkerGroups,
+      getAlertGroups,
+      getEnvironmentList,
+      getStartParamsList
+    } = useStart(ctx)
+
+    const generalWarningTypeListOptions = () => [
+      {
+        value: 'NONE',
+        label: t('project.task.none_send')
+      },
+      {
+        value: 'SUCCESS',
+        label: t('project.task.success_send')
+      },
+      {
+        value: 'FAILURE',
+        label: t('project.task.failure_send')
+      },
+      {
+        value: 'ALL',
+        label: t('project.task.all_send')
+      }
+    ]
+
+    const hideModal = () => {
+      ctx.emit('update:show')
+    }
+
+    const handleStart = () => {
+      handleStartDefinition(props.row.taskCode)
+    }
+
+    const updateWorkerGroup = () => {
+      variables.startForm.environmentCode = null
+    }
+
+    const addStartParams = () => {
+      variables.startState.startParamsList.push({
+        prop: '',
+        value: ''
+      })
+    }
+
+    const updateParamsList = (index: number, param: Array<string>) => {
+      variables.startState.startParamsList[index].prop = param[0]
+      variables.startState.startParamsList[index].value = param[1]
+    }
+
+    const removeStartParams = (index: number) => {
+      variables.startState.startParamsList.splice(index, 1)
+    }
+
+    const trim = getCurrentInstance()?.appContext.config.globalProperties.trim
+
+    onMounted(() => {
+      getWorkerGroups()
+      getAlertGroups()
+      getEnvironmentList()
+    })
+
+    watch(
+      () => props.show,
+      () => {
+        if (props.show) {
+          getStartParamsList(props.row.processDefinitionCode)
+        }
+      }
+    )
+
+    return {
+      t,
+      hideModal,
+      handleStart,
+      updateWorkerGroup,
+      removeStartParams,
+      addStartParams,
+      updateParamsList,
+      generalWarningTypeListOptions,
+      ...toRefs(variables),
+      ...toRefs(variables.startState),
+      ...toRefs(props),
+      trim
+    }
+  },
+
+  render() {
+    const { t } = this
+
+    return (
+      <Modal
+        show={this.show}
+        title={t('project.task.set_parameters_before_starting')}
+        onCancel={this.hideModal}
+        onConfirm={this.handleStart}
+        confirmLoading={this.saving}
+      >
+        <NForm ref='startFormRef' model={this.startForm}>
+          <NFormItem label={t('project.task.task_name')} path='task_name'>
+            <div class={styles.formItem} title={this.row.taskName}>
+              {this.row.taskName}
+            </div>
+          </NFormItem>
+          <NFormItem
+            label={t('project.task.notification_strategy')}
+            path='warningType'
+          >
+            <NSelect
+              options={this.generalWarningTypeListOptions()}
+              v-model:value={this.startForm.warningType}
+            />
+          </NFormItem>
+          <NFormItem label={t('project.task.worker_group')} path='workerGroup'>
+            <NSelect
+              options={this.workerGroups}
+              onUpdateValue={this.updateWorkerGroup}
+              v-model:value={this.startForm.workerGroup}
+            />
+          </NFormItem>
+          <NFormItem
+            label={t('project.task.environment_name')}
+            path='environmentCode'
+          >
+            <NSelect
+              options={this.environmentList.filter((item: any) =>
+                item.workerGroups?.includes(this.startForm.workerGroup)
+              )}
+              v-model:value={this.startForm.environmentCode}
+              clearable
+            />
+          </NFormItem>
+          <NFormItem
+            label={t('project.task.alarm_group')}
+            path='warningGroupId'
+          >
+            <NSelect
+              options={this.alertGroups}
+              placeholder={t('project.task.please_choose')}
+              v-model:value={this.startForm.warningGroupId}
+              clearable
+            />
+          </NFormItem>
+          <NFormItem
+            label={t('project.task.startup_parameter')}
+            path='startup_parameter'
+          >
+            {this.startParamsList.length === 0 ? (
+              <NButton text type='primary' onClick={this.addStartParams}>
+                <NIcon>
+                  <PlusCircleOutlined />
+                </NIcon>
+              </NButton>
+            ) : (
+              <NSpace vertical>
+                {this.startParamsList.map((item, index) => (
+                  <NSpace class={styles.startup} key={Date.now() + index}>
+                    <NInput
+                      allowInput={this.trim}
+                      pair
+                      separator=':'
+                      placeholder={['prop', 'value']}
+                      defaultValue={[item.prop, item.value]}
+                      onUpdateValue={(param) =>
+                        this.updateParamsList(index, param)
+                      }
+                    />
+                    <NButton
+                      text
+                      type='error'
+                      onClick={() => this.removeStartParams(index)}
+                      class='btn-delete-custom-parameter'
+                    >
+                      <NIcon>
+                        <DeleteOutlined />
+                      </NIcon>
+                    </NButton>
+                    <NButton
+                      text
+                      type='primary'
+                      onClick={this.addStartParams}
+                      class='btn-create-custom-parameter'
+                    >
+                      <NIcon>
+                        <PlusCircleOutlined />
+                      </NIcon>
+                    </NButton>
+                  </NSpace>
+                ))}
+              </NSpace>
+            )}
+          </NFormItem>
+          <NFormItem label={t('project.task.whether_dry_run')} path='dryRun'>
+            <NSwitch
+              checkedValue={1}
+              uncheckedValue={0}
+              v-model:value={this.startForm.dryRun}
+            />
+          </NFormItem>
+        </NForm>
+      </Modal>
+    )
+  }
+})
diff --git a/dolphinscheduler-ui/src/views/projects/task/definition/components/use-start.tsx b/dolphinscheduler-ui/src/views/projects/task/definition/components/use-start.tsx
new file mode 100644
index 0000000000..4a1589c279
--- /dev/null
+++ b/dolphinscheduler-ui/src/views/projects/task/definition/components/use-start.tsx
@@ -0,0 +1,143 @@
+/*
+ * 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.
+ */
+
+import { startTaskDefinition } from '@/service/modules/task-definition'
+import _ from 'lodash'
+import { reactive, ref, SetupContext } from 'vue'
+import { useI18n } from 'vue-i18n'
+import { useRoute } from 'vue-router'
+import { queryProcessDefinitionByCode } from '@/service/modules/process-definition'
+import { queryAllWorkerGroups } from '@/service/modules/worker-groups'
+import { queryAllEnvironmentList } from '@/service/modules/environment'
+import { listAlertGroupById } from '@/service/modules/alert-group'
+import type { EnvironmentItem } from '@/service/modules/environment/types'
+import type { IStartState } from '../types'
+
+export const useStart = (
+  ctx: SetupContext<('update:show' | 'update:row' | 'updateList')[]>
+) => {
+  const { t } = useI18n()
+  const route = useRoute()
+
+  const variables = reactive({
+    startFormRef: ref(),
+    startForm: {
+      version: 1,
+      warningType: 'NONE',
+      warningGroupId: null,
+      workerGroup: 'default',
+      environmentCode: null,
+      startParams: null as null | string,
+      dryRun: 0
+    },
+    startState: {
+      projectCode: Number(route.params.projectCode),
+      workerGroups: [],
+      alertGroups: [],
+      environmentList: [],
+      startParamsList: []
+    } as IStartState,
+    saving: false
+  })
+
+  const cachedStartParams = {} as {
+    [key: string]: { prop: string; value: string }[]
+  }
+
+  const getWorkerGroups = () => {
+    queryAllWorkerGroups().then((res: any) => {
+      variables.startState.workerGroups = res.map((item: string) => ({
+        label: item,
+        value: item
+      }))
+    })
+  }
+
+  const getEnvironmentList = () => {
+    queryAllEnvironmentList().then((res: Array<EnvironmentItem>) => {
+      variables.startState.environmentList = res.map((item) => ({
+        label: item.name,
+        value: item.code,
+        workerGroups: item.workerGroups
+      }))
+    })
+  }
+
+  const getAlertGroups = () => {
+    listAlertGroupById().then((res: any) => {
+      variables.startState.alertGroups = res.map((item: any) => ({
+        label: item.groupName,
+        value: item.id
+      }))
+    })
+  }
+
+  const getStartParamsList = (code: number) => {
+    if (cachedStartParams[code]) {
+      variables.startState.startParamsList = _.cloneDeep(
+        cachedStartParams[code]
+      )
+      return
+    }
+    queryProcessDefinitionByCode(code, variables.startState.projectCode).then(
+      (res: any) => {
+        variables.startState.startParamsList =
+          res.processDefinition.globalParamList
+        cachedStartParams[code] = _.cloneDeep(
+          variables.startState.startParamsList
+        )
+      }
+    )
+  }
+
+  const handleStartDefinition = async (code: number) => {
+    await variables.startFormRef.validate()
+
+    if (variables.saving) return
+    variables.saving = true
+    try {
+      const startParams = {} as any
+      for (const item of variables.startState.startParamsList) {
+        if (item.value !== '') {
+          startParams[item.prop] = item.value
+        }
+      }
+      variables.startForm.startParams = !_.isEmpty(startParams)
+        ? JSON.stringify(startParams)
+        : ''
+
+      await startTaskDefinition(variables.startState.projectCode, code, {
+        ...variables.startForm
+      } as any)
+      window.$message.success(t('project.task.success'))
+      variables.saving = false
+      ctx.emit('updateList')
+      ctx.emit('update:show')
+    } catch (err) {
+      variables.saving = false
+    }
+  }
+
+  return {
+    variables,
+    getWorkerGroups,
+    getEnvironmentList,
+    getAlertGroups,
+    getStartParamsList,
+    handleStartDefinition
+  }
+}
diff --git a/dolphinscheduler-ui/src/views/projects/task/definition/index.tsx b/dolphinscheduler-ui/src/views/projects/task/definition/index.tsx
index 5ddeccee14..d762e4718b 100644
--- a/dolphinscheduler-ui/src/views/projects/task/definition/index.tsx
+++ b/dolphinscheduler-ui/src/views/projects/task/definition/index.tsx
@@ -15,207 +15,25 @@
  * limitations under the License.
  */
 
-import { defineComponent, getCurrentInstance, onMounted, toRefs, watch } from 'vue'
-import { useRoute } from 'vue-router'
-import {
-  NButton,
-  NCard,
-  NDataTable,
-  NIcon,
-  NInput,
-  NPagination,
-  NSelect,
-  NSpace
-} from 'naive-ui'
-import { SearchOutlined } from '@vicons/antd'
+import { defineComponent } from 'vue'
 import { useI18n } from 'vue-i18n'
-import { useTable } from './use-table'
-import { useTask } from './use-task'
-import { TASK_TYPES_MAP } from '@/views/projects/task/constants/task-type'
-import Card from '@/components/card'
-import VersionModal from './components/version-modal'
-import MoveModal from './components/move-modal'
-import TaskModal from '@/views/projects/task/components/node/detail-modal'
-import styles from './index.module.scss'
-import type { INodeData } from './types'
+import { NTabPane, NTabs } from 'naive-ui'
+import BatchTaskDefinition from './batch-task'
+import StreamTaskDefinition from './stream-task'
 
 const TaskDefinition = defineComponent({
   name: 'task-definition',
   setup() {
-    const route = useRoute()
-    const projectCode = Number(route.params.projectCode)
     const { t } = useI18n()
-
-    const { task, onToggleShow, onTaskSave, onEditTask, onInitTask } =
-      useTask(projectCode)
-
-    const { variables, getTableData, createColumns } = useTable(onEditTask)
-
-    const requestData = () => {
-      getTableData({
-        pageSize: variables.pageSize,
-        pageNo: variables.page,
-        searchTaskName: variables.searchTaskName,
-        searchWorkflowName: variables.searchWorkflowName,
-        taskType: variables.taskType
-      })
-    }
-
-    const onUpdatePageSize = () => {
-      variables.page = 1
-      requestData()
-    }
-
-    const onSearch = () => {
-      variables.page = 1
-      requestData()
-    }
-
-    const onRefresh = () => {
-      variables.showVersionModalRef = false
-      variables.showMoveModalRef = false
-      requestData()
-    }
-    const onCreate = () => {
-      onToggleShow(true)
-    }
-    const onTaskCancel = () => {
-      onToggleShow(false)
-      onInitTask()
-    }
-    const onTaskSubmit = async (params: { data: INodeData }) => {
-      const result = await onTaskSave(params.data)
-      if (result) {
-        onTaskCancel()
-        onRefresh()
-      }
-    }
-
-    const trim = getCurrentInstance()?.appContext.config.globalProperties.trim
-    onMounted(() => {
-      createColumns(variables)
-      requestData()
-    })
-
-    watch(useI18n().locale, () => {
-      createColumns(variables)
-    })
-
-    return {
-      t,
-      ...toRefs(variables),
-      ...toRefs(task),
-      onSearch,
-      requestData,
-      onUpdatePageSize,
-      onRefresh,
-      onCreate,
-      onTaskSubmit,
-      onTaskCancel,
-      projectCode,
-      trim
-    }
-  },
-  render() {
-    const {
-      t,
-      onSearch,
-      requestData,
-      onUpdatePageSize,
-      onRefresh,
-      onCreate,
-      loadingRef
-    } = this
-
-    return (
-      <>
-        <NCard>
-          <div class={styles['search-card']}>
-            <div>
-              <NButton size='small' type='primary' onClick={onCreate}>
-                {t('project.task.create_task')}
-              </NButton>
-            </div>
-            <NSpace justify='end'>
-              <NInput
-                  allowInput={this.trim}
-                size='small'
-                clearable
-                v-model={[this.searchTaskName, 'value']}
-                placeholder={t('project.task.task_name')}
-              />
-              <NInput
-                  allowInput={this.trim}
-                size='small'
-                clearable
-                v-model={[this.searchWorkflowName, 'value']}
-                placeholder={t('project.task.workflow_name')}
-              />
-              <NSelect
-                v-model={[this.taskType, 'value']}
-                size='small'
-                options={Object.keys(TASK_TYPES_MAP).map((item) => {
-                  return { value: item, label: item }
-                })}
-                placeholder={t('project.task.task_type')}
-                style={{ width: '180px' }}
-                clearable
-              />
-              <NButton size='small' type='primary' onClick={onSearch}>
-                {{
-                  icon: () => (
-                    <NIcon>
-                      <SearchOutlined />
-                    </NIcon>
-                  )
-                }}
-              </NButton>
-            </NSpace>
-          </div>
-        </NCard>
-        <Card class={styles['table-card']}>
-          <NDataTable
-            loading={loadingRef}
-            columns={this.columns}
-            data={this.tableData}
-            scrollX={this.tableWidth}
-          />
-          <div class={styles.pagination}>
-            <NPagination
-              v-model:page={this.page}
-              v-model:page-size={this.pageSize}
-              page-count={this.totalPage}
-              show-size-picker
-              page-sizes={[10, 30, 50]}
-              show-quick-jumper
-              onUpdatePage={requestData}
-              onUpdatePageSize={onUpdatePageSize}
-            />
-          </div>
-        </Card>
-        <VersionModal
-          show={this.showVersionModalRef}
-          row={this.row}
-          onConfirm={() => (this.showVersionModalRef = false)}
-          onRefresh={onRefresh}
-        />
-        <MoveModal
-          show={this.showMoveModalRef}
-          row={this.row}
-          onCancel={() => (this.showMoveModalRef = false)}
-          onRefresh={onRefresh}
-        />
-        <TaskModal
-          show={this.taskShow}
-          data={this.taskData}
-          onSubmit={this.onTaskSubmit}
-          onCancel={this.onTaskCancel}
-          projectCode={this.projectCode}
-          from={1}
-          readonly={this.taskReadonly}
-          saving={this.taskSaving}
-        />
-      </>
+    return () => (
+      <NTabs type='line' animated>
+        <NTabPane name='Batch' tab={t('project.task.batch_task')}>
+          <BatchTaskDefinition />
+        </NTabPane>
+        <NTabPane name='Stream' tab={t('project.task.stream_task')}>
+          <StreamTaskDefinition />
+        </NTabPane>
+      </NTabs>
     )
   }
 })
diff --git a/dolphinscheduler-ui/src/views/projects/task/definition/stream-task.tsx b/dolphinscheduler-ui/src/views/projects/task/definition/stream-task.tsx
new file mode 100644
index 0000000000..2fce2a9138
--- /dev/null
+++ b/dolphinscheduler-ui/src/views/projects/task/definition/stream-task.tsx
@@ -0,0 +1,159 @@
+/*
+ * 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.
+ */
+
+import { useI18n } from 'vue-i18n'
+import { SearchOutlined } from '@vicons/antd'
+import { defineComponent, getCurrentInstance, watch, onMounted } from 'vue'
+import {
+  NInput,
+  NButton,
+  NIcon,
+  NSpace,
+  NDataTable,
+  NPagination
+} from 'naive-ui'
+import { useRoute } from 'vue-router'
+import Card from '@/components/card'
+import TaskModal from '@/views/projects/task/components/node/detail-modal'
+import { useTable } from './use-stream-table'
+import { useTask } from './use-task'
+import StartModal from './components/start-modal'
+import type { INodeData } from './types'
+
+const StreamTaskDefinition = defineComponent({
+  name: 'stream-task-definition',
+  setup() {
+    const { t } = useI18n()
+    const route = useRoute()
+    const projectCode = Number(route.params.projectCode)
+
+    const trim = getCurrentInstance()?.appContext.config.globalProperties.trim
+    const { task, onToggleShow, onEditTask, onInitTask, onUpdateTask } =
+      useTask(projectCode)
+    const { variables, getTableData, createColumns } = useTable(onEditTask)
+
+    const onSearch = () => {
+      variables.page = 1
+      getTableData()
+    }
+
+    const onRefresh = () => {
+      getTableData()
+    }
+
+    const onUpdatePageSize = () => {
+      variables.page = 1
+      getTableData()
+    }
+
+    const onTaskCancel = () => {
+      onToggleShow(false)
+      onInitTask()
+    }
+
+    const onTaskSubmit = async (params: { data: INodeData }) => {
+      const result = await onUpdateTask(params.data)
+      if (result) {
+        onTaskCancel()
+        onRefresh()
+      }
+    }
+
+    onMounted(() => {
+      createColumns(variables)
+      getTableData()
+    })
+
+    watch(useI18n().locale, () => {
+      createColumns(variables)
+    })
+
+    return () => (
+      <NSpace vertical>
+        <Card>
+          <NSpace justify='space-between'>
+            <NSpace />
+            <NSpace>
+              <NInput
+                allowInput={trim}
+                size='small'
+                clearable
+                v-model={[variables.searchTaskName, 'value']}
+                placeholder={t('project.task.task_name')}
+              />
+              <NInput
+                allowInput={trim}
+                size='small'
+                clearable
+                v-model={[variables.searchWorkflowName, 'value']}
+                placeholder={t('project.task.workflow_name')}
+              />
+              <NButton size='small' type='primary' onClick={onSearch}>
+                {{
+                  icon: () => (
+                    <NIcon>
+                      <SearchOutlined />
+                    </NIcon>
+                  )
+                }}
+              </NButton>
+            </NSpace>
+          </NSpace>
+        </Card>
+        <Card>
+          <NSpace vertical>
+            <NDataTable
+              loading={variables.loading}
+              columns={variables.columns}
+              data={variables.tableData}
+              scrollX={variables.tableWidth}
+            />
+            <NSpace justify='center'>
+              <NPagination
+                v-model:page={variables.page}
+                v-model:page-size={variables.pageSize}
+                page-count={variables.totalPage}
+                show-size-picker
+                page-sizes={[10, 30, 50]}
+                show-quick-jumper
+                onUpdatePage={getTableData}
+                onUpdatePageSize={onUpdatePageSize}
+              />
+            </NSpace>
+          </NSpace>
+        </Card>
+        <TaskModal
+          show={task.taskShow}
+          data={task.taskData}
+          onSubmit={onTaskSubmit}
+          onCancel={onTaskCancel}
+          projectCode={projectCode}
+          from={1}
+          readonly={task.taskReadonly}
+          saving={task.taskSaving}
+        />
+        <StartModal
+          v-model:row={variables.row}
+          v-model:show={variables.startShow}
+          onUpdateList={getTableData}
+        />
+      </NSpace>
+    )
+  }
+})
+
+export default StreamTaskDefinition
diff --git a/dolphinscheduler-ui/src/views/projects/task/definition/types.ts b/dolphinscheduler-ui/src/views/projects/task/definition/types.ts
index c15c840355..3ca196bd72 100644
--- a/dolphinscheduler-ui/src/views/projects/task/definition/types.ts
+++ b/dolphinscheduler-ui/src/views/projects/task/definition/types.ts
@@ -15,6 +15,10 @@
  * limitations under the License.
  */
 
+import { IOption } from '@/components/form/types'
+import { IParam } from '../../workflow/definition/components/types'
+import { IEnvironmentNameOption } from '../components/node/types'
+
 export type { ITaskData, INodeData } from '../components/node/types'
 export type { ISingleSaveReq } from '@/service/modules/task-definition/types'
 
@@ -24,4 +28,12 @@ interface IRecord {
   taskName: string
 }
 
-export { IRecord }
+interface IStartState {
+  projectCode: number
+  workerGroups: Array<IOption>
+  alertGroups: Array<IOption>
+  environmentList: Array<IEnvironmentNameOption>
+  startParamsList: Array<IParam>
+}
+
+export { IRecord, IStartState }
diff --git a/dolphinscheduler-ui/src/views/projects/task/definition/use-stream-table.ts b/dolphinscheduler-ui/src/views/projects/task/definition/use-stream-table.ts
new file mode 100644
index 0000000000..88b79d528a
--- /dev/null
+++ b/dolphinscheduler-ui/src/views/projects/task/definition/use-stream-table.ts
@@ -0,0 +1,202 @@
+/*
+ * 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.
+ */
+
+import { reactive, h } from 'vue'
+import { NButton, NIcon, NSpace, NTooltip, NEllipsis } from 'naive-ui'
+import ButtonLink from '@/components/button-link'
+import { useI18n } from 'vue-i18n'
+import { EditOutlined, PlayCircleOutlined } from '@vicons/antd'
+import { queryTaskDefinitionListPaging } from '@/service/modules/task-definition'
+import { useRoute } from 'vue-router'
+import {
+  COLUMN_WIDTH_CONFIG,
+  calculateTableWidth,
+  DefaultTableWidth
+} from '@/common/column-width-config'
+import type {
+  TaskDefinitionItem,
+  TaskDefinitionRes
+} from '@/service/modules/task-definition/types'
+import type { IRecord } from './types'
+
+export function useTable(onEdit: Function) {
+  const { t } = useI18n()
+  const route = useRoute()
+  const projectCode = Number(route.params.projectCode)
+
+  const createColumns = (variables: any) => {
+    variables.columns = [
+      {
+        title: '#',
+        key: 'index',
+        render: (row: any, index: number) => index + 1,
+        ...COLUMN_WIDTH_CONFIG['index']
+      },
+      {
+        title: t('project.task.task_name'),
+        key: 'taskName',
+        ...COLUMN_WIDTH_CONFIG['linkName'],
+        render: (row: IRecord) =>
+          h(
+            ButtonLink,
+            {
+              onClick: () => void onEdit(row, true)
+            },
+            {
+              default: () =>
+                h(
+                  NEllipsis,
+                  COLUMN_WIDTH_CONFIG['linkEllipsis'],
+                  () => row.taskName
+                )
+            }
+          )
+      },
+      {
+        title: t('project.task.version'),
+        key: 'taskVersion',
+        render: (row: TaskDefinitionItem) =>
+          h('span', null, 'v' + row.taskVersion),
+        ...COLUMN_WIDTH_CONFIG['version']
+      },
+      {
+        title: t('project.task.workflow_name'),
+        key: 'processDefinitionName',
+        ...COLUMN_WIDTH_CONFIG['name']
+      },
+      {
+        title: t('project.task.task_type'),
+        key: 'taskType',
+        ...COLUMN_WIDTH_CONFIG['type']
+      },
+      {
+        title: t('project.task.create_time'),
+        key: 'taskCreateTime',
+        ...COLUMN_WIDTH_CONFIG['time']
+      },
+      {
+        title: t('project.task.update_time'),
+        key: 'taskUpdateTime',
+        ...COLUMN_WIDTH_CONFIG['time']
+      },
+      {
+        title: t('project.task.operation'),
+        key: 'operation',
+        ...COLUMN_WIDTH_CONFIG['operation'](2),
+        render(row: any) {
+          return h(NSpace, null, {
+            default: () => [
+              h(
+                NTooltip,
+                {},
+                {
+                  trigger: () =>
+                    h(
+                      NButton,
+                      {
+                        circle: true,
+                        type: 'info',
+                        size: 'small',
+                        onClick: () => onStart(row)
+                      },
+                      {
+                        icon: () =>
+                          h(NIcon, null, {
+                            default: () => h(PlayCircleOutlined)
+                          })
+                      }
+                    ),
+                  default: () => t('project.task.execute')
+                }
+              ),
+              h(
+                NTooltip,
+                {},
+                {
+                  trigger: () =>
+                    h(
+                      NButton,
+                      {
+                        circle: true,
+                        type: 'info',
+                        size: 'small',
+                        onClick: () => onEdit(row, false)
+                      },
+                      {
+                        icon: () =>
+                          h(NIcon, null, { default: () => h(EditOutlined) })
+                      }
+                    ),
+                  default: () => t('project.task.edit')
+                }
+              )
+            ]
+          })
+        }
+      }
+    ]
+    if (variables.tableWidth) {
+      variables.tableWidth = calculateTableWidth(variables.columns)
+    }
+  }
+
+  const variables = reactive({
+    columns: [],
+    tableWidth: DefaultTableWidth,
+    tableData: [],
+    page: 1,
+    pageSize: 10,
+    searchTaskName: null,
+    searchWorkflowName: null,
+    totalPage: 1,
+    row: {},
+    loading: false,
+    startShow: false
+  })
+
+  const getTableData = () => {
+    if (variables.loading) return
+    variables.loading = true
+    const params = {
+      pageSize: variables.pageSize,
+      pageNo: variables.page,
+      searchTaskName: variables.searchTaskName,
+      searchWorkflowName: variables.searchWorkflowName,
+      taskExecuteType: 'STREAM' as 'BATCH' | 'STREAM'
+    } as any
+
+    queryTaskDefinitionListPaging(params, { projectCode })
+      .then((res: TaskDefinitionRes) => {
+        variables.tableData = [...res.totalList] as any
+        variables.totalPage = res.totalPage
+      })
+      .finally(() => {
+        variables.loading = false
+      })
+  }
+
+  const onStart = (row: any) => {
+    variables.row = row
+    variables.startShow = true
+  }
+
+  return {
+    variables,
+    getTableData,
+    createColumns
+  }
+}
diff --git a/dolphinscheduler-ui/src/views/projects/task/definition/use-task.ts b/dolphinscheduler-ui/src/views/projects/task/definition/use-task.ts
index ebc6e57b48..c0092844cd 100644
--- a/dolphinscheduler-ui/src/views/projects/task/definition/use-task.ts
+++ b/dolphinscheduler-ui/src/views/projects/task/definition/use-task.ts
@@ -17,6 +17,7 @@
 
 import { reactive } from 'vue'
 import {
+  updateTask,
   genTaskCodeList,
   saveSingle,
   queryTaskDefinitionByCode,
@@ -98,11 +99,34 @@ export function useTask(projectCode: number) {
     task.taskReadonly = false
   }
 
+  const onUpdateTask = async (data: INodeData) => {
+    if (task.taskSaving || !data.code) return
+    task.taskSaving = true
+
+    const params = {
+      taskExecuteType: 'STREAM',
+      taskDefinitionJsonObj: JSON.stringify(
+        formatData(data).taskDefinitionJsonObj
+      )
+    }
+
+    try {
+      await updateTask(projectCode, data.code, params)
+
+      task.taskSaving = false
+      return true
+    } catch (err) {
+      task.taskSaving = false
+      return false
+    }
+  }
+
   return {
     task,
     onToggleShow,
     onTaskSave,
     onEditTask,
-    onInitTask
+    onInitTask,
+    onUpdateTask
   }
 }
diff --git a/dolphinscheduler-ui/src/views/projects/task/instance/index.tsx b/dolphinscheduler-ui/src/views/projects/task/instance/batch-task.tsx
similarity index 95%
copy from dolphinscheduler-ui/src/views/projects/task/instance/index.tsx
copy to dolphinscheduler-ui/src/views/projects/task/instance/batch-task.tsx
index c742a20d20..d88ab9eb4a 100644
--- a/dolphinscheduler-ui/src/views/projects/task/instance/index.tsx
+++ b/dolphinscheduler-ui/src/views/projects/task/instance/batch-task.tsx
@@ -15,7 +15,13 @@
  * limitations under the License.
  */
 
-import { defineComponent, getCurrentInstance, onMounted, toRefs, watch } from 'vue'
+import {
+  defineComponent,
+  getCurrentInstance,
+  onMounted,
+  toRefs,
+  watch
+} from 'vue'
 import {
   NSpace,
   NInput,
@@ -37,7 +43,7 @@ import { queryLog } from '@/service/modules/log'
 import { stateType } from '@/common/common'
 import styles from './index.module.scss'
 
-const TaskInstance = defineComponent({
+const BatchTaskInstance = defineComponent({
   name: 'task-instance',
   setup() {
     const { t, variables, getTableData, createColumns } = useTable()
@@ -152,28 +158,28 @@ const TaskInstance = defineComponent({
         <NCard>
           <NSpace justify='end' wrap={false}>
             <NInput
-                  allowInput={this.trim}
+              allowInput={this.trim}
               v-model={[this.searchVal, 'value']}
               size='small'
               placeholder={t('project.task.task_name')}
               clearable
             />
             <NInput
-                  allowInput={this.trim}
+              allowInput={this.trim}
               v-model={[this.processInstanceName, 'value']}
               size='small'
               placeholder={t('project.task.workflow_instance')}
               clearable
             />
             <NInput
-                  allowInput={this.trim}
+              allowInput={this.trim}
               v-model={[this.executorName, 'value']}
               size='small'
               placeholder={t('project.task.executor')}
               clearable
             />
             <NInput
-                  allowInput={this.trim}
+              allowInput={this.trim}
               v-model={[this.host, 'value']}
               size='small'
               placeholder={t('project.task.host')}
@@ -239,4 +245,4 @@ const TaskInstance = defineComponent({
   }
 })
 
-export default TaskInstance
+export default BatchTaskInstance
diff --git a/dolphinscheduler-ui/src/views/projects/task/instance/index.tsx b/dolphinscheduler-ui/src/views/projects/task/instance/index.tsx
index c742a20d20..de9e7f690b 100644
--- a/dolphinscheduler-ui/src/views/projects/task/instance/index.tsx
+++ b/dolphinscheduler-ui/src/views/projects/task/instance/index.tsx
@@ -15,228 +15,27 @@
  * limitations under the License.
  */
 
-import { defineComponent, getCurrentInstance, onMounted, toRefs, watch } from 'vue'
-import {
-  NSpace,
-  NInput,
-  NSelect,
-  NDatePicker,
-  NButton,
-  NIcon,
-  NDataTable,
-  NPagination,
-  NCard
-} from 'naive-ui'
-import { SearchOutlined } from '@vicons/antd'
-import { useTable } from './use-table'
+import { defineComponent } from 'vue'
 import { useI18n } from 'vue-i18n'
-import Card from '@/components/card'
-import LogModal from '@/components/log-modal'
-import { useAsyncState } from '@vueuse/core'
-import { queryLog } from '@/service/modules/log'
-import { stateType } from '@/common/common'
-import styles from './index.module.scss'
+import { NTabPane, NTabs } from 'naive-ui'
+import BatchTaskInstance from './batch-task'
+import StreamTaskInstance from './stream-task'
 
-const TaskInstance = defineComponent({
+const TaskDefinition = defineComponent({
   name: 'task-instance',
   setup() {
-    const { t, variables, getTableData, createColumns } = useTable()
-
-    const requestTableData = () => {
-      getTableData({
-        pageSize: variables.pageSize,
-        pageNo: variables.page,
-        searchVal: variables.searchVal,
-        processInstanceId: variables.processInstanceId,
-        host: variables.host,
-        stateType: variables.stateType,
-        datePickerRange: variables.datePickerRange,
-        executorName: variables.executorName,
-        processInstanceName: variables.processInstanceName
-      })
-    }
-
-    const onUpdatePageSize = () => {
-      variables.page = 1
-      requestTableData()
-    }
-
-    const onSearch = () => {
-      variables.page = 1
-      requestTableData()
-    }
-
-    const onConfirmModal = () => {
-      variables.showModalRef = false
-    }
-
-    const getLogs = (row: any) => {
-      const { state } = useAsyncState(
-        queryLog({
-          taskInstanceId: Number(row.id),
-          limit: variables.limit,
-          skipLineNum: variables.skipLineNum
-        }).then((res: any) => {
-          if (res?.message) {
-            variables.logRef += res.message
-            variables.limit += 1000
-            variables.skipLineNum += res.lineNum
-            getLogs(row)
-          } else {
-            variables.logLoadingRef = false
-          }
-        }),
-        {}
-      )
-
-      return state
-    }
-
-    const refreshLogs = (row: any) => {
-      variables.logRef = ''
-      variables.limit = 1000
-      variables.skipLineNum = 0
-      getLogs(row)
-    }
-
-    const trim = getCurrentInstance()?.appContext.config.globalProperties.trim
-
-    onMounted(() => {
-      createColumns(variables)
-      requestTableData()
-    })
-
-    watch(useI18n().locale, () => {
-      createColumns(variables)
-    })
-
-    watch(
-      () => variables.showModalRef,
-      () => {
-        if (variables.showModalRef) {
-          getLogs(variables.row)
-        } else {
-          variables.row = {}
-          variables.logRef = ''
-          variables.logLoadingRef = true
-          variables.skipLineNum = 0
-          variables.limit = 1000
-        }
-      }
-    )
-
-    return {
-      t,
-      ...toRefs(variables),
-      requestTableData,
-      onUpdatePageSize,
-      onSearch,
-      onConfirmModal,
-      refreshLogs,
-      trim
-    }
-  },
-  render() {
-    const {
-      t,
-      requestTableData,
-      onUpdatePageSize,
-      onSearch,
-      onConfirmModal,
-      loadingRef,
-      refreshLogs
-    } = this
-
-    return (
-      <>
-        <NCard>
-          <NSpace justify='end' wrap={false}>
-            <NInput
-                  allowInput={this.trim}
-              v-model={[this.searchVal, 'value']}
-              size='small'
-              placeholder={t('project.task.task_name')}
-              clearable
-            />
-            <NInput
-                  allowInput={this.trim}
-              v-model={[this.processInstanceName, 'value']}
-              size='small'
-              placeholder={t('project.task.workflow_instance')}
-              clearable
-            />
-            <NInput
-                  allowInput={this.trim}
-              v-model={[this.executorName, 'value']}
-              size='small'
-              placeholder={t('project.task.executor')}
-              clearable
-            />
-            <NInput
-                  allowInput={this.trim}
-              v-model={[this.host, 'value']}
-              size='small'
-              placeholder={t('project.task.host')}
-              clearable
-            />
-            <NSelect
-              v-model={[this.stateType, 'value']}
-              size='small'
-              options={stateType(t).slice(1)}
-              placeholder={t('project.task.state')}
-              style={{ width: '180px' }}
-              clearable
-            />
-            <NDatePicker
-              v-model={[this.datePickerRange, 'value']}
-              type='datetimerange'
-              size='small'
-              start-placeholder={t('project.task.start_time')}
-              end-placeholder={t('project.task.end_time')}
-              clearable
-            />
-            <NButton size='small' type='primary' onClick={onSearch}>
-              {{
-                icon: () => (
-                  <NIcon>
-                    <SearchOutlined />
-                  </NIcon>
-                )
-              }}
-            </NButton>
-          </NSpace>
-        </NCard>
-        <Card class={styles['table-card']}>
-          <NDataTable
-            loading={loadingRef}
-            columns={this.columns}
-            data={this.tableData}
-            scrollX={this.tableWidth}
-          />
-          <div class={styles.pagination}>
-            <NPagination
-              v-model:page={this.page}
-              v-model:page-size={this.pageSize}
-              page-count={this.totalPage}
-              show-size-picker
-              page-sizes={[10, 30, 50]}
-              show-quick-jumper
-              onUpdatePage={requestTableData}
-              onUpdatePageSize={onUpdatePageSize}
-            />
-          </div>
-        </Card>
-        <LogModal
-          showModalRef={this.showModalRef}
-          logRef={this.logRef}
-          row={this.row}
-          logLoadingRef={this.logLoadingRef}
-          onConfirmModal={onConfirmModal}
-          onRefreshLogs={refreshLogs}
-        />
-      </>
+    const { t } = useI18n()
+    return () => (
+      <NTabs type='line' animated>
+        <NTabPane name='Batch' tab={t('project.task.batch_task')}>
+          <BatchTaskInstance />
+        </NTabPane>
+        <NTabPane name='Stream' tab={t('project.task.stream_task')}>
+          <StreamTaskInstance />
+        </NTabPane>
+      </NTabs>
     )
   }
 })
 
-export default TaskInstance
+export default TaskDefinition
diff --git a/dolphinscheduler-ui/src/views/projects/task/instance/index.tsx b/dolphinscheduler-ui/src/views/projects/task/instance/stream-task.tsx
similarity index 82%
copy from dolphinscheduler-ui/src/views/projects/task/instance/index.tsx
copy to dolphinscheduler-ui/src/views/projects/task/instance/stream-task.tsx
index c742a20d20..18ab06e802 100644
--- a/dolphinscheduler-ui/src/views/projects/task/instance/index.tsx
+++ b/dolphinscheduler-ui/src/views/projects/task/instance/stream-task.tsx
@@ -15,7 +15,14 @@
  * limitations under the License.
  */
 
-import { defineComponent, getCurrentInstance, onMounted, toRefs, watch } from 'vue'
+import {
+  defineComponent,
+  getCurrentInstance,
+  onMounted,
+  onUnmounted,
+  toRefs,
+  watch
+} from 'vue'
 import {
   NSpace,
   NInput,
@@ -28,42 +35,29 @@ import {
   NCard
 } from 'naive-ui'
 import { SearchOutlined } from '@vicons/antd'
-import { useTable } from './use-table'
+import { useTable } from './use-stream-table'
 import { useI18n } from 'vue-i18n'
 import Card from '@/components/card'
 import LogModal from '@/components/log-modal'
 import { useAsyncState } from '@vueuse/core'
 import { queryLog } from '@/service/modules/log'
-import { stateType } from '@/common/common'
+import { streamStateType } from '@/common/common'
 import styles from './index.module.scss'
 
-const TaskInstance = defineComponent({
+const BatchTaskInstance = defineComponent({
   name: 'task-instance',
   setup() {
+    let setIntervalP: number
     const { t, variables, getTableData, createColumns } = useTable()
 
-    const requestTableData = () => {
-      getTableData({
-        pageSize: variables.pageSize,
-        pageNo: variables.page,
-        searchVal: variables.searchVal,
-        processInstanceId: variables.processInstanceId,
-        host: variables.host,
-        stateType: variables.stateType,
-        datePickerRange: variables.datePickerRange,
-        executorName: variables.executorName,
-        processInstanceName: variables.processInstanceName
-      })
-    }
-
     const onUpdatePageSize = () => {
       variables.page = 1
-      requestTableData()
+      getTableData()
     }
 
     const onSearch = () => {
       variables.page = 1
-      requestTableData()
+      getTableData()
     }
 
     const onConfirmModal = () => {
@@ -103,7 +97,14 @@ const TaskInstance = defineComponent({
 
     onMounted(() => {
       createColumns(variables)
-      requestTableData()
+      getTableData()
+      setIntervalP = setInterval(() => {
+        getTableData()
+      }, 3000)
+    })
+
+    onUnmounted(() => {
+      clearInterval(setIntervalP)
     })
 
     watch(useI18n().locale, () => {
@@ -128,7 +129,7 @@ const TaskInstance = defineComponent({
     return {
       t,
       ...toRefs(variables),
-      requestTableData,
+      getTableData,
       onUpdatePageSize,
       onSearch,
       onConfirmModal,
@@ -139,7 +140,7 @@ const TaskInstance = defineComponent({
   render() {
     const {
       t,
-      requestTableData,
+      getTableData,
       onUpdatePageSize,
       onSearch,
       onConfirmModal,
@@ -152,28 +153,28 @@ const TaskInstance = defineComponent({
         <NCard>
           <NSpace justify='end' wrap={false}>
             <NInput
-                  allowInput={this.trim}
+              allowInput={this.trim}
               v-model={[this.searchVal, 'value']}
               size='small'
               placeholder={t('project.task.task_name')}
               clearable
             />
             <NInput
-                  allowInput={this.trim}
-              v-model={[this.processInstanceName, 'value']}
+              allowInput={this.trim}
+              v-model={[this.processDefinitionName, 'value']}
               size='small'
-              placeholder={t('project.task.workflow_instance')}
+              placeholder={t('project.task.workflow_name')}
               clearable
             />
             <NInput
-                  allowInput={this.trim}
+              allowInput={this.trim}
               v-model={[this.executorName, 'value']}
               size='small'
               placeholder={t('project.task.executor')}
               clearable
             />
             <NInput
-                  allowInput={this.trim}
+              allowInput={this.trim}
               v-model={[this.host, 'value']}
               size='small'
               placeholder={t('project.task.host')}
@@ -182,7 +183,7 @@ const TaskInstance = defineComponent({
             <NSelect
               v-model={[this.stateType, 'value']}
               size='small'
-              options={stateType(t).slice(1)}
+              options={streamStateType(t).slice(1)}
               placeholder={t('project.task.state')}
               style={{ width: '180px' }}
               clearable
@@ -221,7 +222,7 @@ const TaskInstance = defineComponent({
               show-size-picker
               page-sizes={[10, 30, 50]}
               show-quick-jumper
-              onUpdatePage={requestTableData}
+              onUpdatePage={getTableData}
               onUpdatePageSize={onUpdatePageSize}
             />
           </div>
@@ -239,4 +240,4 @@ const TaskInstance = defineComponent({
   }
 })
 
-export default TaskInstance
+export default BatchTaskInstance
diff --git a/dolphinscheduler-ui/src/views/projects/task/instance/use-stream-table.ts b/dolphinscheduler-ui/src/views/projects/task/instance/use-stream-table.ts
new file mode 100644
index 0000000000..ca6f3efa3d
--- /dev/null
+++ b/dolphinscheduler-ui/src/views/projects/task/instance/use-stream-table.ts
@@ -0,0 +1,357 @@
+/*
+ * 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.
+ */
+
+import { useI18n } from 'vue-i18n'
+import { h, reactive } from 'vue'
+import {
+  downloadLog,
+  queryTaskListPaging,
+  savePoint,
+  streamTaskStop
+} from '@/service/modules/task-instances'
+import { NButton, NIcon, NSpace, NTooltip, NSpin } from 'naive-ui'
+import {
+  AlignLeftOutlined,
+  DownloadOutlined,
+  RetweetOutlined,
+  SaveOutlined,
+  StopOutlined
+} from '@vicons/antd'
+import { format } from 'date-fns'
+import { useRoute } from 'vue-router'
+import { parseTime, renderTableTime, tasksState } from '@/common/common'
+import {
+  COLUMN_WIDTH_CONFIG,
+  calculateTableWidth,
+  DefaultTableWidth
+} from '@/common/column-width-config'
+import type { TaskInstancesRes, ITaskState } from './types'
+
+export function useTable() {
+  const { t } = useI18n()
+  const route = useRoute()
+  const projectCode = Number(route.params.projectCode)
+  const processInstanceId = Number(route.params.processInstanceId)
+
+  const variables = reactive({
+    columns: [],
+    tableWidth: DefaultTableWidth,
+    tableData: [] as any[],
+    page: 1,
+    pageSize: 10,
+    searchVal: null,
+    processInstanceId: processInstanceId ? processInstanceId : null,
+    host: null,
+    stateType: null,
+    datePickerRange: null,
+    executorName: null,
+    processDefinitionName: null,
+    totalPage: 1,
+    showModalRef: false,
+    row: {},
+    loadingRef: false,
+    logRef: '',
+    logLoadingRef: true,
+    skipLineNum: 0,
+    limit: 1000
+  })
+
+  const createColumns = (variables: any) => {
+    variables.columns = [
+      {
+        title: '#',
+        key: 'index',
+        render: (row: any, index: number) => index + 1,
+        ...COLUMN_WIDTH_CONFIG['index']
+      },
+      {
+        title: t('project.task.task_name'),
+        key: 'name',
+        ...COLUMN_WIDTH_CONFIG['name']
+      },
+      {
+        title: t('project.task.workflow_name'),
+        key: 'processDefinitionName',
+        ...COLUMN_WIDTH_CONFIG['name']
+      },
+      {
+        title: t('project.task.node_type'),
+        key: 'taskType',
+        ...COLUMN_WIDTH_CONFIG['type']
+      },
+      {
+        title: t('project.task.state'),
+        key: 'state',
+        ...COLUMN_WIDTH_CONFIG['state'],
+        render: (row: any) => renderStateCell(row.state, t)
+      },
+      {
+        title: t('project.task.executor'),
+        key: 'executorName',
+        ...COLUMN_WIDTH_CONFIG['name']
+      },
+      {
+        title: t('project.task.host'),
+        key: 'host',
+        ...COLUMN_WIDTH_CONFIG['name'],
+        render: (row: any) => row.host || '-'
+      },
+      {
+        title: t('project.task.app_id'),
+        key: 'applicationID',
+        ...COLUMN_WIDTH_CONFIG['name'],
+        render: (row: any) => row.applicationID || '-'
+      },
+      {
+        title: t('project.task.dry_run_flag'),
+        key: 'dryRun',
+        ...COLUMN_WIDTH_CONFIG['dryRun'],
+        render: (row: any) => (row.dryRun === 1 ? 'YES' : 'NO')
+      },
+      {
+        title: t('project.task.start_time'),
+        ...COLUMN_WIDTH_CONFIG['time'],
+        key: 'startTime',
+        render: (row: any) => renderTableTime(row.startTime)
+      },
+      {
+        title: t('project.task.end_time'),
+        ...COLUMN_WIDTH_CONFIG['time'],
+        key: 'endTime',
+        render: (row: any) => renderTableTime(row.endTime)
+      },
+      {
+        title: t('project.task.duration'),
+        key: 'duration',
+        ...COLUMN_WIDTH_CONFIG['duration'],
+        render: (row: any) => h('span', null, row.duration ? row.duration : '-')
+      },
+      {
+        title: t('project.task.operation'),
+        key: 'operation',
+        ...COLUMN_WIDTH_CONFIG['operation'](5),
+        render(row: any) {
+          return h(NSpace, null, {
+            default: () => [
+              h(
+                NTooltip,
+                {},
+                {
+                  trigger: () =>
+                    h(
+                      NButton,
+                      {
+                        tag: 'div',
+                        circle: true,
+                        type: 'info',
+                        size: 'small',
+                        onClick: () => onSavePoint(row.id)
+                      },
+                      {
+                        icon: () =>
+                          h(NIcon, null, {
+                            default: () => h(SaveOutlined)
+                          })
+                      }
+                    ),
+                  default: () => t('project.task.savepoint')
+                }
+              ),
+              h(
+                NTooltip,
+                {},
+                {
+                  trigger: () =>
+                    h(
+                      NButton,
+                      {
+                        tag: 'div',
+                        circle: true,
+                        type: 'info',
+                        size: 'small',
+                        onClick: () => onExecute(row.id)
+                      },
+                      {
+                        icon: () =>
+                          h(NIcon, null, {
+                            default: () => h(StopOutlined)
+                          })
+                      }
+                    ),
+                  default: () => t('project.task.stop')
+                }
+              ),
+              h(
+                NTooltip,
+                {},
+                {
+                  trigger: () =>
+                    h(
+                      NButton,
+                      {
+                        circle: true,
+                        type: 'info',
+                        size: 'small',
+                        disabled: !row.host,
+                        onClick: () => handleLog(row)
+                      },
+                      {
+                        icon: () =>
+                          h(NIcon, null, {
+                            default: () => h(AlignLeftOutlined)
+                          })
+                      }
+                    ),
+                  default: () => t('project.task.view_log')
+                }
+              ),
+              h(
+                NTooltip,
+                {},
+                {
+                  trigger: () =>
+                    h(
+                      NButton,
+                      {
+                        circle: true,
+                        type: 'info',
+                        size: 'small',
+                        onClick: () => downloadLog(row.id)
+                      },
+                      {
+                        icon: () =>
+                          h(NIcon, null, { default: () => h(DownloadOutlined) })
+                      }
+                    ),
+                  default: () => t('project.task.download_log')
+                }
+              ),
+              h(
+                NTooltip,
+                {},
+                {
+                  trigger: () =>
+                    h(
+                      NButton,
+                      {
+                        circle: true,
+                        type: 'warning',
+                        size: 'small',
+                        disabled: true,
+                        tag: 'div'
+                      },
+                      {
+                        icon: () =>
+                          h(NIcon, null, {
+                            default: () => h(RetweetOutlined)
+                          })
+                      }
+                    ),
+                  default: () => t('project.task.jump_tip')
+                }
+              )
+            ]
+          })
+        }
+      }
+    ]
+    if (variables.tableWidth) {
+      variables.tableWidth = calculateTableWidth(variables.columns)
+    }
+  }
+
+  const handleLog = (row: any) => {
+    variables.showModalRef = true
+    variables.row = row
+  }
+
+  const getTableData = () => {
+    if (variables.loadingRef) return
+    variables.loadingRef = true
+    const data = {
+      pageSize: variables.pageSize,
+      pageNo: variables.page,
+      searchVal: variables.searchVal,
+      processInstanceId: variables.processInstanceId,
+      host: variables.host,
+      stateType: variables.stateType,
+      startDate: variables.datePickerRange
+        ? format(parseTime(variables.datePickerRange[0]), 'yyyy-MM-dd HH:mm:ss')
+        : '',
+      endDate: variables.datePickerRange
+        ? format(parseTime(variables.datePickerRange[1]), 'yyyy-MM-dd HH:mm:ss')
+        : '',
+      executorName: variables.executorName,
+      processDefinitionName: variables.processDefinitionName,
+      taskExecuteType: 'STREAM' as 'BATCH' | 'STREAM'
+    } as any
+
+    queryTaskListPaging(data, { projectCode })
+      .then((res: TaskInstancesRes) => {
+        variables.tableData = [...res.totalList]
+        variables.totalPage = res.totalPage
+      })
+      .finally(() => (variables.loadingRef = false))
+  }
+
+  const onExecute = (taskId: number) => {
+    streamTaskStop(projectCode, taskId).then(() => {
+      window.$message.success(t('project.task.success'))
+      getTableData()
+    })
+  }
+
+  const onSavePoint = (taskId: number) => {
+    savePoint(projectCode, taskId).then(() => {
+      window.$message.success(t('project.task.success'))
+      getTableData()
+    })
+  }
+
+  return {
+    t,
+    variables,
+    getTableData,
+    createColumns
+  }
+}
+
+export function renderStateCell(state: ITaskState, t: Function) {
+  if (!state) return ''
+
+  const stateOption = tasksState(t)[state]
+
+  const Icon = h(
+    NIcon,
+    {
+      color: stateOption.color,
+      class: stateOption.classNames,
+      style: {
+        display: 'flex'
+      },
+      size: 20
+    },
+    () => h(stateOption.icon)
+  )
+  return h(NTooltip, null, {
+    trigger: () => {
+      if (!stateOption.isSpin) return Icon
+      return h(NSpin, { size: 20 }, { icon: () => Icon })
+    },
+    default: () => stateOption.desc
+  })
+}
diff --git a/dolphinscheduler-ui/src/views/projects/workflow/components/dag/dag.module.scss b/dolphinscheduler-ui/src/views/projects/workflow/components/dag/dag.module.scss
index bdfb9e3ac4..6b8d7e6de7 100644
--- a/dolphinscheduler-ui/src/views/projects/workflow/components/dag/dag.module.scss
+++ b/dolphinscheduler-ui/src/views/projects/workflow/components/dag/dag.module.scss
@@ -116,7 +116,8 @@ $bgLight: #ffffff;
     &.icon-sql {
       background-image: url('/images/task-icons/sql.png');
     }
-    &.icon-flink {
+    &.icon-flink,
+    &.icon-flink_stream {
       background-image: url('/images/task-icons/flink.png');
     }
     &.icon-mr {
@@ -201,7 +202,8 @@ $bgLight: #ffffff;
       &.icon-sql {
         background-image: url('/images/task-icons/sql_hover.png');
       }
-      &.icon-flink {
+      &.icon-flink,
+      &.icon-flink_stream {
         background-image: url('/images/task-icons/flink_hover.png');
       }
       &.icon-mr {
diff --git a/dolphinscheduler-ui/src/views/projects/workflow/components/dag/use-canvas-init.ts b/dolphinscheduler-ui/src/views/projects/workflow/components/dag/use-canvas-init.ts
index bc7a809b19..fca224940f 100644
--- a/dolphinscheduler-ui/src/views/projects/workflow/components/dag/use-canvas-init.ts
+++ b/dolphinscheduler-ui/src/views/projects/workflow/components/dag/use-canvas-init.ts
@@ -100,7 +100,9 @@ export function useCanvasInit(options: Options) {
         // Whether all available ports or nodes are highlighted when you drag the edge
         highlight: true,
         createEdge() {
-          return graph.value?.createEdge({ shape: X6_EDGE_NAME })
+          return graph.value?.createEdge({
+            shape: X6_EDGE_NAME
+          })
         },
         validateConnection(data) {
           const { sourceCell, targetCell } = data
@@ -125,6 +127,20 @@ export function useCanvasInit(options: Options) {
             })
           }
 
+          return true
+        },
+        validateEdge({ edge }) {
+          const sourceData = edge.getSourceNode()?.getData()
+          const targetData = edge.getTargetNode()?.getData()
+          edge?.setAttrs({
+            line: {
+              strokeDasharray:
+                sourceData.taskExecuteType === 'STREAM' ||
+                targetData.taskExecuteType === 'STREAM'
+                  ? '5 5'
+                  : 'none'
+            }
+          })
           return true
         }
       },
diff --git a/dolphinscheduler-ui/src/views/projects/workflow/components/dag/use-cell-active.ts b/dolphinscheduler-ui/src/views/projects/workflow/components/dag/use-cell-active.ts
index 3a12831386..d6b3c8ffbc 100644
--- a/dolphinscheduler-ui/src/views/projects/workflow/components/dag/use-cell-active.ts
+++ b/dolphinscheduler-ui/src/views/projects/workflow/components/dag/use-cell-active.ts
@@ -97,9 +97,11 @@ export function useCellActive(options: Options) {
     let portAttrs = null
 
     if (isHover || isSelected) {
-      img = `${
-        import.meta.env.BASE_URL
-      }images/task-icons/${node.data.taskType.toLocaleLowerCase()}_hover.png`
+      img = `${import.meta.env.BASE_URL}images/task-icons/${(node.data
+        .taskType !== 'FLINK_STREAM'
+        ? node.data.taskType
+        : 'FLINK'
+      ).toLocaleLowerCase()}_hover.png`
       if (isHover) {
         nodeAttrs = nodeHover
         portAttrs = _.merge(portDefault, portHover)
@@ -108,9 +110,11 @@ export function useCellActive(options: Options) {
         portAttrs = _.merge(portDefault, portSelected)
       }
     } else {
-      img = `${
-        import.meta.env.BASE_URL
-      }images/task-icons/${node.data.taskType.toLocaleLowerCase()}.png`
+      img = `${import.meta.env.BASE_URL}images/task-icons/${(node.data
+        .taskType !== 'FLINK_STREAM'
+        ? node.data.taskType
+        : 'FLINK'
+      ).toLocaleLowerCase()}.png`
       nodeAttrs = NODE.attrs
       portAttrs = portDefault
     }
diff --git a/dolphinscheduler-ui/src/views/projects/workflow/components/dag/use-cell-update.ts b/dolphinscheduler-ui/src/views/projects/workflow/components/dag/use-cell-update.ts
index f62c09a8e1..1fa614c3d6 100644
--- a/dolphinscheduler-ui/src/views/projects/workflow/components/dag/use-cell-update.ts
+++ b/dolphinscheduler-ui/src/views/projects/workflow/components/dag/use-cell-update.ts
@@ -59,7 +59,7 @@ export function useCellUpdate(options: Options) {
    */
   function addNode(
     id: string,
-    type: string,
+    type: TaskType,
     name: string,
     flag: string,
     coordinate: Coordinate = { x: 100, y: 100 }
diff --git a/dolphinscheduler-ui/src/views/projects/workflow/components/dag/use-custom-cell-builder.ts b/dolphinscheduler-ui/src/views/projects/workflow/components/dag/use-custom-cell-builder.ts
index 0579ad696b..52e2a4c988 100644
--- a/dolphinscheduler-ui/src/views/projects/workflow/components/dag/use-custom-cell-builder.ts
+++ b/dolphinscheduler-ui/src/views/projects/workflow/components/dag/use-custom-cell-builder.ts
@@ -18,6 +18,10 @@
 import type { Node, Edge } from '@antv/x6'
 import { X6_NODE_NAME, X6_EDGE_NAME } from './dag-config'
 import utils from '@/utils'
+import {
+  TASK_TYPES_MAP,
+  TaskType
+} from '@/views/projects/task/constants/task-type'
 import { WorkflowDefinition, Coordinate } from './types'
 
 export function useCustomCellBuilder() {
@@ -42,7 +46,8 @@ export function useCustomCellBuilder() {
   function buildEdge(
     sourceId: string,
     targetId: string,
-    label = ''
+    label = '',
+    isStream = false
   ): Edge.Metadata {
     return {
       shape: X6_EDGE_NAME,
@@ -52,7 +57,12 @@ export function useCustomCellBuilder() {
       target: {
         cell: targetId
       },
-      labels: label ? [label] : undefined
+      labels: label ? [label] : undefined,
+      attrs: {
+        line: {
+          strokeDasharray: isStream ? '5 5' : 'none'
+        }
+      }
     }
   }
 
@@ -64,7 +74,7 @@ export function useCustomCellBuilder() {
    */
   function buildNode(
     id: string,
-    type: string,
+    type: TaskType,
     taskName: string,
     flag: string,
     coordinate: Coordinate = { x: 100, y: 100 }
@@ -78,14 +88,18 @@ export function useCustomCellBuilder() {
       data: {
         taskType: type,
         taskName: taskName || id,
-        flag: flag
+        flag: flag,
+        taskExecuteType: TASK_TYPES_MAP[type].taskExecuteType
       },
       attrs: {
         image: {
           // Use href instead of xlink:href, you may lose the icon when downloadPNG
           'xlink:href': `${
             import.meta.env.BASE_URL
-          }images/task-icons/${type.toLocaleLowerCase()}.png`
+          }images/task-icons/${(type !== 'FLINK_STREAM'
+            ? type
+            : 'FLINK'
+          ).toLocaleLowerCase()}.png`
         },
         title: {
           text: truncation
@@ -110,6 +124,7 @@ export function useCustomCellBuilder() {
       parseLocationStr(definition.processDefinition.locations) || []
     const tasks = definition.taskDefinitionList
     const connects = definition.processTaskRelationList
+    const taskTypeMap = {} as { [key in string]: TaskType }
 
     tasks.forEach((task) => {
       const location = locations.find((l) => l.taskCode === task.code) || {}
@@ -124,12 +139,23 @@ export function useCustomCellBuilder() {
         }
       )
       nodes.push(node)
+      taskTypeMap[String(task.code)] = task.taskType
     })
 
     connects
       .filter((r) => !!r.preTaskCode)
       .forEach((c) => {
-        const edge = buildEdge(c.preTaskCode + '', c.postTaskCode + '', c.name)
+        const isStream =
+          TASK_TYPES_MAP[taskTypeMap[c.preTaskCode]].taskExecuteType ===
+            'STREAM' ||
+          TASK_TYPES_MAP[taskTypeMap[c.postTaskCode]].taskExecuteType ===
+            'STREAM'
+        const edge = buildEdge(
+          c.preTaskCode + '',
+          c.postTaskCode + '',
+          c.name,
+          isStream
+        )
         edges.push(edge)
       })
     return {
diff --git a/dolphinscheduler-ui/src/views/projects/workflow/definition/tree/index.tsx b/dolphinscheduler-ui/src/views/projects/workflow/definition/tree/index.tsx
index 2edf0c8626..499c35ea03 100644
--- a/dolphinscheduler-ui/src/views/projects/workflow/definition/tree/index.tsx
+++ b/dolphinscheduler-ui/src/views/projects/workflow/definition/tree/index.tsx
@@ -138,7 +138,12 @@ export default defineComponent({
         color: '#d69f5b',
         image: `${import.meta.env.BASE_URL}images/task-icons/dinky.png`
       },
-      { taskType: 'DAG', color: '#bbdde9' }
+      { taskType: 'DAG', color: '#bbdde9' },
+      {
+        taskType: 'FLINK_STREAM',
+        color: '#d68f5b',
+        image: `${import.meta.env.BASE_URL}images/task-icons/flink.png`
+      }
     ])
 
     const initTaskStateMap = () => {
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskKillProcessor.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskKillProcessor.java
index 5db13f147e..f4f85277df 100644
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskKillProcessor.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskKillProcessor.java
@@ -107,6 +107,8 @@ public class TaskKillProcessor implements NettyRequestProcessor {
             return;
         }
 
+        // if processId > 0, it should call cancelApplication to cancel remote application too.
+        this.cancelApplication(taskInstanceId);
         Pair<Boolean, List<String>> result = doKill(taskExecutionContext);
 
         taskExecutionContext.setCurrentExecutionStatus(
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskSavePointProcessor.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskSavePointProcessor.java
new file mode 100644
index 0000000000..5401c66859
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskSavePointProcessor.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.server.worker.processor;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContextCacheManager;
+import org.apache.dolphinscheduler.plugin.task.api.stream.StreamTask;
+import org.apache.dolphinscheduler.remote.command.Command;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+import org.apache.dolphinscheduler.remote.command.TaskSavePointRequestCommand;
+import org.apache.dolphinscheduler.remote.command.TaskSavePointResponseCommand;
+import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
+import org.apache.dolphinscheduler.server.worker.runner.TaskExecuteThread;
+import org.apache.dolphinscheduler.server.worker.runner.WorkerManagerThread;
+
+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;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+
+/**
+ * task save point processor
+ */
+@Component
+public class TaskSavePointProcessor implements NettyRequestProcessor {
+
+    private final Logger logger = LoggerFactory.getLogger(TaskSavePointProcessor.class);
+
+    /**
+     * task execute manager
+     */
+    @Autowired
+    private WorkerManagerThread workerManager;
+
+    /**
+     * task save point process
+     *
+     * @param channel channel channel
+     * @param command command command
+     */
+    @Override
+    public void process(Channel channel, Command command) {
+        Preconditions.checkArgument(CommandType.TASK_SAVEPOINT_REQUEST == command.getType(),
+                                    String.format("invalid command type : %s", command.getType()));
+        TaskSavePointRequestCommand taskSavePointRequestCommand = JSONUtils.parseObject(command.getBody(), TaskSavePointRequestCommand.class);
+        if (taskSavePointRequestCommand == null) {
+            logger.error("task savepoint request command is null");
+            return;
+        }
+        logger.info("task savepoint command : {}", taskSavePointRequestCommand);
+
+        int taskInstanceId = taskSavePointRequestCommand.getTaskInstanceId();
+        TaskExecutionContext taskExecutionContext = TaskExecutionContextCacheManager.getByTaskInstanceId(taskInstanceId);
+        if (taskExecutionContext == null) {
+            logger.error("taskRequest cache is null, taskInstanceId: {}", taskSavePointRequestCommand.getTaskInstanceId());
+            return;
+        }
+
+        doSavePoint(taskInstanceId);
+
+        sendTaskSavePointResponseCommand(channel, taskExecutionContext);
+    }
+
+    private void sendTaskSavePointResponseCommand(Channel channel, TaskExecutionContext taskExecutionContext) {
+        TaskSavePointResponseCommand taskSavePointResponseCommand = new TaskSavePointResponseCommand();
+        taskSavePointResponseCommand.setTaskInstanceId(taskExecutionContext.getTaskInstanceId());
+        channel.writeAndFlush(taskSavePointResponseCommand.convert2Command()).addListener(new ChannelFutureListener() {
+            @Override
+            public void operationComplete(ChannelFuture future) throws Exception {
+                if (!future.isSuccess()) {
+                    logger.error("Submit kill response to master error, kill command: {}", taskSavePointResponseCommand);
+                }
+            }
+        });
+    }
+
+    protected void doSavePoint(int taskInstanceId) {
+        TaskExecuteThread taskExecuteThread = workerManager.getTaskExecuteThread(taskInstanceId);
+        if (taskExecuteThread == null) {
+            logger.warn("taskExecuteThread not found, taskInstanceId:{}", taskInstanceId);
+            return;
+        }
+        AbstractTask task = taskExecuteThread.getTask();
+        if (task == null) {
+            logger.warn("task not found, taskInstanceId:{}", taskInstanceId);
+            return;
+        }
+        if (!(task instanceof StreamTask)) {
+            logger.warn("task is not stream task");
+            return;
+        }
+        try {
+            ((StreamTask)task).savePoint();
+        } catch (Exception e) {
+            logger.error("task save point error", e);
+        }
+    }
+}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/WorkerRpcServer.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/WorkerRpcServer.java
index e599046c57..1150b9102b 100644
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/WorkerRpcServer.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/rpc/WorkerRpcServer.java
@@ -28,6 +28,7 @@ import org.apache.dolphinscheduler.server.worker.processor.TaskExecuteResultAckP
 import org.apache.dolphinscheduler.server.worker.processor.TaskExecuteRunningAckProcessor;
 import org.apache.dolphinscheduler.server.worker.processor.TaskKillProcessor;
 import org.apache.dolphinscheduler.server.worker.processor.TaskRejectAckProcessor;
+import org.apache.dolphinscheduler.server.worker.processor.TaskSavePointProcessor;
 
 import java.io.Closeable;
 
@@ -50,6 +51,9 @@ public class WorkerRpcServer implements Closeable {
     @Autowired
     private TaskRejectAckProcessor taskRejectAckProcessor;
 
+    @Autowired
+    private TaskSavePointProcessor taskSavePointProcessor;
+
     @Autowired
     private TaskExecuteRunningAckProcessor taskExecuteRunningAckProcessor;
 
@@ -78,6 +82,7 @@ public class WorkerRpcServer implements Closeable {
         this.nettyRemotingServer.registerProcessor(CommandType.TASK_EXECUTE_RESULT_ACK, taskExecuteResultAckProcessor);
         this.nettyRemotingServer.registerProcessor(CommandType.TASK_REJECT_ACK, taskRejectAckProcessor);
         this.nettyRemotingServer.registerProcessor(CommandType.PROCESS_HOST_UPDATE_REQUEST, hostUpdateProcessor);
+        this.nettyRemotingServer.registerProcessor(CommandType.TASK_SAVEPOINT_REQUEST, taskSavePointProcessor);
         // logger server
         this.nettyRemotingServer.registerProcessor(CommandType.GET_LOG_BYTES_REQUEST, loggerRequestProcessor);
         this.nettyRemotingServer.registerProcessor(CommandType.ROLL_VIEW_LOG_REQUEST, loggerRequestProcessor);