From c1ee1333924eff051fc2408480cbc10898e4021a Mon Sep 17 00:00:00 2001 From: qiaozhanwei Date: Fri, 14 Feb 2020 22:00:20 +0800 Subject: [PATCH] Refactor Architecture Basic modification #1658 (#1946) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * 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 Co-authored-by: zhukai Co-authored-by: Yelli * 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 Co-authored-by: khadgarmage Co-authored-by: zhukai Co-authored-by: Yelli --- dolphinscheduler-api/pom.xml | 14 +- .../controller/ProcessInstanceController.java | 4 +- .../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 +- .../service/ProcessDefinitionServiceTest.java | 6 +- dolphinscheduler-common/pom.xml | 25 +- .../common/utils/Preconditions.java | 23 +- .../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 ++++ .../remote/command/Command.java | 105 +++++++ .../remote/command/CommandHeader.java | 64 +++++ .../remote/command/CommandType.java | 1 + .../command/ExecuteTaskRequestCommand.java | 1 + .../command/ExecuteTaskResponseCommand.java | 1 + .../dolphinscheduler/remote/command/Ping.java | 74 +++++ .../dolphinscheduler/remote/command/Pong.java | 75 +++++ .../log/GetLogBytesRequestCommand.java | 69 +++++ .../log/GetLogBytesResponseCommand.java | 65 +++++ .../log/RollViewLogRequestCommand.java | 97 +++++++ .../log/RollViewLogResponseCommand.java | 64 +++++ .../command/log/ViewLogRequestCommand.java | 66 +++++ .../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 +++++++++++ .../processor/NettyRequestProcessor.java | 32 +-- .../remote/utils/Address.java | 96 +++++++ .../remote/utils/ChannelUtils.java | 57 ++++ .../remote/utils/Constants.java | 42 ++- .../remote/utils/FastJsonSerializer.java | 60 ++++ .../dolphinscheduler/remote/utils/Pair.java | 53 ++++ .../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 ++++++++++++ .../server/log/LoggerServer.java | 91 ++++++ .../server/master/MasterServer.java | 27 +- .../runner/MasterBaseTaskExecThread.java | 18 +- .../master/runner/MasterExecThread.java | 55 ++-- .../master/runner/MasterSchedulerThread.java | 22 +- .../master/runner/MasterTaskExecThread.java | 10 +- .../runner/SubProcessTaskExecThread.java | 16 +- .../server/monitor/ZKMonitorImpl.java | 2 +- .../server/rpc/LogClient.java | 149 ---------- .../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 +- .../worker/runner/TaskScheduleThread.java | 26 +- .../worker/task/AbstractCommandExecutor.java | 20 +- .../server/worker/task/AbstractYarnTask.java | 12 +- .../server/worker/task/datax/DataxTask.java | 22 +- .../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 +- .../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 +- .../server/zk/ZKMasterClient.java | 18 +- .../server/zk/ZKWorkerClient.java | 2 +- .../server/master/MasterExecThreadTest.java | 18 +- .../shell/ShellCommandExecutorTest.java | 10 +- .../server/worker/sql/SqlExecutorTest.java | 10 +- .../worker/task/datax/DataxTaskTest.java | 22 +- dolphinscheduler-service/pom.xml | 56 ++++ .../bean}/SpringApplicationContext.java | 3 +- .../service/log/LogClientService.java | 166 +++++++++++ .../service/log/LogPromise.java | 81 ++++++ .../service}/permission/PermissionCheck.java | 36 +-- .../service/process/ProcessService.java | 14 +- .../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 +- .../test/java}/queue/BaseTaskQueueTest.java | 5 +- .../test/java}/queue/TaskQueueZKImplTest.java | 2 +- .../src/test/java/queue}/ZKServer.java | 2 +- .../test/java}/utils/PreconditionsTest.java | 5 +- pom.xml | 13 +- 125 files changed, 3534 insertions(+), 1296 deletions(-) delete mode 100644 dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/log/LogClient.java rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/BaseDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/ClickHouseDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/DB2ServerDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/DataSourceFactory.java (98%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/HiveDataSource.java (93%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/MySQLDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/OracleDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/PostgreDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/SQLServerDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/SparkDataSource.java (97%) create mode 100644 dolphinscheduler-remote/pom.xml create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyDecoder.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyEncoder.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandHeader.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Pong.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesResponseCommand.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogResponseCommand.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogResponseCommand.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyClientConfig.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyServerConfig.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingException.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java rename dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZkServer.java => dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/NettyRequestProcessor.java (65%) create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Address.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java rename dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZk.java => dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java (66%) create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/FastJsonSerializer.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Pair.java create mode 100644 dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java delete mode 100644 dolphinscheduler-rpc/pom.xml delete mode 100644 dolphinscheduler-rpc/src/main/proto/scheduler.proto create mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerRequestProcessor.java create mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerServer.java delete mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/rpc/LogClient.java delete mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/rpc/LoggerServer.java create mode 100644 dolphinscheduler-service/pom.xml rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/bean}/SpringApplicationContext.java (96%) create mode 100644 dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java create mode 100644 dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogPromise.java rename {dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/permission/PermissionCheck.java (80%) rename dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/ProcessDao.java => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java (99%) rename {dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/quartz/DruidConnectionProvider.java (99%) rename {dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/quartz/ProcessScheduleJob.java (83%) rename {dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/quartz/QuartzExecutors.java (99%) rename {dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz}/cron/AbstractCycle.java (99%) rename {dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz}/cron/CronUtils.java (98%) rename {dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz}/cron/CycleFactory.java (99%) rename {dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz}/cron/CycleLinks.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/queue/ITaskQueue.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/queue/TaskQueueFactory.java (93%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/queue/TaskQueueZkImpl.java (99%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/zk/AbstractZKClient.java (99%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/zk/DefaultEnsembleProvider.java (96%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/zk/ZookeeperCachedOperator.java (90%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/zk/ZookeeperConfig.java (98%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/zk/ZookeeperOperator.java (98%) rename {dolphinscheduler-common => dolphinscheduler-service}/src/main/resources/quartz.properties (96%) rename {dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao => dolphinscheduler-service/src/test/java}/cron/CronUtilsTest.java (98%) rename {dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/test/java}/queue/BaseTaskQueueTest.java (90%) rename {dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/test/java}/queue/TaskQueueZKImplTest.java (99%) rename {dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk => dolphinscheduler-service/src/test/java/queue}/ZKServer.java (99%) rename {dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/test/java}/utils/PreconditionsTest.java (97%) diff --git a/dolphinscheduler-api/pom.xml b/dolphinscheduler-api/pom.xml index c10f443384..6440805af7 100644 --- a/dolphinscheduler-api/pom.xml +++ b/dolphinscheduler-api/pom.xml @@ -31,12 +31,6 @@ org.apache.dolphinscheduler dolphinscheduler-alert - - - org.apache.dolphinscheduler - dolphinscheduler-dao - - @@ -129,13 +123,13 @@ - com.github.xiaoymin - swagger-bootstrap-ui + org.apache.dolphinscheduler + dolphinscheduler-service - org.apache.dolphinscheduler - dolphinscheduler-rpc + com.github.xiaoymin + swagger-bootstrap-ui 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 150c647f99..80db6c86af 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 3452060ec9..0000000000 --- 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 b95782711f..bafe833fab 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 5d33b46bd2..f6d8903dd8 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 257f15d580..6edd48d499 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 updateProcessInstanceState(Integer processInstanceId, ExecutionStatus executionStatus) { Map 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 ids = new ArrayList<>(); - processDao.recurseFindSubProcessId(processDefineId, ids); + processService.recurseFindSubProcessId(processDefineId, ids); Integer[] idArray = ids.toArray(new Integer[ids.size()]); if (ids.size() > 0){ List 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 schedules = processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefineId); + List schedules = processService.queryReleaseSchedulerListByProcessDefinitionId(processDefineId); List 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 2587290fd3..bff54b6c21 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 e9cfe341dd..123dc8186a 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 87e1a0ede1..2b1f04e6ce 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 taskInstanceList = processDao.findValidTaskListByProcessId(processId); + ProcessInstance processInstance = processService.findProcessInstanceDetailById(processId); + List taskInstanceList = processService.findValidTaskListByProcessId(processId); AddDependResultForTaskList(taskInstanceList); Map 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 deleteProcessInstanceById(User loginUser, String projectName, Integer processInstanceId,ITaskQueue tasksQueue) { + public Map deleteProcessInstanceById(User loginUser, String projectName, Integer processInstanceId, ITaskQueue tasksQueue) { Map 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 taskInstanceList = processDao.findValidTaskListByProcessId(processInstanceId); + ProcessInstance processInstance = processService.findProcessInstanceDetailById(processInstanceId); + List 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 bdce9470ca..72122100a1 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 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 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 74afa2a44e..9690f5c69f 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 1f523edac5..340a389d1c 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 66f57f6a11..f91d3bc68c 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 c7afd76cc6..6f308e7b17 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 66c7a3ebab..07d7477930 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 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 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 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 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 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 c6ab6f8e74..20571577e3 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 51b440b9ca..aa9f5fe20f 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 updateResult = processDefinitionService.updateProcessDefinition(loginUser, projectName, 1, "test", sqlDependentJson, "", "", ""); diff --git a/dolphinscheduler-common/pom.xml b/dolphinscheduler-common/pom.xml index bd2448eee7..2ef61d7f36 100644 --- a/dolphinscheduler-common/pom.xml +++ b/dolphinscheduler-common/pom.xml @@ -85,21 +85,7 @@ com.fasterxml.jackson.core jackson-databind - - org.apache.curator - curator-client - ${curator.version} - - - log4j-1.2-api - org.apache.logging.log4j - - - io.netty - netty - - - + org.apache.commons commons-collections4 @@ -548,6 +534,10 @@ log4j-web org.apache.logging.log4j + + jasper-compiler + tomcat + @@ -601,11 +591,6 @@ compile - - org.springframework - spring-context - - org.codehaus.janino janino 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 92337f5de6..e59cbd1b96 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 checkNotNull(T reference, @Nullable String errorMessage) { + public static 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 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 { * *

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 5b23847ba3..3d51aa830f 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 b3b22c128d..20d19410e2 100644 --- a/dolphinscheduler-dao/pom.xml +++ b/dolphinscheduler-dao/pom.xml @@ -116,21 +116,6 @@ cron-utils - - org.quartz-scheduler - quartz - - - c3p0 - c3p0 - - - - - - org.quartz-scheduler - quartz-jobs - commons-configuration commons-configuration 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 41a9b3a566..a46e5aabcc 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 48550c31cc..e159f81d2e 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 44ee200c5d..3c2366b5b0 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 48ec319eaa..9571f9c9f6 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 cddedd1f73..0a8f527536 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 fa149e67e2..6e2fbe3dd8 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 c3dc3a96df..cefaf879b5 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 4989e7681e..176cba2587 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 8554992efc..07770c06a7 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 5d10c63e5d..81a5ac6f04 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 0000000000..b67b033ffa --- /dev/null +++ b/dolphinscheduler-remote/pom.xml @@ -0,0 +1,44 @@ + + + + + dolphinscheduler + org.apache.dolphinscheduler + 1.2.1-SNAPSHOT + + 4.0.0 + + dolphinscheduler-remote + + dolphinscheduler-remote + + http://www.example.com + + + UTF-8 + 1.7 + 1.7 + + + + + io.netty + netty-all + + + org.slf4j + slf4j-api + + + com.alibaba + fastjson + + + junit + junit + test + + + + 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 0000000000..df0c13ad38 --- /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 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() { + @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 0000000000..c69bf09540 --- /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() { + + @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 0000000000..caa4fbdd17 --- /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 { + + 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 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 0000000000..4e9836a26f --- /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 { + + /** + * 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 0000000000..ee95044764 --- /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-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandHeader.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandHeader.java new file mode 100644 index 0000000000..78948a5c0c --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandHeader.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; + +import java.io.Serializable; + +/** + * command header + */ +public class CommandHeader implements Serializable { + + /** + * type + */ + private byte type; + + /** + * request unique identification + */ + private long opaque; + + /** + * body length + */ + private int bodyLength; + + public int getBodyLength() { + return bodyLength; + } + + public void setBodyLength(int bodyLength) { + this.bodyLength = bodyLength; + } + + public byte getType() { + return type; + } + + public void setType(byte type) { + this.type = type; + } + + public long getOpaque() { + return opaque; + } + + 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 0000000000..b1b24d3303 --- /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 0000000000..a582221cd3 --- /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.concurrent.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 shardItems; public List getShardItems() { return shardItems; } public void setShardItems(List 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 0000000000..0268653b5d --- /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 0000000000..4f32d5f699 --- /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 0000000000..e52cef6d92 --- /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 0000000000..9b064b7136 --- /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 0000000000..deaf9b8d85 --- /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 0000000000..f072c479f4 --- /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 0000000000..591d787200 --- /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 0000000000..5dcefc6233 --- /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 0000000000..dffadade26 --- /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 0000000000..831e05f7e7 --- /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 0000000000..4ec8a0f7a7 --- /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 0000000000..29d48db8f8 --- /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.

