You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dolphinscheduler.apache.org by jo...@apache.org on 2020/02/14 14:00:30 UTC

[incubator-dolphinscheduler] branch dev updated: Refactor Architecture Basic modification #1658 (#1946)

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

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


The following commit(s) were added to refs/heads/dev by this push:
     new c1ee133  Refactor Architecture Basic modification #1658 (#1946)
c1ee133 is described below

commit c1ee1333924eff051fc2408480cbc10898e4021a
Author: qiaozhanwei <qi...@outlook.com>
AuthorDate: Fri Feb 14 22:00:20 2020 +0800

    Refactor Architecture Basic modification #1658 (#1946)
    
    * 1,remove dolphinscheduler-rpc module  2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925)
    
    * 1,remove dolphinscheduler-rpc module
    2,add dolphinscheduler-remote module
    3,add dolphinscheduler-service module
    4,refactor LoggerServer module
    
    * ProcessUtils modify
    
    * Refactor architecture (#1926)
    
    * move version to parent pom
    
    * move version properties to parent pom for easy management
    
    * remove freemarker dependency
    
    * delete CombinedApplicationServer
    
    * #1871 correct spelling
    
    * #1873 some updates for TaskQueueZkImpl
    
    * #1875 remove unused properties in pom
    
    * #1878
    1. remove tomcat dependency
    2. remove combined_logback.xml in api module
    3. format pom.xml for not aligning
    
    * #1885 fix api server startup failure
    1. add jsp-2.1 dependency
    2. remove jasper-runtime dependency
    
    * add stringutils ut (#1921)
    
    * add stringutils ut
    
    * Newfeature for #1675. (#1908)
    
    Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell.
    
    * Add modify user name for process definition (#1919)
    
    * class overrides equals() and should therefore also override hashCode()
    
    * #1862 add modify user in process difinition list
    
    * #1862 add pg-1.2.2 ddl.sql
    
    * modify ScriptRunnerTest
    
    * add updateProessDifinition UT
    
    * modify updateProcessDifinition UT
    
    * modify updateProcessDifinition UT
    
    * modify mysql 1.2.2 ddl.sql&dml.sql
    
    * add scope test to mysql in pom
    
    * modify pg-1.2.2 ddl.sql
    
    * refactor module
    
    * updates
    
    Co-authored-by: khadgarmage <kh...@outlook.com>
    Co-authored-by: zhukai <bo...@qq.com>
    Co-authored-by: Yelli <am...@my.com>
    
    * dolphinscheduler-common remove spring (#1931)
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * SpringApplicationContext class title add license (#1932)
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * add license (#1934)
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * Refactor architecture (#1936)
    
    * move datasource classes to dao module
    
    * fix send4LetterWord bug
    
    * LoggerServiceTest remove ProcessDao (#1944)
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * dolphinscheduler-common remove spring
    
    * LoggerServiceTest remove ProcessDao
    
    * exclude jasper-compiler in case of runtime conflict (#1938)
    
    * move datasource classes to dao module
    
    * fix send4LetterWord bug
    
    * exclude jasper-compiler in case of runtime conflict
    
    * DataAnaylysisServiceTest and ProcessDefinitionService modify
    
    * remote module add comment
    
    * OSUtilsTest modify
    
    * add finally block to close channel (#1951)
    
    * move datasource classes to dao module
    
    * fix send4LetterWord bug
    
    * exclude jasper-compiler in case of runtime conflict
    
    * add finally block to close channel
    
    Co-authored-by: Tboy <te...@yeah.net>
    Co-authored-by: khadgarmage <kh...@outlook.com>
    Co-authored-by: zhukai <bo...@qq.com>
    Co-authored-by: Yelli <am...@my.com>
---
 dolphinscheduler-api/pom.xml                       |  14 +-
 .../api/controller/ProcessInstanceController.java  |   4 +-
 .../apache/dolphinscheduler/api/log/LogClient.java | 137 -----------
 .../api/service/DataAnalysisService.java           |  10 +-
 .../api/service/DataSourceService.java             |   3 +-
 .../api/service/ExecutorService.java               |  34 +--
 .../api/service/LoggerService.java                 |  39 +--
 .../api/service/ProcessDefinitionService.java      |   8 +-
 .../api/service/ProcessInstanceService.java        |  44 ++--
 .../api/service/SchedulerService.java              |  22 +-
 .../api/service/TaskInstanceService.java           |   4 +-
 .../api/utils/FourLetterWordMain.java              |   2 +-
 .../api/utils/ZookeeperMonitor.java                |   4 +-
 .../api/service/DataAnalysisServiceTest.java       |   8 +-
 .../api/service/ExecutorService2Test.java          |  28 +--
 .../api/service/LoggerServiceTest.java             |  10 +-
 .../api/service/ProcessDefinitionServiceTest.java  |   6 +-
 dolphinscheduler-common/pom.xml                    |  25 +-
 .../common/utils/Preconditions.java                |  23 +-
 .../dolphinscheduler/common/utils/OSUtilsTest.java |   1 -
 dolphinscheduler-dao/pom.xml                       |  15 --
 .../dao/datasource}/BaseDataSource.java            |   2 +-
 .../dao/datasource}/ClickHouseDataSource.java      |   2 +-
 .../dao/datasource}/DB2ServerDataSource.java       |   2 +-
 .../dao/datasource}/DataSourceFactory.java         |   4 +-
 .../dao/datasource}/HiveDataSource.java            |   6 +-
 .../dao/datasource}/MySQLDataSource.java           |   2 +-
 .../dao/datasource}/OracleDataSource.java          |   2 +-
 .../dao/datasource}/PostgreDataSource.java         |   2 +-
 .../dao/datasource}/SQLServerDataSource.java       |   2 +-
 .../dao/datasource}/SparkDataSource.java           |   2 +-
 dolphinscheduler-remote/pom.xml                    |  44 ++++
 .../remote/NettyRemotingClient.java                | 272 +++++++++++++++++++++
 .../remote/NettyRemotingServer.java                | 220 +++++++++++++++++
 .../remote/codec/NettyDecoder.java                 | 109 +++++++++
 .../remote/codec/NettyEncoder.java                 |  52 ++++
 .../dolphinscheduler/remote/command/Command.java   | 105 ++++++++
 .../remote/command/CommandHeader.java              |  52 ++--
 .../remote/command/CommandType.java                |   1 +
 .../remote/command/ExecuteTaskRequestCommand.java  |   1 +
 .../remote/command/ExecuteTaskResponseCommand.java |   1 +
 .../dolphinscheduler/remote/command/Ping.java      |  74 ++++++
 .../dolphinscheduler/remote/command/Pong.java      |  75 ++++++
 .../command/log/GetLogBytesRequestCommand.java     |  69 ++++++
 .../command/log/GetLogBytesResponseCommand.java    |  65 +++++
 .../command/log/RollViewLogRequestCommand.java     |  97 ++++++++
 .../command/log/RollViewLogResponseCommand.java    |  64 +++++
 .../remote/command/log/ViewLogRequestCommand.java  |  66 +++++
 .../remote/command/log/ViewLogResponseCommand.java |  64 +++++
 .../remote/config/NettyClientConfig.java           |  91 +++++++
 .../remote/config/NettyServerConfig.java           | 116 +++++++++
 .../remote/exceptions/RemotingException.java       |  94 +++++++
 .../remote/handler/NettyClientHandler.java         | 174 +++++++++++++
 .../remote/handler/NettyServerHandler.java         | 173 +++++++++++++
 .../remote/processor/NettyRequestProcessor.java    |  32 +--
 .../dolphinscheduler/remote/utils/Address.java     |  96 ++++++++
 .../remote/utils/ChannelUtils.java                 |  57 +++++
 .../dolphinscheduler/remote/utils/Constants.java   |  42 ++--
 .../remote/utils/FastJsonSerializer.java           |  60 +++++
 .../apache/dolphinscheduler/remote/utils/Pair.java |  42 ++--
 .../remote/NettyRemotingClientTest.java            |  77 ++++++
 dolphinscheduler-rpc/pom.xml                       | 113 ---------
 .../src/main/proto/scheduler.proto                 | 101 --------
 dolphinscheduler-server/pom.xml                    |   2 +-
 .../server/log/LoggerRequestProcessor.java         | 179 ++++++++++++++
 .../dolphinscheduler/server/log/LoggerServer.java  |  91 +++++++
 .../server/master/MasterServer.java                |  27 +-
 .../master/runner/MasterBaseTaskExecThread.java    |  18 +-
 .../server/master/runner/MasterExecThread.java     |  55 +++--
 .../master/runner/MasterSchedulerThread.java       |  22 +-
 .../server/master/runner/MasterTaskExecThread.java |  10 +-
 .../master/runner/SubProcessTaskExecThread.java    |  16 +-
 .../server/monitor/ZKMonitorImpl.java              |   2 +-
 .../dolphinscheduler/server/rpc/LogClient.java     | 149 -----------
 .../dolphinscheduler/server/rpc/LoggerServer.java  | 238 ------------------
 .../server/utils/ProcessUtils.java                 |  15 +-
 .../server/utils/RemoveZKNode.java                 |   2 +-
 .../server/worker/WorkerServer.java                |  36 +--
 .../server/worker/runner/FetchTaskThread.java      |  29 +--
 .../server/worker/runner/TaskScheduleThread.java   |  26 +-
 .../worker/task/AbstractCommandExecutor.java       |  20 +-
 .../server/worker/task/AbstractYarnTask.java       |  12 +-
 .../server/worker/task/datax/DataxTask.java        |  22 +-
 .../worker/task/dependent/DependentExecute.java    |  16 +-
 .../worker/task/dependent/DependentTask.java       |  12 +-
 .../server/worker/task/flink/FlinkTask.java        |   2 +-
 .../server/worker/task/http/HttpTask.java          |  12 +-
 .../worker/task/processdure/ProcedureTask.java     |  16 +-
 .../server/worker/task/python/PythonTask.java      |  12 +-
 .../server/worker/task/shell/ShellTask.java        |  10 +-
 .../server/worker/task/sql/SqlTask.java            |  29 +--
 .../dolphinscheduler/server/zk/ZKMasterClient.java |  18 +-
 .../dolphinscheduler/server/zk/ZKWorkerClient.java |   2 +-
 .../server/master/MasterExecThreadTest.java        |  18 +-
 .../worker/shell/ShellCommandExecutorTest.java     |  10 +-
 .../server/worker/sql/SqlExecutorTest.java         |  10 +-
 .../server/worker/task/datax/DataxTaskTest.java    |  22 +-
 dolphinscheduler-service/pom.xml                   |  56 +++++
 .../service/bean}/SpringApplicationContext.java    |   3 +-
 .../service/log/LogClientService.java              | 166 +++++++++++++
 .../dolphinscheduler/service/log/LogPromise.java   |  81 ++++++
 .../service}/permission/PermissionCheck.java       |  36 +--
 .../service/process/ProcessService.java            |  14 +-
 .../service}/quartz/DruidConnectionProvider.java   |   3 +-
 .../service}/quartz/ProcessScheduleJob.java        |  27 +-
 .../service}/quartz/QuartzExecutors.java           |   4 +-
 .../service/quartz}/cron/AbstractCycle.java        |   4 +-
 .../service/quartz}/cron/CronUtils.java            |  10 +-
 .../service/quartz}/cron/CycleFactory.java         |   4 +-
 .../service/quartz}/cron/CycleLinks.java           |   4 +-
 .../service}/queue/ITaskQueue.java                 |   2 +-
 .../service}/queue/TaskQueueFactory.java           |   6 +-
 .../service}/queue/TaskQueueZkImpl.java            |   4 +-
 .../service}/zk/AbstractZKClient.java              |   5 +-
 .../service}/zk/DefaultEnsembleProvider.java       |   2 +-
 .../service}/zk/ZookeeperCachedOperator.java       |   7 +-
 .../service}/zk/ZookeeperConfig.java               |   2 +-
 .../service}/zk/ZookeeperOperator.java             |   4 +-
 .../src/main/resources/quartz.properties           |   2 +-
 .../src/test/java}/cron/CronUtilsTest.java         |   8 +-
 .../src/test/java}/queue/BaseTaskQueueTest.java    |   5 +-
 .../src/test/java}/queue/TaskQueueZKImplTest.java  |   2 +-
 .../src/test/java/queue}/ZKServer.java             |   2 +-
 .../src/test/java}/utils/PreconditionsTest.java    |   5 +-
 pom.xml                                            |  13 +-
 125 files changed, 3477 insertions(+), 1330 deletions(-)

diff --git a/dolphinscheduler-api/pom.xml b/dolphinscheduler-api/pom.xml
index c10f443..6440805 100644
--- a/dolphinscheduler-api/pom.xml
+++ b/dolphinscheduler-api/pom.xml
@@ -31,12 +31,6 @@
     <dependency>
       <groupId>org.apache.dolphinscheduler</groupId>
       <artifactId>dolphinscheduler-alert</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.dolphinscheduler</groupId>
-          <artifactId>dolphinscheduler-dao</artifactId>
-        </exclusion>
-      </exclusions>
     </dependency>
 
     <dependency>
@@ -129,13 +123,13 @@
     </dependency>
 
     <dependency>
-      <groupId>com.github.xiaoymin</groupId>
-      <artifactId>swagger-bootstrap-ui</artifactId>
+      <groupId>org.apache.dolphinscheduler</groupId>
+      <artifactId>dolphinscheduler-service</artifactId>
     </dependency>
 
     <dependency>
-      <groupId>org.apache.dolphinscheduler</groupId>
-      <artifactId>dolphinscheduler-rpc</artifactId>
+      <groupId>com.github.xiaoymin</groupId>
+      <artifactId>swagger-bootstrap-ui</artifactId>
     </dependency>
 
     <dependency>
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceController.java
index 150c647..80db6c8 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceController.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceController.java
@@ -22,12 +22,12 @@ import org.apache.dolphinscheduler.api.utils.Result;
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.common.enums.Flag;
-import org.apache.dolphinscheduler.common.queue.ITaskQueue;
-import org.apache.dolphinscheduler.common.queue.TaskQueueFactory;
 import org.apache.dolphinscheduler.common.utils.ParameterUtils;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
 import org.apache.dolphinscheduler.dao.entity.User;
 import io.swagger.annotations.*;
+import org.apache.dolphinscheduler.service.queue.ITaskQueue;
+import org.apache.dolphinscheduler.service.queue.TaskQueueFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/log/LogClient.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/log/LogClient.java
deleted file mode 100644
index 3452060..0000000
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/log/LogClient.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.dolphinscheduler.api.log;
-
-import io.grpc.ManagedChannel;
-import io.grpc.ManagedChannelBuilder;
-import io.grpc.StatusRuntimeException;
-import org.apache.dolphinscheduler.rpc.*;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * log client
- */
-public class LogClient {
-
-    private static final Logger logger = LoggerFactory.getLogger(LogClient.class);
-
-    private final ManagedChannel channel;
-    private final LogViewServiceGrpc.LogViewServiceBlockingStub blockingStub;
-
-    /**
-     * construct client connecting to HelloWorld server at {@code host:port}
-     *
-     * @param host host
-     * @param port port
-     */
-    public LogClient(String host, int port) {
-        this(ManagedChannelBuilder.forAddress(host, port)
-                // Channels are secure by default (via SSL/TLS). For the example we disable TLS to avoid
-                // needing certificates.
-                .usePlaintext(true));
-    }
-
-    /**
-     * construct client for accessing RouteGuide server using the existing channel
-     *
-     */
-    LogClient(ManagedChannelBuilder<?> channelBuilder) {
-        /**
-         * set max read size
-         */
-        channelBuilder.maxInboundMessageSize(Integer.MAX_VALUE);
-        channel = channelBuilder.build();
-        blockingStub = LogViewServiceGrpc.newBlockingStub(channel);
-    }
-
-    /**
-     * shutdown
-     *
-     * @throws InterruptedException InterruptedException
-     */
-    public void shutdown() throws InterruptedException {
-        channel.shutdown().awaitTermination(5, TimeUnit.SECONDS);
-    }
-
-    /**
-     * roll view log
-     *
-     * @param path path
-     * @param skipLineNum skip line number
-     * @param limit limit
-     * @return log content
-     */
-    public String rollViewLog(String path,int skipLineNum,int limit) {
-        logger.info("roll view log : path {},skipLineNum {} ,limit {}", path, skipLineNum, limit);
-        LogParameter pathParameter = LogParameter
-                .newBuilder()
-                .setPath(path)
-                .setSkipLineNum(skipLineNum)
-                .setLimit(limit)
-                .build();
-        RetStrInfo retStrInfo;
-        try {
-            retStrInfo = blockingStub.rollViewLog(pathParameter);
-            return retStrInfo.getMsg();
-        } catch (StatusRuntimeException e) {
-            logger.error("roll view log error", e);
-            return null;
-        }
-    }
-
-    /**
-     * view log
-     *
-     * @param path path
-     * @return log content
-     */
-    public String viewLog(String path) {
-        logger.info("view log path {}",path);
-        PathParameter pathParameter = PathParameter.newBuilder().setPath(path).build();
-        RetStrInfo retStrInfo;
-        try {
-            retStrInfo = blockingStub.viewLog(pathParameter);
-            return retStrInfo.getMsg();
-        } catch (StatusRuntimeException e) {
-            logger.error("view log error", e);
-            return null;
-        }
-    }
-
-    /**
-     * get log size
-     *
-     * @param path log path
-     * @return log content bytes
-     */
-    public byte[] getLogBytes(String path) {
-        logger.info("log path {}",path);
-        PathParameter pathParameter = PathParameter.newBuilder().setPath(path).build();
-        RetByteInfo retByteInfo;
-        try {
-            retByteInfo = blockingStub.getLogBytes(pathParameter);
-            return retByteInfo.getData().toByteArray();
-        } catch (StatusRuntimeException e) {
-            logger.error("log size error", e);
-            return null;
-        }
-    }
-
-}
\ No newline at end of file
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataAnalysisService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataAnalysisService.java
index b957827..bafe833 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataAnalysisService.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataAnalysisService.java
@@ -24,13 +24,13 @@ import org.apache.dolphinscheduler.api.enums.Status;
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.CommandType;
 import org.apache.dolphinscheduler.common.enums.UserType;
-import org.apache.dolphinscheduler.common.queue.ITaskQueue;
-import org.apache.dolphinscheduler.common.queue.TaskQueueFactory;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.*;
 import org.apache.dolphinscheduler.dao.mapper.*;