Note that the detail message associated with + * {@code cause} is not 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 null 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 (cause==null ? null : cause.toString()) + * (which typically contains the class and detail message of + * cause). 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 null 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 0000000000..6aceb5a41b --- /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> 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(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 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 0000000000..eabd6560de --- /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> 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(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 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% 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/processor/NettyRequestProcessor.java index d1a0526309..6966b53d17 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 0000000000..f61dcd615c --- /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 0000000000..d7af5fe165 --- /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 5c3db2d5d1..5733b17790 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 0000000000..e96796a05c --- /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 object type + * @return byte array + */ + public static byte[] serialize(T obj) { + String json = JSON.toJSONString(obj); + return json.getBytes(Constants.UTF8); + } + + /** + * serialize to string + * @param obj object + * @param object type + * @return string + */ + public static String serializeToString(T obj) { + return JSON.toJSONString(obj); + } + + /** + * deserialize + * + * @param src byte array + * @param clazz class + * @param deserialize type + * @return deserialize type + */ + public static T deserialize(byte[] src, Class clazz) { + return JSON.parseObject(new String(src, Constants.UTF8), clazz); + } + +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Pair.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Pair.java new file mode 100644 index 0000000000..2042191486 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Pair.java @@ -0,0 +1,53 @@ +/* + * 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; + + +/** + * key value pair + * + * @param L generic type + * @param R generic type + */ +public class Pair { + + private L left; + + private R right; + + public Pair(L left, R right) { + this.left = left; + this.right = right; + } + + public L getLeft() { + return left; + } + + public void setLeft(L left) { + this.left = left; + } + + public R getRight() { + return right; + } + + 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 0000000000..6f0a802af6 --- /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 680a4a24c0..0000000000 --- a/dolphinscheduler-rpc/pom.xml +++ /dev/null @@ -1,113 +0,0 @@ - - - - - - org.apache.dolphinscheduler - dolphinscheduler - 1.2.1-SNAPSHOT - - 4.0.0 - - dolphinscheduler-rpc - - dolphinscheduler-rpc - https://github.com/apache/incubator-dolphinscheduler - - - UTF-8 - 1.8 - 1.8 - - 3.5.1 - 1.9.0 - - - - - com.google.protobuf - protobuf-java - ${protobuf.version} - - - io.grpc - grpc-netty - ${grpc.version} - - - io.grpc - grpc-protobuf - ${grpc.version} - - - io.grpc - grpc-stub - ${grpc.version} - - - - com.google.guava - guava - - - - - - - kr.motd.maven - os-maven-plugin - 1.5.0.Final - - - - - org.xolstice.maven.plugins - protobuf-maven-plugin - 0.5.0 - - com.google.protobuf:protoc:3.5.1-1:exe:${os.detected.classifier} - grpc-java - io.grpc:protoc-gen-grpc-java:${grpc.version}:exe:${os.detected.classifier} - - - - compile - - compile - - - - compile-custom - - compile-custom - - - - - - org.apache.maven.plugins - maven-compiler-plugin - - ${java.version} - ${java.version} - ${project.build.sourceEncoding} - - - - - diff --git a/dolphinscheduler-rpc/src/main/proto/scheduler.proto b/dolphinscheduler-rpc/src/main/proto/scheduler.proto deleted file mode 100644 index b8b595cb2a..0000000000 --- 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 751fd919a8..080b87ebaa 100644 --- a/dolphinscheduler-server/pom.xml +++ b/dolphinscheduler-server/pom.xml @@ -71,7 +71,7 @@ org.apache.dolphinscheduler - dolphinscheduler-rpc + dolphinscheduler-service org.apache.curator 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 0000000000..4e4404ea1c --- /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 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 readPartFileContent(String filePath, + int skipLine, + int limit){ + try (Stream 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 0000000000..3520fb09ec --- /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 ab4ba5c8ab..6b5063cba4 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 c1552c4621..f8fcb1456d 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 { 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 { * @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 { 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 2b1ff4d23f..f5e31210a0 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 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 schedules = processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId); + List schedules = processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId); List 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 taskInstanceList = processDao.findValidTaskListByProcessId(processInstance.getId()); + List 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 taskInstances = processDao.findValidTaskListByProcessId(processInstance.getId()); + List 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 taskInstanceList = processDao.findValidTaskListByProcessId(processInstance.getId()); + List 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 taskInstanceList = processDao.findValidTaskListByProcessId(this.processInstance.getId()); + List 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> 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 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 a873fb786d..c0ddb1cb5c 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 f2ee66b64a..66d1a3f4c2 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 0026de7c25..fc16b5112b 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 927074012d..5acc8fd931 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 1c6c97b88f..0000000000 --- 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 5ec5df92fc..0000000000 --- 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 responseObserver) { - - logger.info("log parameter path : {} ,skip line : {}, limit : {}", - request.getPath(), - request.getSkipLineNum(), - request.getLimit()); - List 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 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 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 readFile(String path,int skipLine,int limit){ - try (Stream 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 fd0a08cd8e..90711e1d14 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 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 7264c2f59d..5550e750b5 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 99d418f048..ace93079ff 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 @@ -104,13 +104,6 @@ public class WorkerServer implements IStoppable { */ private ExecutorService fetchTaskExecutorService; - /** - * spring application context - * only use it for initialization - */ - @Autowired - private SpringApplicationContext springApplicationContext; - /** * CountDownLatch latch */ @@ -122,6 +115,13 @@ public class WorkerServer implements IStoppable { @Autowired private WorkerConfig workerConfig; + /** + * spring application context + * only use it for initialization + */ + @Autowired + private SpringApplicationContext springApplicationContext; + /** * master server startup * @@ -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 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 221ad069bb..013db83761 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 f179d6344a..a69cffd58d 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 projectRes) throws Exception { int userId = taskInstance.getProcessInstance().getExecutorId(); String[] resNames = projectRes.toArray(new String[projectRes.size()]); - PermissionCheck permissionCheck = new PermissionCheck<>(AuthorizationType.RESOURCE_FILE,processDao,resNames,userId,logger); + PermissionCheck 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 f1c01aff36..c473f3a2aa 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 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 6846617408..39f4dfbb97 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 0de2bbc7c6..ef941cd062 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 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 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 4be65ed49d..b08cabc2e9 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 taskInstanceList = processDao.findValidTaskListByProcessId(processInstance.getId()); + List 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 9af29e01dd..f074d57e6c 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 0fa9e11ce5..c562fbe4dd 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 97e6cb7bee..c925f90b9e 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 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 9b4952bbd2..fb881453e9 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 585d62f154..fc212f866b 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 789a0c5302..5704c8052e 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 eba05a0d21..aae11f5530 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 udfFuncList = processDao.queryUdfFunListByids(idsArray); + List 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 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 permissionCheckUdf = new PermissionCheck(AuthorizationType.UDF,processDao,udfFunIds,userId,logger); + PermissionCheck permissionCheckUdf = new PermissionCheck(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 permissionCheckDataSource = new PermissionCheck(AuthorizationType.DATASOURCE,processDao,new Integer[]{dataSourceId},userId,logger); + PermissionCheck permissionCheckDataSource = new PermissionCheck(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 c6a71ed066..fe4ec9130a 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 needFailoverTaskInstanceList = processDao.queryNeedFailoverTaskInstances(workerHost); + List 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 needFailoverProcessInstanceList = processDao.queryNeedFailoverProcessInstances(masterHost); + List 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 88abfa3071..7ddee3b2a1 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 d7c3de13a5..770ab3cff6 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 04c844827f..5d4263644b 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 7da3f710b6..c395eabe51 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 7a6073e05d..bd7f27530a 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 0000000000..7d775d5497 --- /dev/null +++ b/dolphinscheduler-service/pom.xml @@ -0,0 +1,56 @@ + + + + + dolphinscheduler + org.apache.dolphinscheduler + 1.2.1-SNAPSHOT + + 4.0.0 + + dolphinscheduler-service + + dolphinscheduler-service + + + + org.apache.dolphinscheduler + dolphinscheduler-remote + + + org.apache.dolphinscheduler + dolphinscheduler-dao + + + org.apache.curator + curator-client + ${curator.version} + + + log4j-1.2-api + org.apache.logging.log4j + + + io.netty + netty + + + + + org.quartz-scheduler + quartz + + + c3p0 + c3p0 + + + + + + org.quartz-scheduler + quartz-jobs + + + 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 97618e1b39..ddf1fecf76 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 0000000000..aa6999ef0d --- /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 0000000000..8920b8a527 --- /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 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 63d4c1c8af..027666f053 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 { /** * Authorization Type */ - private ProcessDao processDao; + private ProcessService processService; /** * need check array @@ -53,23 +53,23 @@ public class PermissionCheck { /** * 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 { /** * 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 { 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 { 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 unauthorizedList = processDao.listUnauthorized(userId,needChecks,authorizationType); + List 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 820b2fdaf4..a26044e417 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 8a4ceba927..d51e8e82bf 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 ac461296a9..69a80e65f5 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 054d7903fc..9d96264a60 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 0cda336d7d..0a2e31b610 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 2ad029064e..d03a4a5cdc 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 b2f52566fc..1f807dce7f 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 63824bda8e..9f01b18868 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 5beb8111ad..bed8a11247 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 0a2d943118..6be419f5a9 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 d442c13ebc..874512c361 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 f62e106680..135bfdabc6 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 0cf06c0503..9eedf7a4ca 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 5aa25552d7..dccb768f8b 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 75a9f6c5f4..c6bdfc3b02 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 9442afd7a0..a2cabce805 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 2e3a2a0dc1..9c8930b647 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 5ecc6620dd..6a402b5e67 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 433e4fa30f..a0cc457e22 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 b34a7d6924..d29c5aa610 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 fc39e62ed8..65fb95c02b 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 dcb0e1370e..a1b85f1b12 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 de7ee81403..47e0f3ec79 100644 --- a/pom.xml +++ b/pom.xml @@ -229,7 +229,12 @@ org.apache.dolphinscheduler - dolphinscheduler-rpc + dolphinscheduler-remote + ${project.version} + + + org.apache.dolphinscheduler + dolphinscheduler-service ${project.version} @@ -774,9 +779,6 @@ **/dolphinscheduler-ui/src/view/common/outro.inc **/dolphinscheduler-ui/src/view/common/meta.inc **/dolphinscheduler-ui/src/combo/1.0.0/3rd.css - - **/dolphinscheduler-rpc/src/main/java/org/apache/dolphinscheduler/rpc/LogViewServiceGrpc.java - true @@ -863,8 +865,9 @@ dolphinscheduler-api dolphinscheduler-dao dolphinscheduler-alert - dolphinscheduler-rpc dolphinscheduler-dist + dolphinscheduler-remote + dolphinscheduler-service