+import org.apache.dolphinscheduler.service.process.ProcessService;
+import org.apache.dolphinscheduler.service.queue.ITaskQueue;
+import org.apache.dolphinscheduler.service.queue.TaskQueueFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -69,7 +69,7 @@ public class DataAnalysisService extends BaseService{
     TaskInstanceMapper taskInstanceMapper;
 
     @Autowired
-    ProcessDao processDao;
+    ProcessService processService;
 
     /**
      * statistical task instance status data
@@ -296,7 +296,7 @@ public class DataAnalysisService extends BaseService{
         if(projectId !=0){
             projectIds.add(projectId);
         }else if(loginUser.getUserType() == UserType.GENERAL_USER){
-            projectIds = processDao.getProjectIdListHavePerm(loginUser.getId());
+            projectIds = processService.getProjectIdListHavePerm(loginUser.getId());
             if(projectIds.size() ==0 ){
                 projectIds.add(0);
             }
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataSourceService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataSourceService.java
index 5d33b46..f6d8903 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataSourceService.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataSourceService.java
@@ -21,10 +21,9 @@ 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.DbType;
-import org.apache.dolphinscheduler.common.enums.UserType;
-import org.apache.dolphinscheduler.common.job.db.*;
 import org.apache.dolphinscheduler.common.utils.CommonUtils;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.datasource.*;
 import org.apache.dolphinscheduler.dao.entity.DataSource;
 import org.apache.dolphinscheduler.dao.entity.Resource;
 import org.apache.dolphinscheduler.dao.entity.User;
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 257f15d..6edd48d 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
@@ -25,12 +25,12 @@ import org.apache.dolphinscheduler.common.utils.CollectionUtils;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.*;
 import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
 import org.apache.dolphinscheduler.dao.mapper.ProcessInstanceMapper;
 import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
-import org.apache.dolphinscheduler.dao.utils.cron.CronUtils;
+import org.apache.dolphinscheduler.service.process.ProcessService;
+import org.apache.dolphinscheduler.service.quartz.cron.CronUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -67,7 +67,7 @@ public class ExecutorService extends BaseService{
 
 
     @Autowired
-    private ProcessDao processDao;
+    private ProcessService processService;
 
     /**
      * execute process instance
@@ -186,13 +186,13 @@ public class ExecutorService extends BaseService{
             return checkResult;
         }
 
-        ProcessInstance processInstance = processDao.findProcessInstanceDetailById(processInstanceId);
+        ProcessInstance processInstance = processService.findProcessInstanceDetailById(processInstanceId);
         if (processInstance == null) {
             putMsg(result, Status.PROCESS_INSTANCE_NOT_EXIST, processInstanceId);
             return result;
         }
 
-        ProcessDefinition processDefinition = processDao.findProcessDefineById(processInstance.getProcessDefinitionId());
+        ProcessDefinition processDefinition = processService.findProcessDefineById(processInstance.getProcessDefinitionId());
         if(executeType != ExecuteType.STOP && executeType != ExecuteType.PAUSE){
             result = checkProcessDefinitionValid(processDefinition, processInstance.getProcessDefinitionId());
             if (result.get(Constants.STATUS) != Status.SUCCESS) {
@@ -227,7 +227,7 @@ public class ExecutorService extends BaseService{
                 } else {
                     processInstance.setCommandType(CommandType.STOP);
                     processInstance.addHistoryCmd(CommandType.STOP);
-                    processDao.updateProcessInstance(processInstance);
+                    processService.updateProcessInstance(processInstance);
                     result = updateProcessInstanceState(processInstanceId, ExecutionStatus.READY_STOP);
                 }
                 break;
@@ -237,7 +237,7 @@ public class ExecutorService extends BaseService{
                 } else {
                     processInstance.setCommandType(CommandType.PAUSE);
                     processInstance.addHistoryCmd(CommandType.PAUSE);
-                    processDao.updateProcessInstance(processInstance);
+                    processService.updateProcessInstance(processInstance);
                     result = updateProcessInstanceState(processInstanceId, ExecutionStatus.READY_PAUSE);
                 }
                 break;
@@ -257,7 +257,7 @@ public class ExecutorService extends BaseService{
      */
     private boolean checkTenantSuitable(ProcessDefinition processDefinition) {
         // checkTenantExists();
-        Tenant tenant = processDao.getTenantForProcess(processDefinition.getTenantId(),
+        Tenant tenant = processService.getTenantForProcess(processDefinition.getTenantId(),
                 processDefinition.getUserId());
         if(tenant == null){
             return false;
@@ -319,7 +319,7 @@ public class ExecutorService extends BaseService{
     private Map<String, Object> updateProcessInstanceState(Integer processInstanceId, ExecutionStatus executionStatus) {
         Map<String, Object> result = new HashMap<>(5);
 
-        int update = processDao.updateProcessInstanceState(processInstanceId, executionStatus);
+        int update = processService.updateProcessInstanceState(processInstanceId, executionStatus);
         if (update > 0) {
             putMsg(result, Status.SUCCESS);
         } else {
@@ -347,12 +347,12 @@ public class ExecutorService extends BaseService{
                 CMDPARAM_RECOVER_PROCESS_ID_STRING, instanceId));
         command.setExecutorId(loginUser.getId());
 
-        if(!processDao.verifyIsNeedCreateCommand(command)){
+        if(!processService.verifyIsNeedCreateCommand(command)){
             putMsg(result, Status.PROCESS_INSTANCE_EXECUTING_COMMAND,processDefinitionId);
             return result;
         }
 
-        int create = processDao.createCommand(command);
+        int create = processService.createCommand(command);
 
         if (create > 0) {
             putMsg(result, Status.SUCCESS);
@@ -376,7 +376,7 @@ public class ExecutorService extends BaseService{
             putMsg(result,Status.REQUEST_PARAMS_NOT_VALID_ERROR,"process definition id");
         }
         List<Integer> ids = new ArrayList<>();
-        processDao.recurseFindSubProcessId(processDefineId, ids);
+        processService.recurseFindSubProcessId(processDefineId, ids);
         Integer[] idArray = ids.toArray(new Integer[ids.size()]);
         if (ids.size() > 0){
             List<ProcessDefinition> processDefinitionList;
@@ -506,9 +506,9 @@ public class ExecutorService extends BaseService{
                     cmdParam.put(CMDPARAM_COMPLEMENT_DATA_START_DATE, DateUtils.dateToString(start));
                     cmdParam.put(CMDPARAM_COMPLEMENT_DATA_END_DATE, DateUtils.dateToString(end));
                     command.setCommandParam(JSONUtils.toJson(cmdParam));
-                    return processDao.createCommand(command);
+                    return processService.createCommand(command);
                 }else if (runMode == RunMode.RUN_MODE_PARALLEL){
-                    List<Schedule> schedules = processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefineId);
+                    List<Schedule> schedules = processService.queryReleaseSchedulerListByProcessDefinitionId(processDefineId);
                     List<Date> listDate = new LinkedList<>();
                     if(!CollectionUtils.isEmpty(schedules)){
                         for (Schedule item : schedules) {
@@ -521,7 +521,7 @@ public class ExecutorService extends BaseService{
                             cmdParam.put(CMDPARAM_COMPLEMENT_DATA_START_DATE, DateUtils.dateToString(date));
                             cmdParam.put(CMDPARAM_COMPLEMENT_DATA_END_DATE, DateUtils.dateToString(date));
                             command.setCommandParam(JSONUtils.toJson(cmdParam));
-                            processDao.createCommand(command);
+                            processService.createCommand(command);
                         }
                         return listDate.size();
                     }else{
@@ -532,7 +532,7 @@ public class ExecutorService extends BaseService{
                             cmdParam.put(CMDPARAM_COMPLEMENT_DATA_START_DATE, DateUtils.dateToString(start));
                             cmdParam.put(CMDPARAM_COMPLEMENT_DATA_END_DATE, DateUtils.dateToString(start));
                             command.setCommandParam(JSONUtils.toJson(cmdParam));
-                            processDao.createCommand(command);
+                            processService.createCommand(command);
                             start = DateUtils.getSomeDay(start, 1);
                         }
                         return runCunt;
@@ -544,7 +544,7 @@ public class ExecutorService extends BaseService{
             }
         }else{
             command.setCommandParam(JSONUtils.toJson(cmdParam));
-            return processDao.createCommand(command);
+            return processService.createCommand(command);
         }
 
         return 0;
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java
index 2587290..bff54b6 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java
@@ -17,12 +17,12 @@
 package org.apache.dolphinscheduler.api.service;
 
 import org.apache.dolphinscheduler.api.enums.Status;
-import org.apache.dolphinscheduler.api.log.LogClient;
 import org.apache.dolphinscheduler.api.utils.Result;
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.service.log.LogClientService;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -37,7 +37,7 @@ public class LoggerService {
   private static final Logger logger = LoggerFactory.getLogger(LoggerService.class);
 
   @Autowired
-  private ProcessDao processDao;
+  private ProcessService processService;
 
   /**
    * view log
@@ -49,7 +49,7 @@ public class LoggerService {
    */
   public Result queryLog(int taskInstId, int skipLineNum, int limit) {
 
-    TaskInstance taskInstance = processDao.findTaskInstanceById(taskInstId);
+    TaskInstance taskInstance = processService.findTaskInstanceById(taskInstId);
 
     if (taskInstance == null){
       return new Result(Status.TASK_INSTANCE_NOT_FOUND.getCode(), Status.TASK_INSTANCE_NOT_FOUND.getMsg());
@@ -64,11 +64,17 @@ public class LoggerService {
     Result result = new Result(Status.SUCCESS.getCode(), Status.SUCCESS.getMsg());
 
     logger.info("log host : {} , logPath : {} , logServer port : {}",host,taskInstance.getLogPath(),Constants.RPC_PORT);
-
-    LogClient logClient = new LogClient(host, Constants.RPC_PORT);
-    String log = logClient.rollViewLog(taskInstance.getLogPath(),skipLineNum,limit);
-    result.setData(log);
-    logger.info(log);
+    LogClientService logClient = null;
+    try {
+      logClient = new LogClientService(host, Constants.RPC_PORT);
+      String log = logClient.rollViewLog(taskInstance.getLogPath(),skipLineNum,limit);
+      result.setData(log);
+      logger.info(log);
+    } finally {
+      if(logClient != null){
+        logClient.close();
+      }
+    }
 
     return result;
   }
@@ -80,17 +86,20 @@ public class LoggerService {
    * @return log byte array
    */
   public byte[] getLogBytes(int taskInstId) {
-    TaskInstance taskInstance = processDao.findTaskInstanceById(taskInstId);
+    TaskInstance taskInstance = processService.findTaskInstanceById(taskInstId);
     if (taskInstance == null){
       throw new RuntimeException("task instance is null");
     }
 
     String host = taskInstance.getHost();
-    if(StringUtils.isEmpty(host)){
-      throw new RuntimeException("task instance host is null");
+    LogClientService logClient = null;
+    try {
+      logClient = new LogClientService(host, Constants.RPC_PORT);
+      return logClient.getLogBytes(taskInstance.getLogPath());
+    } finally {
+      if(logClient != null){
+        logClient.close();
+      }
     }
-
-    LogClient logClient = new LogClient(host, Constants.RPC_PORT);
-    return logClient.getLogBytes(taskInstance.getLogPath());
   }
 }
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionService.java
index e9cfe34..123dc81 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionService.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionService.java
@@ -43,9 +43,9 @@ import org.apache.dolphinscheduler.common.utils.CollectionUtils;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.*;
 import org.apache.dolphinscheduler.dao.mapper.*;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -94,7 +94,7 @@ public class ProcessDefinitionService extends BaseDAGService {
     private ScheduleMapper scheduleMapper;
 
     @Autowired
-    private ProcessDao processDao;
+    private ProcessService processService;
 
     @Autowired
     private WorkerGroupMapper workerGroupMapper;
@@ -283,7 +283,7 @@ public class ProcessDefinitionService extends BaseDAGService {
         if ((checkProcessJson.get(Constants.STATUS) != Status.SUCCESS)) {
             return checkProcessJson;
         }
-        ProcessDefinition processDefinition = processDao.findProcessDefineById(id);
+        ProcessDefinition processDefinition = processService.findProcessDefineById(id);
         if (processDefinition == null) {
             // check process definition exists
             putMsg(result, Status.PROCESS_DEFINE_NOT_EXIST, id);
@@ -296,7 +296,7 @@ public class ProcessDefinitionService extends BaseDAGService {
             putMsg(result, Status.SUCCESS);
         }
 
-        ProcessDefinition processDefine = processDao.findProcessDefineById(id);
+        ProcessDefinition processDefine = processService.findProcessDefineById(id);
         Date now = new Date();
 
         processDefine.setId(id);
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java
index 87e1a0e..2b1f04e 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java
@@ -30,15 +30,15 @@ import org.apache.dolphinscheduler.common.graph.DAG;
 import org.apache.dolphinscheduler.common.model.TaskNode;
 import org.apache.dolphinscheduler.common.model.TaskNodeRelation;
 import org.apache.dolphinscheduler.common.process.Property;
-import org.apache.dolphinscheduler.common.queue.ITaskQueue;
 import org.apache.dolphinscheduler.common.utils.*;
 import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import com.alibaba.fastjson.JSON;
 import com.baomidou.mybatisplus.core.metadata.IPage;
 import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
 import org.apache.dolphinscheduler.dao.entity.*;
 import org.apache.dolphinscheduler.dao.mapper.*;
+import org.apache.dolphinscheduler.service.process.ProcessService;
+import org.apache.dolphinscheduler.service.queue.ITaskQueue;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -72,7 +72,7 @@ public class ProcessInstanceService extends BaseDAGService {
     ProjectService projectService;
 
     @Autowired
-    ProcessDao processDao;
+    ProcessService processService;
 
     @Autowired
     ProcessInstanceMapper processInstanceMapper;
@@ -112,7 +112,7 @@ public class ProcessInstanceService extends BaseDAGService {
         if (resultEnum != Status.SUCCESS) {
             return checkResult;
         }
-        ProcessInstance processInstance = processDao.findProcessInstanceDetailById(processId);
+        ProcessInstance processInstance = processService.findProcessInstanceDetailById(processId);
         String workerGroupName = "";
         if(processInstance.getWorkerGroupId() == -1){
             workerGroupName = DEFAULT;
@@ -125,7 +125,7 @@ public class ProcessInstanceService extends BaseDAGService {
             }
         }
         processInstance.setWorkerGroupName(workerGroupName);
-        ProcessDefinition processDefinition = processDao.findProcessDefineById(processInstance.getProcessDefinitionId());
+        ProcessDefinition processDefinition = processService.findProcessDefineById(processInstance.getProcessDefinitionId());
         processInstance.setReceivers(processDefinition.getReceivers());
         processInstance.setReceiversCc(processDefinition.getReceiversCc());
         result.put(Constants.DATA_LIST, processInstance);
@@ -228,8 +228,8 @@ public class ProcessInstanceService extends BaseDAGService {
         if (resultEnum != Status.SUCCESS) {
             return checkResult;
         }
-        ProcessInstance processInstance = processDao.findProcessInstanceDetailById(processId);
-        List<TaskInstance> taskInstanceList = processDao.findValidTaskListByProcessId(processId);
+        ProcessInstance processInstance = processService.findProcessInstanceDetailById(processId);
+        List<TaskInstance> taskInstanceList = processService.findValidTaskListByProcessId(processId);
         AddDependResultForTaskList(taskInstanceList);
         Map<String, Object> resultMap = new HashMap<>();
         resultMap.put(PROCESS_INSTANCE_STATE, processInstance.getState().toString());
@@ -304,7 +304,7 @@ public class ProcessInstanceService extends BaseDAGService {
             return checkResult;
         }
 
-        TaskInstance taskInstance = processDao.findTaskInstanceById(taskId);
+        TaskInstance taskInstance = processService.findTaskInstanceById(taskId);
         if (taskInstance == null) {
             putMsg(result, Status.TASK_INSTANCE_NOT_EXISTS, taskId);
             return result;
@@ -314,7 +314,7 @@ public class ProcessInstanceService extends BaseDAGService {
             return result;
         }
 
-        ProcessInstance subWorkflowInstance = processDao.findSubProcessInstance(
+        ProcessInstance subWorkflowInstance = processService.findSubProcessInstance(
                 taskInstance.getProcessInstanceId(), taskInstance.getId());
         if (subWorkflowInstance == null) {
             putMsg(result, Status.SUB_PROCESS_INSTANCE_NOT_EXIST, taskId);
@@ -356,7 +356,7 @@ public class ProcessInstanceService extends BaseDAGService {
         }
 
         //check process instance exists
-        ProcessInstance processInstance = processDao.findProcessInstanceDetailById(processInstanceId);
+        ProcessInstance processInstance = processService.findProcessInstanceDetailById(processInstanceId);
         if (processInstance == null) {
             putMsg(result, Status.PROCESS_INSTANCE_NOT_EXIST, processInstanceId);
             return result;
@@ -380,7 +380,7 @@ public class ProcessInstanceService extends BaseDAGService {
         String globalParams = null;
         String originDefParams = null;
         int timeout = processInstance.getTimeout();
-        ProcessDefinition processDefinition = processDao.findProcessDefineById(processInstance.getProcessDefinitionId());
+        ProcessDefinition processDefinition = processService.findProcessDefineById(processInstance.getProcessDefinitionId());
         if (StringUtils.isNotEmpty(processInstanceJson)) {
             ProcessData processData = JSONUtils.parseObject(processInstanceJson, ProcessData.class);
             //check workflow json is valid
@@ -396,7 +396,7 @@ public class ProcessInstanceService extends BaseDAGService {
                     processInstance.getCmdTypeIfComplement(), schedule);
             timeout = processData.getTimeout();
             processInstance.setTimeout(timeout);
-            Tenant tenant = processDao.getTenantForProcess(processData.getTenantId(),
+            Tenant tenant = processService.getTenantForProcess(processData.getTenantId(),
                     processDefinition.getUserId());
             if(tenant != null){
                 processInstance.setTenantCode(tenant.getTenantCode());
@@ -406,7 +406,7 @@ public class ProcessInstanceService extends BaseDAGService {
         }
 //        int update = processDao.updateProcessInstance(processInstanceId, processInstanceJson,
 //                globalParams, schedule, flag, locations, connects);
-        int update = processDao.updateProcessInstance(processInstance);
+        int update = processService.updateProcessInstance(processInstance);
         int updateDefine = 1;
         if (syncDefine && StringUtils.isNotEmpty(processInstanceJson)) {
             processDefinition.setProcessDefinitionJson(processInstanceJson);
@@ -445,7 +445,7 @@ public class ProcessInstanceService extends BaseDAGService {
             return checkResult;
         }
 
-        ProcessInstance subInstance = processDao.findProcessInstanceDetailById(subId);
+        ProcessInstance subInstance = processService.findProcessInstanceDetailById(subId);
         if (subInstance == null) {
             putMsg(result, Status.PROCESS_INSTANCE_NOT_EXIST, subId);
             return result;
@@ -455,7 +455,7 @@ public class ProcessInstanceService extends BaseDAGService {
             return result;
         }
 
-        ProcessInstance parentWorkflowInstance = processDao.findParentProcessInstance(subId);
+        ProcessInstance parentWorkflowInstance = processService.findParentProcessInstance(subId);
         if (parentWorkflowInstance == null) {
             putMsg(result, Status.SUB_PROCESS_INSTANCE_NOT_EXIST);
             return result;
@@ -476,7 +476,7 @@ public class ProcessInstanceService extends BaseDAGService {
      * @return delete result code
      */
     @Transactional(rollbackFor = Exception.class)
-    public Map<String, Object> deleteProcessInstanceById(User loginUser, String projectName, Integer processInstanceId,ITaskQueue tasksQueue) {
+    public Map<String, Object> deleteProcessInstanceById(User loginUser, String projectName, Integer processInstanceId, ITaskQueue tasksQueue) {
 
         Map<String, Object> result = new HashMap<>(5);
         Project project = projectMapper.queryByName(projectName);
@@ -486,8 +486,8 @@ public class ProcessInstanceService extends BaseDAGService {
         if (resultEnum != Status.SUCCESS) {
             return checkResult;
         }
-        ProcessInstance processInstance = processDao.findProcessInstanceDetailById(processInstanceId);
-        List<TaskInstance> taskInstanceList = processDao.findValidTaskListByProcessId(processInstanceId);
+        ProcessInstance processInstance = processService.findProcessInstanceDetailById(processInstanceId);
+        List<TaskInstance> taskInstanceList = processService.findValidTaskListByProcessId(processInstanceId);
 
         if (null == processInstance) {
             putMsg(result, Status.PROCESS_INSTANCE_NOT_EXIST, processInstanceId);
@@ -512,7 +512,7 @@ public class ProcessInstanceService extends BaseDAGService {
                         .append(taskInstance.getId())
                         .append(UNDERLINE);
 
-                int taskWorkerGroupId = processDao.getTaskWorkerGroupId(taskInstance);
+                int taskWorkerGroupId = processService.getTaskWorkerGroupId(taskInstance);
                 WorkerGroup workerGroup = workerGroupMapper.selectById(taskWorkerGroupId);
 
                 if(workerGroup == null){
@@ -541,9 +541,9 @@ public class ProcessInstanceService extends BaseDAGService {
         }
 
         // delete database cascade
-        int delete = processDao.deleteWorkProcessInstanceById(processInstanceId);
-        processDao.deleteAllSubWorkProcessByParentId(processInstanceId);
-        processDao.deleteWorkProcessMapByParentId(processInstanceId);
+        int delete = processService.deleteWorkProcessInstanceById(processInstanceId);
+        processService.deleteAllSubWorkProcessByParentId(processInstanceId);
+        processService.deleteWorkProcessMapByParentId(processInstanceId);
 
         if (delete > 0) {
             putMsg(result, Status.SUCCESS);
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/SchedulerService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/SchedulerService.java
index bdce947..7212210 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/SchedulerService.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/SchedulerService.java
@@ -26,7 +26,6 @@ import org.apache.dolphinscheduler.common.model.Server;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
 import org.apache.dolphinscheduler.dao.entity.Project;
 import org.apache.dolphinscheduler.dao.entity.Schedule;
@@ -34,11 +33,12 @@ import org.apache.dolphinscheduler.dao.entity.User;
 import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
 import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
 import org.apache.dolphinscheduler.dao.mapper.ScheduleMapper;
-import org.apache.dolphinscheduler.dao.utils.cron.CronUtils;
-import org.apache.dolphinscheduler.dao.quartz.ProcessScheduleJob;
-import org.apache.dolphinscheduler.dao.quartz.QuartzExecutors;
 import com.baomidou.mybatisplus.core.metadata.IPage;
 import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
+import org.apache.dolphinscheduler.service.process.ProcessService;
+import org.apache.dolphinscheduler.service.quartz.ProcessScheduleJob;
+import org.apache.dolphinscheduler.service.quartz.QuartzExecutors;
+import org.apache.dolphinscheduler.service.quartz.cron.CronUtils;
 import org.quartz.CronExpression;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -68,7 +68,7 @@ public class SchedulerService extends BaseService {
     private MonitorService monitorService;
 
     @Autowired
-    private ProcessDao processDao;
+    private ProcessService processService;
 
     @Autowired
     private ScheduleMapper scheduleMapper;
@@ -119,7 +119,7 @@ public class SchedulerService extends BaseService {
         }
 
         // check work flow define release state
-        ProcessDefinition processDefinition = processDao.findProcessDefineById(processDefineId);
+        ProcessDefinition processDefinition = processService.findProcessDefineById(processDefineId);
         result = executorService.checkProcessDefinitionValid(processDefinition, processDefineId);
         if (result.get(Constants.STATUS) != Status.SUCCESS) {
             return result;
@@ -221,7 +221,7 @@ public class SchedulerService extends BaseService {
             return result;
         }
 
-        ProcessDefinition processDefinition = processDao.findProcessDefineById(schedule.getProcessDefinitionId());
+        ProcessDefinition processDefinition = processService.findProcessDefineById(schedule.getProcessDefinitionId());
         if (processDefinition == null) {
             putMsg(result, Status.PROCESS_DEFINE_NOT_EXIST, schedule.getProcessDefinitionId());
             return result;
@@ -321,7 +321,7 @@ public class SchedulerService extends BaseService {
             putMsg(result, Status.SCHEDULE_CRON_REALEASE_NEED_NOT_CHANGE, scheduleStatus);
             return result;
         }
-        ProcessDefinition processDefinition = processDao.findProcessDefineById(scheduleObj.getProcessDefinitionId());
+        ProcessDefinition processDefinition = processService.findProcessDefineById(scheduleObj.getProcessDefinitionId());
         if (processDefinition == null) {
             putMsg(result, Status.PROCESS_DEFINE_NOT_EXIST, scheduleObj.getProcessDefinitionId());
             return result;
@@ -338,7 +338,7 @@ public class SchedulerService extends BaseService {
             }
             // check sub process definition release state
             List<Integer> subProcessDefineIds = new ArrayList<>();
-            processDao.recurseFindSubProcessId(scheduleObj.getProcessDefinitionId(), subProcessDefineIds);
+            processService.recurseFindSubProcessId(scheduleObj.getProcessDefinitionId(), subProcessDefineIds);
             Integer[] idArray = subProcessDefineIds.toArray(new Integer[subProcessDefineIds.size()]);
             if (subProcessDefineIds.size() > 0){
                 List<ProcessDefinition> subProcessDefinitionList =
@@ -423,7 +423,7 @@ public class SchedulerService extends BaseService {
             return result;
         }
 
-        ProcessDefinition processDefinition = processDao.findProcessDefineById(processDefineId);
+        ProcessDefinition processDefinition = processService.findProcessDefineById(processDefineId);
         if (processDefinition == null) {
             putMsg(result, Status.PROCESS_DEFINE_NOT_EXIST, processDefineId);
             return result;
@@ -472,7 +472,7 @@ public class SchedulerService extends BaseService {
         logger.info("set schedule, project id: {}, scheduleId: {}", projectId, scheduleId);
 
 
-        Schedule schedule = processDao.querySchedule(scheduleId);
+        Schedule schedule = processService.querySchedule(scheduleId);
         if (schedule == null) {
             logger.warn("process schedule info not exists");
             return;
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 74afa2a..9690f5c 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
@@ -24,7 +24,6 @@ import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.common.utils.CollectionUtils;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
 import org.apache.dolphinscheduler.dao.entity.Project;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
@@ -33,6 +32,7 @@ import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
 import org.apache.dolphinscheduler.dao.mapper.TaskInstanceMapper;
 import com.baomidou.mybatisplus.core.metadata.IPage;
 import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -56,7 +56,7 @@ public class TaskInstanceService extends BaseService {
     ProjectService projectService;
 
     @Autowired
-    ProcessDao processDao;
+    ProcessService processService;
 
     @Autowired
     TaskInstanceMapper taskInstanceMapper;
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/FourLetterWordMain.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/FourLetterWordMain.java
index 1f523ed..340a389 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/FourLetterWordMain.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/FourLetterWordMain.java
@@ -17,6 +17,7 @@
 
 package org.apache.dolphinscheduler.api.utils;
 
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -91,5 +92,4 @@ public class FourLetterWordMain {
             throw new IOException("Exception while executing four letter word: " + cmd, e);
         }
     }
-
 }
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/ZookeeperMonitor.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/ZookeeperMonitor.java
index 66f57f6..f91d3bc 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/ZookeeperMonitor.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/ZookeeperMonitor.java
@@ -18,9 +18,9 @@ package org.apache.dolphinscheduler.api.utils;
 
 import org.apache.dolphinscheduler.common.enums.ZKNodeType;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
-import org.apache.dolphinscheduler.common.zk.AbstractZKClient;
 import org.apache.dolphinscheduler.common.model.Server;
 import org.apache.dolphinscheduler.dao.entity.ZookeeperRecord;
+import org.apache.dolphinscheduler.service.zk.AbstractZKClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.stereotype.Component;
@@ -34,7 +34,7 @@ import java.util.List;
  *	monitor zookeeper info
  */
 @Component
-public class ZookeeperMonitor extends AbstractZKClient{
+public class ZookeeperMonitor extends AbstractZKClient {
 
 	private static final Logger LOG = LoggerFactory.getLogger(ZookeeperMonitor.class);
 
diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/DataAnalysisServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/DataAnalysisServiceTest.java
index c7afd76..6f308e7 100644
--- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/DataAnalysisServiceTest.java
+++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/DataAnalysisServiceTest.java
@@ -21,15 +21,15 @@ import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.CommandType;
 import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.common.enums.UserType;
-import org.apache.dolphinscheduler.common.queue.ITaskQueue;
-import org.apache.dolphinscheduler.common.queue.TaskQueueFactory;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.CommandCount;
 import org.apache.dolphinscheduler.dao.entity.ExecuteStatusCount;
 import org.apache.dolphinscheduler.dao.entity.Project;
 import org.apache.dolphinscheduler.dao.entity.User;
 import org.apache.dolphinscheduler.dao.mapper.*;
+import org.apache.dolphinscheduler.service.process.ProcessService;
+import org.apache.dolphinscheduler.service.queue.ITaskQueue;
+import org.apache.dolphinscheduler.service.queue.TaskQueueFactory;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -78,7 +78,7 @@ public class DataAnalysisServiceTest {
     ITaskQueue taskQueue;
 
     @Mock
-    ProcessDao processDao;
+    ProcessService processService;
 
     private Project project;
 
diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java
index 66c7a3e..07d7477 100644
--- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java
+++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java
@@ -22,10 +22,10 @@ import org.apache.dolphinscheduler.common.enums.CommandType;
 import org.apache.dolphinscheduler.common.enums.Priority;
 import org.apache.dolphinscheduler.common.enums.ReleaseState;
 import org.apache.dolphinscheduler.common.enums.RunMode;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.*;
 import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
 import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -52,7 +52,7 @@ public class ExecutorService2Test {
     private ExecutorService executorService;
 
     @Mock
-    private ProcessDao processDao;
+    private ProcessService processService;
 
     @Mock
     private ProcessDefinitionMapper processDefinitionMapper;
@@ -100,8 +100,8 @@ public class ExecutorService2Test {
         Mockito.when(projectMapper.queryByName(projectName)).thenReturn(project);
         Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(checkProjectAndAuth());
         Mockito.when(processDefinitionMapper.selectById(processDefinitionId)).thenReturn(processDefinition);
-        Mockito.when(processDao.getTenantForProcess(tenantId, userId)).thenReturn(new Tenant());
-        Mockito.when(processDao.createCommand(any(Command.class))).thenReturn(1);
+        Mockito.when(processService.getTenantForProcess(tenantId, userId)).thenReturn(new Tenant());
+        Mockito.when(processService.createCommand(any(Command.class))).thenReturn(1);
     }
 
     /**
@@ -111,7 +111,7 @@ public class ExecutorService2Test {
     @Test
     public void testNoComplement() throws ParseException {
         try {
-            Mockito.when(processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList());
+            Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList());
             Map<String, Object> result = executorService.execProcessInstance(loginUser, projectName,
                     processDefinitionId, cronTime, CommandType.START_PROCESS,
                     null, null,
@@ -119,7 +119,7 @@ public class ExecutorService2Test {
                     "", "", RunMode.RUN_MODE_SERIAL,
                     Priority.LOW, 0, 110);
             Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
-            verify(processDao, times(1)).createCommand(any(Command.class));
+            verify(processService, times(1)).createCommand(any(Command.class));
         }catch (Exception e){
             Assert.assertTrue(false);
         }
@@ -132,7 +132,7 @@ public class ExecutorService2Test {
     @Test
     public void testDateError() throws ParseException {
         try {
-            Mockito.when(processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList());
+            Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList());
             Map<String, Object> result = executorService.execProcessInstance(loginUser, projectName,
                     processDefinitionId, "2020-01-31 23:00:00,2020-01-01 00:00:00", CommandType.COMPLEMENT_DATA,
                     null, null,
@@ -140,7 +140,7 @@ public class ExecutorService2Test {
                     "", "", RunMode.RUN_MODE_SERIAL,
                     Priority.LOW, 0, 110);
             Assert.assertEquals(Status.START_PROCESS_INSTANCE_ERROR, result.get(Constants.STATUS));
-            verify(processDao, times(0)).createCommand(any(Command.class));
+            verify(processService, times(0)).createCommand(any(Command.class));
         }catch (Exception e){
             Assert.assertTrue(false);
         }
@@ -153,7 +153,7 @@ public class ExecutorService2Test {
     @Test
     public void testSerial() throws ParseException {
         try {
-            Mockito.when(processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList());
+            Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList());
             Map<String, Object> result = executorService.execProcessInstance(loginUser, projectName,
                     processDefinitionId, cronTime, CommandType.COMPLEMENT_DATA,
                     null, null,
@@ -161,7 +161,7 @@ public class ExecutorService2Test {
                     "", "", RunMode.RUN_MODE_SERIAL,
                     Priority.LOW, 0, 110);
             Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
-            verify(processDao, times(1)).createCommand(any(Command.class));
+            verify(processService, times(1)).createCommand(any(Command.class));
         }catch (Exception e){
             Assert.assertTrue(false);
         }
@@ -174,7 +174,7 @@ public class ExecutorService2Test {
     @Test
     public void testParallelWithOutSchedule() throws ParseException {
         try{
-            Mockito.when(processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList());
+            Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList());
             Map<String, Object> result = executorService.execProcessInstance(loginUser, projectName,
                     processDefinitionId, cronTime, CommandType.COMPLEMENT_DATA,
                     null, null,
@@ -182,7 +182,7 @@ public class ExecutorService2Test {
                     "", "", RunMode.RUN_MODE_PARALLEL,
                     Priority.LOW, 0, 110);
             Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
-            verify(processDao, times(31)).createCommand(any(Command.class));
+            verify(processService, times(31)).createCommand(any(Command.class));
         }catch (Exception e){
             Assert.assertTrue(false);
         }
@@ -195,7 +195,7 @@ public class ExecutorService2Test {
     @Test
     public void testParallelWithSchedule() throws ParseException {
         try{
-            Mockito.when(processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(oneSchedulerList());
+            Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(oneSchedulerList());
             Map<String, Object> result = executorService.execProcessInstance(loginUser, projectName,
                     processDefinitionId, cronTime, CommandType.COMPLEMENT_DATA,
                     null, null,
@@ -203,7 +203,7 @@ public class ExecutorService2Test {
                     "", "", RunMode.RUN_MODE_PARALLEL,
                     Priority.LOW, 0, 110);
             Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
-            verify(processDao, times(15)).createCommand(any(Command.class));
+            verify(processService, times(15)).createCommand(any(Command.class));
         }catch (Exception e){
             Assert.assertTrue(false);
         }
diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/LoggerServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/LoggerServiceTest.java
index c6ab6f8..2057157 100644
--- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/LoggerServiceTest.java
+++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/LoggerServiceTest.java
@@ -18,8 +18,8 @@ package org.apache.dolphinscheduler.api.service;
 
 import org.apache.dolphinscheduler.api.enums.Status;
 import org.apache.dolphinscheduler.api.utils.Result;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -40,14 +40,14 @@ public class LoggerServiceTest {
     @InjectMocks
     private LoggerService loggerService;
     @Mock
-    private ProcessDao processDao;
+    private ProcessService processService;
 
 
     @Test
     public void testQueryDataSourceList(){
 
         TaskInstance taskInstance = new TaskInstance();
-        Mockito.when(processDao.findTaskInstanceById(1)).thenReturn(taskInstance);
+        Mockito.when(processService.findTaskInstanceById(1)).thenReturn(taskInstance);
         Result result = loggerService.queryLog(2,1,1);
         //TASK_INSTANCE_NOT_FOUND
         Assert.assertEquals(Status.TASK_INSTANCE_NOT_FOUND.getCode(),result.getCode().intValue());
@@ -59,7 +59,7 @@ public class LoggerServiceTest {
         //SUCCESS
         taskInstance.setHost("127.0.0.1");
         taskInstance.setLogPath("/temp/log");
-        Mockito.when(processDao.findTaskInstanceById(1)).thenReturn(taskInstance);
+        Mockito.when(processService.findTaskInstanceById(1)).thenReturn(taskInstance);
         result = loggerService.queryLog(1,1,1);
         Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue());
     }
@@ -68,7 +68,7 @@ public class LoggerServiceTest {
     public void testGetLogBytes(){
 
         TaskInstance taskInstance = new TaskInstance();
-        Mockito.when(processDao.findTaskInstanceById(1)).thenReturn(taskInstance);
+        Mockito.when(processService.findTaskInstanceById(1)).thenReturn(taskInstance);
 
         //task instance is null
         try{
diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java
index 51b440b..aa9f5fe 100644
--- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java
+++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java
@@ -28,9 +28,9 @@ import org.apache.dolphinscheduler.common.enums.*;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
 import org.apache.dolphinscheduler.common.utils.FileUtils;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.*;
 import org.apache.dolphinscheduler.dao.mapper.*;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.apache.http.entity.ContentType;
 import org.json.JSONException;
 import org.junit.Assert;
@@ -80,7 +80,7 @@ public class ProcessDefinitionServiceTest {
     private WorkerGroupMapper workerGroupMapper;
 
     @Mock
-    private ProcessDao processDao;
+    private ProcessService processService;
 
     private String sqlDependentJson = "{\"globalParams\":[]," +
             "\"tasks\":[{\"type\":\"SQL\",\"id\":\"tasks-27297\",\"name\":\"sql\"," +
@@ -584,7 +584,7 @@ public class ProcessDefinitionServiceTest {
 
         Mockito.when(projectMapper.queryByName(projectName)).thenReturn(getProject(projectName));
         Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result);
-        Mockito.when(processDao.findProcessDefineById(1)).thenReturn(getProcessDefinition());
+        Mockito.when(processService.findProcessDefineById(1)).thenReturn(getProcessDefinition());
 
         Map<String, Object> updateResult = processDefinitionService.updateProcessDefinition(loginUser, projectName, 1, "test",
                 sqlDependentJson, "", "", "");
diff --git a/dolphinscheduler-common/pom.xml b/dolphinscheduler-common/pom.xml
index bd2448e..2ef61d7 100644
--- a/dolphinscheduler-common/pom.xml
+++ b/dolphinscheduler-common/pom.xml
@@ -85,21 +85,7 @@
 			<groupId>com.fasterxml.jackson.core</groupId>
 			<artifactId>jackson-databind</artifactId>
 		</dependency>
-		<dependency>
-			<groupId>org.apache.curator</groupId>
-			<artifactId>curator-client</artifactId>
-			<version>${curator.version}</version>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j-1.2-api</groupId>
-					<artifactId>org.apache.logging.log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>io.netty</groupId>
-					<artifactId>netty</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
+
 		<dependency>
 			<groupId>org.apache.commons</groupId>
 			<artifactId>commons-collections4</artifactId>
@@ -548,6 +534,10 @@
 					<artifactId>log4j-web</artifactId>
 					<groupId>org.apache.logging.log4j</groupId>
 				</exclusion>
+				<exclusion>
+					<artifactId>jasper-compiler</artifactId>
+					<groupId>tomcat</groupId>
+				</exclusion>
 			</exclusions>
 		</dependency>
 
@@ -602,11 +592,6 @@
         </dependency>
 
 		<dependency>
-			<groupId>org.springframework</groupId>
-			<artifactId>spring-context</artifactId>
-		</dependency>
-
-		<dependency>
 			<groupId>org.codehaus.janino</groupId>
 			<artifactId>janino</artifactId>
 			<version>${codehaus.janino.version}</version>
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/Preconditions.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/Preconditions.java
index 92337f5..e59cbd1 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/Preconditions.java
+++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/Preconditions.java
@@ -16,7 +16,6 @@
  */
 package org.apache.dolphinscheduler.common.utils;
 
-import org.springframework.lang.Nullable;
 
 /**
  * A collection of static utility methods to validate input.
@@ -57,7 +56,7 @@ public final class Preconditions {
      *
      * @throws NullPointerException Thrown, if the passed reference was null.
      */
-    public static <T> T checkNotNull(T reference, @Nullable String errorMessage) {
+    public static <T> T checkNotNull(T reference,  String errorMessage) {
         if (reference == null) {
             throw new NullPointerException(String.valueOf(errorMessage));
         }
@@ -84,8 +83,8 @@ public final class Preconditions {
      * @throws NullPointerException Thrown, if the passed reference was null.
      */
     public static <T> T checkNotNull(T reference,
-                                     @Nullable String errorMessageTemplate,
-                                     @Nullable Object... errorMessageArgs) {
+                                      String errorMessageTemplate,
+                                      Object... errorMessageArgs) {
 
         if (reference == null) {
             throw new NullPointerException(format(errorMessageTemplate, errorMessageArgs));
@@ -121,7 +120,7 @@ public final class Preconditions {
      *
      * @throws IllegalArgumentException Thrown, if the condition is violated.
      */
-    public static void checkArgument(boolean condition, @Nullable Object errorMessage) {
+    public static void checkArgument(boolean condition,  Object errorMessage) {
         if (!condition) {
             throw new IllegalArgumentException(String.valueOf(errorMessage));
         }
@@ -141,8 +140,8 @@ public final class Preconditions {
      * @throws IllegalArgumentException Thrown, if the condition is violated.
      */
     public static void checkArgument(boolean condition,
-                                     @Nullable String errorMessageTemplate,
-                                     @Nullable Object... errorMessageArgs) {
+                                      String errorMessageTemplate,
+                                      Object... errorMessageArgs) {
 
         if (!condition) {
             throw new IllegalArgumentException(format(errorMessageTemplate, errorMessageArgs));
@@ -177,7 +176,7 @@ public final class Preconditions {
      *
      * @throws IllegalStateException Thrown, if the condition is violated.
      */
-    public static void checkState(boolean condition, @Nullable Object errorMessage) {
+    public static void checkState(boolean condition,  Object errorMessage) {
         if (!condition) {
             throw new IllegalStateException(String.valueOf(errorMessage));
         }
@@ -197,8 +196,8 @@ public final class Preconditions {
      * @throws IllegalStateException Thrown, if the condition is violated.
      */
     public static void checkState(boolean condition,
-                                  @Nullable String errorMessageTemplate,
-                                  @Nullable Object... errorMessageArgs) {
+                                   String errorMessageTemplate,
+                                   Object... errorMessageArgs) {
 
         if (!condition) {
             throw new IllegalStateException(format(errorMessageTemplate, errorMessageArgs));
@@ -231,7 +230,7 @@ public final class Preconditions {
      * @throws IllegalArgumentException Thrown, if size is negative.
      * @throws IndexOutOfBoundsException Thrown, if the index negative or greater than or equal to size
      */
-    public static void checkElementIndex(int index, int size, @Nullable String errorMessage) {
+    public static void checkElementIndex(int index, int size,  String errorMessage) {
         checkArgument(size >= 0, "Size was negative.");
         if (index < 0 || index >= size) {
             throw new IndexOutOfBoundsException(String.valueOf(errorMessage) + " Index: " + index + ", Size: " + size);
@@ -248,7 +247,7 @@ public final class Preconditions {
      *
      * <p>This method is taken quasi verbatim from the Guava Preconditions class.
      */
-    private static String format(@Nullable String template, @Nullable Object... args) {
+    private static String format( String template,  Object... args) {
         final int numArgs = args == null ? 0 : args.length;
         template = String.valueOf(template); // null -> "null"
 
diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java
index 5b23847..3d51aa8 100644
--- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java
+++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java
@@ -19,7 +19,6 @@ package org.apache.dolphinscheduler.common.utils;
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.dolphinscheduler.common.Constants;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
diff --git a/dolphinscheduler-dao/pom.xml b/dolphinscheduler-dao/pom.xml
index b3b22c1..20d1941 100644
--- a/dolphinscheduler-dao/pom.xml
+++ b/dolphinscheduler-dao/pom.xml
@@ -116,21 +116,6 @@
 			<artifactId>cron-utils</artifactId>
 		</dependency>
 
-		<dependency>
-			<groupId>org.quartz-scheduler</groupId>
-			<artifactId>quartz</artifactId>
-			<exclusions>
-				<exclusion>
-					<artifactId>c3p0</artifactId>
-					<groupId>c3p0</groupId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.quartz-scheduler</groupId>
-			<artifactId>quartz-jobs</artifactId>
-		</dependency>
         <dependency>
             <groupId>commons-configuration</groupId>
             <artifactId>commons-configuration</artifactId>
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/BaseDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/BaseDataSource.java
similarity index 97%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/BaseDataSource.java
rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/BaseDataSource.java
index 41a9b3a..a46e5aa 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/BaseDataSource.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/BaseDataSource.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.job.db;
+package org.apache.dolphinscheduler.dao.datasource;
 
 /**
  * data source base class
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/ClickHouseDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/ClickHouseDataSource.java
similarity index 97%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/ClickHouseDataSource.java
rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/ClickHouseDataSource.java
index 48550c3..e159f81 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/ClickHouseDataSource.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/ClickHouseDataSource.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.job.db;
+package org.apache.dolphinscheduler.dao.datasource;
 
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DB2ServerDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DB2ServerDataSource.java
similarity index 97%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DB2ServerDataSource.java
rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DB2ServerDataSource.java
index 44ee200..3c2366b 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DB2ServerDataSource.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DB2ServerDataSource.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.job.db;
+package org.apache.dolphinscheduler.dao.datasource;
 
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DataSourceFactory.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DataSourceFactory.java
similarity index 98%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DataSourceFactory.java
rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DataSourceFactory.java
index 48ec319..9571f9c 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DataSourceFactory.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DataSourceFactory.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.job.db;
+package org.apache.dolphinscheduler.dao.datasource;
 
+import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.DbType;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
-import org.apache.dolphinscheduler.common.Constants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/HiveDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/HiveDataSource.java
similarity index 93%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/HiveDataSource.java
rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/HiveDataSource.java
index cddedd1..0a8f527 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/HiveDataSource.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/HiveDataSource.java
@@ -14,14 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.job.db;
+package org.apache.dolphinscheduler.dao.datasource;
 
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
 
 /**
  * data source of hive
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/MySQLDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/MySQLDataSource.java
similarity index 97%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/MySQLDataSource.java
rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/MySQLDataSource.java
index fa149e6..6e2fbe3 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/MySQLDataSource.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/MySQLDataSource.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.job.db;
+package org.apache.dolphinscheduler.dao.datasource;
 
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/OracleDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/OracleDataSource.java
similarity index 97%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/OracleDataSource.java
rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/OracleDataSource.java
index c3dc3a9..cefaf87 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/OracleDataSource.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/OracleDataSource.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.job.db;
+package org.apache.dolphinscheduler.dao.datasource;
 
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/PostgreDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/PostgreDataSource.java
similarity index 97%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/PostgreDataSource.java
rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/PostgreDataSource.java
index 4989e76..176cba2 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/PostgreDataSource.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/PostgreDataSource.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.job.db;
+package org.apache.dolphinscheduler.dao.datasource;
 
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SQLServerDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SQLServerDataSource.java
similarity index 97%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SQLServerDataSource.java
rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SQLServerDataSource.java
index 8554992..07770c0 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SQLServerDataSource.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SQLServerDataSource.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.job.db;
+package org.apache.dolphinscheduler.dao.datasource;
 
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SparkDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SparkDataSource.java
similarity index 97%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SparkDataSource.java
rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SparkDataSource.java
index 5d10c63..81a5ac6 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SparkDataSource.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SparkDataSource.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.job.db;
+package org.apache.dolphinscheduler.dao.datasource;
 
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
diff --git a/dolphinscheduler-remote/pom.xml b/dolphinscheduler-remote/pom.xml
new file mode 100644
index 0000000..b67b033
--- /dev/null
+++ b/dolphinscheduler-remote/pom.xml
@@ -0,0 +1,44 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<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</artifactId>
+        <groupId>org.apache.dolphinscheduler</groupId>
+        <version>1.2.1-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>dolphinscheduler-remote</artifactId>
+
+    <name>dolphinscheduler-remote</name>
+    <!-- FIXME change it to the project's website -->
+    <url>http://www.example.com</url>
+
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <maven.compiler.source>1.7</maven.compiler.source>
+        <maven.compiler.target>1.7</maven.compiler.target>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-all</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.alibaba</groupId>
+            <artifactId>fastjson</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+</project>
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java
new file mode 100644
index 0000000..df0c13a
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java
@@ -0,0 +1,272 @@
+/*
+ * 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;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.*;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.dolphinscheduler.remote.codec.NettyDecoder;
+import org.apache.dolphinscheduler.remote.codec.NettyEncoder;
+import org.apache.dolphinscheduler.remote.command.Command;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+import org.apache.dolphinscheduler.remote.config.NettyClientConfig;
+import org.apache.dolphinscheduler.remote.exceptions.RemotingException;
+import org.apache.dolphinscheduler.remote.handler.NettyClientHandler;
+import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
+import org.apache.dolphinscheduler.remote.utils.Address;
+import org.apache.dolphinscheduler.remote.utils.Constants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetSocketAddress;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ *  remoting netty client
+ */
+public class NettyRemotingClient {
+
+    private final Logger logger = LoggerFactory.getLogger(NettyRemotingClient.class);
+
+    /**
+     *  bootstrap
+     */
+    private final Bootstrap bootstrap = new Bootstrap();
+
+    /**
+     *  encoder
+     */
+    private final NettyEncoder encoder = new NettyEncoder();
+
+    /**
+     *  channels
+     */
+    private final ConcurrentHashMap<Address, Channel> channels = new ConcurrentHashMap();
+
+    /**
+     *  default executor
+     */
+    private final ExecutorService defaultExecutor = Executors.newFixedThreadPool(Constants.CPUS);
+
+    /**
+     *  started flag
+     */
+    private final AtomicBoolean isStarted = new AtomicBoolean(false);
+
+    /**
+     *  worker group
+     */
+    private final NioEventLoopGroup workerGroup;
+
+    /**
+     *  client handler
+     */
+    private final NettyClientHandler clientHandler = new NettyClientHandler(this);
+
+    /**
+     *  netty client config
+     */
+    private final NettyClientConfig clientConfig;
+
+    /**
+     *  netty client init
+     *
+     * @param clientConfig client config
+     */
+    public NettyRemotingClient(final NettyClientConfig clientConfig){
+        this.clientConfig = clientConfig;
+        this.workerGroup = new NioEventLoopGroup(clientConfig.getWorkerThreads(), new ThreadFactory() {
+            private AtomicInteger threadIndex = new AtomicInteger(0);
+
+            @Override
+            public Thread newThread(Runnable r) {
+                return new Thread(r, String.format("NettyClient_%d", this.threadIndex.incrementAndGet()));
+            }
+        });
+        this.start();
+    }
+
+    /**
+     *  netty server start
+     */
+    private void start(){
+
+        this.bootstrap
+                .group(this.workerGroup)
+                .channel(NioSocketChannel.class)
+                .option(ChannelOption.SO_KEEPALIVE, clientConfig.isSoKeepalive())
+                .option(ChannelOption.TCP_NODELAY, clientConfig.isTcpNoDelay())
+                .option(ChannelOption.SO_SNDBUF, clientConfig.getSendBufferSize())
+                .option(ChannelOption.SO_RCVBUF, clientConfig.getReceiveBufferSize())
+                .handler(new ChannelInitializer<SocketChannel>() {
+                    @Override
+                    public void initChannel(SocketChannel ch) throws Exception {
+                        ch.pipeline().addLast(
+                                new NettyDecoder(),
+                                clientHandler,
+                                encoder);
+                    }
+                });
+        isStarted.compareAndSet(false, true);
+    }
+
+    /**
+     *  register processor
+     *
+     * @param commandType command type
+     * @param processor processor
+     */
+    public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) {
+        registerProcessor(commandType, processor, null);
+    }
+
+    /**
+     * register processor
+     *
+     * @param commandType command type
+     * @param processor processor
+     * @param executor thread executor
+     */
+    public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) {
+        this.clientHandler.registerProcessor(commandType, processor, executor);
+    }
+
+    /**
+     *  send connect
+     * @param address address
+     * @param command command
+     * @throws RemotingException
+     */
+    public void send(final Address address, final Command command) throws RemotingException {
+        final Channel channel = getChannel(address);
+        if (channel == null) {
+            throw new RemotingException("network error");
+        }
+        try {
+            channel.writeAndFlush(command).addListener(new ChannelFutureListener(){
+
+                @Override
+                public void operationComplete(ChannelFuture future) throws Exception {
+                    if(future.isSuccess()){
+                        logger.info("sent command {} to {}", command, address);
+                    } else{
+                        logger.error("send command {} to {} failed, error {}", command, address, future.cause());
+                    }
+                }
+            });
+        } catch (Exception ex) {
+            String msg = String.format("send command %s to address %s encounter error", command, address);
+            throw new RemotingException(msg, ex);
+        }
+    }
+
+    /**
+     *  get channel
+     * @param address address
+     * @return channel
+     */
+    public Channel getChannel(Address address) {
+        Channel channel = channels.get(address);
+        if(channel != null && channel.isActive()){
+            return channel;
+        }
+        return createChannel(address, true);
+    }
+
+    /**
+     *  create channel
+     * @param address address
+     * @param isSync is sync
+     * @return channel
+     */
+    public Channel createChannel(Address address, boolean isSync) {
+        ChannelFuture future;
+        try {
+            synchronized (bootstrap){
+                future = bootstrap.connect(new InetSocketAddress(address.getHost(), address.getPort()));
+            }
+            if(isSync){
+                future.sync();
+            }
+            if (future.isSuccess()) {
+                Channel channel = future.channel();
+                channels.put(address, channel);
+                return channel;
+            }
+        } catch (Exception ex) {
+            logger.info("connect to {} error  {}", address, ex);
+        }
+        return null;
+    }
+
+    /**
+     *  get default thread executor
+     * @return thread executor
+     */
+    public ExecutorService getDefaultExecutor() {
+        return defaultExecutor;
+    }
+
+    /**
+     *  close client
+     */
+    public void close() {
+        if(isStarted.compareAndSet(true, false)){
+            try {
+                closeChannels();
+                if(workerGroup != null){
+                    this.workerGroup.shutdownGracefully();
+                }
+                if(defaultExecutor != null){
+                    defaultExecutor.shutdown();
+                }
+            } catch (Exception ex) {
+                logger.error("netty client close exception", ex);
+            }
+            logger.info("netty client closed");
+        }
+    }
+
+    /**
+     *  close channel
+     */
+    private void closeChannels(){
+        for (Channel channel : this.channels.values()) {
+            channel.close();
+        }
+        this.channels.clear();
+    }
+
+    /**
+     *  remove channel
+     * @param address address
+     */
+    public void removeChannel(Address address){
+        Channel channel = this.channels.remove(address);
+        if(channel != null){
+            channel.close();
+        }
+    }
+}
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java
new file mode 100644
index 0000000..c69bf09
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java
@@ -0,0 +1,220 @@
+/*
+ * 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;
+
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.dolphinscheduler.remote.codec.NettyDecoder;
+import org.apache.dolphinscheduler.remote.codec.NettyEncoder;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+import org.apache.dolphinscheduler.remote.config.NettyServerConfig;
+import org.apache.dolphinscheduler.remote.handler.NettyServerHandler;
+import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
+import org.apache.dolphinscheduler.remote.utils.Constants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ *  remoting netty server
+ */
+public class NettyRemotingServer {
+
+    private final Logger logger = LoggerFactory.getLogger(NettyRemotingServer.class);
+
+    /**
+     *  server bootstart
+     */
+    private final ServerBootstrap serverBootstrap = new ServerBootstrap();
+
+    /**
+     *  encoder
+     */
+    private final NettyEncoder encoder = new NettyEncoder();
+
+    /**
+     *  default executor
+     */
+    private final ExecutorService defaultExecutor = Executors.newFixedThreadPool(Constants.CPUS);
+
+    /**
+     * boss group
+     */
+    private final NioEventLoopGroup bossGroup;
+
+    /**
+     *  worker group
+     */
+    private final NioEventLoopGroup workGroup;
+
+    /**
+     *  server config
+     */
+    private final NettyServerConfig serverConfig;
+
+    /**
+     *  server handler
+     */
+    private final NettyServerHandler serverHandler = new NettyServerHandler(this);
+
+    /**
+     * started flag
+     */
+    private final AtomicBoolean isStarted = new AtomicBoolean(false);
+
+    /**
+     *  server init
+     *
+     * @param serverConfig server config
+     */
+    public NettyRemotingServer(final NettyServerConfig serverConfig){
+        this.serverConfig = serverConfig;
+
+        this.bossGroup = new NioEventLoopGroup(1, new ThreadFactory() {
+            private AtomicInteger threadIndex = new AtomicInteger(0);
+
+            @Override
+            public Thread newThread(Runnable r) {
+                return new Thread(r, String.format("NettyServerBossThread_%d", this.threadIndex.incrementAndGet()));
+            }
+        });
+
+        this.workGroup = new NioEventLoopGroup(serverConfig.getWorkerThread(), new ThreadFactory() {
+            private AtomicInteger threadIndex = new AtomicInteger(0);
+
+            @Override
+            public Thread newThread(Runnable r) {
+                return new Thread(r, String.format("NettyServerWorkerThread_%d", this.threadIndex.incrementAndGet()));
+            }
+        });
+    }
+
+    /**
+     *  server start
+     */
+    public void start(){
+
+        if(this.isStarted.get()){
+            return;
+        }
+
+        this.serverBootstrap
+                .group(this.bossGroup, this.workGroup)
+                .channel(NioServerSocketChannel.class)
+                .option(ChannelOption.SO_REUSEADDR, true)
+                .option(ChannelOption.SO_BACKLOG, serverConfig.getSoBacklog())
+                .childOption(ChannelOption.SO_KEEPALIVE, serverConfig.isSoKeepalive())
+                .childOption(ChannelOption.TCP_NODELAY, serverConfig.isTcpNoDelay())
+                .childOption(ChannelOption.SO_SNDBUF, serverConfig.getSendBufferSize())
+                .childOption(ChannelOption.SO_RCVBUF, serverConfig.getReceiveBufferSize())
+                .childHandler(new ChannelInitializer<NioSocketChannel>() {
+
+                    @Override
+                    protected void initChannel(NioSocketChannel ch) throws Exception {
+                        initNettyChannel(ch);
+                    }
+                });
+
+        ChannelFuture future;
+        try {
+            future = serverBootstrap.bind(serverConfig.getListenPort()).sync();
+        } catch (Exception e) {
+            logger.error("NettyRemotingServer bind fail {}, exit", e);
+            throw new RuntimeException(String.format("NettyRemotingServer bind %s fail", serverConfig.getListenPort()));
+        }
+        if (future.isSuccess()) {
+            logger.info("NettyRemotingServer bind success at port : {}", serverConfig.getListenPort());
+        } else if (future.cause() != null) {
+            throw new RuntimeException(String.format("NettyRemotingServer bind %s fail", serverConfig.getListenPort()), future.cause());
+        } else {
+            throw new RuntimeException(String.format("NettyRemotingServer bind %s fail", serverConfig.getListenPort()));
+        }
+        //
+        isStarted.compareAndSet(false, true);
+    }
+
+    /**
+     *  init netty channel
+     * @param ch socket channel
+     * @throws Exception
+     */
+    private void initNettyChannel(NioSocketChannel ch) throws Exception{
+        ChannelPipeline pipeline = ch.pipeline();
+        pipeline.addLast("encoder", encoder);
+        pipeline.addLast("decoder", new NettyDecoder());
+        pipeline.addLast("handler", serverHandler);
+    }
+
+    /**
+     *  register processor
+     * @param commandType command type
+     * @param processor processor
+     */
+    public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) {
+        this.registerProcessor(commandType, processor, null);
+    }
+
+    /**
+     *  register processor
+     *
+     * @param commandType command type
+     * @param processor processor
+     * @param executor thread executor
+     */
+    public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) {
+        this.serverHandler.registerProcessor(commandType, processor, executor);
+    }
+
+    /**
+     *  get default thread executor
+     * @return thread executor
+     */
+    public ExecutorService getDefaultExecutor() {
+        return defaultExecutor;
+    }
+
+    public void close() {
+        if(isStarted.compareAndSet(true, false)){
+            try {
+                if(bossGroup != null){
+                    this.bossGroup.shutdownGracefully();
+                }
+                if(workGroup != null){
+                    this.workGroup.shutdownGracefully();
+                }
+                if(defaultExecutor != null){
+                    defaultExecutor.shutdown();
+                }
+            } catch (Exception ex) {
+                logger.error("netty server close exception", ex);
+            }
+            logger.info("netty server closed");
+        }
+    }
+}
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyDecoder.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyDecoder.java
new file mode 100644
index 0000000..caa4fbd
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyDecoder.java
@@ -0,0 +1,109 @@
+/*
+ * 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.codec;
+
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.ReplayingDecoder;
+import org.apache.dolphinscheduler.remote.command.Command;
+import org.apache.dolphinscheduler.remote.command.CommandHeader;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+
+import java.util.List;
+
+/**
+ *  netty decoder
+ */
+public class NettyDecoder extends ReplayingDecoder<NettyDecoder.State> {
+
+    public NettyDecoder(){
+        super(State.MAGIC);
+    }
+
+    private final CommandHeader commandHeader = new CommandHeader();
+
+    /**
+     * decode
+     *
+     * @param ctx channel handler context
+     * @param in byte buffer
+     * @param out out content
+     * @throws Exception
+     */
+    @Override
+    protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) throws Exception {
+        switch (state()){
+            case MAGIC:
+                checkMagic(in.readByte());
+                checkpoint(State.COMMAND);
+            case COMMAND:
+                commandHeader.setType(in.readByte());
+                checkpoint(State.OPAQUE);
+            case OPAQUE:
+                commandHeader.setOpaque(in.readLong());
+                checkpoint(State.BODY_LENGTH);
+            case BODY_LENGTH:
+                commandHeader.setBodyLength(in.readInt());
+                checkpoint(State.BODY);
+            case BODY:
+                byte[] body = new byte[commandHeader.getBodyLength()];
+                in.readBytes(body);
+                //
+                Command packet = new Command();
+                packet.setType(commandType(commandHeader.getType()));
+                packet.setOpaque(commandHeader.getOpaque());
+                packet.setBody(body);
+                out.add(packet);
+                //
+                checkpoint(State.MAGIC);
+        }
+    }
+
+    /**
+     *  get command type
+     * @param type type
+     * @return
+     */
+    private CommandType commandType(byte type){
+        for(CommandType ct : CommandType.values()){
+            if(ct.ordinal() == type){
+                return ct;
+            }
+        }
+        return null;
+    }
+
+    /**
+     *  check magic
+     * @param magic magic
+     */
+    private void checkMagic(byte magic) {
+        if (magic != Command.MAGIC) {
+            throw new IllegalArgumentException("illegal packet [magic]" + magic);
+        }
+    }
+
+    enum State{
+        MAGIC,
+        COMMAND,
+        OPAQUE,
+        BODY_LENGTH,
+        BODY;
+    }
+}
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyEncoder.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyEncoder.java
new file mode 100644
index 0000000..4e9836a
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyEncoder.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.remote.codec;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandler.Sharable;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.MessageToByteEncoder;
+import org.apache.dolphinscheduler.remote.command.Command;
+
+/**
+ *  netty encoder
+ */
+@Sharable
+public class NettyEncoder extends MessageToByteEncoder<Command> {
+
+    /**
+     * encode
+     *
+     * @param ctx channel handler context
+     * @param msg command
+     * @param out byte buffer
+     * @throws Exception
+     */
+    @Override
+    protected void encode(ChannelHandlerContext ctx, Command msg, ByteBuf out) throws Exception {
+        if(msg == null){
+            throw new Exception("encode msg is null");
+        }
+        out.writeByte(Command.MAGIC);
+        out.writeByte(msg.getType().ordinal());
+        out.writeLong(msg.getOpaque());
+        out.writeInt(msg.getBody().length);
+        out.writeBytes(msg.getBody());
+    }
+
+}
+
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java
new file mode 100644
index 0000000..ee95044
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java
@@ -0,0 +1,105 @@
+/*
+ * 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 java.io.Serializable;
+
+/**
+ *  receive task log request command and content fill
+ *  for netty data serializable transfer
+ */
+public class Command implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final byte MAGIC = (byte) 0xbabe;
+
+    public Command(){
+    }
+
+    public Command(long opaque){
+        this.opaque = opaque;
+    }
+
+    /**
+     * command type
+     */
+    private CommandType type;
+
+    /**
+     *  request unique identification
+     */
+    private long opaque;
+
+    /**
+     *  data body
+     */
+    private byte[] body;
+
+    public CommandType getType() {
+        return type;
+    }
+
+    public void setType(CommandType type) {
+        this.type = type;
+    }
+
+    public long getOpaque() {
+        return opaque;
+    }
+
+    public void setOpaque(long opaque) {
+        this.opaque = opaque;
+    }
+
+    public byte[] getBody() {
+        return body;
+    }
+
+    public void setBody(byte[] body) {
+        this.body = body;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + (int) (opaque ^ (opaque >>> 32));
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+        Command other = (Command) obj;
+        return opaque == other.opaque;
+    }
+
+    @Override
+    public String toString() {
+        return "Command [type=" + type + ", opaque=" + opaque + ", bodyLen=" + (body == null ? 0 : body.length) + "]";
+    }
+
+}
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/DefaultEnsembleProvider.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandHeader.java
similarity index 53%
copy from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/DefaultEnsembleProvider.java
copy to dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandHeader.java
index 0cf06c0..78948a5 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/DefaultEnsembleProvider.java
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandHeader.java
@@ -14,35 +14,51 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.zk;
+package org.apache.dolphinscheduler.remote.command;
 
-import org.apache.curator.ensemble.EnsembleProvider;
-
-import java.io.IOException;
+import java.io.Serializable;
 
 /**
- * default conf provider
+ *  command header
  */
-public class DefaultEnsembleProvider implements EnsembleProvider {
+public class CommandHeader implements Serializable {
+
+    /**
+     * type
+     */
+    private byte type;
+
+    /**
+     * request unique identification
+     */
+    private long opaque;
+
+    /**
+     *  body length
+     */
+    private int bodyLength;
 
-    private final String serverList;
+    public int getBodyLength() {
+        return bodyLength;
+    }
+
+    public void setBodyLength(int bodyLength) {
+        this.bodyLength = bodyLength;
+    }
 
-    public DefaultEnsembleProvider(String serverList){
-        this.serverList = serverList;
+    public byte getType() {
+        return type;
     }
 
-    @Override
-    public void start() throws Exception {
-        //NOP
+    public void setType(byte type) {
+        this.type = type;
     }
 
-    @Override
-    public String getConnectionString() {
-        return serverList;
+    public long getOpaque() {
+        return opaque;
     }
 
-    @Override
-    public void close() throws IOException {
-        //NOP
+    public void setOpaque(long opaque) {
+        this.opaque = opaque;
     }
 }
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
new file mode 100644
index 0000000..b1b24d3
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java
@@ -0,0 +1 @@
+/*
 * 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;


public enum CommandType {

    /**
     *  roll view log request
     */
    ROLL_VIEW_LOG_REQUEST,

    /**
     *  roll view log response
  
    */
    ROLL_VIEW_LOG_RESPONSE,

    /**
     * view whole log request
     */
    VIEW_WHOLE_LOG_REQUEST,

    /**
     * view whole log response
     */
    VIEW_WHOLE_LOG_RESPONSE,

    /**
     * get log bytes request
     */
    GET_LOG_BYTES_REQUEST,

    /**
     * get log bytes response
     */
    GET_LOG_BYTES_RESPONSE,


    WORKER_REQUEST,
    MASTER_RESPONSE,

    /**
     * execute task request
     */
    EXECUTE_TASK_REQUEST,

    /**
     * execute task response
     */
    EXECUTE_TASK_RESPONSE,

    /**
     *  ping
     */
    PING,

    /**
     *  pong
     */
    PONG;
}
\ No newline at end of file
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java
new file mode 100644
index 0000000..a582221
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java
@@ -0,0 +1 @@
+/*
 * 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.remote.utils.FastJsonSerializer;

import java.io.Serializable;
import java.util.List;
import java.util.concu
 rrent.atomic.AtomicLong;

/**
 *  execute task request command
 */
public class ExecuteTaskRequestCommand implements Serializable {

    private static final AtomicLong REQUEST = new AtomicLong(1);

    /**
     *  task id
     */
    private String taskId;

    /**
     *  attempt id
     */
    private String attemptId;

    /**
     *  application name
     */
    private String applicationName;

    /**
     *  group name
     */
    private String groupName;

    /**
     *  task name
     */
    private String taskName;

    /**
     *  connect port
     */
    private int connectorPort;

    /**
     *  description info
     */
    private String description;

    /**
     *  class name
     */
    private String className;

    /**
     *  method name
     */
    private String methodName;

    /**
     *  params
     */
    private String params;

    /**
     *  shard items
     */
    private List<Integer> shardItems;

    public List<Integer> getShardItems() {
        re
 turn shardItems;
    }

    public void setShardItems(List<Integer> shardItems) {
        this.shardItems = shardItems;
    }

    public String getParams() {
        return params;
    }

    public void setParams(String params) {
        this.params = params;
    }

    public String getTaskId() {
        return taskId;
    }

    public void setTaskId(String taskId) {
        this.taskId = taskId;
    }

    public String getApplicationName() {
        return applicationName;
    }

    public void setApplicationName(String applicationName) {
        this.applicationName = applicationName;
    }

    public String getGroupName() {
        return groupName;
    }

    public void setGroupName(String groupName) {
        this.groupName = groupName;
    }

    public String getTaskName() {
        return taskName;
    }

    public void setTaskName(String taskName) {
        this.taskName = taskName;
    }

    public int getConnectorPort() {
        return connectorPort;
    }

   
  public void setConnectorPort(int connectorPort) {
        this.connectorPort = connectorPort;
    }

    public String getDescription() {
        return description;
    }

    public void setDescription(String description) {
        this.description = description;
    }

    public String getClassName() {
        return className;
    }

    public void setClassName(String className) {
        this.className = className;
    }

    public String getMethodName() {
        return methodName;
    }

    public void setMethodName(String methodName) {
        this.methodName = methodName;
    }

    /**
     *  package request command
     *
     * @return command
     */
    public Command convert2Command(){
        Command command = new Command(REQUEST.getAndIncrement());
        command.setType(CommandType.EXECUTE_TASK_REQUEST);
        byte[] body = FastJsonSerializer.serialize(this);
        command.setBody(body);
        return command;
    }
}
\ No newline at end of file
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java
new file mode 100644
index 0000000..0268653
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java
@@ -0,0 +1 @@
+/*
 * 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.remote.utils.FastJsonSerializer;

import java.io.Serializable;
import java.util.concurrent.atomic.AtomicLong
 ;

/**
 * execute taks response command
 */
public class ExecuteTaskResponseCommand implements Serializable {

    private static final AtomicLong REQUEST = new AtomicLong(1);

    /**
     * task id
     */
    private String taskId;

    /**
     *  attempt id
     */
    private String attemptId;

    /**
     *  result info
     */
    private Object result;

    /**
     *  receive time
     */
    private long receivedTime;

    /**
     * execute count
     */
    private int executeCount;

    /**
     *  execute time
     */
    private long executeTime;

    public String getAttemptId() {
        return attemptId;
    }

    public void setAttemptId(String attemptId) {
        this.attemptId = attemptId;
    }

    public String getTaskId() {
        return taskId;
    }

    public void setTaskId(String taskId) {
        this.taskId = taskId;
    }

    public Object getResult() {
        return result;
    }

    public void setResult(Object result) {
        this.result
  = result;
    }

    public long getReceivedTime() {
        return receivedTime;
    }

    public void setReceivedTime(long receivedTime) {
        this.receivedTime = receivedTime;
    }

    public int getExecuteCount() {
        return executeCount;
    }

    public void setExecuteCount(int executeCount) {
        this.executeCount = executeCount;
    }

    public long getExecuteTime() {
        return executeTime;
    }

    public void setExecuteTime(long executeTime) {
        this.executeTime = executeTime;
    }

    /**
     * package response command
     *
     * @return command
     */
    public Command convert2Command(){
        Command command = new Command(REQUEST.getAndIncrement());
        command.setType(CommandType.EXECUTE_TASK_RESPONSE);
        byte[] body = FastJsonSerializer.serialize(this);
        command.setBody(body);
        return command;
    }
}
\ No newline at end of file
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java
new file mode 100644
index 0000000..4f32d5f
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.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.remote.command;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+
+import java.io.Serializable;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ *  ping machine
+ */
+public class Ping implements Serializable {
+
+    private static final AtomicLong ID = new AtomicLong(1);
+
+    /**
+     *  ping body
+     */
+    protected static ByteBuf EMPTY_BODY = Unpooled.EMPTY_BUFFER;
+
+    /**
+     *  request command body
+     */
+    private static byte[] EMPTY_BODY_ARRAY = new byte[0];
+
+    private static final ByteBuf PING_BUF;
+
+    static {
+        ByteBuf ping = Unpooled.buffer();
+        ping.writeByte(Command.MAGIC);
+        ping.writeByte(CommandType.PING.ordinal());
+        ping.writeLong(0);
+        ping.writeInt(0);
+        ping.writeBytes(EMPTY_BODY);
+        PING_BUF = Unpooled.unreleasableBuffer(ping).asReadOnly();
+    }
+
+    /**
+     *  ping connect
+     * @return result
+     */
+    public static ByteBuf pingContent(){
+        return PING_BUF.duplicate();
+    }
+
+    /**
+     * package ping command
+     *
+     * @return command
+     */
+    public static Command create(){
+        Command command = new Command(ID.getAndIncrement());
+        command.setType(CommandType.PING);
+        command.setBody(EMPTY_BODY_ARRAY);
+        return command;
+    }
+}
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Pong.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Pong.java
new file mode 100644
index 0000000..e52cef6
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Pong.java
@@ -0,0 +1,75 @@
+/*
+ * 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 io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+
+import java.io.Serializable;
+
+/**
+ * Pong return after ping
+ */
+public class Pong implements Serializable {
+
+    /**
+     *  pong body
+     */
+    protected static ByteBuf EMPTY_BODY = Unpooled.EMPTY_BUFFER;
+
+    /**
+     *  pong command body
+     */
+    private static byte[] EMPTY_BODY_ARRAY = new byte[0];
+
+    /**
+     *  ping byte buffer
+     */
+    private static final ByteBuf PONG_BUF;
+
+    static {
+        ByteBuf ping = Unpooled.buffer();
+        ping.writeByte(Command.MAGIC);
+        ping.writeByte(CommandType.PONG.ordinal());
+        ping.writeLong(0);
+        ping.writeInt(0);
+        ping.writeBytes(EMPTY_BODY);
+        PONG_BUF = Unpooled.unreleasableBuffer(ping).asReadOnly();
+    }
+
+    /**
+     *  ping content
+     * @return result
+     */
+    public static ByteBuf pingContent(){
+        return PONG_BUF.duplicate();
+    }
+
+    /**
+     * package pong command
+     *
+     * @param opaque request unique identification
+     * @return command
+     */
+    public static Command create(long opaque){
+        Command command = new Command(opaque);
+        command.setType(CommandType.PONG);
+        command.setBody(EMPTY_BODY_ARRAY);
+        return command;
+    }
+}
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java
new file mode 100644
index 0000000..9b064b7
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java
@@ -0,0 +1,69 @@
+/*
+ * 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.log;
+
+import org.apache.dolphinscheduler.remote.command.Command;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer;
+
+import java.io.Serializable;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ *  get log bytes request command
+ */
+public class GetLogBytesRequestCommand implements Serializable {
+
+    /**
+     *  request id
+     */
+    private static final AtomicLong REQUEST = new AtomicLong(1);
+
+    /**
+     *  log path
+     */
+    private String path;
+
+    public GetLogBytesRequestCommand() {
+    }
+
+    public GetLogBytesRequestCommand(String path) {
+        this.path = path;
+    }
+
+    public String getPath() {
+        return path;
+    }
+
+    public void setPath(String path) {
+        this.path = path;
+    }
+
+    /**
+     * package request command
+     *
+     * @return command
+     */
+    public Command convert2Command(){
+        Command command = new Command(REQUEST.getAndIncrement());
+        command.setType(CommandType.GET_LOG_BYTES_REQUEST);
+        byte[] body = FastJsonSerializer.serialize(this);
+        command.setBody(body);
+        return command;
+    }
+}
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesResponseCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesResponseCommand.java
new file mode 100644
index 0000000..deaf9b8
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesResponseCommand.java
@@ -0,0 +1,65 @@
+/*
+ * 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.log;
+
+import org.apache.dolphinscheduler.remote.command.Command;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer;
+
+import java.io.Serializable;
+
+/**
+ *  get log bytes response command
+ */
+public class GetLogBytesResponseCommand implements Serializable {
+
+    /**
+     *  log byte data
+     */
+    private byte[] data;
+
+    public GetLogBytesResponseCommand() {
+    }
+
+    public GetLogBytesResponseCommand(byte[] data) {
+        this.data = data;
+    }
+
+    public byte[] getData() {
+        return data;
+    }
+
+    public void setData(byte[] data) {
+        this.data = data;
+    }
+
+    /**
+     * package response command
+     *
+     * @param opaque request unique identification
+     * @return command
+     */
+    public Command convert2Command(long opaque){
+        Command command = new Command(opaque);
+        command.setType(CommandType.GET_LOG_BYTES_RESPONSE);
+        byte[] body = FastJsonSerializer.serialize(this);
+        command.setBody(body);
+        return command;
+    }
+
+}
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java
new file mode 100644
index 0000000..f072c47
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java
@@ -0,0 +1,97 @@
+/*
+ * 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.log;
+
+import org.apache.dolphinscheduler.remote.command.Command;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer;
+
+import java.io.Serializable;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ *  roll view log request command
+ */
+public class RollViewLogRequestCommand implements Serializable {
+
+    /**
+     *  request id
+     */
+    private static final AtomicLong REQUEST = new AtomicLong(1);
+
+    /**
+     *  log path
+     */
+    private String path;
+
+    /**
+     *  skip line number
+     */
+    private int skipLineNum;
+
+    /**
+     *  query log line number limit
+     */
+    private int limit;
+
+    public RollViewLogRequestCommand() {
+    }
+
+    public RollViewLogRequestCommand(String path, int skipLineNum, int limit) {
+        this.path = path;
+        this.skipLineNum = skipLineNum;
+        this.limit = limit;
+    }
+
+    public String getPath() {
+        return path;
+    }
+
+    public void setPath(String path) {
+        this.path = path;
+    }
+
+    public int getSkipLineNum() {
+        return skipLineNum;
+    }
+
+    public void setSkipLineNum(int skipLineNum) {
+        this.skipLineNum = skipLineNum;
+    }
+
+    public int getLimit() {
+        return limit;
+    }
+
+    public void setLimit(int limit) {
+        this.limit = limit;
+    }
+
+    /**
+     *  package request command
+     *
+     * @return command
+     */
+    public Command convert2Command(){
+        Command command = new Command(REQUEST.getAndIncrement());
+        command.setType(CommandType.ROLL_VIEW_LOG_REQUEST);
+        byte[] body = FastJsonSerializer.serialize(this);
+        command.setBody(body);
+        return command;
+    }
+}
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogResponseCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogResponseCommand.java
new file mode 100644
index 0000000..591d787
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogResponseCommand.java
@@ -0,0 +1,64 @@
+/*
+ * 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.log;
+
+import org.apache.dolphinscheduler.remote.command.Command;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer;
+
+import java.io.Serializable;
+
+/**
+ *  roll view log response command
+ */
+public class RollViewLogResponseCommand implements Serializable {
+
+    /**
+     *  response data
+     */
+    private String msg;
+
+    public RollViewLogResponseCommand() {
+    }
+
+    public RollViewLogResponseCommand(String msg) {
+        this.msg = msg;
+    }
+
+    public String getMsg() {
+        return msg;
+    }
+
+    public void setMsg(String msg) {
+        this.msg = msg;
+    }
+
+    /**
+     * package response command
+     *
+     * @param opaque request unique identification
+     * @return command
+     */
+    public Command convert2Command(long opaque){
+        Command command = new Command(opaque);
+        command.setType(CommandType.ROLL_VIEW_LOG_RESPONSE);
+        byte[] body = FastJsonSerializer.serialize(this);
+        command.setBody(body);
+        return command;
+    }
+}
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java
new file mode 100644
index 0000000..5dcefc6
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java
@@ -0,0 +1,66 @@
+/*
+ * 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.log;
+
+import org.apache.dolphinscheduler.remote.command.Command;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer;
+
+import java.io.Serializable;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ *  view log request command
+ */
+public class ViewLogRequestCommand implements Serializable {
+
+    /**
+     *  request id
+     */
+    private static final AtomicLong REQUEST = new AtomicLong(1);
+
+    private String path;
+
+    public ViewLogRequestCommand() {
+    }
+
+    public ViewLogRequestCommand(String path) {
+        this.path = path;
+    }
+
+    public String getPath() {
+        return path;
+    }
+
+    public void setPath(String path) {
+        this.path = path;
+    }
+
+    /**
+     * package request command
+     *
+     * @return command
+     */
+    public Command convert2Command(){
+        Command command = new Command(REQUEST.getAndIncrement());
+        command.setType(CommandType.VIEW_WHOLE_LOG_REQUEST);
+        byte[] body = FastJsonSerializer.serialize(this);
+        command.setBody(body);
+        return command;
+    }
+}
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogResponseCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogResponseCommand.java
new file mode 100644
index 0000000..dffadad
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogResponseCommand.java
@@ -0,0 +1,64 @@
+/*
+ * 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.log;
+
+import org.apache.dolphinscheduler.remote.command.Command;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer;
+
+import java.io.Serializable;
+
+/**
+ *  view log response command
+ */
+public class ViewLogResponseCommand implements Serializable {
+
+    /**
+     *  response data
+     */
+    private String msg;
+
+    public ViewLogResponseCommand() {
+    }
+
+    public ViewLogResponseCommand(String msg) {
+        this.msg = msg;
+    }
+
+    public String getMsg() {
+        return msg;
+    }
+
+    public void setMsg(String msg) {
+        this.msg = msg;
+    }
+
+    /**
+     * package response command
+     *
+     * @param opaque request unique identification
+     * @return command
+     */
+    public Command convert2Command(long opaque){
+        Command command = new Command(opaque);
+        command.setType(CommandType.VIEW_WHOLE_LOG_RESPONSE);
+        byte[] body = FastJsonSerializer.serialize(this);
+        command.setBody(body);
+        return command;
+    }
+}
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyClientConfig.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyClientConfig.java
new file mode 100644
index 0000000..831e05f
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyClientConfig.java
@@ -0,0 +1,91 @@
+/*
+ * 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.config;
+
+import org.apache.dolphinscheduler.remote.utils.Constants;
+
+/**
+ *  netty client config
+ */
+public class NettyClientConfig {
+
+    /**
+     *  worker threads,default get machine cpus
+     */
+    private int workerThreads = Constants.CPUS;
+
+    /**
+     *  whether tpc delay
+     */
+    private boolean tcpNoDelay = true;
+
+    /**
+     * whether keep alive
+     */
+    private boolean soKeepalive = true;
+
+    /**
+     *  send buffer size
+     */
+    private int sendBufferSize = 65535;
+
+    /**
+     *  receive buffer size
+     */
+    private int receiveBufferSize = 65535;
+
+    public int getWorkerThreads() {
+        return workerThreads;
+    }
+
+    public void setWorkerThreads(int workerThreads) {
+        this.workerThreads = workerThreads;
+    }
+
+    public boolean isTcpNoDelay() {
+        return tcpNoDelay;
+    }
+
+    public void setTcpNoDelay(boolean tcpNoDelay) {
+        this.tcpNoDelay = tcpNoDelay;
+    }
+
+    public boolean isSoKeepalive() {
+        return soKeepalive;
+    }
+
+    public void setSoKeepalive(boolean soKeepalive) {
+        this.soKeepalive = soKeepalive;
+    }
+
+    public int getSendBufferSize() {
+        return sendBufferSize;
+    }
+
+    public void setSendBufferSize(int sendBufferSize) {
+        this.sendBufferSize = sendBufferSize;
+    }
+
+    public int getReceiveBufferSize() {
+        return receiveBufferSize;
+    }
+
+    public void setReceiveBufferSize(int receiveBufferSize) {
+        this.receiveBufferSize = receiveBufferSize;
+    }
+
+}
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyServerConfig.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyServerConfig.java
new file mode 100644
index 0000000..4ec8a0f
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyServerConfig.java
@@ -0,0 +1,116 @@
+/*
+ * 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.config;
+
+import org.apache.dolphinscheduler.remote.utils.Constants;
+
+/**
+ *  netty server config
+ */
+public class NettyServerConfig {
+
+    /**
+     * init the server connectable queue
+     */
+    private int soBacklog = 1024;
+
+    /**
+     *  whether tpc delay
+     */
+    private boolean tcpNoDelay = true;
+
+    /**
+     *  whether keep alive
+     */
+    private boolean soKeepalive = true;
+
+    /**
+     *  send buffer size
+     */
+    private int sendBufferSize = 65535;
+
+    /**
+     *  receive buffer size
+     */
+    private int receiveBufferSize = 65535;
+
+    /**
+     *  worker threads,default get machine cpus
+     */
+    private int workerThread = Constants.CPUS;
+
+    /**
+     *  listen port
+     */
+    private int listenPort = 12346;
+
+    public int getListenPort() {
+        return listenPort;
+    }
+
+    public void setListenPort(int listenPort) {
+        this.listenPort = listenPort;
+    }
+
+    public int getSoBacklog() {
+        return soBacklog;
+    }
+
+    public void setSoBacklog(int soBacklog) {
+        this.soBacklog = soBacklog;
+    }
+
+    public boolean isTcpNoDelay() {
+        return tcpNoDelay;
+    }
+
+    public void setTcpNoDelay(boolean tcpNoDelay) {
+        this.tcpNoDelay = tcpNoDelay;
+    }
+
+    public boolean isSoKeepalive() {
+        return soKeepalive;
+    }
+
+    public void setSoKeepalive(boolean soKeepalive) {
+        this.soKeepalive = soKeepalive;
+    }
+
+    public int getSendBufferSize() {
+        return sendBufferSize;
+    }
+
+    public void setSendBufferSize(int sendBufferSize) {
+        this.sendBufferSize = sendBufferSize;
+    }
+
+    public int getReceiveBufferSize() {
+        return receiveBufferSize;
+    }
+
+    public void setReceiveBufferSize(int receiveBufferSize) {
+        this.receiveBufferSize = receiveBufferSize;
+    }
+
+    public int getWorkerThread() {
+        return workerThread;
+    }
+
+    public void setWorkerThread(int workerThread) {
+        this.workerThread = workerThread;
+    }
+}
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingException.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingException.java
new file mode 100644
index 0000000..29d48db
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingException.java
@@ -0,0 +1,94 @@
+/*
+ * 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.exceptions;
+
+/**
+ *  remote exception
+ */
+public class RemotingException extends Exception {
+
+    public RemotingException() {
+        super();
+    }
+
+    /** Constructs a new runtime exception with the specified detail message.
+     * The cause is not initialized, and may subsequently be initialized by a
+     * call to {@link #initCause}.
+     *
+     * @param   message   the detail message. The detail message is saved for
+     *          later retrieval by the {@link #getMessage()} method.
+     */
+    public RemotingException(String message) {
+        super(message);
+    }
+
+    /**
+     * Constructs a new runtime exception with the specified detail message and
+     * cause.  <p>Note that the detail message associated with
+     * {@code cause} is <i>not</i> automatically incorporated in
+     * this runtime exception's detail message.
+     *
+     * @param  message the detail message (which is saved for later retrieval
+     *         by the {@link #getMessage()} method).
+     * @param  cause the cause (which is saved for later retrieval by the
+     *         {@link #getCause()} method).  (A <tt>null</tt> value is
+     *         permitted, and indicates that the cause is nonexistent or
+     *         unknown.)
+     * @since  1.4
+     */
+    public RemotingException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    /** Constructs a new runtime exception with the specified cause and a
+     * detail message of <tt>(cause==null ? null : cause.toString())</tt>
+     * (which typically contains the class and detail message of
+     * <tt>cause</tt>).  This constructor is useful for runtime exceptions
+     * that are little more than wrappers for other throwables.
+     *
+     * @param  cause the cause (which is saved for later retrieval by the
+     *         {@link #getCause()} method).  (A <tt>null</tt> value is
+     *         permitted, and indicates that the cause is nonexistent or
+     *         unknown.)
+     * @since  1.4
+     */
+    public RemotingException(Throwable cause) {
+        super(cause);
+    }
+
+    /**
+     * Constructs a new runtime exception with the specified detail
+     * message, cause, suppression enabled or disabled, and writable
+     * stack trace enabled or disabled.
+     *
+     * @param  message the detail message.
+     * @param cause the cause.  (A {@code null} value is permitted,
+     * and indicates that the cause is nonexistent or unknown.)
+     * @param enableSuppression whether or not suppression is enabled
+     *                          or disabled
+     * @param writableStackTrace whether or not the stack trace should
+     *                           be writable
+     *
+     * @since 1.7
+     */
+    protected RemotingException(String message, Throwable cause,
+                                boolean enableSuppression,
+                                boolean writableStackTrace) {
+        super(message, cause, enableSuppression, writableStackTrace);
+    }
+}
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java
new file mode 100644
index 0000000..6aceb5a
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java
@@ -0,0 +1,174 @@
+/*
+ * 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.handler;
+
+import io.netty.channel.*;
+import org.apache.dolphinscheduler.remote.NettyRemotingClient;
+import org.apache.dolphinscheduler.remote.command.Command;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
+import org.apache.dolphinscheduler.remote.utils.ChannelUtils;
+import org.apache.dolphinscheduler.remote.utils.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.RejectedExecutionException;
+
+/**
+ *  netty client request handler
+ */
+@ChannelHandler.Sharable
+public class NettyClientHandler extends ChannelInboundHandlerAdapter {
+
+    private final Logger logger = LoggerFactory.getLogger(NettyClientHandler.class);
+
+    /**
+     *  netty remote client
+     */
+    private final NettyRemotingClient nettyRemotingClient;
+
+    /**
+     *  client processors queue
+     */
+    private final ConcurrentHashMap<CommandType, Pair<NettyRequestProcessor, ExecutorService>> processors = new ConcurrentHashMap();
+
+    public NettyClientHandler(NettyRemotingClient nettyRemotingClient){
+        this.nettyRemotingClient = nettyRemotingClient;
+    }
+
+    /**
+     *  When the current channel is not active,
+     *  the current channel has reached the end of its life cycle
+     *
+     * @param ctx channel handler context
+     * @throws Exception
+     */
+    @Override
+    public void channelInactive(ChannelHandlerContext ctx) throws Exception {
+        nettyRemotingClient.removeChannel(ChannelUtils.toAddress(ctx.channel()));
+        ctx.channel().close();
+    }
+
+    /**
+     *  The current channel reads data from the remote
+     *
+     * @param ctx channel handler context
+     * @param msg message
+     * @throws Exception
+     */
+    @Override
+    public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+        processReceived(ctx.channel(), (Command)msg);
+    }
+
+    /**
+     *  register processor
+     *
+     * @param commandType command type
+     * @param processor processor
+     */
+    public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) {
+         this.registerProcessor(commandType, processor, nettyRemotingClient.getDefaultExecutor());
+    }
+
+    /**
+     * register processor
+     *
+     * @param commandType command type
+     * @param processor processor
+     * @param executor thread executor
+     */
+    public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) {
+        ExecutorService executorRef = executor;
+        if(executorRef == null){
+            executorRef = nettyRemotingClient.getDefaultExecutor();
+        }
+        this.processors.putIfAbsent(commandType, new Pair<NettyRequestProcessor, ExecutorService>(processor, executorRef));
+    }
+
+    /**
+     *  process received logic
+     *
+     * @param channel channel
+     * @param msg message
+     */
+    private void processReceived(final Channel channel, final Command msg) {
+        final CommandType commandType = msg.getType();
+        final Pair<NettyRequestProcessor, ExecutorService> pair = processors.get(commandType);
+        if (pair != null) {
+            Runnable r = new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        pair.getLeft().process(channel, msg);
+                    } catch (Throwable ex) {
+                        logger.error("process msg {} error : {}", msg, ex);
+                    }
+                }
+            };
+            try {
+                pair.getRight().submit(r);
+            } catch (RejectedExecutionException e) {
+                logger.warn("thread pool is full, discard msg {} from {}", msg, ChannelUtils.getRemoteAddress(channel));
+            }
+        } else {
+            logger.warn("commandType {} not support", commandType);
+        }
+    }
+
+    /**
+     *  caught exception
+     *
+     * @param ctx channel handler context
+     * @param cause cause
+     * @throws Exception
+     */
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
+        logger.error("exceptionCaught : {}", cause);
+        nettyRemotingClient.removeChannel(ChannelUtils.toAddress(ctx.channel()));
+        ctx.channel().close();
+    }
+
+    /**
+     *  channel write changed
+     * @param ctx channel handler context
+     * @throws Exception
+     */
+    @Override
+    public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception {
+        Channel ch = ctx.channel();
+        ChannelConfig config = ch.config();
+
+        if (!ch.isWritable()) {
+            if (logger.isWarnEnabled()) {
+                logger.warn("{} is not writable, over high water level : {}",
+                        new Object[]{ch, config.getWriteBufferHighWaterMark()});
+            }
+
+            config.setAutoRead(false);
+        } else {
+            if (logger.isWarnEnabled()) {
+                logger.warn("{} is writable, to low water : {}",
+                        new Object[]{ch, config.getWriteBufferLowWaterMark()});
+            }
+            config.setAutoRead(true);
+        }
+    }
+}
\ No newline at end of file
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java
new file mode 100644
index 0000000..eabd656
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java
@@ -0,0 +1,173 @@
+/*
+ * 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.handler;
+
+import io.netty.channel.*;
+import org.apache.dolphinscheduler.remote.NettyRemotingServer;
+import org.apache.dolphinscheduler.remote.command.Command;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
+import org.apache.dolphinscheduler.remote.utils.ChannelUtils;
+import org.apache.dolphinscheduler.remote.utils.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.RejectedExecutionException;
+
+/**
+ *  netty server request handler
+ */
+@ChannelHandler.Sharable
+public class NettyServerHandler extends ChannelInboundHandlerAdapter {
+
+    private final Logger logger = LoggerFactory.getLogger(NettyServerHandler.class);
+
+    /**
+     *  netty remote server
+     */
+    private final NettyRemotingServer nettyRemotingServer;
+
+    /**
+     *  server processors queue
+     */
+    private final ConcurrentHashMap<CommandType, Pair<NettyRequestProcessor, ExecutorService>> processors = new ConcurrentHashMap();
+
+    public NettyServerHandler(NettyRemotingServer nettyRemotingServer){
+        this.nettyRemotingServer = nettyRemotingServer;
+    }
+
+    /**
+     *  When the current channel is not active,
+     *  the current channel has reached the end of its life cycle
+     * @param ctx channel handler context
+     * @throws Exception
+     */
+    @Override
+    public void channelInactive(ChannelHandlerContext ctx) throws Exception {
+        ctx.channel().close();
+    }
+
+    /**
+     * The current channel reads data from the remote end
+     *
+     * @param ctx channel handler context
+     * @param msg message
+     * @throws Exception
+     */
+    @Override
+    public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+        processReceived(ctx.channel(), (Command)msg);
+    }
+
+    /**
+     * register processor
+     *
+     * @param commandType command type
+     * @param processor processor
+     */
+    public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) {
+        this.registerProcessor(commandType, processor, null);
+    }
+
+    /**
+     *  register processor
+     *
+     * @param commandType command type
+     * @param processor processor
+     * @param executor thread executor
+     */
+    public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) {
+        ExecutorService executorRef = executor;
+        if(executorRef == null){
+            executorRef = nettyRemotingServer.getDefaultExecutor();
+        }
+        this.processors.putIfAbsent(commandType, new Pair<NettyRequestProcessor, ExecutorService>(processor, executorRef));
+    }
+
+    /**
+     *  process received logic
+     * @param channel channel
+     * @param msg message
+     */
+    private void processReceived(final Channel channel, final Command msg) {
+        final CommandType commandType = msg.getType();
+        final Pair<NettyRequestProcessor, ExecutorService> pair = processors.get(commandType);
+        if (pair != null) {
+            Runnable r = new Runnable() {
+
+                @Override
+                public void run() {
+                    try {
+                        pair.getLeft().process(channel, msg);
+                    } catch (Throwable ex) {
+                        logger.error("process msg {} error : {}", msg, ex);
+                    }
+                }
+            };
+            try {
+                pair.getRight().submit(r);
+            } catch (RejectedExecutionException e) {
+                logger.warn("thread pool is full, discard msg {} from {}", msg, ChannelUtils.getRemoteAddress(channel));
+            }
+        } else {
+            logger.warn("commandType {} not support", commandType);
+        }
+    }
+
+    /**
+     *  caught exception
+     *
+     * @param ctx channel handler context
+     * @param cause cause
+     * @throws Exception
+     */
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
+        logger.error("exceptionCaught : {}", cause);
+        ctx.channel().close();
+    }
+
+    /**
+     *  channel write changed
+     *
+     * @param ctx  channel handler context
+     * @throws Exception
+     */
+    @Override
+    public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception {
+        Channel ch = ctx.channel();
+        ChannelConfig config = ch.config();
+
+        if (!ch.isWritable()) {
+            if (logger.isWarnEnabled()) {
+                logger.warn("{} is not writable, over high water level : {}",
+                        new Object[]{ch, config.getWriteBufferHighWaterMark()});
+            }
+
+            config.setAutoRead(false);
+        } else {
+            if (logger.isWarnEnabled()) {
+                logger.warn("{} is writable, to low water : {}",
+                        new Object[]{ch, config.getWriteBufferLowWaterMark()});
+            }
+            config.setAutoRead(true);
+        }
+    }
+}
\ No newline at end of file
diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZkServer.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/NettyRequestProcessor.java
similarity index 65%
copy from dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZkServer.java
copy to dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/NettyRequestProcessor.java
index d1a0526..6966b53 100644
--- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZkServer.java
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/NettyRequestProcessor.java
@@ -14,30 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.zk;
+package org.apache.dolphinscheduler.remote.processor;
 
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import io.netty.channel.Channel;
+import org.apache.dolphinscheduler.remote.command.Command;
 
 /**
- * demo for using zkServer
+ *  netty request processor
  */
-public class TestZkServer {
+public interface NettyRequestProcessor {
 
-    @Before
-    public void before(){
-        ZKServer.start();
-    }
-
-    @Test
-    public void test(){
-        Assert.assertTrue(ZKServer.isStarted());
-    }
-
-    @After
-    public void after(){
-        ZKServer.stop();
-    }
+    /**
+     *  process logic
+     * @param channel channel
+     * @param command command
+     */
+    void process(final Channel channel, final Command command);
 }
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Address.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Address.java
new file mode 100644
index 0000000..f61dcd6
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Address.java
@@ -0,0 +1,96 @@
+/*
+ * 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.utils;
+
+import java.io.Serializable;
+
+/**
+ *  server address
+ */
+public class Address implements Serializable {
+
+    /**
+     *  host
+     */
+    private String host;
+
+    /**
+     *  port
+     */
+    private int port;
+
+    public Address(){
+        //NOP
+    }
+
+    public Address(String host, int port){
+        this.host = host;
+        this.port = port;
+    }
+
+    public String getHost() {
+        return host;
+    }
+
+    public void setHost(String host) {
+        this.host = host;
+    }
+
+    public int getPort() {
+        return port;
+    }
+
+    public void setPort(int port) {
+        this.port = port;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((host == null) ? 0 : host.hashCode());
+        result = prime * result + port;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+        Address other = (Address) obj;
+        if (host == null) {
+            if (other.host != null) {
+                return false;
+            }
+        } else if (!host.equals(other.host)) {
+            return false;
+        }
+        return port == other.port;
+    }
+
+    @Override
+    public String toString() {
+        return "Address [host=" + host + ", port=" + port + "]";
+    }
+}
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java
new file mode 100644
index 0000000..d7af5fe
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.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.remote.utils;
+
+import io.netty.channel.Channel;
+
+import java.net.InetSocketAddress;
+
+/**
+ *  channel utils
+ */
+public class ChannelUtils {
+
+    /**
+     *  get local address
+     *
+     * @param channel channel
+     * @return local address
+     */
+    public static String getLocalAddress(Channel channel){
+        return ((InetSocketAddress)channel.localAddress()).getAddress().getHostAddress();
+    }
+
+    /**
+     *  get remote address
+     * @param channel channel
+     * @return remote address
+     */
+    public static String getRemoteAddress(Channel channel){
+        return ((InetSocketAddress)channel.remoteAddress()).getAddress().getHostAddress();
+    }
+
+    /**
+     *  channel to address
+     * @param channel channel
+     * @return address
+     */
+    public static Address toAddress(Channel channel){
+        InetSocketAddress socketAddress = ((InetSocketAddress)channel.remoteAddress());
+        return new Address(socketAddress.getAddress().getHostAddress(), socketAddress.getPort());
+    }
+
+}
diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZk.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java
similarity index 66%
rename from dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZk.java
rename to dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java
index 5c3db2d..5733b17 100644
--- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZk.java
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java
@@ -14,30 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.zk;
+package org.apache.dolphinscheduler.remote.utils;
+
+import java.nio.charset.Charset;
 
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
 
 /**
- * demo for using zkServer
+ *  constant
  */
-public class TestZk {
-
-    @Before
-    public void before(){
-        ZKServer.start();
-    }
-
-    @Test
-    public void test(){
-        Assert.assertTrue(ZKServer.isStarted());
-    }
-
-    @After
-    public void after(){
-        ZKServer.stop();
-    }
+public class Constants {
+
+    public static final String COMMA = ",";
+
+    public static final String SLASH = "/";
+
+    /**
+     *  charset
+     */
+    public static final Charset UTF8 = Charset.forName("UTF-8");
+
+    /**
+     *  cpus
+     */
+    public static final int CPUS = Runtime.getRuntime().availableProcessors();
+
 }
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/FastJsonSerializer.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/FastJsonSerializer.java
new file mode 100644
index 0000000..e96796a
--- /dev/null
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/FastJsonSerializer.java
@@ -0,0 +1,60 @@
+/*
+ * 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.utils;
+
+import com.alibaba.fastjson.JSON;
+
+/**
+ *  json serialize or deserialize
+ */
+public class FastJsonSerializer {
+
+	/**
+	 * serialize to byte
+	 *
+	 * @param obj object
+	 * @param <T> object type
+	 * @return byte array
+	 */
+	public static <T> byte[] serialize(T obj)  {
+		String json = JSON.toJSONString(obj);
+		return json.getBytes(Constants.UTF8);
+	}
+
+	/**
+	 *  serialize to string
+	 * @param obj object
+	 * @param <T> object type
+	 * @return string
+	 */
+	public static <T> String serializeToString(T obj)  {
+		return JSON.toJSONString(obj);
+	}
+
+	/**
+	 *  deserialize
+	 *
+	 * @param src byte array
+	 * @param clazz class
+	 * @param <T> deserialize type
+	 * @return deserialize type
+	 */
+	public static <T> T deserialize(byte[] src, Class<T> clazz) {
+		return JSON.parseObject(new String(src, Constants.UTF8), clazz);
+	}
+
+}
diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZkServer.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Pair.java
similarity index 61%
rename from dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZkServer.java
rename to dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Pair.java
index d1a0526..2042191 100644
--- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZkServer.java
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Pair.java
@@ -14,30 +14,40 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.zk;
 
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+package org.apache.dolphinscheduler.remote.utils;
+
 
 /**
- * demo for using zkServer
+ * key value pair
+ *
+ * @param <L> L generic type
+ * @param <R> R generic type
  */
-public class TestZkServer {
+public class Pair<L, R> {
+
+    private L left;
+
+    private R right;
+
+    public Pair(L left, R right) {
+        this.left = left;
+        this.right = right;
+    }
+
+    public L getLeft() {
+        return left;
+    }
 
-    @Before
-    public void before(){
-        ZKServer.start();
+    public void setLeft(L left) {
+        this.left = left;
     }
 
-    @Test
-    public void test(){
-        Assert.assertTrue(ZKServer.isStarted());
+    public R getRight() {
+        return right;
     }
 
-    @After
-    public void after(){
-        ZKServer.stop();
+    public void setRight(R right) {
+        this.right = right;
     }
 }
diff --git a/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java b/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java
new file mode 100644
index 0000000..6f0a802
--- /dev/null
+++ b/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java
@@ -0,0 +1,77 @@
+package org.apache.dolphinscheduler.remote;/*
+ * 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 io.netty.channel.Channel;
+import org.apache.dolphinscheduler.remote.NettyRemotingClient;
+import org.apache.dolphinscheduler.remote.NettyRemotingServer;
+import org.apache.dolphinscheduler.remote.command.Command;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+import org.apache.dolphinscheduler.remote.command.Ping;
+import org.apache.dolphinscheduler.remote.command.Pong;
+import org.apache.dolphinscheduler.remote.config.NettyClientConfig;
+import org.apache.dolphinscheduler.remote.config.NettyServerConfig;
+import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
+import org.apache.dolphinscheduler.remote.utils.Address;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ *  netty remote client test
+ */
+public class NettyRemotingClientTest {
+
+
+    /**
+     *  test ping
+     */
+    @Test
+    public void testSend(){
+        NettyServerConfig serverConfig = new NettyServerConfig();
+
+        NettyRemotingServer server = new NettyRemotingServer(serverConfig);
+        server.registerProcessor(CommandType.PING, new NettyRequestProcessor() {
+            @Override
+            public void process(Channel channel, Command command) {
+                channel.writeAndFlush(Pong.create(command.getOpaque()));
+            }
+        });
+        server.start();
+        //
+        CountDownLatch latch = new CountDownLatch(1);
+        AtomicLong opaque = new AtomicLong(1);
+        final NettyClientConfig clientConfig = new NettyClientConfig();
+        NettyRemotingClient client = new NettyRemotingClient(clientConfig);
+        client.registerProcessor(CommandType.PONG, new NettyRequestProcessor() {
+            @Override
+            public void process(Channel channel, Command command) {
+                opaque.set(command.getOpaque());
+                latch.countDown();
+            }
+        });
+        Command commandPing = Ping.create();
+        try {
+            client.send(new Address("127.0.0.1", serverConfig.getListenPort()), commandPing);
+            latch.await();
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+        Assert.assertEquals(opaque.get(), commandPing.getOpaque());
+    }
+}
diff --git a/dolphinscheduler-rpc/pom.xml b/dolphinscheduler-rpc/pom.xml
deleted file mode 100644
index 680a4a2..0000000
--- a/dolphinscheduler-rpc/pom.xml
+++ /dev/null
@@ -1,113 +0,0 @@
-<?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>
-        <groupId>org.apache.dolphinscheduler</groupId>
-        <artifactId>dolphinscheduler</artifactId>
-        <version>1.2.1-SNAPSHOT</version>
-    </parent>
-    <modelVersion>4.0.0</modelVersion>
-
-    <artifactId>dolphinscheduler-rpc</artifactId>
-
-    <name>dolphinscheduler-rpc</name>
-    <url>https://github.com/apache/incubator-dolphinscheduler</url>
-
-    <properties>
-        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-        <maven.compiler.source>1.8</maven.compiler.source>
-        <maven.compiler.target>1.8</maven.compiler.target>
-
-        <protobuf.version>3.5.1</protobuf.version>
-        <grpc.version>1.9.0</grpc.version>
-    </properties>
-
-    <dependencies>
-        <dependency>
-            <groupId>com.google.protobuf</groupId>
-            <artifactId>protobuf-java</artifactId>
-            <version>${protobuf.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>io.grpc</groupId>
-            <artifactId>grpc-netty</artifactId>
-            <version>${grpc.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>io.grpc</groupId>
-            <artifactId>grpc-protobuf</artifactId>
-            <version>${grpc.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>io.grpc</groupId>
-            <artifactId>grpc-stub</artifactId>
-            <version>${grpc.version}</version>
-        </dependency>
-
-        <dependency>
-            <groupId>com.google.guava</groupId>
-            <artifactId>guava</artifactId>
-        </dependency>
-    </dependencies>
-
-    <build>
-        <extensions>
-            <extension>
-                <groupId>kr.motd.maven</groupId>
-                <artifactId>os-maven-plugin</artifactId>
-                <version>1.5.0.Final</version>
-            </extension>
-        </extensions>
-        <plugins>
-            <plugin>
-                <groupId>org.xolstice.maven.plugins</groupId>
-                <artifactId>protobuf-maven-plugin</artifactId>
-                <version>0.5.0</version>
-                <configuration>
-                    <protocArtifact>com.google.protobuf:protoc:3.5.1-1:exe:${os.detected.classifier}</protocArtifact>
-                    <pluginId>grpc-java</pluginId>
-                    <pluginArtifact>io.grpc:protoc-gen-grpc-java:${grpc.version}:exe:${os.detected.classifier}</pluginArtifact>
-                </configuration>
-                <executions>
-                    <execution>
-                        <id>compile</id>
-                        <goals>
-                            <goal>compile</goal>
-                        </goals>
-                    </execution>
-                    <execution>
-                        <id>compile-custom</id>
-                        <goals>
-                            <goal>compile-custom</goal>
-                        </goals>
-                    </execution>
-                </executions>
-            </plugin>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-compiler-plugin</artifactId>
-                <configuration>
-                    <source>${java.version}</source>
-                    <target>${java.version}</target>
-                    <encoding>${project.build.sourceEncoding}</encoding>
-                </configuration>
-            </plugin>
-        </plugins>
-    </build>
-</project>
diff --git a/dolphinscheduler-rpc/src/main/proto/scheduler.proto b/dolphinscheduler-rpc/src/main/proto/scheduler.proto
deleted file mode 100644
index b8b595c..0000000
--- a/dolphinscheduler-rpc/src/main/proto/scheduler.proto
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-syntax = "proto3";
-
-package schduler;
-
-option java_multiple_files = true;
-option java_package = "org.apache.dolphinscheduler.rpc";
-option java_outer_classname = "SchdulerProto";
-
-
-/**
- *  return str info
- */
-message RetStrInfo {
-  /**
-   *  str msg info
-   */
-  string msg = 1 ;
-}
-
-/**
- * return byte info
- */
-message RetByteInfo {
-  /**
-   *  byte data info
-   */
-  bytes data  = 1;
-}
-
-/**
- *  log parameter
- */
-message LogParameter {
-
-  /**
-   * path
-   */
-  string path = 1 ;
-
-   /**
-    * skip line num
-   */
-  int32 skipLineNum = 2 ;
-
-  /**
-   * display limt num
-  */
-  int32 limit = 3 ;
-}
-
-
-/**
- *  path parameter
- */
-message PathParameter {
-
-  /**
-   * path
-   */
-  string path = 1 ;
-}
-
-/**
- *  log view service
- */
-service LogViewService {
-
-  /**
-   *  roll view log
-   */
-  rpc rollViewLog(LogParameter) returns (RetStrInfo) {};
-
-  /**
-     * view all log
-   */
-  rpc viewLog(PathParameter) returns (RetStrInfo) {};
-
-  /**
-    * get log bytes
-   */
-  rpc getLogBytes(PathParameter) returns (RetByteInfo) {};
-}
-
diff --git a/dolphinscheduler-server/pom.xml b/dolphinscheduler-server/pom.xml
index 751fd91..080b87e 100644
--- a/dolphinscheduler-server/pom.xml
+++ b/dolphinscheduler-server/pom.xml
@@ -71,7 +71,7 @@
 
 		<dependency>
 			<groupId>org.apache.dolphinscheduler</groupId>
-			<artifactId>dolphinscheduler-rpc</artifactId>
+			<artifactId>dolphinscheduler-service</artifactId>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.curator</groupId>
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerRequestProcessor.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerRequestProcessor.java
new file mode 100644
index 0000000..4e4404e
--- /dev/null
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerRequestProcessor.java
@@ -0,0 +1,179 @@
+/*
+ * 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.log;
+
+import io.netty.channel.Channel;
+import org.apache.dolphinscheduler.remote.command.Command;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+import org.apache.dolphinscheduler.remote.command.log.*;
+import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
+import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.*;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ *  logger request process logic
+ */
+public class LoggerRequestProcessor implements NettyRequestProcessor {
+
+    private final Logger logger = LoggerFactory.getLogger(LoggerRequestProcessor.class);
+
+    private final ThreadPoolExecutor executor;
+
+    public LoggerRequestProcessor(){
+        this.executor = new ThreadPoolExecutor(4, 4, 10, TimeUnit.SECONDS, new LinkedBlockingQueue<>(100));
+    }
+
+    @Override
+    public void process(Channel channel, Command command) {
+        logger.info("received command : {}", command);
+
+        /**
+         * reuqest task log command type
+         */
+        final CommandType commandType = command.getType();
+        switch (commandType){
+            case GET_LOG_BYTES_REQUEST:
+                GetLogBytesRequestCommand getLogRequest = FastJsonSerializer.deserialize(
+                        command.getBody(), GetLogBytesRequestCommand.class);
+                byte[] bytes = getFileContentBytes(getLogRequest.getPath());
+                GetLogBytesResponseCommand getLogResponse = new GetLogBytesResponseCommand(bytes);
+                channel.writeAndFlush(getLogResponse.convert2Command(command.getOpaque()));
+                break;
+            case VIEW_WHOLE_LOG_REQUEST:
+                ViewLogRequestCommand viewLogRequest = FastJsonSerializer.deserialize(
+                        command.getBody(), ViewLogRequestCommand.class);
+                String msg = readWholeFileContent(viewLogRequest.getPath());
+                ViewLogResponseCommand viewLogResponse = new ViewLogResponseCommand(msg);
+                channel.writeAndFlush(viewLogResponse.convert2Command(command.getOpaque()));
+                break;
+            case ROLL_VIEW_LOG_REQUEST:
+                RollViewLogRequestCommand rollViewLogRequest = FastJsonSerializer.deserialize(
+                        command.getBody(), RollViewLogRequestCommand.class);
+                List<String> lines = readPartFileContent(rollViewLogRequest.getPath(),
+                        rollViewLogRequest.getSkipLineNum(), rollViewLogRequest.getLimit());
+                StringBuilder builder = new StringBuilder();
+                for (String line : lines){
+                    builder.append(line + "\r\n");
+                }
+                RollViewLogResponseCommand rollViewLogRequestResponse = new RollViewLogResponseCommand(builder.toString());
+                channel.writeAndFlush(rollViewLogRequestResponse.convert2Command(command.getOpaque()));
+                break;
+            default:
+                throw new IllegalArgumentException("unknown commandType");
+        }
+    }
+
+    public ExecutorService getExecutor(){
+        return this.executor;
+    }
+
+    /**
+     * get files content bytes,for down load file
+     *
+     * @param filePath file path
+     * @return byte array of file
+     * @throws Exception exception
+     */
+    private byte[] getFileContentBytes(String filePath){
+        InputStream in = null;
+        ByteArrayOutputStream bos = null;
+        try {
+            in = new FileInputStream(filePath);
+            bos  = new ByteArrayOutputStream();
+            byte[] buf = new byte[1024];
+            int len;
+            while ((len = in.read(buf)) != -1) {
+                bos.write(buf, 0, len);
+            }
+            return bos.toByteArray();
+        }catch (IOException e){
+            logger.error("get file bytes error",e);
+        }finally {
+            if (bos != null){
+                try {
+                    bos.close();
+                } catch (IOException ignore) {}
+            }
+            if (in != null){
+                try {
+                    in.close();
+                } catch (IOException ignore) {}
+            }
+        }
+        return new byte[0];
+    }
+
+    /**
+     * read part file content,can skip any line and read some lines
+     *
+     * @param filePath file path
+     * @param skipLine skip line
+     * @param limit read lines limit
+     * @return part file content
+     */
+    private List<String> readPartFileContent(String filePath,
+                                            int skipLine,
+                                            int limit){
+        try (Stream<String> stream = Files.lines(Paths.get(filePath))) {
+            return stream.skip(skipLine).limit(limit).collect(Collectors.toList());
+        } catch (IOException e) {
+            logger.error("read file error",e);
+        }
+        return Collections.EMPTY_LIST;
+    }
+
+    /**
+     * read whole file content
+     *
+     * @param filePath file path
+     * @return whole file content
+     */
+    private String readWholeFileContent(String filePath){
+        BufferedReader br = null;
+        String line;
+        StringBuilder sb = new StringBuilder();
+        try {
+            br = new BufferedReader(new InputStreamReader(new FileInputStream(filePath)));
+            while ((line = br.readLine()) != null){
+                sb.append(line + "\r\n");
+            }
+            return sb.toString();
+        }catch (IOException e){
+            logger.error("read file error",e);
+        }finally {
+            try {
+                if (br != null){
+                    br.close();
+                }
+            } catch (IOException ignore) {}
+        }
+        return "";
+    }
+}
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerServer.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerServer.java
new file mode 100644
index 0000000..3520fb0
--- /dev/null
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerServer.java
@@ -0,0 +1,91 @@
+/*
+ * 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.log;
+
+
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.remote.NettyRemotingServer;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+import org.apache.dolphinscheduler.remote.config.NettyServerConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *  logger server
+ */
+public class LoggerServer {
+
+    private static  final Logger logger = LoggerFactory.getLogger(LoggerServer.class);
+
+    /**
+     *  netty server
+     */
+    private final NettyRemotingServer server;
+
+    /**
+     *  netty server config
+     */
+    private final NettyServerConfig serverConfig;
+
+    /**
+     *  loggger request processor
+     */
+    private final LoggerRequestProcessor requestProcessor;
+
+    public LoggerServer(){
+        this.serverConfig = new NettyServerConfig();
+        this.serverConfig.setListenPort(Constants.RPC_PORT);
+        this.server = new NettyRemotingServer(serverConfig);
+        this.requestProcessor = new LoggerRequestProcessor();
+        this.server.registerProcessor(CommandType.GET_LOG_BYTES_REQUEST, requestProcessor, requestProcessor.getExecutor());
+        this.server.registerProcessor(CommandType.ROLL_VIEW_LOG_REQUEST, requestProcessor, requestProcessor.getExecutor());
+        this.server.registerProcessor(CommandType.VIEW_WHOLE_LOG_REQUEST, requestProcessor, requestProcessor.getExecutor());
+    }
+
+    /**
+     * main launches the server from the command line.
+     * @param args arguments
+     */
+    public static void main(String[] args)  {
+        final LoggerServer server = new LoggerServer();
+        server.start();
+    }
+
+    /**
+     * server start
+     */
+    public void start()  {
+        this.server.start();
+        logger.info("logger server started, listening on port : {}" , Constants.RPC_PORT);
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                LoggerServer.this.stop();
+            }
+        });
+    }
+
+    /**
+     * stop
+     */
+    public void stop() {
+        this.server.close();
+        logger.info("logger server shut down");
+    }
+
+}
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java
index ab4ba5c..6b5063c 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java
@@ -22,14 +22,14 @@ import org.apache.dolphinscheduler.common.thread.Stopper;
 import org.apache.dolphinscheduler.common.thread.ThreadPoolExecutors;
 import org.apache.dolphinscheduler.common.thread.ThreadUtils;
 import org.apache.dolphinscheduler.common.utils.OSUtils;
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.server.master.config.MasterConfig;
 import org.apache.dolphinscheduler.server.master.runner.MasterSchedulerThread;
-import org.apache.dolphinscheduler.dao.quartz.ProcessScheduleJob;
-import org.apache.dolphinscheduler.dao.quartz.QuartzExecutors;
 import org.apache.dolphinscheduler.server.zk.ZKMasterClient;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
+import org.apache.dolphinscheduler.service.quartz.ProcessScheduleJob;
+import org.apache.dolphinscheduler.service.quartz.QuartzExecutors;
 import org.quartz.SchedulerException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -66,10 +66,10 @@ public class MasterServer implements IStoppable {
     private ScheduledExecutorService heartbeatMasterService;
 
     /**
-     *  dolphinscheduler database interface
+     *  process service
      */
     @Autowired
-    protected ProcessDao processDao;
+    protected ProcessService processService;
 
     /**
      *  master exec thread pool
@@ -77,17 +77,18 @@ public class MasterServer implements IStoppable {
     private ExecutorService masterSchedulerService;
 
     /**
-     *  spring application context
-     *  only use it for initialization
+     * master config
      */
     @Autowired
-    private SpringApplicationContext springApplicationContext;
+    private MasterConfig masterConfig;
+
 
     /**
-     * master config
+     *  spring application context
+     *  only use it for initialization
      */
     @Autowired
-    private MasterConfig masterConfig;
+    private SpringApplicationContext springApplicationContext;
 
 
     /**
@@ -126,7 +127,7 @@ public class MasterServer implements IStoppable {
         // master scheduler thread
         MasterSchedulerThread masterSchedulerThread = new MasterSchedulerThread(
                 zkMasterClient,
-                processDao,
+                processService,
                 masterConfig.getMasterExecThreads());
 
         // submit master scheduler thread
@@ -136,7 +137,7 @@ public class MasterServer implements IStoppable {
         // what system should do if exception
         try {
             logger.info("start Quartz server...");
-            ProcessScheduleJob.init(processDao);
+            ProcessScheduleJob.init(processService);
             QuartzExecutors.getInstance().start();
         } catch (Exception e) {
             try {
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterBaseTaskExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterBaseTaskExecThread.java
index c1552c4..f8fcb14 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterBaseTaskExecThread.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterBaseTaskExecThread.java
@@ -16,15 +16,15 @@
  */
 package org.apache.dolphinscheduler.server.master.runner;
 
-import org.apache.dolphinscheduler.common.queue.ITaskQueue;
-import org.apache.dolphinscheduler.common.queue.TaskQueueFactory;
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
 import org.apache.dolphinscheduler.dao.AlertDao;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.dao.utils.BeanContext;
 import org.apache.dolphinscheduler.server.master.config.MasterConfig;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
+import org.apache.dolphinscheduler.service.queue.ITaskQueue;
+import org.apache.dolphinscheduler.service.queue.TaskQueueFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -41,9 +41,9 @@ public class MasterBaseTaskExecThread implements Callable<Boolean> {
     private static final Logger logger = LoggerFactory.getLogger(MasterBaseTaskExecThread.class);
 
     /**
-     * process dao
+     * process service
      */
-    protected ProcessDao processDao;
+    protected ProcessService processService;
 
     /**
      * alert database access
@@ -81,7 +81,7 @@ public class MasterBaseTaskExecThread implements Callable<Boolean> {
      * @param processInstance   process instance
      */
     public MasterBaseTaskExecThread(TaskInstance taskInstance, ProcessInstance processInstance){
-        this.processDao = BeanContext.getBean(ProcessDao.class);
+        this.processService = BeanContext.getBean(ProcessService.class);
         this.alertDao = BeanContext.getBean(AlertDao.class);
         this.processInstance = processInstance;
         this.taskQueue = TaskQueueFactory.getTaskQueueInstance();
@@ -121,14 +121,14 @@ public class MasterBaseTaskExecThread implements Callable<Boolean> {
             try {
                 if(!submitDB){
                     // submit task to db
-                    task = processDao.submitTask(taskInstance, processInstance);
+                    task = processService.submitTask(taskInstance, processInstance);
                     if(task != null && task.getId() != 0){
                         submitDB = true;
                     }
                 }
                 if(submitDB && !submitQueue){
                     // submit task to queue
-                    submitQueue = processDao.submitTaskToQueue(task);
+                    submitQueue = processService.submitTaskToQueue(task);
                 }
                 if(submitDB && submitQueue){
                     return task;
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java
index 2b1ff4d..f5e3121 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java
@@ -28,14 +28,15 @@ import org.apache.dolphinscheduler.common.process.ProcessDag;
 import org.apache.dolphinscheduler.common.thread.Stopper;
 import org.apache.dolphinscheduler.common.thread.ThreadUtils;
 import org.apache.dolphinscheduler.common.utils.*;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
 import org.apache.dolphinscheduler.dao.entity.Schedule;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.dao.utils.DagHelper;
-import org.apache.dolphinscheduler.dao.utils.cron.CronUtils;
 import org.apache.dolphinscheduler.server.master.config.MasterConfig;
 import org.apache.dolphinscheduler.server.utils.AlertManager;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
+import org.apache.dolphinscheduler.service.quartz.cron.CronUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -124,9 +125,9 @@ public class MasterExecThread implements Runnable {
     private DAG<String,TaskNode,TaskNodeRelation> dag;
 
     /**
-     *  process dao
+     *  process service
      */
-    private ProcessDao processDao;
+    private ProcessService processService;
 
     /**
      * master config
@@ -136,10 +137,10 @@ public class MasterExecThread implements Runnable {
     /**
      * constructor of MasterExecThread
      * @param processInstance   process instance
-     * @param processDao        process dao
+     * @param processService        process dao
      */
-    public MasterExecThread(ProcessInstance processInstance,ProcessDao processDao){
-        this.processDao = processDao;
+    public MasterExecThread(ProcessInstance processInstance, ProcessService processService){
+        this.processService = processService;
 
         this.processInstance = processInstance;
         this.masterConfig = SpringApplicationContext.getBean(MasterConfig.class);
@@ -177,7 +178,7 @@ public class MasterExecThread implements Runnable {
             logger.error("process execute failed, process id:{}", processInstance.getId());
             processInstance.setState(ExecutionStatus.FAILURE);
             processInstance.setEndTime(new Date());
-            processDao.updateProcessInstance(processInstance);
+            processService.updateProcessInstance(processInstance);
         }finally {
             taskExecService.shutdown();
             // post handle
@@ -205,11 +206,11 @@ public class MasterExecThread implements Runnable {
 
         Date startDate = DateUtils.getScheduleDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_START_DATE));
         Date endDate = DateUtils.getScheduleDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_END_DATE));
-        processDao.saveProcessInstance(processInstance);
+        processService.saveProcessInstance(processInstance);
 
         // get schedules
         int processDefinitionId = processInstance.getProcessDefinitionId();
-        List<Schedule> schedules = processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId);
+        List<Schedule> schedules = processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId);
         List<Date> listDate = Lists.newLinkedList();
         if(!CollectionUtils.isEmpty(schedules)){
             for (Schedule schedule : schedules) {
@@ -223,7 +224,7 @@ public class MasterExecThread implements Runnable {
             iterator = listDate.iterator();
             scheduleDate = iterator.next();
             processInstance.setScheduleTime(scheduleDate);
-            processDao.updateProcessInstance(processInstance);
+            processService.updateProcessInstance(processInstance);
         }else{
             scheduleDate = processInstance.getScheduleTime();
             if(scheduleDate == null){
@@ -239,7 +240,7 @@ public class MasterExecThread implements Runnable {
                 logger.error("process {} dag is null, please check out parameters",
                         processInstance.getId());
                 processInstance.setState(ExecutionStatus.SUCCESS);
-                processDao.updateProcessInstance(processInstance);
+                processService.updateProcessInstance(processInstance);
                 return;
             }
 
@@ -281,10 +282,10 @@ public class MasterExecThread implements Runnable {
                 processInstance.setCommandParam(JSONUtils.toJson(cmdParam));
             }
 
-            List<TaskInstance> taskInstanceList = processDao.findValidTaskListByProcessId(processInstance.getId());
+            List<TaskInstance> taskInstanceList = processService.findValidTaskListByProcessId(processInstance.getId());
             for(TaskInstance taskInstance : taskInstanceList){
                 taskInstance.setFlag(Flag.NO);
-                processDao.updateTaskInstance(taskInstance);
+                processService.updateTaskInstance(taskInstance);
             }
             processInstance.setState(ExecutionStatus.RUNNING_EXEUTION);
             processInstance.setGlobalParams(ParameterUtils.curingGlobalParams(
@@ -292,7 +293,7 @@ public class MasterExecThread implements Runnable {
                     processInstance.getProcessDefinition().getGlobalParamList(),
                     CommandType.COMPLEMENT_DATA, processInstance.getScheduleTime()));
 
-            processDao.saveProcessInstance(processInstance);
+            processService.saveProcessInstance(processInstance);
         }
 
         // flow end
@@ -320,11 +321,11 @@ public class MasterExecThread implements Runnable {
      */
     private void endProcess() {
         processInstance.setEndTime(new Date());
-        processDao.updateProcessInstance(processInstance);
+        processService.updateProcessInstance(processInstance);
         if(processInstance.getState().typeIsWaittingThread()){
-            processDao.createRecoveryWaitingThreadCommand(null, processInstance);
+            processService.createRecoveryWaitingThreadCommand(null, processInstance);
         }
-        List<TaskInstance> taskInstances = processDao.findValidTaskListByProcessId(processInstance.getId());
+        List<TaskInstance> taskInstances = processService.findValidTaskListByProcessId(processInstance.getId());
         alertManager.sendAlertProcessInstance(processInstance, taskInstances);
     }
 
@@ -361,7 +362,7 @@ public class MasterExecThread implements Runnable {
         dependFailedTask.clear();
         completeTaskList.clear();
         errorTaskList.clear();
-        List<TaskInstance> taskInstanceList = processDao.findValidTaskListByProcessId(processInstance.getId());
+        List<TaskInstance> taskInstanceList = processService.findValidTaskListByProcessId(processInstance.getId());
         for(TaskInstance task : taskInstanceList){
             if(task.isTaskComplete()){
                 completeTaskList.put(task.getName(), task);
@@ -417,7 +418,7 @@ public class MasterExecThread implements Runnable {
      * @return TaskInstance
      */
     private TaskInstance findTaskIfExists(String taskName){
-        List<TaskInstance> taskInstanceList = processDao.findValidTaskListByProcessId(this.processInstance.getId());
+        List<TaskInstance> taskInstanceList = processService.findValidTaskListByProcessId(this.processInstance.getId());
         for(TaskInstance taskInstance : taskInstanceList){
             if(taskInstance.getName().equals(taskName)){
                 return taskInstance;
@@ -706,7 +707,7 @@ public class MasterExecThread implements Runnable {
      * @return process instance execution status
      */
     private ExecutionStatus getProcessInstanceState(){
-        ProcessInstance instance = processDao.findProcessInstanceById(processInstance.getId());
+        ProcessInstance instance = processService.findProcessInstanceById(processInstance.getId());
         ExecutionStatus state = instance.getState();
 
         if(activeTaskNode.size() > 0){
@@ -784,10 +785,10 @@ public class MasterExecThread implements Runnable {
                     processInstance.getState().toString(), state.toString(),
                     processInstance.getCommandType().toString());
             processInstance.setState(state);
-            ProcessInstance instance = processDao.findProcessInstanceById(processInstance.getId());
+            ProcessInstance instance = processService.findProcessInstanceById(processInstance.getId());
             instance.setState(state);
             instance.setProcessDefinition(processInstance.getProcessDefinition());
-            processDao.updateProcessInstance(instance);
+            processService.updateProcessInstance(instance);
             processInstance = instance;
         }
     }
@@ -845,7 +846,7 @@ public class MasterExecThread implements Runnable {
             // send warning email if process time out.
             if( !sendTimeWarning && checkProcessTimeOut(processInstance) ){
                 alertManager.sendProcessTimeoutAlert(processInstance,
-                        processDao.findProcessDefineById(processInstance.getProcessDefinitionId()));
+                        processService.findProcessDefineById(processInstance.getProcessDefinitionId()));
                 sendTimeWarning = true;
             }
             for(Map.Entry<MasterBaseTaskExecThread,Future<Boolean>> entry: activeTaskNode.entrySet()) {
@@ -903,7 +904,7 @@ public class MasterExecThread implements Runnable {
                     if(completeTask.getState()== ExecutionStatus.PAUSE){
                         completeTask.setState(ExecutionStatus.KILL);
                         completeTaskList.put(entry.getKey(), completeTask);
-                        processDao.updateTaskInstance(completeTask);
+                        processService.updateTaskInstance(completeTask);
                     }
                 }
             }
@@ -961,7 +962,7 @@ public class MasterExecThread implements Runnable {
             Future<Boolean> future = entry.getValue();
 
             TaskInstance taskInstance = taskExecThread.getTaskInstance();
-            taskInstance = processDao.findTaskInstanceById(taskInstance.getId());
+            taskInstance = processService.findTaskInstanceById(taskInstance.getId());
             if(taskInstance.getState().typeIsFinished()){
                 continue;
             }
@@ -1031,7 +1032,7 @@ public class MasterExecThread implements Runnable {
         }
         try {
             Integer intId = Integer.valueOf(taskId);
-            TaskInstance task = processDao.findTaskInstanceById(intId);
+            TaskInstance task = processService.findTaskInstanceById(intId);
             if(task == null){
                 logger.error("start node id cannot be found: {}",  taskId);
             }else {
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java
index a873fb7..c0ddb1c 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java
@@ -22,13 +22,13 @@ import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.thread.Stopper;
 import org.apache.dolphinscheduler.common.thread.ThreadUtils;
 import org.apache.dolphinscheduler.common.utils.OSUtils;
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
-import org.apache.dolphinscheduler.common.zk.AbstractZKClient;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.Command;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
 import org.apache.dolphinscheduler.server.master.config.MasterConfig;
 import org.apache.dolphinscheduler.server.zk.ZKMasterClient;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
+import org.apache.dolphinscheduler.service.zk.AbstractZKClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -53,7 +53,7 @@ public class MasterSchedulerThread implements Runnable {
     /**
      * dolphinscheduler database interface
      */
-    private final ProcessDao processDao;
+    private final ProcessService processService;
 
     /**
      * zookeeper master client
@@ -74,11 +74,11 @@ public class MasterSchedulerThread implements Runnable {
     /**
      * constructor of MasterSchedulerThread
      * @param zkClient              zookeeper master client
-     * @param processDao            process dao
+     * @param processService            process service
      * @param masterExecThreadNum   master exec thread num
      */
-    public MasterSchedulerThread(ZKMasterClient zkClient, ProcessDao processDao, int masterExecThreadNum){
-        this.processDao = processDao;
+    public MasterSchedulerThread(ZKMasterClient zkClient, ProcessService processService, int masterExecThreadNum){
+        this.processService = processService;
         this.zkMasterClient = zkClient;
         this.masterExecThreadNum = masterExecThreadNum;
         this.masterExecService = ThreadUtils.newDaemonFixedThreadExecutor("Master-Exec-Thread",masterExecThreadNum);
@@ -115,19 +115,19 @@ public class MasterSchedulerThread implements Runnable {
                     ThreadPoolExecutor poolExecutor = (ThreadPoolExecutor) masterExecService;
                     int activeCount = poolExecutor.getActiveCount();
                     // make sure to scan and delete command  table in one transaction
-                    Command command = processDao.findOneCommand();
+                    Command command = processService.findOneCommand();
                     if (command != null) {
                         logger.info(String.format("find one command: id: %d, type: %s", command.getId(),command.getCommandType().toString()));
 
                         try{
-                            processInstance = processDao.handleCommand(logger, OSUtils.getHost(), this.masterExecThreadNum - activeCount, command);
+                            processInstance = processService.handleCommand(logger, OSUtils.getHost(), this.masterExecThreadNum - activeCount, command);
                             if (processInstance != null) {
                                 logger.info("start master exec thread , split DAG ...");
-                                masterExecService.execute(new MasterExecThread(processInstance,processDao));
+                                masterExecService.execute(new MasterExecThread(processInstance, processService));
                             }
                         }catch (Exception e){
                             logger.error("scan command error ", e);
-                            processDao.moveToErrorCommand(command, e.toString());
+                            processService.moveToErrorCommand(command, e.toString());
                         }
                     } else{
                         //indicate that no command ,sleep for 1s
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java
index f2ee66b..66d1a3f 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java
@@ -82,7 +82,7 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread {
             result = waitTaskQuit();
         }
         taskInstance.setEndTime(new Date());
-        processDao.updateTaskInstance(taskInstance);
+        processService.updateTaskInstance(taskInstance);
         logger.info("task :{} id:{}, process id:{}, exec thread completed ",
                 this.taskInstance.getName(),taskInstance.getId(), processInstance.getId() );
         return result;
@@ -94,7 +94,7 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread {
      */
     public Boolean waitTaskQuit(){
         // query new state
-        taskInstance = processDao.findTaskInstanceById(taskInstance.getId());
+        taskInstance = processService.findTaskInstanceById(taskInstance.getId());
         logger.info("wait task: process id: {}, task id:{}, task name:{} complete",
                 this.taskInstance.getProcessInstanceId(), this.taskInstance.getId(), this.taskInstance.getName());
         // task time out
@@ -126,15 +126,15 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread {
                     if (remainTime < 0) {
                         logger.warn("task id: {} execution time out",taskInstance.getId());
                         // process define
-                        ProcessDefinition processDefine = processDao.findProcessDefineById(processInstance.getProcessDefinitionId());
+                        ProcessDefinition processDefine = processService.findProcessDefineById(processInstance.getProcessDefinitionId());
                         // send warn mail
                         alertDao.sendTaskTimeoutAlert(processInstance.getWarningGroupId(),processDefine.getReceivers(),processDefine.getReceiversCc(),taskInstance.getId(),taskInstance.getName());
                         checkTimeout = false;
                     }
                 }
                 // updateProcessInstance task instance
-                taskInstance = processDao.findTaskInstanceById(taskInstance.getId());
-                processInstance = processDao.findProcessInstanceById(processInstance.getId());
+                taskInstance = processService.findTaskInstanceById(taskInstance.getId());
+                processInstance = processService.findProcessInstanceById(processInstance.getId());
                 Thread.sleep(Constants.SLEEP_TIME_MILLIS);
             } catch (Exception e) {
                 logger.error("exception",e);
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/SubProcessTaskExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/SubProcessTaskExecThread.java
index 0026de7..fc16b51 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/SubProcessTaskExecThread.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/SubProcessTaskExecThread.java
@@ -64,7 +64,7 @@ public class SubProcessTaskExecThread extends MasterBaseTaskExecThread {
             }
             setTaskInstanceState();
             waitTaskQuit();
-            subProcessInstance = processDao.findSubProcessInstance(processInstance.getId(), taskInstance.getId());
+            subProcessInstance = processService.findSubProcessInstance(processInstance.getId(), taskInstance.getId());
 
             // at the end of the subflow , the task state is changed to the subflow state
             if(subProcessInstance != null){
@@ -75,7 +75,7 @@ public class SubProcessTaskExecThread extends MasterBaseTaskExecThread {
                 }
             }
             taskInstance.setEndTime(new Date());
-            processDao.updateTaskInstance(taskInstance);
+            processService.updateTaskInstance(taskInstance);
             logger.info("subflow task :{} id:{}, process id:{}, exec thread completed ",
                     this.taskInstance.getName(),taskInstance.getId(), processInstance.getId() );
             result = true;
@@ -96,14 +96,14 @@ public class SubProcessTaskExecThread extends MasterBaseTaskExecThread {
      * @return
      */
     private Boolean setTaskInstanceState(){
-        subProcessInstance = processDao.findSubProcessInstance(processInstance.getId(), taskInstance.getId());
+        subProcessInstance = processService.findSubProcessInstance(processInstance.getId(), taskInstance.getId());
         if(subProcessInstance == null || taskInstance.getState().typeIsFinished()){
             return false;
         }
 
         taskInstance.setState(ExecutionStatus.RUNNING_EXEUTION);
         taskInstance.setStartTime(new Date());
-        processDao.updateTaskInstance(taskInstance);
+        processService.updateTaskInstance(taskInstance);
         return true;
     }
 
@@ -111,7 +111,7 @@ public class SubProcessTaskExecThread extends MasterBaseTaskExecThread {
      *  updateProcessInstance parent state
      */
     private void updateParentProcessState(){
-        ProcessInstance parentProcessInstance = processDao.findProcessInstanceById(this.processInstance.getId());
+        ProcessInstance parentProcessInstance = processService.findProcessInstanceById(this.processInstance.getId());
 
         if(parentProcessInstance == null){
             logger.error("parent work flow instance is null ,  please check it! work flow id {}", processInstance.getId());
@@ -145,7 +145,7 @@ public class SubProcessTaskExecThread extends MasterBaseTaskExecThread {
                     continue;
                 }
             }
-            subProcessInstance = processDao.findProcessInstanceById(subProcessInstance.getId());
+            subProcessInstance = processService.findProcessInstanceById(subProcessInstance.getId());
             updateParentProcessState();
             if (subProcessInstance.getState().typeIsFinished()){
                 break;
@@ -171,7 +171,7 @@ public class SubProcessTaskExecThread extends MasterBaseTaskExecThread {
             return;
         }
         subProcessInstance.setState(ExecutionStatus.READY_STOP);
-        processDao.updateProcessInstance(subProcessInstance);
+        processService.updateProcessInstance(subProcessInstance);
     }
 
     /**
@@ -183,6 +183,6 @@ public class SubProcessTaskExecThread extends MasterBaseTaskExecThread {
             return;
         }
         subProcessInstance.setState(ExecutionStatus.READY_PAUSE);
-        processDao.updateProcessInstance(subProcessInstance);
+        processService.updateProcessInstance(subProcessInstance);
     }
 }
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/monitor/ZKMonitorImpl.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/monitor/ZKMonitorImpl.java
index 9270740..5acc8fd 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/monitor/ZKMonitorImpl.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/monitor/ZKMonitorImpl.java
@@ -16,7 +16,7 @@
  */
 package org.apache.dolphinscheduler.server.monitor;
 
-import org.apache.dolphinscheduler.common.zk.ZookeeperOperator;
+import org.apache.dolphinscheduler.service.zk.ZookeeperOperator;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Component;
 
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/rpc/LogClient.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/rpc/LogClient.java
deleted file mode 100644
index 1c6c97b..0000000
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/rpc/LogClient.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.dolphinscheduler.server.rpc;
-
-import io.grpc.ManagedChannel;
-import io.grpc.ManagedChannelBuilder;
-import io.grpc.StatusRuntimeException;
-import org.apache.dolphinscheduler.rpc.*;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- *  log client
- */
-public class LogClient {
-
-    /**
-     * logger of LogClient
-     */
-    private static  final Logger logger = LoggerFactory.getLogger(LogClient.class);
-
-    /**
-     * managed channel
-     */
-    private final ManagedChannel channel;
-
-    /**
-     * blocking stub
-     */
-    private final LogViewServiceGrpc.LogViewServiceBlockingStub blockingStub;
-
-    /**
-     * Construct client connecting to HelloWorld server at host:port.
-     *
-     * @param host host
-     * @param port port
-     */
-    public LogClient(String host, int port) {
-        this(ManagedChannelBuilder.forAddress(host, port)
-                // Channels are secure by default (via SSL/TLS). For the example we disable TLS to avoid
-                // needing certificates.
-                .usePlaintext(true));
-    }
-
-    /**
-     * Construct client for accessing RouteGuide server using the existing channel.
-     *
-     * @param channelBuilder channel builder
-     */
-    LogClient(ManagedChannelBuilder<?> channelBuilder) {
-        /**
-         *  set max message read size
-         */
-        channelBuilder.maxInboundMessageSize(Integer.MAX_VALUE);
-        channel = channelBuilder.build();
-        blockingStub = LogViewServiceGrpc.newBlockingStub(channel);
-    }
-
-    /**
-     * shut down channel
-     *
-     * @throws InterruptedException interrupted exception
-     */
-    public void shutdown() throws InterruptedException {
-        channel.shutdown().awaitTermination(5, TimeUnit.SECONDS);
-    }
-
-    /**
-     * roll view log
-     *
-     * @param path          log path
-     * @param skipLineNum   skip line num
-     * @param limit         limit
-     * @return log content
-     */
-    public String rollViewLog(String path,int skipLineNum,int limit) {
-        logger.info("roll view log , path : {},skipLineNum : {} ,limit :{}", path, skipLineNum, limit);
-        LogParameter pathParameter = LogParameter
-                .newBuilder()
-                .setPath(path)
-                .setSkipLineNum(skipLineNum)
-                .setLimit(limit)
-                .build();
-        RetStrInfo retStrInfo;
-        try {
-            retStrInfo = blockingStub.rollViewLog(pathParameter);
-            return retStrInfo.getMsg();
-        } catch (StatusRuntimeException e) {
-            logger.error("roll view log failed", e);
-            return null;
-        }
-    }
-
-    /**
-     * view all log
-     *
-     * @param path log path
-     * @return log content
-     */
-    public String viewLog(String path) {
-        logger.info("view log path : {}",path);
-
-        PathParameter pathParameter = PathParameter.newBuilder().setPath(path).build();
-        RetStrInfo retStrInfo;
-        try {
-            retStrInfo = blockingStub.viewLog(pathParameter);
-            return retStrInfo.getMsg();
-        } catch (StatusRuntimeException e) {
-            logger.error("view log failed", e);
-            return null;
-        }
-    }
-
-    /**
-     * get log bytes
-     *
-     * @param path log path
-     * @return log content
-     */
-    public byte[] getLogBytes(String path) {
-        logger.info("get log bytes {}",path);
-
-        PathParameter pathParameter = PathParameter.newBuilder().setPath(path).build();
-        RetByteInfo retByteInfo;
-        try {
-            retByteInfo = blockingStub.getLogBytes(pathParameter);
-            return retByteInfo.getData().toByteArray();
-        } catch (StatusRuntimeException e) {
-            logger.error("get log bytes failed ", e);
-            return null;
-        }
-    }
-}
\ No newline at end of file
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/rpc/LoggerServer.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/rpc/LoggerServer.java
deleted file mode 100644
index 5ec5df9..0000000
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/rpc/LoggerServer.java
+++ /dev/null
@@ -1,238 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.dolphinscheduler.server.rpc;
-
-import io.grpc.stub.StreamObserver;
-import org.apache.dolphinscheduler.common.Constants;
-import com.google.protobuf.ByteString;
-import io.grpc.Server;
-import io.grpc.ServerBuilder;
-import org.apache.dolphinscheduler.rpc.*;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.*;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.util.List;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-/**
- * logger server
- */
-public class LoggerServer {
-
-    private static  final Logger logger = LoggerFactory.getLogger(LoggerServer.class);
-
-    /**
-     * server
-     */
-    private Server server;
-
-    /**
-     * server start
-     * @throws IOException io exception
-     */
-    public void start() throws IOException {
-	    /* The port on which the server should run */
-        int port = Constants.RPC_PORT;
-        server = ServerBuilder.forPort(port)
-                .addService(new LogViewServiceGrpcImpl())
-                .build()
-                .start();
-        logger.info("server started, listening on port : {}" , port);
-        Runtime.getRuntime().addShutdownHook(new Thread() {
-            @Override
-            public void run() {
-                // Use stderr here since the logger may have been reset by its JVM shutdown hook.
-                logger.info("shutting down gRPC server since JVM is shutting down");
-                LoggerServer.this.stop();
-                logger.info("server shut down");
-            }
-        });
-    }
-
-    /**
-     * stop
-     */
-    private void stop() {
-        if (server != null) {
-            server.shutdown();
-        }
-    }
-
-    /**
-     * await termination on the main thread since the grpc library uses daemon threads.
-     */
-    private void blockUntilShutdown() throws InterruptedException {
-        if (server != null) {
-            server.awaitTermination();
-        }
-    }
-
-    /**
-     * main launches the server from the command line.
-     */
-
-    /**
-     * main launches the server from the command line.
-     * @param args arguments
-     * @throws IOException          io exception
-     * @throws InterruptedException interrupted exception
-     */
-    public static void main(String[] args) throws IOException, InterruptedException {
-        final LoggerServer server = new LoggerServer();
-        server.start();
-        server.blockUntilShutdown();
-    }
-
-    /**
-     * Log View Service Grpc Implementation
-     */
-    static class LogViewServiceGrpcImpl extends LogViewServiceGrpc.LogViewServiceImplBase {
-        @Override
-        public void rollViewLog(LogParameter request, StreamObserver<RetStrInfo> responseObserver) {
-
-            logger.info("log parameter path : {} ,skip line : {}, limit : {}",
-                    request.getPath(),
-                    request.getSkipLineNum(),
-                    request.getLimit());
-            List<String> list = readFile(request.getPath(), request.getSkipLineNum(), request.getLimit());
-            StringBuilder sb = new StringBuilder();
-            boolean errorLineFlag = false;
-            for (String line : list){
-                sb.append(line + "\r\n");
-            }
-            RetStrInfo retInfoBuild = RetStrInfo.newBuilder().setMsg(sb.toString()).build();
-            responseObserver.onNext(retInfoBuild);
-            responseObserver.onCompleted();
-        }
-
-        @Override
-        public void viewLog(PathParameter request, StreamObserver<RetStrInfo> responseObserver) {
-            logger.info("task path is : {} " , request.getPath());
-            RetStrInfo retInfoBuild = RetStrInfo.newBuilder().setMsg(readFile(request.getPath())).build();
-            responseObserver.onNext(retInfoBuild);
-            responseObserver.onCompleted();
-        }
-
-        @Override
-        public void getLogBytes(PathParameter request, StreamObserver<RetByteInfo> responseObserver) {
-            try {
-                ByteString bytes = ByteString.copyFrom(getFileBytes(request.getPath()));
-                RetByteInfo.Builder builder = RetByteInfo.newBuilder();
-                builder.setData(bytes);
-                responseObserver.onNext(builder.build());
-                responseObserver.onCompleted();
-            }catch (Exception e){
-                logger.error("get log bytes failed",e);
-            }
-        }
-    }
-
-    /**
-     * get files bytes
-     *
-     * @param path path
-     * @return byte array of file
-     * @throws Exception exception
-     */
-    private static byte[] getFileBytes(String path){
-        InputStream in = null;
-        ByteArrayOutputStream bos = null;
-        try {
-            in = new FileInputStream(path);
-            bos  = new ByteArrayOutputStream();
-            byte[] buf = new byte[1024];
-            int len = 0;
-            while ((len = in.read(buf)) != -1) {
-                bos.write(buf, 0, len);
-            }
-            return bos.toByteArray();
-        }catch (IOException e){
-            logger.error("get file bytes error",e);
-        }finally {
-            if (bos != null){
-                try {
-                    bos.close();
-                } catch (IOException e) {
-                    e.printStackTrace();
-                }
-            }
-            if (in != null){
-                try {
-                    in.close();
-                } catch (IOException e) {
-                    e.printStackTrace();
-                }
-            }
-        }
-        return null;
-    }
-
-    /**
-     * read file content
-     *
-     * @param path
-     * @param skipLine
-     * @param limit
-     * @return
-     */
-    private static List<String> readFile(String path,int skipLine,int limit){
-        try (Stream<String> stream = Files.lines(Paths.get(path))) {
-            return stream.skip(skipLine).limit(limit).collect(Collectors.toList());
-        } catch (IOException e) {
-            logger.error("read file failed",e);
-        }
-        return null;
-    }
-
-    /**
-     * read  file content
-     *
-     * @param path path
-     * @return string of file content
-     * @throws Exception exception
-     */
-    private static String readFile(String path){
-        BufferedReader br = null;
-        String line = null;
-        StringBuilder sb = new StringBuilder();
-        try {
-            br = new BufferedReader(new InputStreamReader(new FileInputStream(path)));
-            boolean errorLineFlag = false;
-            while ((line = br.readLine()) != null){
-                sb.append(line + "\r\n");
-            }
-
-            return sb.toString();
-        }catch (IOException e){
-            logger.error("read file failed",e);
-        }finally {
-            try {
-                if (br != null){
-                    br.close();
-                }
-            } catch (IOException e) {
-                logger.error(e.getMessage(),e);
-            }
-        }
-        return null;
-    }
-
-}
\ No newline at end of file
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java
index fd0a08c..90711e1 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java
@@ -22,8 +22,8 @@ import org.apache.dolphinscheduler.common.utils.LoggerUtils;
 import org.apache.dolphinscheduler.common.utils.OSUtils;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
-import org.apache.dolphinscheduler.server.rpc.LogClient;
 import org.apache.commons.io.FileUtils;
+import org.apache.dolphinscheduler.service.log.LogClientService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -375,9 +375,16 @@ public class ProcessUtils {
   public static void killYarnJob(TaskInstance taskInstance) {
     try {
       Thread.sleep(Constants.SLEEP_TIME_MILLIS);
-      LogClient logClient = new LogClient(taskInstance.getHost(), Constants.RPC_PORT);
-
-      String log = logClient.viewLog(taskInstance.getLogPath());
+      LogClientService logClient = null;
+      String log = null;
+      try {
+        logClient = new LogClientService(taskInstance.getHost(), Constants.RPC_PORT);
+        log = logClient.viewLog(taskInstance.getLogPath());
+      } finally {
+        if(logClient != null){
+          logClient.close();
+        }
+      }
       if (StringUtils.isNotEmpty(log)) {
         List<String> appIds = LoggerUtils.getAppIds(log, logger);
         String workerDir = taskInstance.getExecutePath();
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/RemoveZKNode.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/RemoveZKNode.java
index 7264c2f..5550e75 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/RemoveZKNode.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/RemoveZKNode.java
@@ -16,7 +16,7 @@
  */
 package org.apache.dolphinscheduler.server.utils;
 
-import org.apache.dolphinscheduler.common.zk.ZookeeperOperator;
+import org.apache.dolphinscheduler.service.zk.ZookeeperOperator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java
index 99d418f..ace9307 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java
@@ -22,22 +22,22 @@ import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.IStoppable;
 import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.common.enums.TaskType;
-import org.apache.dolphinscheduler.common.queue.ITaskQueue;
-import org.apache.dolphinscheduler.common.queue.TaskQueueFactory;
 import org.apache.dolphinscheduler.common.thread.Stopper;
 import org.apache.dolphinscheduler.common.thread.ThreadPoolExecutors;
 import org.apache.dolphinscheduler.common.thread.ThreadUtils;
 import org.apache.dolphinscheduler.common.utils.CollectionUtils;
 import org.apache.dolphinscheduler.common.utils.OSUtils;
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
-import org.apache.dolphinscheduler.common.zk.AbstractZKClient;
 import org.apache.dolphinscheduler.dao.AlertDao;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.server.utils.ProcessUtils;
 import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
 import org.apache.dolphinscheduler.server.worker.runner.FetchTaskThread;
 import org.apache.dolphinscheduler.server.zk.ZKWorkerClient;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
+import org.apache.dolphinscheduler.service.queue.ITaskQueue;
+import org.apache.dolphinscheduler.service.queue.TaskQueueFactory;
+import org.apache.dolphinscheduler.service.zk.AbstractZKClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -73,10 +73,10 @@ public class WorkerServer implements IStoppable {
 
 
     /**
-     *  process database access
+     *  process service
      */
     @Autowired
-    private ProcessDao processDao;
+    private ProcessService processService;
 
     /**
      *  alert database access
@@ -105,13 +105,6 @@ public class WorkerServer implements IStoppable {
     private ExecutorService fetchTaskExecutorService;
 
     /**
-     *  spring application context
-     *  only use it for initialization
-     */
-    @Autowired
-    private SpringApplicationContext springApplicationContext;
-
-    /**
      * CountDownLatch latch
      */
     private CountDownLatch latch;
@@ -123,6 +116,13 @@ public class WorkerServer implements IStoppable {
     private WorkerConfig workerConfig;
 
     /**
+     *  spring application context
+     *  only use it for initialization
+     */
+    @Autowired
+    private SpringApplicationContext springApplicationContext;
+
+    /**
      * master server startup
      *
      * master server not use web service
@@ -167,7 +167,7 @@ public class WorkerServer implements IStoppable {
         killExecutorService.execute(killProcessThread);
 
         // new fetch task thread
-        FetchTaskThread fetchTaskThread = new FetchTaskThread(zkWorkerClient, processDao, taskQueue);
+        FetchTaskThread fetchTaskThread = new FetchTaskThread(zkWorkerClient, processService, taskQueue);
 
         // submit fetch task thread
         fetchTaskExecutorService.execute(fetchTaskThread);
@@ -297,7 +297,7 @@ public class WorkerServer implements IStoppable {
                     Set<String> taskInfoSet = taskQueue.smembers(Constants.DOLPHINSCHEDULER_TASKS_KILL);
                     if (CollectionUtils.isNotEmpty(taskInfoSet)){
                         for (String taskInfo : taskInfoSet){
-                            killTask(taskInfo, processDao);
+                            killTask(taskInfo, processService);
                             removeKillInfoFromQueue(taskInfo);
                         }
                     }
@@ -319,7 +319,7 @@ public class WorkerServer implements IStoppable {
      * @param taskInfo  task info
      * @param pd        process dao
      */
-    private void killTask(String taskInfo, ProcessDao pd) {
+    private void killTask(String taskInfo, ProcessService pd) {
         logger.info("get one kill command from tasks kill queue: " + taskInfo);
         String[] taskInfoArray = taskInfo.split("-");
         if(taskInfoArray.length != 2){
@@ -357,7 +357,7 @@ public class WorkerServer implements IStoppable {
      * @param taskInstance
      * @param pd process dao
      */
-    private void deleteTaskFromQueue(TaskInstance taskInstance, ProcessDao pd){
+    private void deleteTaskFromQueue(TaskInstance taskInstance, ProcessService pd){
         // creating distributed locks, lock path /dolphinscheduler/lock/worker
         InterProcessMutex mutex = null;
         logger.info("delete task from tasks queue: " + taskInstance.getId());
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/FetchTaskThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/FetchTaskThread.java
index 221ad06..013db83 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/FetchTaskThread.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/FetchTaskThread.java
@@ -19,17 +19,18 @@ package org.apache.dolphinscheduler.server.worker.runner;
 import org.apache.curator.framework.recipes.locks.InterProcessMutex;
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
-import org.apache.dolphinscheduler.common.queue.ITaskQueue;
 import org.apache.dolphinscheduler.common.thread.Stopper;
 import org.apache.dolphinscheduler.common.thread.ThreadUtils;
 import org.apache.dolphinscheduler.common.utils.*;
-import org.apache.dolphinscheduler.common.zk.AbstractZKClient;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.dao.entity.Tenant;
 import org.apache.dolphinscheduler.dao.entity.WorkerGroup;
 import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
 import org.apache.dolphinscheduler.server.zk.ZKWorkerClient;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
+import org.apache.dolphinscheduler.service.queue.ITaskQueue;
+import org.apache.dolphinscheduler.service.zk.AbstractZKClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -63,7 +64,7 @@ public class FetchTaskThread implements Runnable{
     /**
      *  process database access
      */
-    private final ProcessDao processDao;
+    private final ProcessService processService;
 
     /**
      *  worker thread pool executor
@@ -91,10 +92,10 @@ public class FetchTaskThread implements Runnable{
     private WorkerConfig workerConfig;
 
     public FetchTaskThread(ZKWorkerClient zkWorkerClient,
-                           ProcessDao processDao,
+                           ProcessService processService,
                            ITaskQueue taskQueue){
         this.zkWorkerClient = zkWorkerClient;
-        this.processDao = processDao;
+        this.processService = processService;
         this.taskQueue = taskQueue;
         this.workerConfig = SpringApplicationContext.getBean(WorkerConfig.class);
         this.taskNum = workerConfig.getWorkerFetchTaskNum();
@@ -112,12 +113,12 @@ public class FetchTaskThread implements Runnable{
      */
     private boolean checkWorkerGroup(TaskInstance taskInstance, String host){
 
-        int taskWorkerGroupId = processDao.getTaskWorkerGroupId(taskInstance);
+        int taskWorkerGroupId = processService.getTaskWorkerGroupId(taskInstance);
 
         if(taskWorkerGroupId <= 0){
             return true;
         }
-        WorkerGroup workerGroup = processDao.queryWorkerGroupById(taskWorkerGroupId);
+        WorkerGroup workerGroup = processService.queryWorkerGroupById(taskWorkerGroupId);
         if(workerGroup == null ){
             logger.info("task {} cannot find the worker group, use all worker instead.", taskInstance.getId());
             return true;
@@ -184,7 +185,7 @@ public class FetchTaskThread implements Runnable{
                     // mainly to wait for the master insert task to succeed
                     waitForTaskInstance();
 
-                    taskInstance = processDao.getTaskInstanceDetailByTaskId(taskInstId);
+                    taskInstance = processService.getTaskInstanceDetailByTaskId(taskInstId);
 
                     // verify task instance is null
                     if (verifyTaskInstanceIsNull(taskInstance)) {
@@ -200,7 +201,7 @@ public class FetchTaskThread implements Runnable{
                     // if process definition is null ,process definition already deleted
                     int userId = taskInstance.getProcessDefine() == null ? 0 : taskInstance.getProcessDefine().getUserId();
 
-                    Tenant tenant = processDao.getTenantForProcess(
+                    Tenant tenant = processService.getTenantForProcess(
                             taskInstance.getProcessInstance().getTenantId(),
                             userId);
 
@@ -212,7 +213,7 @@ public class FetchTaskThread implements Runnable{
                     }
 
                     // set queue for process instance, user-specified queue takes precedence over tenant queue
-                    String userQueue = processDao.queryUserQueueByProcessInstanceId(taskInstance.getProcessInstanceId());
+                    String userQueue = processService.queryUserQueueByProcessInstanceId(taskInstance.getProcessInstanceId());
                     taskInstance.getProcessInstance().setQueue(StringUtils.isEmpty(userQueue) ? tenant.getQueue() : userQueue);
                     taskInstance.getProcessInstance().setTenantCode(tenant.getTenantCode());
 
@@ -234,7 +235,7 @@ public class FetchTaskThread implements Runnable{
 
                     logger.info("task : {} ready to submit to task scheduler thread",taskInstId);
                     // submit task
-                    workerExecService.submit(new TaskScheduleThread(taskInstance, processDao));
+                    workerExecService.submit(new TaskScheduleThread(taskInstance, processService));
 
                     // remove node from zk
                     removeNodeFromTaskQueue(taskQueueStr);
@@ -259,7 +260,7 @@ public class FetchTaskThread implements Runnable{
         removeNodeFromTaskQueue(taskQueueStr);
 
         if (taskInstance != null){
-            processDao.changeTaskState(ExecutionStatus.FAILURE,
+            processService.changeTaskState(ExecutionStatus.FAILURE,
                     taskInstance.getStartTime(),
                     taskInstance.getHost(),
                     null,
@@ -347,7 +348,7 @@ public class FetchTaskThread implements Runnable{
         int retryTimes = 30;
         while (taskInstance == null && retryTimes > 0) {
             Thread.sleep(Constants.SLEEP_TIME_MILLIS);
-            taskInstance = processDao.findTaskInstanceById(taskInstId);
+            taskInstance = processService.findTaskInstanceById(taskInstId);
             retryTimes--;
         }
     }
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java
index f179d63..a69cffd 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java
@@ -31,15 +31,15 @@ import org.apache.dolphinscheduler.common.task.TaskTimeoutParameter;
 import org.apache.dolphinscheduler.common.utils.CommonUtils;
 import org.apache.dolphinscheduler.common.utils.HadoopUtils;
 import org.apache.dolphinscheduler.common.utils.TaskParametersUtils;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
-import org.apache.dolphinscheduler.dao.permission.PermissionCheck;
 import org.apache.dolphinscheduler.common.utils.LoggerUtils;
 import org.apache.dolphinscheduler.common.log.TaskLogDiscriminator;
 import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
 import org.apache.dolphinscheduler.server.worker.task.TaskManager;
 import org.apache.dolphinscheduler.server.worker.task.TaskProps;
+import org.apache.dolphinscheduler.service.permission.PermissionCheck;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -64,9 +64,9 @@ public class TaskScheduleThread implements Runnable {
     private TaskInstance taskInstance;
 
     /**
-     *  process database access
+     *  process service
      */
-    private final ProcessDao processDao;
+    private final ProcessService processService;
 
     /**
      *  abstract task
@@ -77,10 +77,10 @@ public class TaskScheduleThread implements Runnable {
      * constructor
      *
      * @param taskInstance  task instance
-     * @param processDao    process dao
+     * @param processService    process dao
      */
-    public TaskScheduleThread(TaskInstance taskInstance, ProcessDao processDao){
-        this.processDao = processDao;
+    public TaskScheduleThread(TaskInstance taskInstance, ProcessService processService){
+        this.processService = processService;
         this.taskInstance = taskInstance;
     }
 
@@ -152,7 +152,7 @@ public class TaskScheduleThread implements Runnable {
             logger.error("task scheduler failure", e);
             kill();
             // update task instance state
-            processDao.changeTaskState(ExecutionStatus.FAILURE,
+            processService.changeTaskState(ExecutionStatus.FAILURE,
                     new Date(),
                     taskInstance.getId());
         }
@@ -161,7 +161,7 @@ public class TaskScheduleThread implements Runnable {
                 taskInstance.getId(),
                 task.getExitStatus());
         // update task instance state
-        processDao.changeTaskState(task.getExitStatus(),
+        processService.changeTaskState(task.getExitStatus(),
                 new Date(),
                 taskInstance.getId());
     }
@@ -191,14 +191,14 @@ public class TaskScheduleThread implements Runnable {
         // update task status is running
         if(taskType.equals(TaskType.SQL.name())  ||
                 taskType.equals(TaskType.PROCEDURE.name())){
-            processDao.changeTaskState(ExecutionStatus.RUNNING_EXEUTION,
+            processService.changeTaskState(ExecutionStatus.RUNNING_EXEUTION,
                     taskInstance.getStartTime(),
                     taskInstance.getHost(),
                     null,
                     getTaskLogPath(),
                     taskInstance.getId());
         }else{
-            processDao.changeTaskState(ExecutionStatus.RUNNING_EXEUTION,
+            processService.changeTaskState(ExecutionStatus.RUNNING_EXEUTION,
                     taskInstance.getStartTime(),
                     taskInstance.getHost(),
                     taskInstance.getExecutePath(),
@@ -311,7 +311,7 @@ public class TaskScheduleThread implements Runnable {
             if (!resFile.exists()) {
                 try {
                     // query the tenant code of the resource according to the name of the resource
-                    String tentnCode = processDao.queryTenantCodeByResName(res);
+                    String tentnCode = processService.queryTenantCodeByResName(res);
                     String resHdfsPath = HadoopUtils.getHdfsFilename(tentnCode, res);
 
                     logger.info("get resource file from hdfs :{}", resHdfsPath);
@@ -334,7 +334,7 @@ public class TaskScheduleThread implements Runnable {
     private void checkDownloadPermission(List<String> projectRes) throws Exception {
         int userId = taskInstance.getProcessInstance().getExecutorId();
         String[] resNames = projectRes.toArray(new String[projectRes.size()]);
-        PermissionCheck<String> permissionCheck = new PermissionCheck<>(AuthorizationType.RESOURCE_FILE,processDao,resNames,userId,logger);
+        PermissionCheck<String> permissionCheck = new PermissionCheck<>(AuthorizationType.RESOURCE_FILE, processService,resNames,userId,logger);
         permissionCheck.checkPermission();
     }
 }
\ No newline at end of file
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java
index f1c01af..c473f3a 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java
@@ -21,10 +21,10 @@ import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.common.thread.ThreadUtils;
 import org.apache.dolphinscheduler.common.utils.HadoopUtils;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.common.utils.LoggerUtils;
 import org.apache.dolphinscheduler.server.utils.ProcessUtils;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.slf4j.Logger;
 
 import java.io.*;
@@ -121,10 +121,10 @@ public abstract class AbstractCommandExecutor {
      * task specific execution logic
      *
      * @param execCommand   exec command
-     * @param processDao    process dao
+     * @param processService    process dao
      * @return exit status code
      */
-    public int run(String execCommand, ProcessDao processDao) {
+    public int run(String execCommand, ProcessService processService) {
         int exitStatusCode;
 
         try {
@@ -147,7 +147,7 @@ public abstract class AbstractCommandExecutor {
             // get process id
             int pid = getProcessId(process);
 
-            processDao.updatePidByTaskInstId(taskInstId, pid, "");
+            processService.updatePidByTaskInstId(taskInstId, pid, "");
 
             logger.info("process start, process id is: {}", pid);
 
@@ -161,10 +161,10 @@ public abstract class AbstractCommandExecutor {
                 exitStatusCode = process.exitValue();
                 logger.info("process has exited, work dir:{}, pid:{} ,exitStatusCode:{}", taskDir, pid,exitStatusCode);
                 //update process state to db
-                exitStatusCode = updateState(processDao, exitStatusCode, pid, taskInstId);
+                exitStatusCode = updateState(processService, exitStatusCode, pid, taskInstId);
 
             } else {
-                TaskInstance taskInstance = processDao.findTaskInstanceById(taskInstId);
+                TaskInstance taskInstance = processService.findTaskInstanceById(taskInstId);
                 if (taskInstance == null) {
                     logger.error("task instance id:{} not exist", taskInstId);
                 } else {
@@ -219,23 +219,23 @@ public abstract class AbstractCommandExecutor {
     /**
      * update process state to db
      *
-     * @param processDao        process dao
+     * @param processService        process dao
      * @param exitStatusCode    exit status code
      * @param pid               process id
      * @param taskInstId        task instance id
      * @return exit status code
      */
-    private int updateState(ProcessDao processDao, int exitStatusCode, int pid, int taskInstId) {
+    private int updateState(ProcessService processService, int exitStatusCode, int pid, int taskInstId) {
         //get yarn state by log
         if (exitStatusCode == 0) {
-            TaskInstance taskInstance = processDao.findTaskInstanceById(taskInstId);
+            TaskInstance taskInstance = processService.findTaskInstanceById(taskInstId);
             logger.info("process id is {}", pid);
 
             List<String> appIds = getAppLinks(taskInstance.getLogPath());
             if (appIds.size() > 0) {
                 String appUrl = String.join(Constants.COMMA, appIds);
                 logger.info("yarn log url:{}",appUrl);
-                processDao.updatePidByTaskInstId(taskInstId, pid, appUrl);
+                processService.updatePidByTaskInstId(taskInstId, pid, appUrl);
             }
 
             // check if all operations are completed
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractYarnTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractYarnTask.java
index 6846617..39f4dfb 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractYarnTask.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractYarnTask.java
@@ -16,10 +16,10 @@
  */
 package org.apache.dolphinscheduler.server.worker.task;
 
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.server.utils.ProcessUtils;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.slf4j.Logger;
 
 /**
@@ -39,7 +39,7 @@ public abstract class AbstractYarnTask extends AbstractTask {
   /**
    *  process database access
    */
-  protected ProcessDao processDao;
+  protected ProcessService processService;
 
   /**
    * Abstract Yarn Task
@@ -48,7 +48,7 @@ public abstract class AbstractYarnTask extends AbstractTask {
    */
   public AbstractYarnTask(TaskProps taskProps, Logger logger) {
     super(taskProps, logger);
-    this.processDao = SpringApplicationContext.getBean(ProcessDao.class);
+    this.processService = SpringApplicationContext.getBean(ProcessService.class);
     this.shellCommandExecutor = new ShellCommandExecutor(this::logHandle,
             taskProps.getTaskDir(),
             taskProps.getTaskAppId(),
@@ -64,7 +64,7 @@ public abstract class AbstractYarnTask extends AbstractTask {
   public void handle() throws Exception {
     try {
       // construct process
-      exitStatusCode = shellCommandExecutor.run(buildCommand(), processDao);
+      exitStatusCode = shellCommandExecutor.run(buildCommand(), processService);
     } catch (Exception e) {
       logger.error("yarn process failure", e);
       exitStatusCode = -1;
@@ -82,7 +82,7 @@ public abstract class AbstractYarnTask extends AbstractTask {
     cancel = true;
     // cancel process
     shellCommandExecutor.cancelApplication();
-    TaskInstance taskInstance = processDao.findTaskInstanceById(taskProps.getTaskInstId());
+    TaskInstance taskInstance = processService.findTaskInstanceById(taskProps.getTaskInstId());
     if (status && taskInstance != null){
       ProcessUtils.killYarnJob(taskInstance);
     }
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java
index 0de2bbc..ef941cd 100755
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java
@@ -39,23 +39,23 @@ import java.util.Set;
 import org.apache.commons.io.FileUtils;
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.DbType;
-import org.apache.dolphinscheduler.common.job.db.BaseDataSource;
-import org.apache.dolphinscheduler.common.job.db.DataSourceFactory;
 import org.apache.dolphinscheduler.common.process.Property;
 import org.apache.dolphinscheduler.common.task.AbstractParameters;
 import org.apache.dolphinscheduler.common.task.datax.DataxParameters;
 import org.apache.dolphinscheduler.common.utils.CollectionUtils;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
 import org.apache.dolphinscheduler.common.utils.ParameterUtils;
-import org.apache.dolphinscheduler.dao.ProcessDao;
+import org.apache.dolphinscheduler.dao.datasource.BaseDataSource;
+import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory;
 import org.apache.dolphinscheduler.dao.entity.DataSource;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
 import org.apache.dolphinscheduler.server.utils.DataxUtils;
 import org.apache.dolphinscheduler.server.utils.ParamUtils;
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
 import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
 import org.apache.dolphinscheduler.server.worker.task.ShellCommandExecutor;
 import org.apache.dolphinscheduler.server.worker.task.TaskProps;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.slf4j.Logger;
 
 import com.alibaba.druid.sql.ast.SQLStatement;
@@ -106,9 +106,9 @@ public class DataxTask extends AbstractTask {
     private ShellCommandExecutor shellCommandExecutor;
 
     /**
-     * process database access
+     * process dao
      */
-    private ProcessDao processDao;
+    private ProcessService processService;
 
     /**
      * constructor
@@ -128,7 +128,7 @@ public class DataxTask extends AbstractTask {
             props.getTaskInstId(), props.getTenantCode(), props.getEnvFile(), props.getTaskStartTime(),
             props.getTaskTimeout(), logger);
 
-        this.processDao = SpringApplicationContext.getBean(ProcessDao.class);
+        this.processService = SpringApplicationContext.getBean(ProcessService.class);
     }
 
     /**
@@ -160,7 +160,7 @@ public class DataxTask extends AbstractTask {
             // run datax process
             String jsonFilePath = buildDataxJsonFile();
             String shellCommandFilePath = buildShellCommandFile(jsonFilePath);
-            exitStatusCode = shellCommandExecutor.run(shellCommandFilePath, processDao);
+            exitStatusCode = shellCommandExecutor.run(shellCommandFilePath, processService);
         }
         catch (Exception e) {
             exitStatusCode = -1;
@@ -220,11 +220,11 @@ public class DataxTask extends AbstractTask {
      */
     private List<JSONObject> buildDataxJobContentJson()
         throws SQLException {
-        DataSource dataSource = processDao.findDataSourceById(dataXParameters.getDataSource());
+        DataSource dataSource = processService.findDataSourceById(dataXParameters.getDataSource());
         BaseDataSource dataSourceCfg = DataSourceFactory.getDatasource(dataSource.getType(),
             dataSource.getConnectionParams());
 
-        DataSource dataTarget = processDao.findDataSourceById(dataXParameters.getDataTarget());
+        DataSource dataTarget = processService.findDataSourceById(dataXParameters.getDataTarget());
         BaseDataSource dataTargetCfg = DataSourceFactory.getDatasource(dataTarget.getType(),
             dataTarget.getConnectionParams());
 
@@ -355,7 +355,7 @@ public class DataxTask extends AbstractTask {
         String dataxCommand = sbr.toString();
 
         // find process instance by task id
-        ProcessInstance processInstance = processDao.findProcessInstanceByTaskId(taskProps.getTaskInstId());
+        ProcessInstance processInstance = processService.findProcessInstanceByTaskId(taskProps.getTaskInstId());
 
         // combining local and global parameters
         Map<String, Property> paramsMap = ParamUtils.convert(taskProps.getUserDefParamsMap(),
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentExecute.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentExecute.java
index 4be65ed..b08cabc 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentExecute.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentExecute.java
@@ -23,10 +23,10 @@ import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.common.model.DateInterval;
 import org.apache.dolphinscheduler.common.model.DependentItem;
 import org.apache.dolphinscheduler.common.utils.DependentUtils;
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -37,9 +37,9 @@ import java.util.*;
  */
 public class DependentExecute {
     /**
-     * process dao
+     * process service
      */
-    private final ProcessDao processDao = SpringApplicationContext.getBean(ProcessDao.class);
+    private final ProcessService processService = SpringApplicationContext.getBean(ProcessService.class);
 
     /**
      * depend item list
@@ -108,7 +108,7 @@ public class DependentExecute {
                 result = getDependResultByState(processInstance.getState());
             }else{
                 TaskInstance taskInstance = null;
-                List<TaskInstance> taskInstanceList = processDao.findValidTaskListByProcessId(processInstance.getId());
+                List<TaskInstance> taskInstanceList = processService.findValidTaskListByProcessId(processInstance.getId());
 
                 for(TaskInstance task : taskInstanceList){
                     if(task.getName().equals(dependentItem.getDepTasks())){
@@ -141,16 +141,16 @@ public class DependentExecute {
      */
     private ProcessInstance findLastProcessInterval(int definitionId, DateInterval dateInterval) {
 
-        ProcessInstance runningProcess = processDao.findLastRunningProcess(definitionId, dateInterval);
+        ProcessInstance runningProcess = processService.findLastRunningProcess(definitionId, dateInterval);
         if(runningProcess != null){
             return runningProcess;
         }
 
-        ProcessInstance lastSchedulerProcess = processDao.findLastSchedulerProcessInterval(
+        ProcessInstance lastSchedulerProcess = processService.findLastSchedulerProcessInterval(
                 definitionId, dateInterval
         );
 
-        ProcessInstance lastManualProcess = processDao.findLastManualProcessInterval(
+        ProcessInstance lastManualProcess = processService.findLastManualProcessInterval(
                 definitionId, dateInterval
         );
 
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentTask.java
index 9af29e0..f074d57 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentTask.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentTask.java
@@ -25,11 +25,11 @@ import org.apache.dolphinscheduler.common.task.dependent.DependentParameters;
 import org.apache.dolphinscheduler.common.thread.Stopper;
 import org.apache.dolphinscheduler.common.utils.DependentUtils;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
 import org.apache.dolphinscheduler.server.worker.task.TaskProps;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.slf4j.Logger;
 
 import java.util.*;
@@ -63,9 +63,9 @@ public class DependentTask extends AbstractTask {
     private Date dependentDate;
 
     /**
-     * process dao
+     * process service
      */
-    private ProcessDao processDao;
+    private ProcessService processService;
 
     /**
      * constructor
@@ -88,7 +88,7 @@ public class DependentTask extends AbstractTask {
                             taskModel.getDependItemList(), taskModel.getRelation()));
         }
 
-        this.processDao = SpringApplicationContext.getBean(ProcessDao.class);
+        this.processService = SpringApplicationContext.getBean(ProcessService.class);
 
         if(taskProps.getScheduleTime() != null){
             this.dependentDate = taskProps.getScheduleTime();
@@ -107,7 +107,7 @@ public class DependentTask extends AbstractTask {
         try{
             TaskInstance taskInstance = null;
             while(Stopper.isRunning()){
-                taskInstance = processDao.findTaskInstanceById(this.taskProps.getTaskInstId());
+                taskInstance = processService.findTaskInstanceById(this.taskProps.getTaskInstId());
 
                 if(taskInstance == null){
                     exitStatusCode = -1;
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/flink/FlinkTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/flink/FlinkTask.java
index 0fa9e11..c562fbe 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/flink/FlinkTask.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/flink/FlinkTask.java
@@ -68,7 +68,7 @@ public class FlinkTask extends AbstractYarnTask {
     if (StringUtils.isNotEmpty(flinkParameters.getMainArgs())) {
       String args = flinkParameters.getMainArgs();
       // get process instance by task instance id
-      ProcessInstance processInstance = processDao.findProcessInstanceByTaskId(taskProps.getTaskInstId());
+      ProcessInstance processInstance = processService.findProcessInstanceByTaskId(taskProps.getTaskInstId());
 
       /**
        *  combining local and global parameters
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/http/HttpTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/http/HttpTask.java
index 97e6cb7..c925f90 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/http/HttpTask.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/http/HttpTask.java
@@ -28,13 +28,13 @@ import org.apache.dolphinscheduler.common.task.AbstractParameters;
 import org.apache.dolphinscheduler.common.task.http.HttpParameters;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
 import org.apache.dolphinscheduler.common.utils.ParameterUtils;
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
 import org.apache.dolphinscheduler.common.utils.StringUtils;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
 import org.apache.dolphinscheduler.server.utils.ParamUtils;
 import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
 import org.apache.dolphinscheduler.server.worker.task.TaskProps;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.apache.http.HttpEntity;
 import org.apache.http.ParseException;
 import org.apache.http.client.config.RequestConfig;
@@ -66,9 +66,9 @@ public class HttpTask extends AbstractTask {
     private HttpParameters httpParameters;
 
     /**
-     *  process database access
+     *  process service
      */
-    private ProcessDao processDao;
+    private ProcessService processService;
 
     /**
      * Convert mill seconds to second unit
@@ -92,7 +92,7 @@ public class HttpTask extends AbstractTask {
      */
     public HttpTask(TaskProps props, Logger logger) {
         super(props, logger);
-        this.processDao = SpringApplicationContext.getBean(ProcessDao.class);
+        this.processService = SpringApplicationContext.getBean(ProcessService.class);
     }
 
     @Override
@@ -138,7 +138,7 @@ public class HttpTask extends AbstractTask {
      */
     protected CloseableHttpResponse sendRequest(CloseableHttpClient client) throws IOException {
         RequestBuilder builder = createRequestBuilder();
-        ProcessInstance processInstance = processDao.findProcessInstanceByTaskId(taskProps.getTaskInstId());
+        ProcessInstance processInstance = processService.findProcessInstanceByTaskId(taskProps.getTaskInstId());
 
         Map<String, Property> paramsMap = ParamUtils.convert(taskProps.getUserDefParamsMap(),
                 taskProps.getDefinedParams(),
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java
index 9b4952b..fb88145 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java
@@ -22,19 +22,19 @@ import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.DataType;
 import org.apache.dolphinscheduler.common.enums.Direct;
 import org.apache.dolphinscheduler.common.enums.TaskTimeoutStrategy;
-import org.apache.dolphinscheduler.common.job.db.BaseDataSource;
-import org.apache.dolphinscheduler.common.job.db.DataSourceFactory;
 import org.apache.dolphinscheduler.common.process.Property;
 import org.apache.dolphinscheduler.common.task.AbstractParameters;
 import org.apache.dolphinscheduler.common.task.procedure.ProcedureParameters;
 import org.apache.dolphinscheduler.common.utils.CollectionUtils;
 import org.apache.dolphinscheduler.common.utils.ParameterUtils;
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
-import org.apache.dolphinscheduler.dao.ProcessDao;
+import org.apache.dolphinscheduler.dao.datasource.BaseDataSource;
+import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory;
 import org.apache.dolphinscheduler.dao.entity.DataSource;
 import org.apache.dolphinscheduler.server.utils.ParamUtils;
 import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
 import org.apache.dolphinscheduler.server.worker.task.TaskProps;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.slf4j.Logger;
 
 import java.sql.*;
@@ -56,9 +56,9 @@ public class ProcedureTask extends AbstractTask {
     private ProcedureParameters procedureParameters;
 
     /**
-     *  process database access
+     *  process service
      */
-    private ProcessDao processDao;
+    private ProcessService processService;
 
     /**
      * base datasource
@@ -82,7 +82,7 @@ public class ProcedureTask extends AbstractTask {
             throw new RuntimeException("procedure task params is not valid");
         }
 
-        this.processDao = SpringApplicationContext.getBean(ProcessDao.class);
+        this.processService = SpringApplicationContext.getBean(ProcessService.class);
     }
 
     @Override
@@ -97,7 +97,7 @@ public class ProcedureTask extends AbstractTask {
                 procedureParameters.getMethod(),
                 procedureParameters.getLocalParams());
 
-        DataSource dataSource = processDao.findDataSourceById(procedureParameters.getDatasource());
+        DataSource dataSource = processService.findDataSourceById(procedureParameters.getDatasource());
         if (dataSource == null){
             logger.error("datasource not exists");
             exitStatusCode = -1;
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/python/PythonTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/python/PythonTask.java
index 585d62f..fc212f8 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/python/PythonTask.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/python/PythonTask.java
@@ -22,12 +22,12 @@ import org.apache.dolphinscheduler.common.task.AbstractParameters;
 import org.apache.dolphinscheduler.common.task.python.PythonParameters;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
 import org.apache.dolphinscheduler.common.utils.ParameterUtils;
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.server.utils.ParamUtils;
 import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
 import org.apache.dolphinscheduler.server.worker.task.PythonCommandExecutor;
 import org.apache.dolphinscheduler.server.worker.task.TaskProps;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.slf4j.Logger;
 
 import java.util.Map;
@@ -53,9 +53,9 @@ public class PythonTask extends AbstractTask {
   private PythonCommandExecutor pythonCommandExecutor;
 
   /**
-   * process database access
+   * process service
    */
-  private ProcessDao processDao;
+  private ProcessService processService;
 
   /**
    * constructor
@@ -76,7 +76,7 @@ public class PythonTask extends AbstractTask {
             taskProps.getTaskStartTime(),
             taskProps.getTaskTimeout(),
             logger);
-    this.processDao = SpringApplicationContext.getBean(ProcessDao.class);
+    this.processService = SpringApplicationContext.getBean(ProcessService.class);
   }
 
   @Override
@@ -94,7 +94,7 @@ public class PythonTask extends AbstractTask {
   public void handle() throws Exception {
     try {
       //  construct process
-      exitStatusCode = pythonCommandExecutor.run(buildCommand(), processDao);
+      exitStatusCode = pythonCommandExecutor.run(buildCommand(), processService);
     } catch (Exception e) {
       logger.error("python task failure", e);
       exitStatusCode = -1;
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java
index 789a0c5..5704c80 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java
@@ -23,12 +23,12 @@ import org.apache.dolphinscheduler.common.task.AbstractParameters;
 import org.apache.dolphinscheduler.common.task.shell.ShellParameters;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
 import org.apache.dolphinscheduler.common.utils.ParameterUtils;
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.server.utils.ParamUtils;
 import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
 import org.apache.dolphinscheduler.server.worker.task.ShellCommandExecutor;
 import org.apache.dolphinscheduler.server.worker.task.TaskProps;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.slf4j.Logger;
 
 import java.io.File;
@@ -64,7 +64,7 @@ public class ShellTask extends AbstractTask {
   /**
    * process database access
    */
-  private ProcessDao processDao;
+  private ProcessService processService;
 
   /**
    * constructor
@@ -84,7 +84,7 @@ public class ShellTask extends AbstractTask {
             taskProps.getTaskStartTime(),
             taskProps.getTaskTimeout(),
             logger);
-    this.processDao = SpringApplicationContext.getBean(ProcessDao.class);
+    this.processService = SpringApplicationContext.getBean(ProcessService.class);
   }
 
   @Override
@@ -102,7 +102,7 @@ public class ShellTask extends AbstractTask {
   public void handle() throws Exception {
     try {
       // construct process
-      exitStatusCode = shellCommandExecutor.run(buildCommand(), processDao);
+      exitStatusCode = shellCommandExecutor.run(buildCommand(), processService);
     } catch (Exception e) {
       logger.error("shell task failure", e);
       exitStatusCode = -1;
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java
index eba05a0..aae11f5 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java
@@ -27,8 +27,6 @@ import org.apache.dolphinscheduler.common.enums.AuthorizationType;
 import org.apache.dolphinscheduler.common.enums.ShowType;
 import org.apache.dolphinscheduler.common.enums.TaskTimeoutStrategy;
 import org.apache.dolphinscheduler.common.enums.UdfType;
-import org.apache.dolphinscheduler.common.job.db.BaseDataSource;
-import org.apache.dolphinscheduler.common.job.db.DataSourceFactory;
 import org.apache.dolphinscheduler.common.process.Property;
 import org.apache.dolphinscheduler.common.task.AbstractParameters;
 import org.apache.dolphinscheduler.common.task.sql.SqlBinds;
@@ -36,16 +34,19 @@ import org.apache.dolphinscheduler.common.task.sql.SqlParameters;
 import org.apache.dolphinscheduler.common.task.sql.SqlType;
 import org.apache.dolphinscheduler.common.utils.*;
 import org.apache.dolphinscheduler.dao.AlertDao;
-import org.apache.dolphinscheduler.dao.ProcessDao;
+import org.apache.dolphinscheduler.dao.datasource.BaseDataSource;
+import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory;
 import org.apache.dolphinscheduler.dao.entity.DataSource;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
 import org.apache.dolphinscheduler.dao.entity.UdfFunc;
 import org.apache.dolphinscheduler.dao.entity.User;
-import org.apache.dolphinscheduler.dao.permission.PermissionCheck;
 import org.apache.dolphinscheduler.server.utils.ParamUtils;
 import org.apache.dolphinscheduler.server.utils.UDFUtils;
 import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
 import org.apache.dolphinscheduler.server.worker.task.TaskProps;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.permission.PermissionCheck;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.slf4j.Logger;
 
 import java.sql.*;
@@ -67,9 +68,9 @@ public class SqlTask extends AbstractTask {
     private SqlParameters sqlParameters;
 
     /**
-     *  process database access
+     *  process service
      */
-    private ProcessDao processDao;
+    private ProcessService processService;
 
     /**
      *  alert dao
@@ -96,7 +97,7 @@ public class SqlTask extends AbstractTask {
         if (!sqlParameters.checkParameters()) {
             throw new RuntimeException("sql task params is not valid");
         }
-        this.processDao = SpringApplicationContext.getBean(ProcessDao.class);
+        this.processService = SpringApplicationContext.getBean(ProcessService.class);
         this.alertDao = SpringApplicationContext.getBean(AlertDao.class);
     }
 
@@ -122,7 +123,7 @@ public class SqlTask extends AbstractTask {
             return;
         }
 
-        dataSource= processDao.findDataSourceById(sqlParameters.getDatasource());
+        dataSource= processService.findDataSourceById(sqlParameters.getDatasource());
 
         // data source is null
         if (dataSource == null){
@@ -171,7 +172,7 @@ public class SqlTask extends AbstractTask {
                 }
                 // check udf permission
                 checkUdfPermission(ArrayUtils.toObject(idsArray));
-                List<UdfFunc> udfFuncList = processDao.queryUdfFunListByids(idsArray);
+                List<UdfFunc> udfFuncList = processService.queryUdfFunListByids(idsArray);
                 createFuncs = UDFUtils.createFuncs(udfFuncList, taskProps.getTenantCode(), logger);
             }
 
@@ -383,7 +384,7 @@ public class SqlTask extends AbstractTask {
     public void sendAttachment(String title,String content){
 
         //  process instance
-        ProcessInstance instance = processDao.findProcessInstanceByTaskId(taskProps.getTaskInstId());
+        ProcessInstance instance = processService.findProcessInstanceByTaskId(taskProps.getTaskInstId());
 
         List<User> users = alertDao.queryUserByAlertGroupId(instance.getWarningGroupId());
 
@@ -470,10 +471,10 @@ public class SqlTask extends AbstractTask {
      */
     private void checkUdfPermission(Integer[] udfFunIds) throws Exception{
         //  process instance
-        ProcessInstance processInstance = processDao.findProcessInstanceByTaskId(taskProps.getTaskInstId());
+        ProcessInstance processInstance = processService.findProcessInstanceByTaskId(taskProps.getTaskInstId());
         int userId = processInstance.getExecutorId();
 
-        PermissionCheck<Integer> permissionCheckUdf = new PermissionCheck<Integer>(AuthorizationType.UDF,processDao,udfFunIds,userId,logger);
+        PermissionCheck<Integer> permissionCheckUdf = new PermissionCheck<Integer>(AuthorizationType.UDF, processService,udfFunIds,userId,logger);
         permissionCheckUdf.checkPermission();
     }
 
@@ -484,10 +485,10 @@ public class SqlTask extends AbstractTask {
      */
     private void checkDataSourcePermission(int dataSourceId) throws Exception{
         //  process instance
-        ProcessInstance processInstance = processDao.findProcessInstanceByTaskId(taskProps.getTaskInstId());
+        ProcessInstance processInstance = processService.findProcessInstanceByTaskId(taskProps.getTaskInstId());
         int userId = processInstance.getExecutorId();
 
-        PermissionCheck<Integer> permissionCheckDataSource = new PermissionCheck<Integer>(AuthorizationType.DATASOURCE,processDao,new Integer[]{dataSourceId},userId,logger);
+        PermissionCheck<Integer> permissionCheckDataSource = new PermissionCheck<Integer>(AuthorizationType.DATASOURCE, processService,new Integer[]{dataSourceId},userId,logger);
         permissionCheckDataSource.checkPermission();
     }
 
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java
index c6a71ed..fe4ec91 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java
@@ -21,10 +21,8 @@ import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.common.enums.ZKNodeType;
 import org.apache.dolphinscheduler.common.model.Server;
-import org.apache.dolphinscheduler.common.zk.AbstractZKClient;
 import org.apache.dolphinscheduler.dao.AlertDao;
 import org.apache.dolphinscheduler.dao.DaoFactory;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.server.utils.ProcessUtils;
@@ -32,6 +30,8 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.recipes.locks.InterProcessMutex;
 import org.apache.curator.utils.ThreadUtils;
+import org.apache.dolphinscheduler.service.process.ProcessService;
+import org.apache.dolphinscheduler.service.zk.AbstractZKClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -70,10 +70,10 @@ public class ZKMasterClient extends AbstractZKClient {
 	 */
 	private AlertDao alertDao = null;
 	/**
-	 *  flow database access
+	 *  process service
 	 */
 	@Autowired
-	private ProcessDao processDao;
+	private ProcessService processService;
 
 	/**
 	 * default constructor
@@ -374,7 +374,7 @@ public class ZKMasterClient extends AbstractZKClient {
 	private void failoverWorker(String workerHost, boolean needCheckWorkerAlive) throws Exception {
 		logger.info("start worker[{}] failover ...", workerHost);
 
-		List<TaskInstance> needFailoverTaskInstanceList = processDao.queryNeedFailoverTaskInstances(workerHost);
+		List<TaskInstance> needFailoverTaskInstanceList = processService.queryNeedFailoverTaskInstances(workerHost);
 		for(TaskInstance taskInstance : needFailoverTaskInstanceList){
 			if(needCheckWorkerAlive){
 				if(!checkTaskInstanceNeedFailover(taskInstance)){
@@ -382,7 +382,7 @@ public class ZKMasterClient extends AbstractZKClient {
                 }
 			}
 
-			ProcessInstance instance = processDao.findProcessInstanceDetailById(taskInstance.getProcessInstanceId());
+			ProcessInstance instance = processService.findProcessInstanceDetailById(taskInstance.getProcessInstanceId());
 			if(instance!=null){
 				taskInstance.setProcessInstance(instance);
 			}
@@ -390,7 +390,7 @@ public class ZKMasterClient extends AbstractZKClient {
 			ProcessUtils.killYarnJob(taskInstance);
 
 			taskInstance.setState(ExecutionStatus.NEED_FAULT_TOLERANCE);
-			processDao.saveTaskInstance(taskInstance);
+			processService.saveTaskInstance(taskInstance);
 		}
 		logger.info("end worker[{}] failover ...", workerHost);
 	}
@@ -403,11 +403,11 @@ public class ZKMasterClient extends AbstractZKClient {
 	private void failoverMaster(String masterHost) {
 		logger.info("start master failover ...");
 
-		List<ProcessInstance> needFailoverProcessInstanceList = processDao.queryNeedFailoverProcessInstances(masterHost);
+		List<ProcessInstance> needFailoverProcessInstanceList = processService.queryNeedFailoverProcessInstances(masterHost);
 
 		//updateProcessInstance host is null and insert into command
 		for(ProcessInstance processInstance : needFailoverProcessInstanceList){
-			processDao.processNeedFailoverProcessInstances(processInstance);
+			processService.processNeedFailoverProcessInstances(processInstance);
 		}
 
 		logger.info("master failover end");
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKWorkerClient.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKWorkerClient.java
index 88abfa3..7ddee3b 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKWorkerClient.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKWorkerClient.java
@@ -19,9 +19,9 @@ package org.apache.dolphinscheduler.server.zk;
 import org.apache.curator.framework.recipes.cache.TreeCacheEvent;
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.ZKNodeType;
-import org.apache.dolphinscheduler.common.zk.AbstractZKClient;
 import org.apache.commons.lang.StringUtils;
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.dolphinscheduler.service.zk.AbstractZKClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.stereotype.Component;
diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java
index d7c3de1..770ab3c 100644
--- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java
+++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java
@@ -20,13 +20,13 @@ import com.alibaba.fastjson.JSONObject;
 import org.apache.dolphinscheduler.common.enums.*;
 import org.apache.dolphinscheduler.common.graph.DAG;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
 import org.apache.dolphinscheduler.dao.entity.Schedule;
 import org.apache.dolphinscheduler.server.master.config.MasterConfig;
 import org.apache.dolphinscheduler.server.master.runner.MasterExecThread;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -57,7 +57,7 @@ public class MasterExecThreadTest {
 
     private ProcessInstance processInstance;
 
-    private ProcessDao processDao;
+    private ProcessService processService;
 
     private int processDefinitionId = 1;
 
@@ -67,7 +67,7 @@ public class MasterExecThreadTest {
 
     @Before
     public void init() throws Exception{
-        processDao = mock(ProcessDao.class);
+        processService = mock(ProcessService.class);
 
         applicationContext = mock(ApplicationContext.class);
         config = new MasterConfig();
@@ -91,7 +91,7 @@ public class MasterExecThreadTest {
         processDefinition.setGlobalParamList(Collections.EMPTY_LIST);
         Mockito.when(processInstance.getProcessDefinition()).thenReturn(processDefinition);
 
-        masterExecThread = PowerMockito.spy(new MasterExecThread(processInstance, processDao));
+        masterExecThread = PowerMockito.spy(new MasterExecThread(processInstance, processService));
         // prepareProcess init dag
         Field dag = MasterExecThread.class.getDeclaredField("dag");
         dag.setAccessible(true);
@@ -110,12 +110,12 @@ public class MasterExecThreadTest {
     @Test
     public void testParallelWithOutSchedule() throws ParseException {
         try{
-            Mockito.when(processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList());
+            Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList());
             Method method = MasterExecThread.class.getDeclaredMethod("executeComplementProcess");
             method.setAccessible(true);
             method.invoke(masterExecThread);
             // one create save, and 1-30 for next save, and last day 31 no save
-            verify(processDao, times(31)).saveProcessInstance(processInstance);
+            verify(processService, times(31)).saveProcessInstance(processInstance);
         }catch (Exception e){
             e.printStackTrace();
             Assert.assertTrue(false);
@@ -129,12 +129,12 @@ public class MasterExecThreadTest {
     @Test
     public void testParallelWithSchedule() throws ParseException {
         try{
-            Mockito.when(processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(oneSchedulerList());
+            Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(oneSchedulerList());
             Method method = MasterExecThread.class.getDeclaredMethod("executeComplementProcess");
             method.setAccessible(true);
             method.invoke(masterExecThread);
             // one create save, and 15(1 to 31 step 2) for next save, and last day 31 no save
-            verify(processDao, times(15)).saveProcessInstance(processInstance);
+            verify(processService, times(15)).saveProcessInstance(processInstance);
         }catch (Exception e){
             Assert.assertTrue(false);
         }
diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/shell/ShellCommandExecutorTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/shell/ShellCommandExecutorTest.java
index 04c8448..5d42636 100644
--- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/shell/ShellCommandExecutorTest.java
+++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/shell/ShellCommandExecutorTest.java
@@ -20,13 +20,13 @@ import com.alibaba.fastjson.JSONObject;
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.common.model.TaskNode;
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.common.utils.LoggerUtils;
 import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
 import org.apache.dolphinscheduler.server.worker.task.TaskManager;
 import org.apache.dolphinscheduler.server.worker.task.TaskProps;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -43,11 +43,11 @@ public class ShellCommandExecutorTest {
 
     private static final Logger logger = LoggerFactory.getLogger(ShellCommandExecutorTest.class);
 
-    private ProcessDao processDao = null;
+    private ProcessService processService = null;
 
     @Before
     public void before(){
-        processDao = SpringApplicationContext.getBean(ProcessDao.class);
+        processService = SpringApplicationContext.getBean(ProcessService.class);
     }
 
     @Test
@@ -65,7 +65,7 @@ public class ShellCommandExecutorTest {
 
 
 
-        TaskInstance taskInstance = processDao.findTaskInstanceById(7657);
+        TaskInstance taskInstance = processService.findTaskInstanceById(7657);
 
         String taskJson = taskInstance.getTaskJson();
         TaskNode taskNode = JSONObject.parseObject(taskJson, TaskNode.class);
diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/sql/SqlExecutorTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/sql/SqlExecutorTest.java
index 7da3f71..c395eab 100644
--- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/sql/SqlExecutorTest.java
+++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/sql/SqlExecutorTest.java
@@ -21,13 +21,13 @@ import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.CommandType;
 import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.common.model.TaskNode;
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.TaskInstance;
 import org.apache.dolphinscheduler.common.utils.LoggerUtils;
 import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
 import org.apache.dolphinscheduler.server.worker.task.TaskManager;
 import org.apache.dolphinscheduler.server.worker.task.TaskProps;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -44,11 +44,11 @@ public class SqlExecutorTest {
 
     private static final Logger logger = LoggerFactory.getLogger(SqlExecutorTest.class);
 
-    private ProcessDao processDao = null;
+    private ProcessService processService = null;
 
     @Before
     public void before(){
-        processDao = SpringApplicationContext.getBean(ProcessDao.class);
+        processService = SpringApplicationContext.getBean(ProcessService.class);
     }
 
     @Test
@@ -109,7 +109,7 @@ public class SqlExecutorTest {
         taskProps.setCmdTypeIfComplement(CommandType.START_PROCESS);
 
 
-        TaskInstance taskInstance = processDao.findTaskInstanceById(taskInstId);
+        TaskInstance taskInstance = processService.findTaskInstanceById(taskInstId);
 
         String taskJson = taskInstance.getTaskJson();
         TaskNode taskNode = JSONObject.parseObject(taskJson, TaskNode.class);
diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java
index 7a6073e..bd7f275 100644
--- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java
+++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java
@@ -25,15 +25,15 @@ import java.util.List;
 import com.alibaba.fastjson.JSONObject;
 import org.apache.dolphinscheduler.common.enums.CommandType;
 import org.apache.dolphinscheduler.common.enums.DbType;
-import org.apache.dolphinscheduler.common.job.db.BaseDataSource;
-import org.apache.dolphinscheduler.common.job.db.DataSourceFactory;
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
-import org.apache.dolphinscheduler.dao.ProcessDao;
+import org.apache.dolphinscheduler.dao.datasource.BaseDataSource;
+import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory;
 import org.apache.dolphinscheduler.dao.entity.DataSource;
 import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
 import org.apache.dolphinscheduler.server.utils.DataxUtils;
 import org.apache.dolphinscheduler.server.worker.task.ShellCommandExecutor;
 import org.apache.dolphinscheduler.server.worker.task.TaskProps;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -53,7 +53,7 @@ public class DataxTaskTest {
 
     private DataxTask dataxTask;
 
-    private ProcessDao processDao;
+    private ProcessService processService;
 
     private ShellCommandExecutor shellCommandExecutor;
 
@@ -62,13 +62,13 @@ public class DataxTaskTest {
     @Before
     public void before()
         throws Exception {
-        processDao = Mockito.mock(ProcessDao.class);
+        processService = Mockito.mock(ProcessService.class);
         shellCommandExecutor = Mockito.mock(ShellCommandExecutor.class);
 
         applicationContext = Mockito.mock(ApplicationContext.class);
         SpringApplicationContext springApplicationContext = new SpringApplicationContext();
         springApplicationContext.setApplicationContext(applicationContext);
-        Mockito.when(applicationContext.getBean(ProcessDao.class)).thenReturn(processDao);
+        Mockito.when(applicationContext.getBean(ProcessService.class)).thenReturn(processService);
 
         TaskProps props = new TaskProps();
         props.setTaskDir("/tmp");
@@ -83,12 +83,12 @@ public class DataxTaskTest {
         dataxTask = PowerMockito.spy(new DataxTask(props, logger));
         dataxTask.init();
 
-        Mockito.when(processDao.findDataSourceById(1)).thenReturn(getDataSource());
-        Mockito.when(processDao.findDataSourceById(2)).thenReturn(getDataSource());
-        Mockito.when(processDao.findProcessInstanceByTaskId(1)).thenReturn(getProcessInstance());
+        Mockito.when(processService.findDataSourceById(1)).thenReturn(getDataSource());
+        Mockito.when(processService.findDataSourceById(2)).thenReturn(getDataSource());
+        Mockito.when(processService.findProcessInstanceByTaskId(1)).thenReturn(getProcessInstance());
 
         String fileName = String.format("%s/%s_node.sh", props.getTaskDir(), props.getTaskAppId());
-        Mockito.when(shellCommandExecutor.run(fileName, processDao)).thenReturn(0);
+        Mockito.when(shellCommandExecutor.run(fileName, processService)).thenReturn(0);
     }
 
     private DataSource getDataSource() {
diff --git a/dolphinscheduler-service/pom.xml b/dolphinscheduler-service/pom.xml
new file mode 100644
index 0000000..7d775d5
--- /dev/null
+++ b/dolphinscheduler-service/pom.xml
@@ -0,0 +1,56 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<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</artifactId>
+        <groupId>org.apache.dolphinscheduler</groupId>
+        <version>1.2.1-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>dolphinscheduler-service</artifactId>
+
+    <name>dolphinscheduler-service</name>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.dolphinscheduler</groupId>
+            <artifactId>dolphinscheduler-remote</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.dolphinscheduler</groupId>
+            <artifactId>dolphinscheduler-dao</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-client</artifactId>
+            <version>${curator.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>log4j-1.2-api</groupId>
+                    <artifactId>org.apache.logging.log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>io.netty</groupId>
+                    <artifactId>netty</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.quartz-scheduler</groupId>
+            <artifactId>quartz</artifactId>
+            <exclusions>
+                <exclusion>
+                    <artifactId>c3p0</artifactId>
+                    <groupId>c3p0</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.quartz-scheduler</groupId>
+            <artifactId>quartz-jobs</artifactId>
+        </dependency>
+    </dependencies>
+</project>
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/SpringApplicationContext.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/bean/SpringApplicationContext.java
similarity index 96%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/SpringApplicationContext.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/bean/SpringApplicationContext.java
index 97618e1..ddf1fec 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/SpringApplicationContext.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/bean/SpringApplicationContext.java
@@ -14,14 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.utils;
+package org.apache.dolphinscheduler.service.bean;
 
 import org.springframework.beans.BeansException;
 import org.springframework.context.ApplicationContext;
 import org.springframework.context.ApplicationContextAware;
 import org.springframework.stereotype.Component;
 
-
 @Component
 public class SpringApplicationContext implements ApplicationContextAware {
 
diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java
new file mode 100644
index 0000000..aa6999e
--- /dev/null
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java
@@ -0,0 +1,166 @@
+/*
+ * 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.service.log;
+
+import io.netty.channel.Channel;
+import org.apache.dolphinscheduler.remote.NettyRemotingClient;
+import org.apache.dolphinscheduler.remote.command.Command;
+import org.apache.dolphinscheduler.remote.command.CommandType;
+import org.apache.dolphinscheduler.remote.command.log.*;
+import org.apache.dolphinscheduler.remote.config.NettyClientConfig;
+import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
+import org.apache.dolphinscheduler.remote.utils.Address;
+import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * log client
+ */
+public class LogClientService implements NettyRequestProcessor {
+
+    private static final Logger logger = LoggerFactory.getLogger(LogClientService.class);
+
+    private final NettyClientConfig clientConfig;
+
+    private final NettyRemotingClient client;
+
+    private final Address address;
+
+    /**
+     *  request time out
+     */
+    private final long logRequestTimeout = 10 * 1000;
+
+    /**
+     * construct client
+     * @param host host
+     * @param port port
+     */
+    public LogClientService(String host, int port) {
+        this.address = new Address(host, port);
+        this.clientConfig = new NettyClientConfig();
+        this.clientConfig.setWorkerThreads(1);
+        this.client = new NettyRemotingClient(clientConfig);
+        this.client.registerProcessor(CommandType.ROLL_VIEW_LOG_RESPONSE,this);
+        this.client.registerProcessor(CommandType.VIEW_WHOLE_LOG_RESPONSE, this);
+        this.client.registerProcessor(CommandType.GET_LOG_BYTES_RESPONSE, this);
+
+    }
+
+    /**
+     * close
+     */
+    public void close()  {
+        this.client.close();
+        logger.info("logger client closed");
+    }
+
+    /**
+     * roll view log
+     * @param path path
+     * @param skipLineNum skip line number
+     * @param limit limit
+     * @return log content
+     */
+    public String rollViewLog(String path,int skipLineNum,int limit) {
+        logger.info("roll view log, path {}, skipLineNum {} ,limit {}", path, skipLineNum, limit);
+        RollViewLogRequestCommand request = new RollViewLogRequestCommand(path, skipLineNum, limit);
+        String result = "";
+        try {
+            Command command = request.convert2Command();
+            this.client.send(address, command);
+            LogPromise promise = new LogPromise(command.getOpaque(), logRequestTimeout);
+            result = ((String)promise.getResult());
+        } catch (Exception e) {
+            logger.error("roll view log error", e);
+        }
+        return result;
+    }
+
+    /**
+     * view log
+     * @param path path
+     * @return log content
+     */
+    public String viewLog(String path) {
+        logger.info("view log path {}", path);
+        ViewLogRequestCommand request = new ViewLogRequestCommand(path);
+        String result = "";
+        try {
+            Command command = request.convert2Command();
+            this.client.send(address, command);
+            LogPromise promise = new LogPromise(command.getOpaque(), logRequestTimeout);
+            result = ((String)promise.getResult());
+        } catch (Exception e) {
+            logger.error("view log error", e);
+        }
+        return result;
+    }
+
+    /**
+     * get log size
+     * @param path log path
+     * @return log content bytes
+     */
+    public byte[] getLogBytes(String path) {
+        logger.info("log path {}", path);
+        GetLogBytesRequestCommand request = new GetLogBytesRequestCommand(path);
+        byte[] result = null;
+        try {
+            Command command = request.convert2Command();
+            this.client.send(address, command);
+            LogPromise promise = new LogPromise(command.getOpaque(), logRequestTimeout);
+            result = (byte[])promise.getResult();
+        } catch (Exception e) {
+            logger.error("get log size error", e);
+        }
+        return result;
+    }
+
+    @Override
+    public void process(Channel channel, Command command) {
+        logger.info("received log response : {}", command);
+        switch (command.getType()){
+            case ROLL_VIEW_LOG_RESPONSE:
+                RollViewLogResponseCommand rollReviewLog = FastJsonSerializer.deserialize(
+                        command.getBody(), RollViewLogResponseCommand.class);
+                LogPromise.notify(command.getOpaque(), rollReviewLog.getMsg());
+                break;
+            case VIEW_WHOLE_LOG_RESPONSE:
+                ViewLogResponseCommand viewLog = FastJsonSerializer.deserialize(
+                        command.getBody(), ViewLogResponseCommand.class);
+                LogPromise.notify(command.getOpaque(), viewLog.getMsg());
+                break;
+            case GET_LOG_BYTES_RESPONSE:
+                GetLogBytesResponseCommand getLog = FastJsonSerializer.deserialize(
+                        command.getBody(), GetLogBytesResponseCommand.class);
+                LogPromise.notify(command.getOpaque(), getLog.getData());
+                break;
+            default:
+                throw new UnsupportedOperationException(String.format("command type : %s is not supported ", command.getType()));
+        }
+    }
+
+    public static void main(String[] args) throws Exception{
+        LogClientService logClient = new LogClientService("192.168.220.247", 50051);
+        byte[] logBytes = logClient.getLogBytes("/opt/program/incubator-dolphinscheduler/logs/1/463/540.log");
+        System.out.println(new String(logBytes));
+    }
+
+}
\ No newline at end of file
diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogPromise.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogPromise.java
new file mode 100644
index 0000000..8920b8a
--- /dev/null
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogPromise.java
@@ -0,0 +1,81 @@
+/*
+ * 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.service.log;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+/**
+ *  log asyc callback
+ */
+public class LogPromise {
+
+    private static final ConcurrentHashMap<Long, LogPromise> PROMISES = new ConcurrentHashMap<>();
+
+    private long opaque;
+
+    private final long start;
+
+    private final long timeout;
+
+    private final CountDownLatch latch;
+
+    private Object result;
+
+    public LogPromise(long opaque, long timeout){
+        this.opaque = opaque;
+        this.timeout = timeout;
+        this.start = System.currentTimeMillis();
+        this.latch = new CountDownLatch(1);
+        PROMISES.put(opaque, this);
+    }
+
+
+    /**
+     *  notify client finish
+     * @param opaque unique identification
+     * @param result result
+     */
+    public static void notify(long opaque, Object result){
+        LogPromise promise = PROMISES.remove(opaque);
+        if(promise != null){
+            promise.doCountDown(result);
+        }
+    }
+
+    private void doCountDown(Object result){
+        this.result = result;
+        this.latch.countDown();
+    }
+
+    public boolean isTimeout(){
+        return System.currentTimeMillis() - start > timeout;
+    }
+
+    public Object getResult(){
+        try {
+            latch.await(timeout, TimeUnit.MILLISECONDS);
+        } catch (InterruptedException ignore) {
+        }
+        PROMISES.remove(opaque);
+        return this.result;
+    }
+
+
+}
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/permission/PermissionCheck.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/permission/PermissionCheck.java
similarity index 80%
rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/permission/PermissionCheck.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/permission/PermissionCheck.java
index 63d4c1c..027666f 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/permission/PermissionCheck.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/permission/PermissionCheck.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.dao.permission;
+package org.apache.dolphinscheduler.service.permission;
 
 import org.apache.dolphinscheduler.common.enums.AuthorizationType;
 import org.apache.dolphinscheduler.common.enums.UserType;
 import org.apache.dolphinscheduler.common.utils.CollectionUtils;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.User;
+import org.apache.dolphinscheduler.service.process.ProcessService;
 import org.slf4j.Logger;
 
 import java.util.List;
@@ -38,7 +38,7 @@ public class PermissionCheck<T> {
     /**
      * Authorization Type
      */
-    private ProcessDao processDao;
+    private ProcessService processService;
 
     /**
      * need check array
@@ -53,23 +53,23 @@ public class PermissionCheck<T> {
     /**
      * permission check
      * @param authorizationType authorization type
-     * @param processDao        process dao
+     * @param processService        process dao
      */
-    public PermissionCheck(AuthorizationType authorizationType, ProcessDao processDao) {
+    public PermissionCheck(AuthorizationType authorizationType, ProcessService processService) {
         this.authorizationType = authorizationType;
-        this.processDao = processDao;
+        this.processService = processService;
     }
 
     /**
      * permission check
      * @param authorizationType
-     * @param processDao
+     * @param processService
      * @param needChecks
      * @param userId
      */
-    public PermissionCheck(AuthorizationType authorizationType, ProcessDao processDao, T[] needChecks, int userId) {
+    public PermissionCheck(AuthorizationType authorizationType, ProcessService processService, T[] needChecks, int userId) {
         this.authorizationType = authorizationType;
-        this.processDao = processDao;
+        this.processService = processService;
         this.needChecks = needChecks;
         this.userId = userId;
     }
@@ -77,14 +77,14 @@ public class PermissionCheck<T> {
     /**
      * permission check
      * @param authorizationType
-     * @param processDao
+     * @param processService
      * @param needChecks
      * @param userId
      * @param logger
      */
-    public PermissionCheck(AuthorizationType authorizationType, ProcessDao processDao, T[] needChecks, int userId,Logger logger) {
+    public PermissionCheck(AuthorizationType authorizationType, ProcessService processService, T[] needChecks, int userId, Logger logger) {
         this.authorizationType = authorizationType;
-        this.processDao = processDao;
+        this.processService = processService;
         this.needChecks = needChecks;
         this.userId = userId;
         this.logger = logger;
@@ -98,12 +98,12 @@ public class PermissionCheck<T> {
         this.authorizationType = authorizationType;
     }
 
-    public ProcessDao getProcessDao() {
-        return processDao;
+    public ProcessService getProcessService() {
+        return processService;
     }
 
-    public void setProcessDao(ProcessDao processDao) {
-        this.processDao = processDao;
+    public void setProcessService(ProcessService processService) {
+        this.processService = processService;
     }
 
     public T[] getNeedChecks() {
@@ -142,9 +142,9 @@ public class PermissionCheck<T> {
     public void checkPermission() throws Exception{
         if(this.needChecks.length > 0){
             // get user type in order to judge whether the user is admin
-            User user = processDao.getUserById(userId);
+            User user = processService.getUserById(userId);
             if (user.getUserType() != UserType.ADMIN_USER){
-                List<T> unauthorizedList = processDao.listUnauthorized(userId,needChecks,authorizationType);
+                List<T> unauthorizedList = processService.listUnauthorized(userId,needChecks,authorizationType);
                 // if exist unauthorized resource
                 if(CollectionUtils.isNotEmpty(unauthorizedList)){
                     logger.error("user {} didn't has permission of {}: {}", user.getUserName(), authorizationType.getDescp(),unauthorizedList.toString());
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/ProcessDao.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java
similarity index 99%
rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/ProcessDao.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java
index 820b2fd..a26044e 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/ProcessDao.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.dao;
+package org.apache.dolphinscheduler.service.process;
 
 import com.alibaba.fastjson.JSONObject;
 import com.cronutils.model.Cron;
@@ -24,16 +24,12 @@ import org.apache.dolphinscheduler.common.enums.*;
 import org.apache.dolphinscheduler.common.model.DateInterval;
 import org.apache.dolphinscheduler.common.model.TaskNode;
 import org.apache.dolphinscheduler.common.process.Property;
-import org.apache.dolphinscheduler.common.queue.ITaskQueue;
 import org.apache.dolphinscheduler.common.task.subprocess.SubProcessParameters;
-import org.apache.dolphinscheduler.common.utils.DateUtils;
-import org.apache.dolphinscheduler.common.utils.IpUtils;
-import org.apache.dolphinscheduler.common.utils.JSONUtils;
-import org.apache.dolphinscheduler.common.utils.ParameterUtils;
-import org.apache.dolphinscheduler.common.utils.StringUtils;
+import org.apache.dolphinscheduler.common.utils.*;
 import org.apache.dolphinscheduler.dao.entity.*;
 import org.apache.dolphinscheduler.dao.mapper.*;
-import org.apache.dolphinscheduler.dao.utils.cron.CronUtils;
+import org.apache.dolphinscheduler.service.quartz.cron.CronUtils;
+import org.apache.dolphinscheduler.service.queue.ITaskQueue;
 import org.quartz.CronExpression;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,7 +47,7 @@ import static org.apache.dolphinscheduler.common.Constants.*;
  * process relative dao that some mappers in this.
  */
 @Component
-public class ProcessDao {
+public class ProcessService {
 
     private final Logger logger = LoggerFactory.getLogger(getClass());
 
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/DruidConnectionProvider.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/DruidConnectionProvider.java
similarity index 99%
rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/DruidConnectionProvider.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/DruidConnectionProvider.java
index 8a4ceba..d51e8e8 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/DruidConnectionProvider.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/DruidConnectionProvider.java
@@ -14,11 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.dao.quartz;
+package org.apache.dolphinscheduler.service.quartz;
 
 import com.alibaba.druid.pool.DruidDataSource;
 import org.quartz.SchedulerException;
 import org.quartz.utils.ConnectionProvider;
+
 import java.sql.Connection;
 import java.sql.SQLException;
 
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/ProcessScheduleJob.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/ProcessScheduleJob.java
similarity index 83%
rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/ProcessScheduleJob.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/ProcessScheduleJob.java
index ac46129..69a80e6 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/ProcessScheduleJob.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/ProcessScheduleJob.java
@@ -14,17 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.dao.quartz;
+package org.apache.dolphinscheduler.service.quartz;
 
 
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.CommandType;
 import org.apache.dolphinscheduler.common.enums.ReleaseState;
-import org.apache.dolphinscheduler.dao.ProcessDao;
 import org.apache.dolphinscheduler.dao.entity.Command;
 import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
 import org.apache.dolphinscheduler.dao.entity.Schedule;
-import org.quartz.*;
+import org.apache.dolphinscheduler.service.process.ProcessService;
+import org.quartz.Job;
+import org.quartz.JobDataMap;
+import org.quartz.JobExecutionContext;
+import org.quartz.JobExecutionException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.util.Assert;
@@ -42,17 +45,17 @@ public class ProcessScheduleJob implements Job {
     private static final Logger logger = LoggerFactory.getLogger(ProcessScheduleJob.class);
 
     /**
-     * process dao
+     * process service
      */
-    private static ProcessDao processDao;
+    private static ProcessService processService;
 
 
     /**
      * init
-     * @param processDao process dao
+     * @param processService process dao
      */
-    public static void init(ProcessDao processDao) {
-        ProcessScheduleJob.processDao = processDao;
+    public static void init(ProcessService processService) {
+        ProcessScheduleJob.processService = processService;
     }
 
     /**
@@ -64,7 +67,7 @@ public class ProcessScheduleJob implements Job {
     @Override
     public void execute(JobExecutionContext context) throws JobExecutionException {
 
-        Assert.notNull(processDao, "please call init() method first");
+        Assert.notNull(processService, "please call init() method first");
 
         JobDataMap dataMap = context.getJobDetail().getJobDataMap();
 
@@ -80,7 +83,7 @@ public class ProcessScheduleJob implements Job {
         logger.info("scheduled fire time :{}, fire time :{}, process id :{}", scheduledFireTime, fireTime, scheduleId);
 
         // query schedule
-        Schedule schedule = processDao.querySchedule(scheduleId);
+        Schedule schedule = processService.querySchedule(scheduleId);
         if (schedule == null) {
             logger.warn("process schedule does not exist in db,delete schedule job in quartz, projectId:{}, scheduleId:{}", projectId, scheduleId);
             deleteJob(projectId, scheduleId);
@@ -88,7 +91,7 @@ public class ProcessScheduleJob implements Job {
         }
 
 
-        ProcessDefinition processDefinition = processDao.findProcessDefineById(schedule.getProcessDefinitionId());
+        ProcessDefinition processDefinition = processService.findProcessDefineById(schedule.getProcessDefinitionId());
         // release state : online/offline
         ReleaseState releaseState = processDefinition.getReleaseState();
         if (processDefinition == null || releaseState == ReleaseState.OFFLINE) {
@@ -108,7 +111,7 @@ public class ProcessScheduleJob implements Job {
         command.setWarningType(schedule.getWarningType());
         command.setProcessInstancePriority(schedule.getProcessInstancePriority());
 
-        processDao.createCommand(command);
+        processService.createCommand(command);
     }
 
 
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/QuartzExecutors.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/QuartzExecutors.java
similarity index 99%
rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/QuartzExecutors.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/QuartzExecutors.java
index 054d790..9d96264 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/QuartzExecutors.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/QuartzExecutors.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.dao.quartz;
+package org.apache.dolphinscheduler.service.quartz;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.utils.JSONUtils;
 import org.apache.dolphinscheduler.dao.entity.Schedule;
-import org.apache.commons.lang.StringUtils;
 import org.quartz.*;
 import org.quartz.impl.StdSchedulerFactory;
 import org.quartz.impl.matchers.GroupMatcher;
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/AbstractCycle.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/AbstractCycle.java
similarity index 99%
rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/AbstractCycle.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/AbstractCycle.java
index 0cda336..0a2e31b 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/AbstractCycle.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/AbstractCycle.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.dao.utils.cron;
+package org.apache.dolphinscheduler.service.quartz.cron;
 
-import org.apache.dolphinscheduler.common.enums.CycleEnum;
 import com.cronutils.model.Cron;
 import com.cronutils.model.field.CronField;
 import com.cronutils.model.field.CronFieldName;
 import com.cronutils.model.field.expression.*;
+import org.apache.dolphinscheduler.common.enums.CycleEnum;
 
 /**
  * Cycle
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CronUtils.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CronUtils.java
similarity index 98%
rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CronUtils.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CronUtils.java
index 2ad0290..d03a4a5 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CronUtils.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CronUtils.java
@@ -14,15 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.dao.utils.cron;
+package org.apache.dolphinscheduler.service.quartz.cron;
 
 
-import org.apache.dolphinscheduler.common.enums.CycleEnum;
-import org.apache.dolphinscheduler.common.thread.Stopper;
-import org.apache.dolphinscheduler.common.utils.DateUtils;
 import com.cronutils.model.Cron;
 import com.cronutils.model.definition.CronDefinitionBuilder;
 import com.cronutils.parser.CronParser;
+import org.apache.dolphinscheduler.common.enums.CycleEnum;
+import org.apache.dolphinscheduler.common.thread.Stopper;
+import org.apache.dolphinscheduler.common.utils.DateUtils;
 import org.quartz.CronExpression;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -31,7 +31,7 @@ import java.text.ParseException;
 import java.util.*;
 
 import static com.cronutils.model.CronType.QUARTZ;
-import static org.apache.dolphinscheduler.dao.utils.cron.CycleFactory.*;
+import static org.apache.dolphinscheduler.service.quartz.cron.CycleFactory.*;
 
 
 /**
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CycleFactory.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CycleFactory.java
similarity index 99%
rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CycleFactory.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CycleFactory.java
index b2f5256..1f807dc 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CycleFactory.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CycleFactory.java
@@ -14,12 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.dao.utils.cron;
+package org.apache.dolphinscheduler.service.quartz.cron;
 
-import org.apache.dolphinscheduler.common.enums.CycleEnum;
 import com.cronutils.model.Cron;
 import com.cronutils.model.field.expression.Always;
 import com.cronutils.model.field.expression.QuestionMark;
+import org.apache.dolphinscheduler.common.enums.CycleEnum;
 
 /**
  * Crontab Cycle Tool Factory
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CycleLinks.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CycleLinks.java
similarity index 97%
rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CycleLinks.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CycleLinks.java
index 63824bd..9f01b18 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CycleLinks.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CycleLinks.java
@@ -14,10 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.dao.utils.cron;
+package org.apache.dolphinscheduler.service.quartz.cron;
 
-import org.apache.dolphinscheduler.common.enums.CycleEnum;
 import com.cronutils.model.Cron;
+import org.apache.dolphinscheduler.common.enums.CycleEnum;
 
 import java.util.ArrayList;
 import java.util.List;
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/ITaskQueue.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/ITaskQueue.java
similarity index 97%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/ITaskQueue.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/ITaskQueue.java
index 5beb811..bed8a11 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/ITaskQueue.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/ITaskQueue.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.queue;
+package org.apache.dolphinscheduler.service.queue;
 
 import java.util.List;
 import java.util.Set;
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/TaskQueueFactory.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueFactory.java
similarity index 93%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/TaskQueueFactory.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueFactory.java
index 0a2d943..6be419f 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/TaskQueueFactory.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueFactory.java
@@ -14,11 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.queue;
+package org.apache.dolphinscheduler.service.queue;
 
-import org.apache.dolphinscheduler.common.utils.CommonUtils;
 import org.apache.commons.lang.StringUtils;
-import org.apache.dolphinscheduler.common.utils.SpringApplicationContext;
+import org.apache.dolphinscheduler.common.utils.CommonUtils;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/TaskQueueZkImpl.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueZkImpl.java
similarity index 99%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/TaskQueueZkImpl.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueZkImpl.java
index d442c13..874512c 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/TaskQueueZkImpl.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueZkImpl.java
@@ -14,13 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.queue;
+package org.apache.dolphinscheduler.service.queue;
 
 
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.utils.IpUtils;
 import org.apache.dolphinscheduler.common.utils.OSUtils;
-import org.apache.dolphinscheduler.common.zk.ZookeeperOperator;
+import org.apache.dolphinscheduler.service.zk.ZookeeperOperator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/AbstractZKClient.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/AbstractZKClient.java
similarity index 99%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/AbstractZKClient.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/AbstractZKClient.java
index f62e106..135bfda 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/AbstractZKClient.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/AbstractZKClient.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.zk;
+package org.apache.dolphinscheduler.service.zk;
 
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.imps.CuratorFrameworkState;
@@ -31,12 +31,13 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.*;
+
 import static org.apache.dolphinscheduler.common.Constants.*;
 
 /**
  * abstract zookeeper client
  */
-public abstract class AbstractZKClient extends ZookeeperCachedOperator{
+public abstract class AbstractZKClient extends ZookeeperCachedOperator {
 
 	private static final Logger logger = LoggerFactory.getLogger(AbstractZKClient.class);
 
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/DefaultEnsembleProvider.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/DefaultEnsembleProvider.java
similarity index 96%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/DefaultEnsembleProvider.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/DefaultEnsembleProvider.java
index 0cf06c0..9eedf7a 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/DefaultEnsembleProvider.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/DefaultEnsembleProvider.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.zk;
+package org.apache.dolphinscheduler.service.zk;
 
 import org.apache.curator.ensemble.EnsembleProvider;
 
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperCachedOperator.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperCachedOperator.java
similarity index 90%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperCachedOperator.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperCachedOperator.java
index 5aa2555..dccb768 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperCachedOperator.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperCachedOperator.java
@@ -14,22 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.zk;
+package org.apache.dolphinscheduler.service.zk;
 
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.recipes.cache.ChildData;
 import org.apache.curator.framework.recipes.cache.TreeCache;
 import org.apache.curator.framework.recipes.cache.TreeCacheEvent;
-import org.apache.curator.framework.recipes.cache.TreeCacheListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.stereotype.Component;
 
 import java.nio.charset.StandardCharsets;
-import java.util.concurrent.ConcurrentHashMap;
-
-import static org.apache.dolphinscheduler.common.utils.Preconditions.*;
-import static org.apache.dolphinscheduler.common.utils.Preconditions.checkNotNull;
 
 @Component
 public class ZookeeperCachedOperator extends ZookeeperOperator {
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperConfig.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperConfig.java
similarity index 98%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperConfig.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperConfig.java
index 75a9f6c..c6bdfc3 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperConfig.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperConfig.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.zk;
+package org.apache.dolphinscheduler.service.zk;
 
 import org.springframework.beans.factory.annotation.Value;
 import org.springframework.context.annotation.PropertySource;
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperOperator.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperOperator.java
similarity index 98%
rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperOperator.java
rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperOperator.java
index 9442afd..a2cabce 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperOperator.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperOperator.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.zk;
+package org.apache.dolphinscheduler.service.zk;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.curator.framework.CuratorFramework;
@@ -33,12 +33,10 @@ import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.InitializingBean;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Component;
-import org.springframework.util.CollectionUtils;
 
 import java.nio.charset.StandardCharsets;
 import java.util.List;
 
-import static org.apache.dolphinscheduler.common.utils.Preconditions.*;
 import static org.apache.dolphinscheduler.common.utils.Preconditions.checkNotNull;
 
 /**
diff --git a/dolphinscheduler-common/src/main/resources/quartz.properties b/dolphinscheduler-service/src/main/resources/quartz.properties
similarity index 96%
rename from dolphinscheduler-common/src/main/resources/quartz.properties
rename to dolphinscheduler-service/src/main/resources/quartz.properties
index 2e3a2a0..9c8930b 100644
--- a/dolphinscheduler-common/src/main/resources/quartz.properties
+++ b/dolphinscheduler-service/src/main/resources/quartz.properties
@@ -59,6 +59,6 @@ org.quartz.jobStore.dataSource = myDs
 #============================================================================
 # Configure Datasources  
 #============================================================================
-org.quartz.dataSource.myDs.connectionProvider.class = org.apache.dolphinscheduler.dao.quartz.DruidConnectionProvider
+org.quartz.dataSource.myDs.connectionProvider.class = org.apache.dolphinscheduler.service.quartz.DruidConnectionProvider
 org.quartz.dataSource.myDs.maxConnections = 10
 org.quartz.dataSource.myDs.validationQuery = select 1
\ No newline at end of file
diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/cron/CronUtilsTest.java b/dolphinscheduler-service/src/test/java/cron/CronUtilsTest.java
similarity index 98%
rename from dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/cron/CronUtilsTest.java
rename to dolphinscheduler-service/src/test/java/cron/CronUtilsTest.java
index 5ecc662..6a402b5 100644
--- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/cron/CronUtilsTest.java
+++ b/dolphinscheduler-service/src/test/java/cron/CronUtilsTest.java
@@ -14,11 +14,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.dao.cron;
+package cron;
 
-import org.apache.dolphinscheduler.common.enums.CycleEnum;
-import org.apache.dolphinscheduler.common.utils.DateUtils;
-import org.apache.dolphinscheduler.dao.utils.cron.CronUtils;
 import com.cronutils.builder.CronBuilder;
 import com.cronutils.model.Cron;
 import com.cronutils.model.CronType;
@@ -26,6 +23,9 @@ import com.cronutils.model.definition.CronDefinitionBuilder;
 import com.cronutils.model.field.CronField;
 import com.cronutils.model.field.CronFieldName;
 import com.cronutils.model.field.expression.*;
+import org.apache.dolphinscheduler.common.enums.CycleEnum;
+import org.apache.dolphinscheduler.common.utils.DateUtils;
+import org.apache.dolphinscheduler.service.quartz.cron.CronUtils;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/queue/BaseTaskQueueTest.java b/dolphinscheduler-service/src/test/java/queue/BaseTaskQueueTest.java
similarity index 90%
rename from dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/queue/BaseTaskQueueTest.java
rename to dolphinscheduler-service/src/test/java/queue/BaseTaskQueueTest.java
index 433e4fa..a0cc457 100644
--- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/queue/BaseTaskQueueTest.java
+++ b/dolphinscheduler-service/src/test/java/queue/BaseTaskQueueTest.java
@@ -14,9 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.queue;
+package queue;
 
-import org.apache.dolphinscheduler.common.zk.ZKServer;
+import org.apache.dolphinscheduler.service.queue.ITaskQueue;
+import org.apache.dolphinscheduler.service.queue.TaskQueueFactory;
 import org.junit.*;
 
 /**
diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/queue/TaskQueueZKImplTest.java b/dolphinscheduler-service/src/test/java/queue/TaskQueueZKImplTest.java
similarity index 99%
rename from dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/queue/TaskQueueZKImplTest.java
rename to dolphinscheduler-service/src/test/java/queue/TaskQueueZKImplTest.java
index b34a7d6..d29c5aa 100644
--- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/queue/TaskQueueZKImplTest.java
+++ b/dolphinscheduler-service/src/test/java/queue/TaskQueueZKImplTest.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.queue;
+package queue;
 
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.utils.IpUtils;
diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/ZKServer.java b/dolphinscheduler-service/src/test/java/queue/ZKServer.java
similarity index 99%
rename from dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/ZKServer.java
rename to dolphinscheduler-service/src/test/java/queue/ZKServer.java
index fc39e62..65fb95c 100644
--- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/ZKServer.java
+++ b/dolphinscheduler-service/src/test/java/queue/ZKServer.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.zk;
+package queue;
 
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/PreconditionsTest.java b/dolphinscheduler-service/src/test/java/utils/PreconditionsTest.java
similarity index 97%
rename from dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/PreconditionsTest.java
rename to dolphinscheduler-service/src/test/java/utils/PreconditionsTest.java
index dcb0e13..a1b85f1 100644
--- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/PreconditionsTest.java
+++ b/dolphinscheduler-service/src/test/java/utils/PreconditionsTest.java
@@ -14,8 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dolphinscheduler.common.utils;
+package utils;
 
+import org.apache.dolphinscheduler.common.utils.Preconditions;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -35,7 +36,7 @@ public class PreconditionsTest {
     public void testCheckNotNull() throws Exception {
         String testReference = "test reference";
         //test  reference is not null
-        Assert.assertEquals(testReference,Preconditions.checkNotNull(testReference));
+        Assert.assertEquals(testReference, Preconditions.checkNotNull(testReference));
         Assert.assertEquals(testReference,Preconditions.checkNotNull(testReference,"reference is null"));
         Assert.assertEquals(testReference,Preconditions.checkNotNull(testReference,"%s is null",testReference));
 
diff --git a/pom.xml b/pom.xml
index de7ee81..47e0f3e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -229,7 +229,12 @@
             </dependency>
             <dependency>
                 <groupId>org.apache.dolphinscheduler</groupId>
-                <artifactId>dolphinscheduler-rpc</artifactId>
+                <artifactId>dolphinscheduler-remote</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.dolphinscheduler</groupId>
+                <artifactId>dolphinscheduler-service</artifactId>
                 <version>${project.version}</version>
             </dependency>
             <dependency>
@@ -774,9 +779,6 @@
                         <exclude>**/dolphinscheduler-ui/src/view/common/outro.inc</exclude>
                         <exclude>**/dolphinscheduler-ui/src/view/common/meta.inc</exclude>
                         <exclude>**/dolphinscheduler-ui/src/combo/1.0.0/3rd.css</exclude>
-                        <exclude>
-                            **/dolphinscheduler-rpc/src/main/java/org/apache/dolphinscheduler/rpc/LogViewServiceGrpc.java
-                        </exclude>
                     </excludes>
                     <consoleOutput>true</consoleOutput>
                 </configuration>
@@ -863,8 +865,9 @@
         <module>dolphinscheduler-api</module>
         <module>dolphinscheduler-dao</module>
         <module>dolphinscheduler-alert</module>
-        <module>dolphinscheduler-rpc</module>
         <module>dolphinscheduler-dist</module>
+        <module>dolphinscheduler-remote</module>
+        <module>dolphinscheduler-service</module>
     </modules>
 
 </project>