Browse Source

[SPI][TASK]TaskPlugin (#6122)

* [Improvement][dao]When I search for the keyword description, the web UI shows empty (#5952)

* [Bug][WorkerServer] SqlTask NullPointerException #5549

* [Improvement][dao]When I search for the keyword Modify User, the web UI shows empty #5428

* [Improvement][dao]When I search for the keyword Modify User, the web UI shows empty #5428

* [Improvement][dao]When I search for the keyword Modify User, the web UI shows empty #5428

* [Improvement][dao]When I search for the keyword Modify User, the web UI shows empty #5428

* [Improvement][dao]When I search for the keyword Modify User, the web UI shows empty #5428

* [Improvement][dao]When I search for the keyword Modify User, the web UI shows empty #5428

* [Improvement][dao]When I search for the keyword description, the web UI shows empty #5428

* fix the readme typing issue (#5998)

* Fix unchecked type conversions

* Use indentation level reported by checkstyle

* Reorganize CI workflows to fasten the wasted time and resources (#6011)

* Add standalone server module to make it easier to develop (#6022)

* Task node of SWITCH (#5939)

* [Feature-#5273][server-master] Task node of SWITCH (#5922)



Co-authored-by: wangxj <wangxj31>

* remove description of bonecp (#6030)

Co-authored-by: shaojwu <shaojwu@ebay.com>

* [Improvement][Api Module]split alert group list-paging interface (#5941)

* [Improvement][Api Module]split alert group list-paging interface

* [FIX-#6007]Wrong complement date (#6026)

* [FIX-#6007]Wrong complement date

* [style]Wrong complement date

* [Improvement-6024][dist] Remove useless packaging commands (#6029)

·Remove useless packaging commands in dolphinscheduler-bin.xml

This closes #6024

Co-authored-by: mask <liuhu@zhiyoutec.com>

* [FIX-5908][MasterServer] When executing an compensation task, the execution thread would have a NPE (#5909)

* fix the npe in MasterExec

* fix the compile error

* Add `.asf.yaml` to easily set the GitHub metadata (#6035)

* fix dead server cannot stop (#6046)

* Enhancement Translation (#6042)

* replaced Loading... with i18n

* modified Edit zh_CN translation

* Delete zh_CN.js

Co-authored-by: David <dailidong66@gmail.com>

* fix bug #6053 zh_CN.js is lost

* [Fix-6038][ui] width of "SQL Statement" in Dag FormLineModal will be shrunk if sql line is too long (#6040)

This closes #6038

* [Improvement] Fix inefficient map iterator (#6004)

* Fix inefficient map iterator

* Use forEach and remove call to valueOf

* Modify AbstractParameters

* Enhance `StandaloneServer` so that we don't need to update the version number manually (#6074)

* Remove invalid character in `.asf.yaml` (#6075)

* Remove invalid character `\n` in `.asf.yaml` (#6077)

It turns out that the invalid character is `\n`

* Add alert server into standalone-server as well and some minor polish (#6087)

* Support starting standalone server in Docker image (#6102)

Also remove unused class

* [Feature-4355][Master-Worker-API] improvements of master and scheduler module (#6095)

* [Feature-4355][Master-Worker-API] improvements of master and scheduler module (#6085)

* master refactor:
   1. spi for task submit and other actions(pause, kill)
   2. remove threads for process instance and task instance.
   3. add events for process instance and task instance

* ut npe

* add try catch

* code style

* fix critical bugs

* fix critical bugs

* fix critical bugs

* fix critical bugs

* Remove unused params in SwitchTaskTest (#6109)

* [Feature-5987][Server] Support to set multiple environment configs for a certain worker. (#6082)

* support multi environments

* add some test cases

* add an environment vue component

* improve environment form

* improve environment form

* add environment worker group relation

* add environment worker group relation

* add the environment choice for formModel

* set an environment for the task

* modify the modal form of starting process

* add the environment config to TaskExecutionContext

* add the environment config to the timing form

* fix conflicts

* fix issues of the code style

* fix some issues of the code style

* fix some issues of the code style

* fix some issues of the code style

* fix some issues of the code style

* fix some issues of the code style

* fix some bugs in the code review

* add the same table and columns to support H2

* fix some bugs

* [Plugin][Task]Task Spi

* [Plugin][Task]Task Spi

* [Plugin][Task]Task Spi

* [Plugin][Task]Task Spi

* [Plugin][Task]Task Spi

* [Plugin][Task]Task Spi

Co-authored-by: didiaode18 <563646039@qq.com>
Co-authored-by: Roy <yongjuncao1213@gmail.com>
Co-authored-by: lyxell <alyxell@kth.se>
Co-authored-by: Wenjun Ruan <wenjun@apache.org>
Co-authored-by: kezhenxu94 <kezhenxu94@apache.org>
Co-authored-by: myangle1120 <942542838@qq.com>
Co-authored-by: wangxj3 <857234426@qq.com>
Co-authored-by: gabry.wu <gabrywu@apache.org>
Co-authored-by: shaojwu <shaojwu@ebay.com>
Co-authored-by: Shukun Zhang <60541766+andream7@users.noreply.github.com>
Co-authored-by: linquan <1175687813@qq.com>
Co-authored-by: mask <39329477+Narcasserun@users.noreply.github.com>
Co-authored-by: mask <liuhu@zhiyoutec.com>
Co-authored-by: kyoty <echohlne@gmail.com>
Co-authored-by: RichardStark <49977764+RichardStark@users.noreply.github.com>
Co-authored-by: David <dailidong66@gmail.com>
Co-authored-by: lenboo <baoliang.leon@gmail.com>
Co-authored-by: lilyzhou <lj_zhou@outlook.com>
Co-authored-by: OS <29528966+lenboo@users.noreply.github.com>
Co-authored-by: junfan.zhang <zuston.shacha@gmail.com>
Co-authored-by: Hua Jiang <jianghuachinacom@163.com>
2.0.7-release
Kirs 3 years ago committed by GitHub
parent
commit
9508d1e978
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 5
      .asf.yaml
  2. 15
      docker/build/conf/dolphinscheduler/supervisor/supervisor.ini
  3. 7
      docker/build/startup.sh
  4. 7
      dolphinscheduler-alert/pom.xml
  5. 85
      dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/AlertServer.java
  6. 2
      dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/plugin/AlertPluginManager.java
  7. 7
      dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/processor/AlertRequestProcessor.java
  8. 5
      dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/runner/AlertSender.java
  9. 60
      dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/FuncUtilsTest.java
  10. 240
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/EnvironmentController.java
  11. 6
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ExecutorController.java
  12. 9
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/SchedulerController.java
  13. 129
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/dto/EnvironmentDto.java
  14. 17
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java
  15. 102
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/EnvironmentService.java
  16. 41
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/EnvironmentWorkerGroupRelationService.java
  17. 3
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java
  18. 8
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/SchedulerService.java
  19. 463
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/EnvironmentServiceImpl.java
  20. 76
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/EnvironmentWorkerGroupRelationServiceImpl.java
  21. 21
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ExecutorServiceImpl.java
  22. 5
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProcessInstanceServiceImpl.java
  23. 10
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/SchedulerServiceImpl.java
  24. 34
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/CheckUtils.java
  25. 208
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/EnvironmentControllerTest.java
  26. 310
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/EnvironmentServiceTest.java
  27. 69
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/EnvironmentWorkerGroupRelationServiceTest.java
  28. 15
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java
  29. 12
      dolphinscheduler-common/pom.xml
  30. 2
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java
  31. 111
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/StateEvent.java
  32. 38
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/StateEventType.java
  33. 15
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java
  34. 3
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/AbstractParameters.java
  35. 6
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sql/SqlParameters.java
  36. 70
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Command.java
  37. 142
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Environment.java
  38. 117
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/EnvironmentWorkerGroupRelation.java
  39. 63
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ErrorCommand.java
  40. 15
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java
  41. 14
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Schedule.java
  42. 14
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinition.java
  43. 1
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinitionLog.java
  44. 27
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java
  45. 8
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/CommandMapper.java
  46. 71
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/EnvironmentMapper.java
  47. 57
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/EnvironmentWorkerGroupRelationMapper.java
  48. 1
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/shell/CreateDolphinScheduler.java
  49. 7
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/CommandMapper.xml
  50. 55
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/EnvironmentMapper.xml
  51. 40
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/EnvironmentWorkerGroupRelationMapper.xml
  52. 2
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.xml
  53. 4
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ScheduleMapper.xml
  54. 4
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionLogMapper.xml
  55. 4
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.xml
  56. 4
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml
  57. 199
      dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/EnvironmentMapperTest.java
  58. 109
      dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/EnvironmentWorkerGroupRelationMapperTest.java
  59. 2
      dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionLogMapperTest.java
  60. 2
      dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapperTest.java
  61. 10
      dolphinscheduler-remote/pom.xml
  62. 17
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java
  63. 72
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/HostUpdateCommand.java
  64. 83
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/HostUpdateResponseCommand.java
  65. 131
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/StateEventChangeCommand.java
  66. 78
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/StateEventResponseCommand.java
  67. 14
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskExecuteAckCommand.java
  68. 16
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskExecuteResponseCommand.java
  69. 2
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/NettyRemoteChannel.java
  70. 125
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/StateEventCallbackService.java
  71. 11
      dolphinscheduler-server/pom.xml
  72. 5
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/builder/TaskExecutionContextBuilder.java
  73. 15
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/entity/TaskExecutionContext.java
  74. 26
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java
  75. 11
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterConfig.java
  76. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/executor/NettyExecutorManager.java
  77. 175
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/future/TaskFuture.java
  78. 42
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/HostUpdateResponseProcessor.java
  79. 74
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/StateEventProcessor.java
  80. 11
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskAckProcessor.java
  81. 12
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskResponseProcessor.java
  82. 149
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/StateEventResponseService.java
  83. 17
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskResponseEvent.java
  84. 35
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskResponseService.java
  85. 58
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistryClient.java
  86. 62
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/registry/ServerNodeManager.java
  87. 195
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/EventExecuteService.java
  88. 92
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java
  89. 154
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/StateWheelExecuteThread.java
  90. 676
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThread.java
  91. 112
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseTaskProcessor.java
  92. 33
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessFactory.java
  93. 175
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessor.java
  94. 32
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ConditionTaskProcessFactory.java
  95. 210
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ConditionTaskProcessor.java
  96. 33
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/DependentTaskProcessFactory.java
  97. 220
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/DependentTaskProcessor.java
  98. 25
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ITaskProcessFactory.java
  99. 39
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ITaskProcessor.java
  100. 32
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/SubTaskProcessFactory.java
  101. Some files were not shown because too many files have changed in this diff Show More

5
.asf.yaml

@ -16,10 +16,7 @@
#
github:
description: |
Apache DolphinScheduler is a distributed and extensible workflow scheduler platform with powerful DAG
visual interfaces, dedicated to solving complex job dependencies in the data pipeline and providing
various types of jobs available `out of the box`.
description: Apache DolphinScheduler is a distributed and extensible workflow scheduler platform with powerful DAG visual interfaces, dedicated to solving complex job dependencies in the data pipeline and providing various types of jobs available out of box.
homepage: https://dolphinscheduler.apache.org/
labels:
- airflow

15
docker/build/conf/dolphinscheduler/supervisor/supervisor.ini

@ -90,3 +90,18 @@ killasgroup=true
redirect_stderr=true
stdout_logfile=/dev/fd/1
stdout_logfile_maxbytes=0
[program:standalone]
command=%(ENV_DOLPHINSCHEDULER_BIN)s/dolphinscheduler-daemon.sh start standalone-server
directory=%(ENV_DOLPHINSCHEDULER_HOME)s
priority=999
autostart=%(ENV_STANDALONE_START_ENABLED)s
autorestart=true
startsecs=5
stopwaitsecs=3
exitcodes=0
stopasgroup=true
killasgroup=true
redirect_stderr=true
stdout_logfile=/dev/fd/1
stdout_logfile_maxbytes=0

7
docker/build/startup.sh

@ -24,6 +24,7 @@ export WORKER_START_ENABLED=false
export API_START_ENABLED=false
export ALERT_START_ENABLED=false
export LOGGER_START_ENABLED=false
export STANDALONE_START_ENABLED=false
# wait database
waitDatabase() {
@ -67,12 +68,13 @@ waitZK() {
printUsage() {
echo -e "Dolphin Scheduler is a distributed and easy-to-expand visual DAG workflow scheduling system,"
echo -e "dedicated to solving the complex dependencies in data processing, making the scheduling system out of the box for data processing.\n"
echo -e "Usage: [ all | master-server | worker-server | api-server | alert-server ]\n"
echo -e "Usage: [ all | master-server | worker-server | api-server | alert-server | standalone-server ]\n"
printf "%-13s: %s\n" "all" "Run master-server, worker-server, api-server and alert-server"
printf "%-13s: %s\n" "master-server" "MasterServer is mainly responsible for DAG task split, task submission monitoring."
printf "%-13s: %s\n" "worker-server" "WorkerServer is mainly responsible for task execution and providing log services."
printf "%-13s: %s\n" "api-server" "ApiServer is mainly responsible for processing requests and providing the front-end UI layer."
printf "%-13s: %s\n" "alert-server" "AlertServer mainly include Alarms."
printf "%-13s: %s\n" "standalone-server" "Standalone server that uses embedded zookeeper and database, only for testing and demostration."
}
# init config file
@ -110,6 +112,9 @@ case "$1" in
waitDatabase
export ALERT_START_ENABLED=true
;;
(standalone-server)
export STANDALONE_START_ENABLED=true
;;
(help)
printUsage
exit 1

7
dolphinscheduler-alert/pom.xml

@ -72,8 +72,13 @@
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
<exclusions>
<exclusion>
<artifactId>jsr305</artifactId>
<groupId>com.google.code.findbugs</groupId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>ch.qos.logback</groupId>
<artifactId>logback-classic</artifactId>

85
dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/AlertServer.java

@ -24,8 +24,6 @@ import org.apache.dolphinscheduler.alert.plugin.AlertPluginManager;
import org.apache.dolphinscheduler.alert.processor.AlertRequestProcessor;
import org.apache.dolphinscheduler.alert.runner.AlertSender;
import org.apache.dolphinscheduler.alert.utils.Constants;
import org.apache.dolphinscheduler.spi.plugin.DolphinPluginLoader;
import org.apache.dolphinscheduler.spi.plugin.DolphinPluginManagerConfig;
import org.apache.dolphinscheduler.common.thread.Stopper;
import org.apache.dolphinscheduler.common.utils.PropertyUtils;
import org.apache.dolphinscheduler.dao.AlertDao;
@ -35,6 +33,8 @@ import org.apache.dolphinscheduler.dao.entity.Alert;
import org.apache.dolphinscheduler.remote.NettyRemotingServer;
import org.apache.dolphinscheduler.remote.command.CommandType;
import org.apache.dolphinscheduler.remote.config.NettyServerConfig;
import org.apache.dolphinscheduler.spi.plugin.DolphinPluginLoader;
import org.apache.dolphinscheduler.spi.plugin.DolphinPluginManagerConfig;
import org.apache.dolphinscheduler.spi.utils.StringUtils;
import java.util.List;
@ -44,45 +44,29 @@ import org.slf4j.LoggerFactory;
import com.google.common.collect.ImmutableList;
/**
* alert of start
*/
public class AlertServer {
private static final Logger logger = LoggerFactory.getLogger(AlertServer.class);
/**
* Plugin Dao
*/
private PluginDao pluginDao = DaoFactory.getDaoInstance(PluginDao.class);
private final PluginDao pluginDao = DaoFactory.getDaoInstance(PluginDao.class);
/**
* Alert Dao
*/
private AlertDao alertDao = DaoFactory.getDaoInstance(AlertDao.class);
private AlertSender alertSender;
private final AlertDao alertDao = DaoFactory.getDaoInstance(AlertDao.class);
private AlertPluginManager alertPluginManager;
private DolphinPluginManagerConfig alertPluginManagerConfig;
public static final String ALERT_PLUGIN_BINDING = "alert.plugin.binding";
public static final String ALERT_PLUGIN_DIR = "alert.plugin.dir";
public static final String MAVEN_LOCAL_REPOSITORY = "maven.local.repository";
/**
* netty server
*/
private NettyRemotingServer server;
private static class AlertServerHolder {
private static final AlertServer INSTANCE = new AlertServer();
}
public static final AlertServer getInstance() {
public static AlertServer getInstance() {
return AlertServerHolder.INSTANCE;
}
@ -98,8 +82,7 @@ public class AlertServer {
}
private void initPlugin() {
alertPluginManager = new AlertPluginManager();
alertPluginManagerConfig = new DolphinPluginManagerConfig();
DolphinPluginManagerConfig alertPluginManagerConfig = new DolphinPluginManagerConfig();
alertPluginManagerConfig.setPlugins(PropertyUtils.getString(ALERT_PLUGIN_BINDING));
if (StringUtils.isNotBlank(PropertyUtils.getString(ALERT_PLUGIN_DIR))) {
alertPluginManagerConfig.setInstalledPluginsDir(PropertyUtils.getString(ALERT_PLUGIN_DIR, Constants.ALERT_PLUGIN_PATH).trim());
@ -109,6 +92,7 @@ public class AlertServer {
alertPluginManagerConfig.setMavenLocalRepository(PropertyUtils.getString(MAVEN_LOCAL_REPOSITORY).trim());
}
alertPluginManager = new AlertPluginManager();
DolphinPluginLoader alertPluginLoader = new DolphinPluginLoader(alertPluginManagerConfig, ImmutableList.of(alertPluginManager));
try {
alertPluginLoader.loadPlugins();
@ -117,9 +101,6 @@ public class AlertServer {
}
}
/**
* init netty remoting server
*/
private void initRemoteServer() {
NettyServerConfig serverConfig = new NettyServerConfig();
serverConfig.setListenPort(ALERT_RPC_PORT);
@ -128,30 +109,10 @@ public class AlertServer {
this.server.start();
}
/**
* Cyclic alert info sending alert
*/
private void runSender() {
while (Stopper.isRunning()) {
try {
Thread.sleep(Constants.ALERT_SCAN_INTERVAL);
} catch (InterruptedException e) {
logger.error(e.getMessage(), e);
Thread.currentThread().interrupt();
}
if (alertPluginManager == null || alertPluginManager.getAlertChannelMap().size() == 0) {
logger.warn("No Alert Plugin . Cannot send alert info. ");
} else {
List<Alert> alerts = alertDao.listWaitExecutionAlert();
alertSender = new AlertSender(alerts, alertDao, alertPluginManager);
alertSender.run();
}
}
new Thread(new Sender()).start();
}
/**
* start
*/
public void start() {
PropertyUtils.loadPropertyFile(ALERT_PROPERTIES_PATH);
checkTable();
@ -161,23 +122,35 @@ public class AlertServer {
runSender();
}
/**
* stop
*/
public void stop() {
this.server.close();
logger.info("alert server shut down");
}
public static void main(String[] args) {
AlertServer alertServer = AlertServer.getInstance();
alertServer.start();
Runtime.getRuntime().addShutdownHook(new Thread() {
final class Sender implements Runnable {
@Override
public void run() {
alertServer.stop();
while (Stopper.isRunning()) {
try {
Thread.sleep(Constants.ALERT_SCAN_INTERVAL);
} catch (InterruptedException e) {
logger.error(e.getMessage(), e);
Thread.currentThread().interrupt();
}
});
if (alertPluginManager == null || alertPluginManager.getAlertChannelMap().size() == 0) {
logger.warn("No Alert Plugin . Cannot send alert info. ");
} else {
List<Alert> alerts = alertDao.listWaitExecutionAlert();
new AlertSender(alerts, alertDao, alertPluginManager).run();
}
}
}
}
public static void main(String[] args) {
AlertServer alertServer = AlertServer.getInstance();
alertServer.start();
Runtime.getRuntime().addShutdownHook(new Thread(alertServer::stop));
}
}

2
dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/plugin/AlertPluginManager.java

@ -56,7 +56,7 @@ public class AlertPluginManager extends AbstractDolphinPluginManager {
*/
private final Map<Integer, String> pluginDefineMap = new HashMap<>();
private PluginDao pluginDao = DaoFactory.getDaoInstance(PluginDao.class);
private final PluginDao pluginDao = DaoFactory.getDaoInstance(PluginDao.class);
private void addAlertChannelFactory(AlertChannelFactory alertChannelFactory) {
requireNonNull(alertChannelFactory, "alertChannelFactory is null");

7
dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/processor/AlertRequestProcessor.java

@ -33,14 +33,11 @@ import org.slf4j.LoggerFactory;
import io.netty.channel.Channel;
/**
* alert request processor
*/
public class AlertRequestProcessor implements NettyRequestProcessor {
private final Logger logger = LoggerFactory.getLogger(AlertRequestProcessor.class);
private AlertDao alertDao;
private AlertPluginManager alertPluginManager;
private final AlertDao alertDao;
private final AlertPluginManager alertPluginManager;
public AlertRequestProcessor(AlertDao alertDao, AlertPluginManager alertPluginManager) {
this.alertDao = alertDao;

5
dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/runner/AlertSender.java

@ -38,16 +38,13 @@ import java.util.Map;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* alert sender
*/
public class AlertSender {
private static final Logger logger = LoggerFactory.getLogger(AlertSender.class);
private List<Alert> alertList;
private AlertDao alertDao;
private AlertPluginManager alertPluginManager;
private final AlertPluginManager alertPluginManager;
public AlertSender(AlertPluginManager alertPluginManager) {
this.alertPluginManager = alertPluginManager;

60
dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/FuncUtilsTest.java

@ -1,60 +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.alert.utils;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import java.util.Arrays;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class FuncUtilsTest {
private static final Logger logger = LoggerFactory.getLogger(FuncUtilsTest.class);
/**
* Test mkString
*/
@Test
public void testMKString() {
//Define users list
Iterable<String> users = Arrays.asList("user1", "user2", "user3");
//Define split
String split = "|";
//Invoke mkString with correctParams
String result = FuncUtils.mkString(users, split);
logger.info(result);
//Expected result string
assertEquals("user1|user2|user3", result);
//Null list expected return null
result = FuncUtils.mkString(null, split);
assertNull(result);
//Null split expected return null
result = FuncUtils.mkString(users, null);
assertNull(result);
}
}

240
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/EnvironmentController.java

@ -0,0 +1,240 @@
/*
* 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.controller;
import static org.apache.dolphinscheduler.api.enums.Status.CREATE_ENVIRONMENT_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.DELETE_ENVIRONMENT_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.QUERY_ENVIRONMENT_BY_CODE_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.QUERY_ENVIRONMENT_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.UPDATE_ENVIRONMENT_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.VERIFY_ENVIRONMENT_ERROR;
import org.apache.dolphinscheduler.api.aspect.AccessLogAnnotation;
import org.apache.dolphinscheduler.api.exceptions.ApiException;
import org.apache.dolphinscheduler.api.service.EnvironmentService;
import org.apache.dolphinscheduler.api.utils.Result;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.utils.ParameterUtils;
import org.apache.dolphinscheduler.dao.entity.User;
import java.util.Map;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.http.HttpStatus;
import org.springframework.web.bind.annotation.GetMapping;
import org.springframework.web.bind.annotation.PostMapping;
import org.springframework.web.bind.annotation.RequestAttribute;
import org.springframework.web.bind.annotation.RequestMapping;
import org.springframework.web.bind.annotation.RequestParam;
import org.springframework.web.bind.annotation.ResponseStatus;
import org.springframework.web.bind.annotation.RestController;
import io.swagger.annotations.Api;
import io.swagger.annotations.ApiImplicitParam;
import io.swagger.annotations.ApiImplicitParams;
import io.swagger.annotations.ApiOperation;
import springfox.documentation.annotations.ApiIgnore;
/**
* environment controller
*/
@Api(tags = "ENVIRONMENT_TAG")
@RestController
@RequestMapping("environment")
public class EnvironmentController extends BaseController {
@Autowired
private EnvironmentService environmentService;
/**
* create environment
*
* @param loginUser login user
* @param name environment name
* @param config config
* @param description description
* @return returns an error if it exists
*/
@ApiOperation(value = "createEnvironment", notes = "CREATE_ENVIRONMENT_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "name", value = "ENVIRONMENT_NAME", required = true, dataType = "String"),
@ApiImplicitParam(name = "config", value = "CONFIG", required = true, dataType = "String"),
@ApiImplicitParam(name = "description", value = "ENVIRONMENT_DESC", dataType = "String"),
@ApiImplicitParam(name = "workerGroups", value = "WORKER_GROUP_LIST", dataType = "String")
})
@PostMapping(value = "/create")
@ResponseStatus(HttpStatus.CREATED)
@ApiException(CREATE_ENVIRONMENT_ERROR)
@AccessLogAnnotation(ignoreRequestArgs = "loginUser")
public Result createProject(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@RequestParam("name") String name,
@RequestParam("config") String config,
@RequestParam(value = "description", required = false) String description,
@RequestParam(value = "workerGroups", required = false) String workerGroups) {
Map<String, Object> result = environmentService.createEnvironment(loginUser, name, config, description, workerGroups);
return returnDataList(result);
}
/**
* update environment
*
* @param loginUser login user
* @param code environment code
* @param name environment name
* @param config environment config
* @param description description
* @return update result code
*/
@ApiOperation(value = "updateEnvironment", notes = "UPDATE_ENVIRONMENT_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "code", value = "ENVIRONMENT_CODE", required = true, dataType = "Long", example = "100"),
@ApiImplicitParam(name = "name", value = "ENVIRONMENT_NAME", required = true, dataType = "String"),
@ApiImplicitParam(name = "config", value = "ENVIRONMENT_CONFIG", required = true, dataType = "String"),
@ApiImplicitParam(name = "description", value = "ENVIRONMENT_DESC", dataType = "String"),
@ApiImplicitParam(name = "workerGroups", value = "WORKER_GROUP_LIST", dataType = "String")
})
@PostMapping(value = "/update")
@ResponseStatus(HttpStatus.OK)
@ApiException(UPDATE_ENVIRONMENT_ERROR)
@AccessLogAnnotation(ignoreRequestArgs = "loginUser")
public Result updateEnvironment(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@RequestParam("code") Long code,
@RequestParam("name") String name,
@RequestParam("config") String config,
@RequestParam(value = "description", required = false) String description,
@RequestParam(value = "workerGroups", required = false) String workerGroups) {
Map<String, Object> result = environmentService.updateEnvironmentByCode(loginUser, code, name, config, description, workerGroups);
return returnDataList(result);
}
/**
* query environment details by code
*
* @param environmentCode environment code
* @return environment detail information
*/
@ApiOperation(value = "queryEnvironmentByCode", notes = "QUERY_ENVIRONMENT_BY_CODE_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "environmentCode", value = "ENVIRONMENT_CODE", required = true, dataType = "Long", example = "100")
})
@GetMapping(value = "/query-by-code")
@ResponseStatus(HttpStatus.OK)
@ApiException(QUERY_ENVIRONMENT_BY_CODE_ERROR)
@AccessLogAnnotation(ignoreRequestArgs = "loginUser")
public Result queryEnvironmentByCode(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@RequestParam("environmentCode") Long environmentCode) {
Map<String, Object> result = environmentService.queryEnvironmentByCode(environmentCode);
return returnDataList(result);
}
/**
* query environment list paging
*
* @param searchVal search value
* @param pageSize page size
* @param pageNo page number
* @return environment list which the login user have permission to see
*/
@ApiOperation(value = "queryEnvironmentListPaging", notes = "QUERY_ENVIRONMENT_LIST_PAGING_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "searchVal", value = "SEARCH_VAL", dataType = "String"),
@ApiImplicitParam(name = "pageSize", value = "PAGE_SIZE", required = true, dataType = "Int", example = "20"),
@ApiImplicitParam(name = "pageNo", value = "PAGE_NO", required = true, dataType = "Int", example = "1")
})
@GetMapping(value = "/list-paging")
@ResponseStatus(HttpStatus.OK)
@ApiException(QUERY_ENVIRONMENT_ERROR)
@AccessLogAnnotation(ignoreRequestArgs = "loginUser")
public Result queryEnvironmentListPaging(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@RequestParam(value = "searchVal", required = false) String searchVal,
@RequestParam("pageSize") Integer pageSize,
@RequestParam("pageNo") Integer pageNo
) {
Result result = checkPageParams(pageNo, pageSize);
if (!result.checkResult()) {
return result;
}
searchVal = ParameterUtils.handleEscapes(searchVal);
result = environmentService.queryEnvironmentListPaging(pageNo, pageSize, searchVal);
return result;
}
/**
* delete environment by code
*
* @param loginUser login user
* @param environmentCode environment code
* @return delete result code
*/
@ApiOperation(value = "deleteEnvironmentByCode", notes = "DELETE_ENVIRONMENT_BY_CODE_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "environmentCode", value = "ENVIRONMENT_CODE", required = true, dataType = "Long", example = "100")
})
@PostMapping(value = "/delete")
@ResponseStatus(HttpStatus.OK)
@ApiException(DELETE_ENVIRONMENT_ERROR)
@AccessLogAnnotation(ignoreRequestArgs = "loginUser")
public Result deleteEnvironment(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@RequestParam("environmentCode") Long environmentCode
) {
Map<String, Object> result = environmentService.deleteEnvironmentByCode(loginUser, environmentCode);
return returnDataList(result);
}
/**
* query all environment list
*
* @param loginUser login user
* @return all environment list
*/
@ApiOperation(value = "queryAllEnvironmentList", notes = "QUERY_ALL_ENVIRONMENT_LIST_NOTES")
@GetMapping(value = "/query-environment-list")
@ResponseStatus(HttpStatus.OK)
@ApiException(QUERY_ENVIRONMENT_ERROR)
@AccessLogAnnotation(ignoreRequestArgs = "loginUser")
public Result queryAllEnvironmentList(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser) {
Map<String, Object> result = environmentService.queryAllEnvironmentList();
return returnDataList(result);
}
/**
* verify environment and environment name
*
* @param loginUser login user
* @param environmentName environment name
* @return true if the environment name not exists, otherwise return false
*/
@ApiOperation(value = "verifyEnvironment", notes = "VERIFY_ENVIRONMENT_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "environmentName", value = "ENVIRONMENT_NAME", required = true, dataType = "String")
})
@PostMapping(value = "/verify-environment")
@ResponseStatus(HttpStatus.OK)
@ApiException(VERIFY_ENVIRONMENT_ERROR)
@AccessLogAnnotation(ignoreRequestArgs = "loginUser")
public Result verifyEnvironment(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@RequestParam(value = "environmentName") String environmentName
) {
Map<String, Object> result = environmentService.verifyEnvironment(environmentName);
return returnDataList(result);
}
}

6
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ExecutorController.java

@ -99,8 +99,9 @@ public class ExecutorController extends BaseController {
@ApiImplicitParam(name = "runMode", value = "RUN_MODE", dataType = "RunMode"),
@ApiImplicitParam(name = "processInstancePriority", value = "PROCESS_INSTANCE_PRIORITY", required = true, dataType = "Priority"),
@ApiImplicitParam(name = "workerGroup", value = "WORKER_GROUP", dataType = "String", example = "default"),
@ApiImplicitParam(name = "environmentCode", value = "ENVIRONMENT_CODE", dataType = "Long", example = "default"),
@ApiImplicitParam(name = "timeout", value = "TIMEOUT", dataType = "Int", example = "100"),
@ApiImplicitParam(name = "expectedParallelismNumber", value = "EXPECTED_PARALLELISM_NUMBER", dataType = "Int", example = "8"),
@ApiImplicitParam(name = "expectedParallelismNumber", value = "EXPECTED_PARALLELISM_NUMBER", dataType = "Int", example = "8")
})
@PostMapping(value = "start-process-instance")
@ResponseStatus(HttpStatus.OK)
@ -119,6 +120,7 @@ public class ExecutorController extends BaseController {
@RequestParam(value = "runMode", required = false) RunMode runMode,
@RequestParam(value = "processInstancePriority", required = false) Priority processInstancePriority,
@RequestParam(value = "workerGroup", required = false, defaultValue = "default") String workerGroup,
@RequestParam(value = "environmentCode", required = false, defaultValue = "-1") Long environmentCode,
@RequestParam(value = "timeout", required = false) Integer timeout,
@RequestParam(value = "startParams", required = false) String startParams,
@RequestParam(value = "expectedParallelismNumber", required = false) Integer expectedParallelismNumber
@ -133,7 +135,7 @@ public class ExecutorController extends BaseController {
}
Map<String, Object> result = execService.execProcessInstance(loginUser, projectName, processDefinitionId, scheduleTime, execType, failureStrategy,
startNodeList, taskDependType, warningType,
warningGroupId, runMode, processInstancePriority, workerGroup, timeout, startParamMap, expectedParallelismNumber);
warningGroupId, runMode, processInstancePriority, workerGroup, environmentCode, timeout, startParamMap, expectedParallelismNumber);
return returnDataList(result);
}

9
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/SchedulerController.java

@ -74,7 +74,6 @@ public class SchedulerController extends BaseController {
@Autowired
private SchedulerService schedulerService;
/**
* create schedule
*
@ -99,6 +98,7 @@ public class SchedulerController extends BaseController {
@ApiImplicitParam(name = "failureStrategy", value = "FAILURE_STRATEGY", type = "FailureStrategy"),
@ApiImplicitParam(name = "workerGroupId", value = "WORKER_GROUP_ID", dataType = "Int", example = "100"),
@ApiImplicitParam(name = "workerGroup", value = "WORKER_GROUP", dataType = "String"),
@ApiImplicitParam(name = "environmentCode", value = "ENVIRONMENT_CODE", dataType = "Long"),
@ApiImplicitParam(name = "processInstancePriority", value = "PROCESS_INSTANCE_PRIORITY", type = "Priority"),
})
@PostMapping("/create")
@ -113,9 +113,10 @@ public class SchedulerController extends BaseController {
@RequestParam(value = "warningGroupId", required = false, defaultValue = DEFAULT_NOTIFY_GROUP_ID) int warningGroupId,
@RequestParam(value = "failureStrategy", required = false, defaultValue = DEFAULT_FAILURE_POLICY) FailureStrategy failureStrategy,
@RequestParam(value = "workerGroup", required = false, defaultValue = "default") String workerGroup,
@RequestParam(value = "environmentCode", required = false, defaultValue = "-1") Long environmentCode,
@RequestParam(value = "processInstancePriority", required = false, defaultValue = DEFAULT_PROCESS_INSTANCE_PRIORITY) Priority processInstancePriority) {
Map<String, Object> result = schedulerService.insertSchedule(loginUser, projectName, processDefinitionId, schedule,
warningType, warningGroupId, failureStrategy, processInstancePriority, workerGroup);
warningType, warningGroupId, failureStrategy, processInstancePriority, workerGroup, environmentCode);
return returnDataList(result);
}
@ -145,6 +146,7 @@ public class SchedulerController extends BaseController {
@ApiImplicitParam(name = "failureStrategy", value = "FAILURE_STRATEGY", type = "FailureStrategy"),
@ApiImplicitParam(name = "workerGroupId", value = "WORKER_GROUP_ID", dataType = "Int", example = "100"),
@ApiImplicitParam(name = "workerGroup", value = "WORKER_GROUP", dataType = "String"),
@ApiImplicitParam(name = "environmentCode", value = "ENVIRONMENT_CODE", dataType = "Long"),
@ApiImplicitParam(name = "processInstancePriority", value = "PROCESS_INSTANCE_PRIORITY", type = "Priority")
})
@PostMapping("/update")
@ -158,10 +160,11 @@ public class SchedulerController extends BaseController {
@RequestParam(value = "warningGroupId", required = false) int warningGroupId,
@RequestParam(value = "failureStrategy", required = false, defaultValue = "END") FailureStrategy failureStrategy,
@RequestParam(value = "workerGroup", required = false, defaultValue = "default") String workerGroup,
@RequestParam(value = "environmentCode", required = false, defaultValue = "-1") Long environmentCode,
@RequestParam(value = "processInstancePriority", required = false) Priority processInstancePriority) {
Map<String, Object> result = schedulerService.updateSchedule(loginUser, projectName, id, schedule,
warningType, warningGroupId, failureStrategy, null, processInstancePriority, workerGroup);
warningType, warningGroupId, failureStrategy, null, processInstancePriority, workerGroup, environmentCode);
return returnDataList(result);
}

129
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/dto/EnvironmentDto.java

@ -0,0 +1,129 @@
/*
* 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.dto;
import java.util.Date;
import java.util.List;
/**
* EnvironmentDto
*/
public class EnvironmentDto {
private int id;
/**
* environment code
*/
private Long code;
/**
* environment name
*/
private String name;
/**
* config content
*/
private String config;
private String description;
private List<String> workerGroups;
/**
* operator user id
*/
private Integer operator;
private Date createTime;
private Date updateTime;
public int getId() {
return id;
}
public void setId(int id) {
this.id = id;
}
public String getName() {
return name;
}
public void setName(String name) {
this.name = name;
}
public Long getCode() {
return this.code;
}
public void setCode(Long code) {
this.code = code;
}
public String getConfig() {
return this.config;
}
public void setConfig(String config) {
this.config = config;
}
public String getDescription() {
return this.description;
}
public void setDescription(String description) {
this.description = description;
}
public Integer getOperator() {
return this.operator;
}
public void setOperator(Integer operator) {
this.operator = operator;
}
public Date getCreateTime() {
return createTime;
}
public void setCreateTime(Date createTime) {
this.createTime = createTime;
}
public Date getUpdateTime() {
return updateTime;
}
public void setUpdateTime(Date updateTime) {
this.updateTime = updateTime;
}
public List<String> getWorkerGroups() {
return workerGroups;
}
public void setWorkerGroups(List<String> workerGroups) {
this.workerGroups = workerGroups;
}
}

17
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java

@ -310,7 +310,22 @@ public enum Status {
LIST_PAGING_ALERT_PLUGIN_INSTANCE_ERROR(110011, "query plugin instance page error", "分页查询告警实例失败"),
DELETE_ALERT_PLUGIN_INSTANCE_ERROR_HAS_ALERT_GROUP_ASSOCIATED(110012, "failed to delete the alert instance, there is an alarm group associated with this alert instance",
"删除告警实例失败,存在与此告警实例关联的警报组"),
PROCESS_DEFINITION_VERSION_IS_USED(110013,"this process definition version is used","此工作流定义版本被使用");
PROCESS_DEFINITION_VERSION_IS_USED(110013,"this process definition version is used","此工作流定义版本被使用"),
CREATE_ENVIRONMENT_ERROR(120001, "create environment error", "创建环境失败"),
ENVIRONMENT_NAME_EXISTS(120002,"this enviroment name [{0}] already exists","环境名称[{0}]已经存在"),
ENVIRONMENT_NAME_IS_NULL(120003,"this enviroment name shouldn't be empty.","环境名称不能为空"),
ENVIRONMENT_CONFIG_IS_NULL(120004,"this enviroment config shouldn't be empty.","环境配置信息不能为空"),
UPDATE_ENVIRONMENT_ERROR(120005, "update environment [{0}] info error", "更新环境[{0}]信息失败"),
DELETE_ENVIRONMENT_ERROR(120006, "delete environment error", "删除环境信息失败"),
DELETE_ENVIRONMENT_RELATED_TASK_EXISTS(120007, "this environment has been used in tasks,so you can't delete it.", "该环境已经被任务使用,所以不能删除该环境信息"),
QUERY_ENVIRONMENT_BY_NAME_ERROR(1200008, "not found environment [{0}] ", "查询环境名称[{0}]信息不存在"),
QUERY_ENVIRONMENT_BY_CODE_ERROR(1200009, "not found environment [{0}] ", "查询环境编码[{0}]不存在"),
QUERY_ENVIRONMENT_ERROR(1200010, "login user query environment error", "分页查询环境列表错误"),
VERIFY_ENVIRONMENT_ERROR(1200011, "verify environment error", "验证环境信息错误"),
ENVIRONMENT_WORKER_GROUPS_IS_INVALID(1200012, "environment worker groups is invalid format", "环境关联的工作组参数解析错误"),
UPDATE_ENVIRONMENT_WORKER_GROUP_RELATION_ERROR(1200013,"You can't modify the worker group, because the worker group [{0}] and this environment [{1}] already be used in the task [{2}]",
"您不能修改工作组选项,因为该工作组 [{0}] 和 该环境 [{1}] 已经被用在任务 [{2}] 中");
private final int code;

102
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/EnvironmentService.java

@ -0,0 +1,102 @@
/*
* 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.service;
import org.apache.dolphinscheduler.api.utils.Result;
import org.apache.dolphinscheduler.dao.entity.User;
import java.util.Map;
/**
* environment service
*/
public interface EnvironmentService {
/**
* create environment
*
* @param loginUser login user
* @param name environment name
* @param config environment config
* @param desc environment desc
* @param workerGroups worker groups
*/
Map<String, Object> createEnvironment(User loginUser, String name, String config, String desc, String workerGroups);
/**
* query environment
*
* @param name environment name
*/
Map<String, Object> queryEnvironmentByName(String name);
/**
* query environment
*
* @param code environment code
*/
Map<String, Object> queryEnvironmentByCode(Long code);
/**
* delete environment
*
* @param loginUser login user
* @param code environment code
*/
Map<String, Object> deleteEnvironmentByCode(User loginUser, Long code);
/**
* update environment
*
* @param loginUser login user
* @param code environment code
* @param name environment name
* @param config environment config
* @param desc environment desc
* @param workerGroups worker groups
*/
Map<String, Object> updateEnvironmentByCode(User loginUser, Long code, String name, String config, String desc, String workerGroups);
/**
* query environment paging
*
* @param pageNo page number
* @param searchVal search value
* @param pageSize page size
* @return environment list page
*/
Result queryEnvironmentListPaging(Integer pageNo, Integer pageSize, String searchVal);
/**
* query all environment
*
* @return all environment list
*/
Map<String, Object> queryAllEnvironmentList();
/**
* verify environment name
*
* @param environmentName environment name
* @return true if the environment name not exists, otherwise return false
*/
Map<String, Object> verifyEnvironment(String environmentName);
}

41
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/EnvironmentWorkerGroupRelationService.java

@ -0,0 +1,41 @@
/*
* 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.service;
import java.util.Map;
/**
* environment worker group relation service
*/
public interface EnvironmentWorkerGroupRelationService {
/**
* query environment worker group relation
*
* @param environmentCode environment code
*/
Map<String, Object> queryEnvironmentWorkerGroupRelation(Long environmentCode);
/**
* query all environment worker group relation
*
* @return all relation list
*/
Map<String, Object> queryAllEnvironmentWorkerGroupRelationList();
}

3
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java

@ -49,6 +49,7 @@ public interface ExecutorService {
* @param warningGroupId notify group id
* @param processInstancePriority process instance priority
* @param workerGroup worker group name
* @param environmentCode environment code
* @param runMode run mode
* @param timeout timeout
* @param startParams the global param values which pass to new process instance
@ -60,7 +61,7 @@ public interface ExecutorService {
FailureStrategy failureStrategy, String startNodeList,
TaskDependType taskDependType, WarningType warningType, int warningGroupId,
RunMode runMode,
Priority processInstancePriority, String workerGroup, Integer timeout,
Priority processInstancePriority, String workerGroup, Long environmentCode, Integer timeout,
Map<String, String> startParams, Integer expectedParallelismNumber);
/**

8
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/SchedulerService.java

@ -43,6 +43,7 @@ public interface SchedulerService {
* @param failureStrategy failure strategy
* @param processInstancePriority process instance priority
* @param workerGroup worker group
* @param environmentCode environment code
* @return create result code
*/
Map<String, Object> insertSchedule(User loginUser, String projectName,
@ -52,7 +53,8 @@ public interface SchedulerService {
int warningGroupId,
FailureStrategy failureStrategy,
Priority processInstancePriority,
String workerGroup);
String workerGroup,
Long environmentCode);
/**
* updateProcessInstance schedule
@ -65,6 +67,7 @@ public interface SchedulerService {
* @param warningGroupId warning group id
* @param failureStrategy failure strategy
* @param workerGroup worker group
* @param environmentCode environment code
* @param processInstancePriority process instance priority
* @param scheduleStatus schedule status
* @return update result code
@ -78,7 +81,8 @@ public interface SchedulerService {
FailureStrategy failureStrategy,
ReleaseState scheduleStatus,
Priority processInstancePriority,
String workerGroup);
String workerGroup,
Long environmentCode);
/**

463
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/EnvironmentServiceImpl.java

@ -0,0 +1,463 @@
/*
* 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.service.impl;
import org.apache.dolphinscheduler.api.dto.EnvironmentDto;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.service.EnvironmentService;
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.utils.CollectionUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.SnowFlakeUtils;
import org.apache.dolphinscheduler.common.utils.SnowFlakeUtils.SnowFlakeException;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.dao.entity.Environment;
import org.apache.dolphinscheduler.dao.entity.EnvironmentWorkerGroupRelation;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.EnvironmentMapper;
import org.apache.dolphinscheduler.dao.mapper.EnvironmentWorkerGroupRelationMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
import org.apache.commons.collections4.SetUtils;
import java.util.ArrayList;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.TreeSet;
import java.util.stream.Collectors;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.BeanUtils;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import org.springframework.transaction.annotation.Transactional;
import com.baomidou.mybatisplus.core.conditions.query.QueryWrapper;
import com.baomidou.mybatisplus.core.conditions.update.UpdateWrapper;
import com.baomidou.mybatisplus.core.metadata.IPage;
import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
import com.fasterxml.jackson.core.type.TypeReference;
/**
* task definition service impl
*/
@Service
public class EnvironmentServiceImpl extends BaseServiceImpl implements EnvironmentService {
private static final Logger logger = LoggerFactory.getLogger(EnvironmentServiceImpl.class);
@Autowired
private EnvironmentMapper environmentMapper;
@Autowired
private EnvironmentWorkerGroupRelationMapper relationMapper;
@Autowired
private TaskDefinitionMapper taskDefinitionMapper;
/**
* create environment
*
* @param loginUser login user
* @param name environment name
* @param config environment config
* @param desc environment desc
* @param workerGroups worker groups
*/
@Transactional(rollbackFor = RuntimeException.class)
@Override
public Map<String, Object> createEnvironment(User loginUser, String name, String config, String desc, String workerGroups) {
Map<String, Object> result = new HashMap<>();
if (isNotAdmin(loginUser, result)) {
return result;
}
Map<String, Object> checkResult = checkParams(name,config,workerGroups);
if (checkResult.get(Constants.STATUS) != Status.SUCCESS) {
return checkResult;
}
Environment environment = environmentMapper.queryByEnvironmentName(name);
if (environment != null) {
putMsg(result, Status.ENVIRONMENT_NAME_EXISTS, name);
return result;
}
Environment env = new Environment();
env.setName(name);
env.setConfig(config);
env.setDescription(desc);
env.setOperator(loginUser.getId());
env.setCreateTime(new Date());
env.setUpdateTime(new Date());
long code = 0L;
try {
code = SnowFlakeUtils.getInstance().nextId();
env.setCode(code);
} catch (SnowFlakeException e) {
logger.error("Environment code get error, ", e);
}
if (code == 0L) {
putMsg(result, Status.INTERNAL_SERVER_ERROR_ARGS, "Error generating environment code");
return result;
}
if (environmentMapper.insert(env) > 0) {
if (StringUtils.isNotEmpty(workerGroups)) {
List<String> workerGroupList = JSONUtils.parseObject(workerGroups, new TypeReference<List<String>>(){});
if (CollectionUtils.isNotEmpty(workerGroupList)) {
workerGroupList.stream().forEach(workerGroup -> {
if (StringUtils.isNotEmpty(workerGroup)) {
EnvironmentWorkerGroupRelation relation = new EnvironmentWorkerGroupRelation();
relation.setEnvironmentCode(env.getCode());
relation.setWorkerGroup(workerGroup);
relation.setOperator(loginUser.getId());
relation.setCreateTime(new Date());
relation.setUpdateTime(new Date());
relationMapper.insert(relation);
}
});
}
}
result.put(Constants.DATA_LIST, env.getCode());
putMsg(result, Status.SUCCESS);
} else {
putMsg(result, Status.CREATE_ENVIRONMENT_ERROR);
}
return result;
}
/**
* query environment paging
*
* @param pageNo page number
* @param searchVal search value
* @param pageSize page size
* @return environment list page
*/
@Override
public Result queryEnvironmentListPaging(Integer pageNo, Integer pageSize, String searchVal) {
Result result = new Result();
Page<Environment> page = new Page<>(pageNo, pageSize);
IPage<Environment> environmentIPage = environmentMapper.queryEnvironmentListPaging(page, searchVal);
PageInfo<EnvironmentDto> pageInfo = new PageInfo<>(pageNo, pageSize);
pageInfo.setTotal((int) environmentIPage.getTotal());
if (CollectionUtils.isNotEmpty(environmentIPage.getRecords())) {
Map<Long, List<String>> relationMap = relationMapper.selectList(null).stream()
.collect(Collectors.groupingBy(EnvironmentWorkerGroupRelation::getEnvironmentCode,Collectors.mapping(EnvironmentWorkerGroupRelation::getWorkerGroup,Collectors.toList())));
List<EnvironmentDto> dtoList = environmentIPage.getRecords().stream().map(environment -> {
EnvironmentDto dto = new EnvironmentDto();
BeanUtils.copyProperties(environment,dto);
List<String> workerGroups = relationMap.getOrDefault(environment.getCode(),new ArrayList<String>());
dto.setWorkerGroups(workerGroups);
return dto;
}).collect(Collectors.toList());
pageInfo.setTotalList(dtoList);
} else {
pageInfo.setTotalList(new ArrayList<>());
}
result.setData(pageInfo);
putMsg(result, Status.SUCCESS);
return result;
}
/**
* query all environment
*
* @return all environment list
*/
@Override
public Map<String, Object> queryAllEnvironmentList() {
Map<String,Object> result = new HashMap<>();
List<Environment> environmentList = environmentMapper.queryAllEnvironmentList();
if (CollectionUtils.isNotEmpty(environmentList)) {
Map<Long, List<String>> relationMap = relationMapper.selectList(null).stream()
.collect(Collectors.groupingBy(EnvironmentWorkerGroupRelation::getEnvironmentCode,Collectors.mapping(EnvironmentWorkerGroupRelation::getWorkerGroup,Collectors.toList())));
List<EnvironmentDto> dtoList = environmentList.stream().map(environment -> {
EnvironmentDto dto = new EnvironmentDto();
BeanUtils.copyProperties(environment,dto);
List<String> workerGroups = relationMap.getOrDefault(environment.getCode(),new ArrayList<String>());
dto.setWorkerGroups(workerGroups);
return dto;
}).collect(Collectors.toList());
result.put(Constants.DATA_LIST,dtoList);
} else {
result.put(Constants.DATA_LIST, new ArrayList<>());
}
putMsg(result,Status.SUCCESS);
return result;
}
/**
* query environment
*
* @param code environment code
*/
@Override
public Map<String, Object> queryEnvironmentByCode(Long code) {
Map<String, Object> result = new HashMap<>();
Environment env = environmentMapper.queryByEnvironmentCode(code);
if (env == null) {
putMsg(result, Status.QUERY_ENVIRONMENT_BY_CODE_ERROR, code);
} else {
List<String> workerGroups = relationMapper.queryByEnvironmentCode(env.getCode()).stream()
.map(item -> item.getWorkerGroup())
.collect(Collectors.toList());
EnvironmentDto dto = new EnvironmentDto();
BeanUtils.copyProperties(env,dto);
dto.setWorkerGroups(workerGroups);
result.put(Constants.DATA_LIST, dto);
putMsg(result, Status.SUCCESS);
}
return result;
}
/**
* query environment
*
* @param name environment name
*/
@Override
public Map<String, Object> queryEnvironmentByName(String name) {
Map<String, Object> result = new HashMap<>();
Environment env = environmentMapper.queryByEnvironmentName(name);
if (env == null) {
putMsg(result, Status.QUERY_ENVIRONMENT_BY_NAME_ERROR, name);
} else {
List<String> workerGroups = relationMapper.queryByEnvironmentCode(env.getCode()).stream()
.map(item -> item.getWorkerGroup())
.collect(Collectors.toList());
EnvironmentDto dto = new EnvironmentDto();
BeanUtils.copyProperties(env,dto);
dto.setWorkerGroups(workerGroups);
result.put(Constants.DATA_LIST, dto);
putMsg(result, Status.SUCCESS);
}
return result;
}
/**
* delete environment
*
* @param loginUser login user
* @param code environment code
*/
@Transactional(rollbackFor = RuntimeException.class)
@Override
public Map<String, Object> deleteEnvironmentByCode(User loginUser, Long code) {
Map<String, Object> result = new HashMap<>();
if (isNotAdmin(loginUser, result)) {
return result;
}
Integer relatedTaskNumber = taskDefinitionMapper
.selectCount(new QueryWrapper<TaskDefinition>().lambda().eq(TaskDefinition::getEnvironmentCode,code));
if (relatedTaskNumber > 0) {
putMsg(result, Status.DELETE_ENVIRONMENT_RELATED_TASK_EXISTS);
return result;
}
int delete = environmentMapper.deleteByCode(code);
if (delete > 0) {
relationMapper.delete(new QueryWrapper<EnvironmentWorkerGroupRelation>()
.lambda()
.eq(EnvironmentWorkerGroupRelation::getEnvironmentCode,code));
putMsg(result, Status.SUCCESS);
} else {
putMsg(result, Status.DELETE_ENVIRONMENT_ERROR);
}
return result;
}
/**
* update environment
*
* @param loginUser login user
* @param code environment code
* @param name environment name
* @param config environment config
* @param desc environment desc
* @param workerGroups worker groups
*/
@Transactional(rollbackFor = RuntimeException.class)
@Override
public Map<String, Object> updateEnvironmentByCode(User loginUser, Long code, String name, String config, String desc, String workerGroups) {
Map<String, Object> result = new HashMap<>();
if (isNotAdmin(loginUser, result)) {
return result;
}
Map<String, Object> checkResult = checkParams(name,config,workerGroups);
if (checkResult.get(Constants.STATUS) != Status.SUCCESS) {
return checkResult;
}
Environment environment = environmentMapper.queryByEnvironmentName(name);
if (environment != null && !environment.getCode().equals(code)) {
putMsg(result, Status.ENVIRONMENT_NAME_EXISTS, name);
return result;
}
Set<String> workerGroupSet;
if (StringUtils.isNotEmpty(workerGroups)) {
workerGroupSet = JSONUtils.parseObject(workerGroups, new TypeReference<Set<String>>() {});
} else {
workerGroupSet = new TreeSet<>();
}
Set<String> existWorkerGroupSet = relationMapper
.queryByEnvironmentCode(code)
.stream()
.map(item -> item.getWorkerGroup())
.collect(Collectors.toSet());
Set<String> deleteWorkerGroupSet = SetUtils.difference(existWorkerGroupSet,workerGroupSet).toSet();
Set<String> addWorkerGroupSet = SetUtils.difference(workerGroupSet,existWorkerGroupSet).toSet();
// verify whether the relation of this environment and worker groups can be adjusted
checkResult = checkUsedEnvironmentWorkerGroupRelation(deleteWorkerGroupSet, name, code);
if (checkResult.get(Constants.STATUS) != Status.SUCCESS) {
return checkResult;
}
Environment env = new Environment();
env.setCode(code);
env.setName(name);
env.setConfig(config);
env.setDescription(desc);
env.setOperator(loginUser.getId());
env.setUpdateTime(new Date());
int update = environmentMapper.update(env, new UpdateWrapper<Environment>().lambda().eq(Environment::getCode,code));
if (update > 0) {
deleteWorkerGroupSet.stream().forEach(key -> {
if (StringUtils.isNotEmpty(key)) {
relationMapper.delete(new QueryWrapper<EnvironmentWorkerGroupRelation>()
.lambda()
.eq(EnvironmentWorkerGroupRelation::getEnvironmentCode,code));
}
});
addWorkerGroupSet.stream().forEach(key -> {
if (StringUtils.isNotEmpty(key)) {
EnvironmentWorkerGroupRelation relation = new EnvironmentWorkerGroupRelation();
relation.setEnvironmentCode(code);
relation.setWorkerGroup(key);
relation.setUpdateTime(new Date());
relation.setCreateTime(new Date());
relation.setOperator(loginUser.getId());
relationMapper.insert(relation);
}
});
putMsg(result, Status.SUCCESS);
} else {
putMsg(result, Status.UPDATE_ENVIRONMENT_ERROR, name);
}
return result;
}
/**
* verify environment name
*
* @param environmentName environment name
* @return true if the environment name not exists, otherwise return false
*/
@Override
public Map<String, Object> verifyEnvironment(String environmentName) {
Map<String, Object> result = new HashMap<>();
if (StringUtils.isEmpty(environmentName)) {
putMsg(result, Status.ENVIRONMENT_NAME_IS_NULL);
return result;
}
Environment environment = environmentMapper.queryByEnvironmentName(environmentName);
if (environment != null) {
putMsg(result, Status.ENVIRONMENT_NAME_EXISTS, environmentName);
return result;
}
result.put(Constants.STATUS, Status.SUCCESS);
return result;
}
private Map<String, Object> checkUsedEnvironmentWorkerGroupRelation(Set<String> deleteKeySet,String environmentName, Long environmentCode) {
Map<String, Object> result = new HashMap<>();
for (String workerGroup : deleteKeySet) {
TaskDefinition taskDefinition = taskDefinitionMapper
.selectOne(new QueryWrapper<TaskDefinition>().lambda()
.eq(TaskDefinition::getEnvironmentCode,environmentCode)
.eq(TaskDefinition::getWorkerGroup,workerGroup));
if (Objects.nonNull(taskDefinition)) {
putMsg(result, Status.UPDATE_ENVIRONMENT_WORKER_GROUP_RELATION_ERROR,workerGroup,environmentName,taskDefinition.getName());
return result;
}
}
result.put(Constants.STATUS, Status.SUCCESS);
return result;
}
public Map<String, Object> checkParams(String name, String config, String workerGroups) {
Map<String, Object> result = new HashMap<>();
if (StringUtils.isEmpty(name)) {
putMsg(result, Status.ENVIRONMENT_NAME_IS_NULL);
return result;
}
if (StringUtils.isEmpty(config)) {
putMsg(result, Status.ENVIRONMENT_CONFIG_IS_NULL);
return result;
}
if (StringUtils.isNotEmpty(workerGroups)) {
List<String> workerGroupList = JSONUtils.parseObject(workerGroups, new TypeReference<List<String>>(){});
if (Objects.isNull(workerGroupList)) {
putMsg(result, Status.ENVIRONMENT_WORKER_GROUPS_IS_INVALID);
return result;
}
}
result.put(Constants.STATUS, Status.SUCCESS);
return result;
}
}

76
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/EnvironmentWorkerGroupRelationServiceImpl.java

@ -0,0 +1,76 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.api.service.impl;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.service.EnvironmentWorkerGroupRelationService;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.dao.entity.EnvironmentWorkerGroupRelation;
import org.apache.dolphinscheduler.dao.mapper.EnvironmentWorkerGroupRelationMapper;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
/**
* task definition service impl
*/
@Service
public class EnvironmentWorkerGroupRelationServiceImpl extends BaseServiceImpl implements
EnvironmentWorkerGroupRelationService {
private static final Logger logger = LoggerFactory.getLogger(EnvironmentWorkerGroupRelationServiceImpl.class);
@Autowired
private EnvironmentWorkerGroupRelationMapper environmentWorkerGroupRelationMapper;
/**
* query environment worker group relation
*
* @param environmentCode environment code
*/
@Override
public Map<String, Object> queryEnvironmentWorkerGroupRelation(Long environmentCode) {
Map<String, Object> result = new HashMap<>();
List<EnvironmentWorkerGroupRelation> relations = environmentWorkerGroupRelationMapper.queryByEnvironmentCode(environmentCode);
result.put(Constants.DATA_LIST, relations);
putMsg(result, Status.SUCCESS);
return result;
}
/**
* query all environment worker group relation
*
* @return all relation list
*/
@Override
public Map<String, Object> queryAllEnvironmentWorkerGroupRelationList() {
Map<String, Object> result = new HashMap<>();
List<EnvironmentWorkerGroupRelation> relations = environmentWorkerGroupRelationMapper.selectList(null);
result.put(Constants.DATA_LIST,relations);
putMsg(result,Status.SUCCESS);
return result;
}
}

21
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ExecutorServiceImpl.java

@ -53,6 +53,8 @@ import org.apache.dolphinscheduler.dao.entity.User;
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.remote.command.StateEventChangeCommand;
import org.apache.dolphinscheduler.remote.processor.StateEventCallbackService;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.service.quartz.cron.CronUtils;
@ -98,6 +100,9 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
@Autowired
private ProcessService processService;
@Autowired
StateEventCallbackService stateEventCallbackService;
/**
* execute process instance
*
@ -113,6 +118,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
* @param warningGroupId notify group id
* @param processInstancePriority process instance priority
* @param workerGroup worker group name
* @param environmentCode environment code
* @param runMode run mode
* @param timeout timeout
* @param startParams the global param values which pass to new process instance
@ -125,7 +131,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
FailureStrategy failureStrategy, String startNodeList,
TaskDependType taskDependType, WarningType warningType, int warningGroupId,
RunMode runMode,
Priority processInstancePriority, String workerGroup, Integer timeout,
Priority processInstancePriority, String workerGroup, Long environmentCode, Integer timeout,
Map<String, String> startParams, Integer expectedParallelismNumber) {
Map<String, Object> result = new HashMap<>();
// timeout is invalid
@ -163,7 +169,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
*/
int create = this.createCommand(commandType, processDefinitionId,
taskDependType, failureStrategy, startNodeList, cronTime, warningType, loginUser.getId(),
warningGroupId, runMode, processInstancePriority, workerGroup, startParams, expectedParallelismNumber);
warningGroupId, runMode, processInstancePriority, workerGroup, environmentCode, startParams, expectedParallelismNumber);
if (create > 0) {
processDefinition.setWarningGroupId(warningGroupId);
@ -383,6 +389,13 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
// determine whether the process is normal
if (update > 0) {
String host = processInstance.getHost();
String address = host.split(":")[0];
int port = Integer.parseInt(host.split(":")[1]);
StateEventChangeCommand stateEventChangeCommand = new StateEventChangeCommand(
processInstance.getId(), 0, processInstance.getState(), processInstance.getId(), 0
);
stateEventCallbackService.sendResult(address, port, stateEventChangeCommand.convert2Command());
putMsg(result, Status.SUCCESS);
} else {
putMsg(result, Status.EXECUTE_PROCESS_INSTANCE_ERROR);
@ -483,13 +496,14 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
* @param runMode runMode
* @param processInstancePriority processInstancePriority
* @param workerGroup workerGroup
* @param environmentCode environmentCode
* @return command id
*/
private int createCommand(CommandType commandType, int processDefineId,
TaskDependType nodeDep, FailureStrategy failureStrategy,
String startNodeList, String schedule, WarningType warningType,
int executorId, int warningGroupId,
RunMode runMode, Priority processInstancePriority, String workerGroup,
RunMode runMode, Priority processInstancePriority, String workerGroup, Long environmentCode,
Map<String, String> startParams, Integer expectedParallelismNumber) {
/**
@ -525,6 +539,7 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
command.setWarningGroupId(warningGroupId);
command.setProcessInstancePriority(processInstancePriority);
command.setWorkerGroup(workerGroup);
command.setEnvironmentCode(environmentCode);
Date start = null;
Date end = null;

5
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProcessInstanceServiceImpl.java

@ -592,7 +592,12 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
return result;
}
try {
processService.removeTaskLogFile(processInstanceId);
} catch (Exception e) {
logger.error("remove task log failed", e);
}
// delete database cascade
int delete = processService.deleteWorkProcessInstanceById(processInstanceId);

10
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/SchedulerServiceImpl.java

@ -106,6 +106,7 @@ public class SchedulerServiceImpl extends BaseServiceImpl implements SchedulerSe
* @param failureStrategy failure strategy
* @param processInstancePriority process instance priority
* @param workerGroup worker group
* @param environmentCode environment code
* @return create result code
*/
@Override
@ -117,7 +118,8 @@ public class SchedulerServiceImpl extends BaseServiceImpl implements SchedulerSe
int warningGroupId,
FailureStrategy failureStrategy,
Priority processInstancePriority,
String workerGroup) {
String workerGroup,
Long environmentCode) {
Map<String, Object> result = new HashMap<>();
@ -169,6 +171,7 @@ public class SchedulerServiceImpl extends BaseServiceImpl implements SchedulerSe
scheduleObj.setReleaseState(ReleaseState.OFFLINE);
scheduleObj.setProcessInstancePriority(processInstancePriority);
scheduleObj.setWorkerGroup(workerGroup);
scheduleObj.setEnvironmentCode(environmentCode);
scheduleMapper.insert(scheduleObj);
/**
@ -196,6 +199,7 @@ public class SchedulerServiceImpl extends BaseServiceImpl implements SchedulerSe
* @param warningGroupId warning group id
* @param failureStrategy failure strategy
* @param workerGroup worker group
* @param environmentCode environment code
* @param processInstancePriority process instance priority
* @param scheduleStatus schedule status
* @return update result code
@ -211,7 +215,8 @@ public class SchedulerServiceImpl extends BaseServiceImpl implements SchedulerSe
FailureStrategy failureStrategy,
ReleaseState scheduleStatus,
Priority processInstancePriority,
String workerGroup) {
String workerGroup,
Long environmentCode) {
Map<String, Object> result = new HashMap<>();
Project project = projectMapper.queryByName(projectName);
@ -277,6 +282,7 @@ public class SchedulerServiceImpl extends BaseServiceImpl implements SchedulerSe
schedule.setReleaseState(scheduleStatus);
}
schedule.setWorkerGroup(workerGroup);
schedule.setEnvironmentCode(environmentCode);
schedule.setUpdateTime(now);
schedule.setProcessInstancePriority(processInstancePriority);
scheduleMapper.updateById(schedule);

34
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/CheckUtils.java

@ -14,7 +14,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.api.utils;
import org.apache.dolphinscheduler.api.enums.Status;
@ -31,6 +30,7 @@ import java.util.HashMap;
import java.util.Map;
import java.util.regex.Pattern;
/**
* check utils
*/
@ -53,8 +53,7 @@ public class CheckUtils {
/**
* check email
*
* @param email
* email
* @param email email
* @return true if email regex valid, otherwise return false
*/
public static boolean checkEmail(String email) {
@ -68,8 +67,7 @@ public class CheckUtils {
/**
* check project description
*
* @param desc
* desc
* @param desc desc
* @return true if description regex valid, otherwise return false
*/
public static Map<String, Object> checkDesc(String desc) {
@ -87,8 +85,7 @@ public class CheckUtils {
/**
* check extra info
*
* @param otherParams
* other parames
* @param otherParams other parames
* @return true if other parameters are valid, otherwise return false
*/
public static boolean checkOtherParams(String otherParams) {
@ -98,8 +95,7 @@ public class CheckUtils {
/**
* check password
*
* @param password
* password
* @param password password
* @return true if password regex valid, otherwise return false
*/
public static boolean checkPassword(String password) {
@ -109,8 +105,7 @@ public class CheckUtils {
/**
* check phone phone can be empty.
*
* @param phone
* phone
* @param phone phone
* @return true if phone regex valid, otherwise return false
*/
public static boolean checkPhone(String phone) {
@ -120,8 +115,7 @@ public class CheckUtils {
/**
* check task node parameter
*
* @param taskNode
* TaskNode
* @param taskNode TaskNode
* @return true if task node parameters are valid, otherwise return false
*/
public static boolean checkTaskNodeParameters(TaskNode taskNode) {
@ -132,6 +126,8 @@ public class CheckUtils {
}
if (TaskType.DEPENDENT.getDesc().equalsIgnoreCase(taskType)) {
abstractParameters = TaskParametersUtils.getParameters(taskType.toUpperCase(), taskNode.getDependence());
} else if (TaskType.SWITCH.getDesc().equalsIgnoreCase(taskType)) {
abstractParameters = TaskParametersUtils.getParameters(taskType.toUpperCase(), taskNode.getSwitchResult());
} else {
abstractParameters = TaskParametersUtils.getParameters(taskType.toUpperCase(), taskNode.getParams());
}
@ -146,14 +142,10 @@ public class CheckUtils {
/**
* check params
*
* @param userName
* user name
* @param password
* password
* @param email
* email
* @param phone
* phone
* @param userName user name
* @param password password
* @param email email
* @param phone phone
* @return true if user parameters are valid, other return false
*/
public static boolean checkUserParams(String userName, String password, String email, String phone) {

208
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/EnvironmentControllerTest.java

@ -0,0 +1,208 @@
/*
* 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.controller;
import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get;
import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post;
import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content;
import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.utils.Result;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.Preconditions;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.http.MediaType;
import org.springframework.test.web.servlet.MvcResult;
import org.springframework.util.LinkedMultiValueMap;
import org.springframework.util.MultiValueMap;
import com.fasterxml.jackson.core.type.TypeReference;
/**
* environment controller test
*/
public class EnvironmentControllerTest extends AbstractControllerTest {
private static Logger logger = LoggerFactory.getLogger(EnvironmentControllerTest.class);
private String environmentCode;
public static final String environmentName = "Env1";
public static final String config = "this is config content";
public static final String desc = "this is environment description";
@Before
public void before() throws Exception {
testCreateEnvironment();
}
@After
public void after() throws Exception {
testDeleteEnvironment();
}
public void testCreateEnvironment() throws Exception {
MultiValueMap<String, String> paramsMap = new LinkedMultiValueMap<>();
paramsMap.add("name",environmentName);
paramsMap.add("config",config);
paramsMap.add("description",desc);
MvcResult mvcResult = mockMvc.perform(post("/environment/create")
.header(SESSION_ID, sessionId)
.params(paramsMap))
.andExpect(status().isCreated())
.andExpect(content().contentType(MediaType.APPLICATION_JSON_UTF8))
.andReturn();
Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), new TypeReference<Result<String>>() {});
logger.info(result.toString());
Assert.assertTrue(result != null && result.isSuccess());
Assert.assertNotNull(result.getData());
logger.info("create environment return result:{}", mvcResult.getResponse().getContentAsString());
environmentCode = (String)result.getData();
}
@Test
public void testUpdateEnvironment() throws Exception {
MultiValueMap<String, String> paramsMap = new LinkedMultiValueMap<>();
paramsMap.add("code", environmentCode);
paramsMap.add("name","environment_test_update");
paramsMap.add("config","this is config content");
paramsMap.add("desc","the test environment update");
MvcResult mvcResult = mockMvc.perform(post("/environment/update")
.header(SESSION_ID, sessionId)
.params(paramsMap))
.andExpect(status().isOk())
.andExpect(content().contentType(MediaType.APPLICATION_JSON_UTF8))
.andReturn();
Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class);
logger.info(result.toString());
Assert.assertTrue(result != null && result.isSuccess());
logger.info("update environment return result:{}", mvcResult.getResponse().getContentAsString());
}
@Test
public void testQueryEnvironmentByCode() throws Exception {
MultiValueMap<String, String> paramsMap = new LinkedMultiValueMap<>();
paramsMap.add("environmentCode", environmentCode);
MvcResult mvcResult = mockMvc.perform(get("/environment/query-by-code")
.header(SESSION_ID, sessionId)
.params(paramsMap))
.andExpect(status().isOk())
.andExpect(content().contentType(MediaType.APPLICATION_JSON_UTF8))
.andReturn();
Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class);
logger.info(result.toString());
Assert.assertTrue(result != null && result.isSuccess());
logger.info(mvcResult.getResponse().getContentAsString());
logger.info("query environment by id :{}, return result:{}", environmentCode, mvcResult.getResponse().getContentAsString());
}
@Test
public void testQueryEnvironmentListPaging() throws Exception {
MultiValueMap<String, String> paramsMap = new LinkedMultiValueMap<>();
paramsMap.add("searchVal","test");
paramsMap.add("pageSize","2");
paramsMap.add("pageNo","2");
MvcResult mvcResult = mockMvc.perform(get("/environment/list-paging")
.header(SESSION_ID, sessionId)
.params(paramsMap))
.andExpect(status().isOk())
.andExpect(content().contentType(MediaType.APPLICATION_JSON_UTF8))
.andReturn();
Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class);
logger.info(result.toString());
Assert.assertTrue(result != null && result.isSuccess());
logger.info("query list-paging environment return result:{}", mvcResult.getResponse().getContentAsString());
}
@Test
public void testQueryAllEnvironmentList() throws Exception {
MultiValueMap<String, String> paramsMap = new LinkedMultiValueMap<>();
MvcResult mvcResult = mockMvc.perform(get("/environment/query-environment-list")
.header(SESSION_ID, sessionId)
.params(paramsMap))
.andExpect(status().isOk())
.andExpect(content().contentType(MediaType.APPLICATION_JSON_UTF8))
.andReturn();
Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class);
logger.info(result.toString());
Assert.assertTrue(result != null && result.isSuccess());
logger.info("query all environment return result:{}", mvcResult.getResponse().getContentAsString());
}
@Test
public void testVerifyEnvironment() throws Exception {
MultiValueMap<String, String> paramsMap = new LinkedMultiValueMap<>();
paramsMap.add("environmentName",environmentName);
MvcResult mvcResult = mockMvc.perform(post("/environment/verify-environment")
.header(SESSION_ID, sessionId)
.params(paramsMap))
.andExpect(status().isOk())
.andExpect(content().contentType(MediaType.APPLICATION_JSON_UTF8))
.andReturn();
Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class);
logger.info(result.toString());
Assert.assertTrue(result.isStatus(Status.ENVIRONMENT_NAME_EXISTS));
logger.info("verify environment return result:{}", mvcResult.getResponse().getContentAsString());
}
private void testDeleteEnvironment() throws Exception {
Preconditions.checkNotNull(environmentCode);
MultiValueMap<String, String> paramsMap = new LinkedMultiValueMap<>();
paramsMap.add("environmentCode", environmentCode);
MvcResult mvcResult = mockMvc.perform(post("/environment/delete")
.header(SESSION_ID, sessionId)
.params(paramsMap))
.andExpect(status().isOk())
.andExpect(content().contentType(MediaType.APPLICATION_JSON_UTF8))
.andReturn();
Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class);
logger.info(result.toString());
Assert.assertTrue(result != null && result.isSuccess());
logger.info("delete environment return result:{}", mvcResult.getResponse().getContentAsString());
}
}

310
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/EnvironmentServiceTest.java

@ -0,0 +1,310 @@
/*
* 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.service;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.service.impl.EnvironmentServiceImpl;
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.UserType;
import org.apache.dolphinscheduler.common.utils.CollectionUtils;
import org.apache.dolphinscheduler.dao.entity.Environment;
import org.apache.dolphinscheduler.dao.entity.EnvironmentWorkerGroupRelation;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.EnvironmentMapper;
import org.apache.dolphinscheduler.dao.mapper.EnvironmentWorkerGroupRelationMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.assertj.core.util.Lists;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.InjectMocks;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.baomidou.mybatisplus.core.conditions.Wrapper;
import com.baomidou.mybatisplus.core.conditions.query.LambdaQueryWrapper;
import com.baomidou.mybatisplus.core.metadata.IPage;
import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
/**
* environment service test
*/
@RunWith(MockitoJUnitRunner.class)
public class EnvironmentServiceTest {
public static final Logger logger = LoggerFactory.getLogger(EnvironmentServiceTest.class);
@InjectMocks
private EnvironmentServiceImpl environmentService;
@Mock
private EnvironmentMapper environmentMapper;
@Mock
private EnvironmentWorkerGroupRelationMapper relationMapper;
@Mock
private TaskDefinitionMapper taskDefinitionMapper;
public static final String testUserName = "environmentServerTest";
public static final String environmentName = "Env1";
public static final String workerGroups = "[\"default\"]";
@Before
public void setUp(){
}
@After
public void after(){
}
@Test
public void testCreateEnvironment() {
User loginUser = getGeneralUser();
Map<String, Object> result = environmentService.createEnvironment(loginUser,environmentName,getConfig(),getDesc(),workerGroups);
logger.info(result.toString());
Assert.assertEquals(Status.USER_NO_OPERATION_PERM, result.get(Constants.STATUS));
loginUser = getAdminUser();
result = environmentService.createEnvironment(loginUser,environmentName,"",getDesc(),workerGroups);
logger.info(result.toString());
Assert.assertEquals(Status.ENVIRONMENT_CONFIG_IS_NULL, result.get(Constants.STATUS));
result = environmentService.createEnvironment(loginUser,"",getConfig(),getDesc(),workerGroups);
logger.info(result.toString());
Assert.assertEquals(Status.ENVIRONMENT_NAME_IS_NULL, result.get(Constants.STATUS));
result = environmentService.createEnvironment(loginUser,environmentName,getConfig(),getDesc(),"test");
logger.info(result.toString());
Assert.assertEquals(Status.ENVIRONMENT_WORKER_GROUPS_IS_INVALID, result.get(Constants.STATUS));
Mockito.when(environmentMapper.queryByEnvironmentName(environmentName)).thenReturn(getEnvironment());
result = environmentService.createEnvironment(loginUser,environmentName,getConfig(),getDesc(),workerGroups);
logger.info(result.toString());
Assert.assertEquals(Status.ENVIRONMENT_NAME_EXISTS, result.get(Constants.STATUS));
Mockito.when(environmentMapper.insert(Mockito.any(Environment.class))).thenReturn(1);
Mockito.when(relationMapper.insert(Mockito.any(EnvironmentWorkerGroupRelation.class))).thenReturn(1);
result = environmentService.createEnvironment(loginUser,"testName","test","test",workerGroups);
logger.info(result.toString());
Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
}
@Test
public void testCheckParams() {
Map<String, Object> result = environmentService.checkParams(environmentName,getConfig(),"test");
Assert.assertEquals(Status.ENVIRONMENT_WORKER_GROUPS_IS_INVALID, result.get(Constants.STATUS));
}
@Test
public void testUpdateEnvironmentByCode() {
User loginUser = getGeneralUser();
Map<String, Object> result = environmentService.updateEnvironmentByCode(loginUser,1L,environmentName,getConfig(),getDesc(),workerGroups);
logger.info(result.toString());
Assert.assertEquals(Status.USER_NO_OPERATION_PERM, result.get(Constants.STATUS));
loginUser = getAdminUser();
result = environmentService.updateEnvironmentByCode(loginUser,1L,environmentName,"",getDesc(),workerGroups);
logger.info(result.toString());
Assert.assertEquals(Status.ENVIRONMENT_CONFIG_IS_NULL, result.get(Constants.STATUS));
result = environmentService.updateEnvironmentByCode(loginUser,1L,"",getConfig(),getDesc(),workerGroups);
logger.info(result.toString());
Assert.assertEquals(Status.ENVIRONMENT_NAME_IS_NULL, result.get(Constants.STATUS));
result = environmentService.updateEnvironmentByCode(loginUser,1L,environmentName,getConfig(),getDesc(),"test");
logger.info(result.toString());
Assert.assertEquals(Status.ENVIRONMENT_WORKER_GROUPS_IS_INVALID, result.get(Constants.STATUS));
Mockito.when(environmentMapper.queryByEnvironmentName(environmentName)).thenReturn(getEnvironment());
result = environmentService.updateEnvironmentByCode(loginUser,2L,environmentName,getConfig(),getDesc(),workerGroups);
logger.info(result.toString());
Assert.assertEquals(Status.ENVIRONMENT_NAME_EXISTS, result.get(Constants.STATUS));
Mockito.when(environmentMapper.update(Mockito.any(Environment.class),Mockito.any(Wrapper.class))).thenReturn(1);
result = environmentService.updateEnvironmentByCode(loginUser,1L,"testName","test","test",workerGroups);
logger.info(result.toString());
Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
}
@Test
public void testQueryAllEnvironmentList() {
Mockito.when(environmentMapper.queryAllEnvironmentList()).thenReturn(Lists.newArrayList(getEnvironment()));
Map<String, Object> result = environmentService.queryAllEnvironmentList();
logger.info(result.toString());
Assert.assertEquals(Status.SUCCESS,result.get(Constants.STATUS));
List<Environment> list = (List<Environment>)(result.get(Constants.DATA_LIST));
Assert.assertEquals(1,list.size());
}
@Test
public void testQueryEnvironmentListPaging() {
IPage<Environment> page = new Page<>(1, 10);
page.setRecords(getList());
page.setTotal(1L);
Mockito.when(environmentMapper.queryEnvironmentListPaging(Mockito.any(Page.class), Mockito.eq(environmentName))).thenReturn(page);
Result result = environmentService.queryEnvironmentListPaging(1, 10, environmentName);
logger.info(result.toString());
PageInfo<Environment> pageInfo = (PageInfo<Environment>) result.getData();
Assert.assertTrue(CollectionUtils.isNotEmpty(pageInfo.getTotalList()));
}
@Test
public void testQueryEnvironmentByName() {
Mockito.when(environmentMapper.queryByEnvironmentName(environmentName)).thenReturn(null);
Map<String, Object> result = environmentService.queryEnvironmentByName(environmentName);
logger.info(result.toString());
Assert.assertEquals(Status.QUERY_ENVIRONMENT_BY_NAME_ERROR,result.get(Constants.STATUS));
Mockito.when(environmentMapper.queryByEnvironmentName(environmentName)).thenReturn(getEnvironment());
result = environmentService.queryEnvironmentByName(environmentName);
logger.info(result.toString());
Assert.assertEquals(Status.SUCCESS,result.get(Constants.STATUS));
}
@Test
public void testQueryEnvironmentByCode() {
Mockito.when(environmentMapper.queryByEnvironmentCode(1L)).thenReturn(null);
Map<String, Object> result = environmentService.queryEnvironmentByCode(1L);
logger.info(result.toString());
Assert.assertEquals(Status.QUERY_ENVIRONMENT_BY_CODE_ERROR,result.get(Constants.STATUS));
Mockito.when(environmentMapper.queryByEnvironmentCode(1L)).thenReturn(getEnvironment());
result = environmentService.queryEnvironmentByCode(1L);
logger.info(result.toString());
Assert.assertEquals(Status.SUCCESS,result.get(Constants.STATUS));
}
@Test
public void testDeleteEnvironmentByCode() {
User loginUser = getGeneralUser();
Map<String, Object> result = environmentService.deleteEnvironmentByCode(loginUser,1L);
logger.info(result.toString());
Assert.assertEquals(Status.USER_NO_OPERATION_PERM, result.get(Constants.STATUS));
loginUser = getAdminUser();
Mockito.when(taskDefinitionMapper.selectCount(Mockito.any(LambdaQueryWrapper.class))).thenReturn(1);
result = environmentService.deleteEnvironmentByCode(loginUser,1L);
logger.info(result.toString());
Assert.assertEquals(Status.DELETE_ENVIRONMENT_RELATED_TASK_EXISTS, result.get(Constants.STATUS));
Mockito.when(taskDefinitionMapper.selectCount(Mockito.any(LambdaQueryWrapper.class))).thenReturn(0);
Mockito.when(environmentMapper.deleteByCode(1L)).thenReturn(1);
result = environmentService.deleteEnvironmentByCode(loginUser,1L);
logger.info(result.toString());
Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
}
@Test
public void testVerifyEnvironment() {
Map<String, Object> result = environmentService.verifyEnvironment("");
logger.info(result.toString());
Assert.assertEquals(Status.ENVIRONMENT_NAME_IS_NULL, result.get(Constants.STATUS));
Mockito.when(environmentMapper.queryByEnvironmentName(environmentName)).thenReturn(getEnvironment());
result = environmentService.verifyEnvironment(environmentName);
logger.info(result.toString());
Assert.assertEquals(Status.ENVIRONMENT_NAME_EXISTS, result.get(Constants.STATUS));
}
private Environment getEnvironment() {
Environment environment = new Environment();
environment.setId(1);
environment.setCode(1L);
environment.setName(environmentName);
environment.setConfig(getConfig());
environment.setDescription(getDesc());
environment.setOperator(1);
return environment;
}
/**
* create an environment description
*/
private String getDesc() {
return "create an environment to test ";
}
/**
* create an environment config
*/
private String getConfig() {
return "export HADOOP_HOME=/opt/hadoop-2.6.5\n"
+ "export HADOOP_CONF_DIR=/etc/hadoop/conf\n"
+ "export SPARK_HOME1=/opt/soft/spark1\n"
+ "export SPARK_HOME2=/opt/soft/spark2\n"
+ "export PYTHON_HOME=/opt/soft/python\n"
+ "export JAVA_HOME=/opt/java/jdk1.8.0_181-amd64\n"
+ "export HIVE_HOME=/opt/soft/hive\n"
+ "export FLINK_HOME=/opt/soft/flink\n"
+ "export DATAX_HOME=/opt/soft/datax\n"
+ "export YARN_CONF_DIR=\"/etc/hadoop/conf\"\n"
+ "\n"
+ "export PATH=$HADOOP_HOME/bin:$SPARK_HOME1/bin:$SPARK_HOME2/bin:$PYTHON_HOME/bin:$JAVA_HOME/bin:$HIVE_HOME/bin:$FLINK_HOME/bin:$DATAX_HOME/bin:$PATH\n"
+ "\n"
+ "export HADOOP_CLASSPATH=`hadoop classpath`\n"
+ "\n"
+ "#echo \"HADOOP_CLASSPATH=\"$HADOOP_CLASSPATH";
}
/**
* create general user
*/
private User getGeneralUser() {
User loginUser = new User();
loginUser.setUserType(UserType.GENERAL_USER);
loginUser.setUserName(testUserName);
loginUser.setId(1);
return loginUser;
}
/**
* create admin user
*/
private User getAdminUser() {
User loginUser = new User();
loginUser.setUserType(UserType.ADMIN_USER);
loginUser.setUserName(testUserName);
loginUser.setId(1);
return loginUser;
}
private List<Environment> getList() {
List<Environment> list = new ArrayList<>();
list.add(getEnvironment());
return list;
}
}

69
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/EnvironmentWorkerGroupRelationServiceTest.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.api.service;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.service.impl.EnvironmentWorkerGroupRelationServiceImpl;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.dao.entity.EnvironmentWorkerGroupRelation;
import org.apache.dolphinscheduler.dao.mapper.EnvironmentWorkerGroupRelationMapper;
import java.util.Map;
import org.assertj.core.util.Lists;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.InjectMocks;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* environment service test
*/
@RunWith(MockitoJUnitRunner.class)
public class EnvironmentWorkerGroupRelationServiceTest {
public static final Logger logger = LoggerFactory.getLogger(EnvironmentWorkerGroupRelationServiceTest.class);
@InjectMocks
private EnvironmentWorkerGroupRelationServiceImpl relationService;
@Mock
private EnvironmentWorkerGroupRelationMapper relationMapper;
@Test
public void testQueryEnvironmentWorkerGroupRelation() {
Mockito.when(relationMapper.queryByEnvironmentCode(1L)).thenReturn(Lists.newArrayList(new EnvironmentWorkerGroupRelation()));
Map<String, Object> result = relationService.queryEnvironmentWorkerGroupRelation(1L);
logger.info(result.toString());
Assert.assertEquals(Status.SUCCESS,result.get(Constants.STATUS));
}
@Test
public void testQueryAllEnvironmentWorkerGroupRelationList() {
Mockito.when(relationMapper.selectList(Mockito.any())).thenReturn(Lists.newArrayList(new EnvironmentWorkerGroupRelation()));
Map<String, Object> result = relationService.queryAllEnvironmentWorkerGroupRelationList();
logger.info(result.toString());
Assert.assertEquals(Status.SUCCESS,result.get(Constants.STATUS));
}
}

15
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java

@ -153,7 +153,7 @@ public class ExecutorService2Test {
null, null,
null, null, 0,
RunMode.RUN_MODE_SERIAL,
Priority.LOW, Constants.DEFAULT_WORKER_GROUP, 110, null, 4);
Priority.LOW, Constants.DEFAULT_WORKER_GROUP,-1L, 110, null, 4);
Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
verify(processService, times(1)).createCommand(any(Command.class));
@ -171,13 +171,12 @@ public class ExecutorService2Test {
null, "n1,n2",
null, null, 0,
RunMode.RUN_MODE_SERIAL,
Priority.LOW, Constants.DEFAULT_WORKER_GROUP, 110, null, null);
Priority.LOW, Constants.DEFAULT_WORKER_GROUP,-1L, 110, null, null);
Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
verify(processService, times(1)).createCommand(any(Command.class));
}
/**
* date error
*/
@ -190,7 +189,7 @@ public class ExecutorService2Test {
null, null,
null, null, 0,
RunMode.RUN_MODE_SERIAL,
Priority.LOW, Constants.DEFAULT_WORKER_GROUP, 110, null, null);
Priority.LOW, Constants.DEFAULT_WORKER_GROUP,-1L, 110, null, null);
Assert.assertEquals(Status.START_PROCESS_INSTANCE_ERROR, result.get(Constants.STATUS));
verify(processService, times(0)).createCommand(any(Command.class));
}
@ -207,7 +206,7 @@ public class ExecutorService2Test {
null, null,
null, null, 0,
RunMode.RUN_MODE_SERIAL,
Priority.LOW, Constants.DEFAULT_WORKER_GROUP, 110, null, null);
Priority.LOW, Constants.DEFAULT_WORKER_GROUP,-1L, 110, null, null);
Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
verify(processService, times(1)).createCommand(any(Command.class));
@ -225,7 +224,7 @@ public class ExecutorService2Test {
null, null,
null, null, 0,
RunMode.RUN_MODE_PARALLEL,
Priority.LOW, Constants.DEFAULT_WORKER_GROUP, 110, null, null);
Priority.LOW, Constants.DEFAULT_WORKER_GROUP,-1L, 110, null, null);
Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
verify(processService, times(31)).createCommand(any(Command.class));
@ -243,7 +242,7 @@ public class ExecutorService2Test {
null, null,
null, null, 0,
RunMode.RUN_MODE_PARALLEL,
Priority.LOW, Constants.DEFAULT_WORKER_GROUP, 110, null, 4);
Priority.LOW, Constants.DEFAULT_WORKER_GROUP,-1L, 110, null, 4);
Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
verify(processService, times(4)).createCommand(any(Command.class));
@ -258,7 +257,7 @@ public class ExecutorService2Test {
null, null,
null, null, 0,
RunMode.RUN_MODE_PARALLEL,
Priority.LOW, Constants.DEFAULT_WORKER_GROUP, 110, null, 4);
Priority.LOW, Constants.DEFAULT_WORKER_GROUP,-1L, 110, null, 4);
Assert.assertEquals(result.get(Constants.STATUS), Status.MASTER_NOT_EXISTS);
}

12
dolphinscheduler-common/pom.xml

@ -58,6 +58,13 @@
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
<scope>provided</scope>
<exclusions>
<exclusion>
<artifactId>jsr305</artifactId>
<groupId>com.google.code.findbugs</groupId>
</exclusion>
</exclusions>
</dependency>
<dependency>
@ -636,5 +643,10 @@
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
<scope>compile</scope>
</dependency>
</dependencies>
</project>

2
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java

@ -435,6 +435,8 @@ public final class Constants {
*/
public static final String DATASOURCE_PROPERTIES = "/datasource.properties";
public static final String COMMON_TASK_TYPE = "common";
public static final String DEFAULT = "Default";
public static final String USER = "user";
public static final String PASSWORD = "password";

111
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/StateEvent.java

@ -0,0 +1,111 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.common.enums;
import io.netty.channel.Channel;
/**
* state event
*/
public class StateEvent {
/**
* origin_pid-origin_task_id-process_instance_id-task_instance_id
*/
private String key;
private StateEventType type;
private ExecutionStatus executionStatus;
private int taskInstanceId;
private int processInstanceId;
private String context;
private Channel channel;
public ExecutionStatus getExecutionStatus() {
return executionStatus;
}
public void setExecutionStatus(ExecutionStatus executionStatus) {
this.executionStatus = executionStatus;
}
public int getTaskInstanceId() {
return taskInstanceId;
}
public int getProcessInstanceId() {
return processInstanceId;
}
public void setProcessInstanceId(int processInstanceId) {
this.processInstanceId = processInstanceId;
}
public String getContext() {
return context;
}
public void setContext(String context) {
this.context = context;
}
public void setTaskInstanceId(int taskInstanceId) {
this.taskInstanceId = taskInstanceId;
}
public Channel getChannel() {
return channel;
}
public void setChannel(Channel channel) {
this.channel = channel;
}
@Override
public String toString() {
return "State Event :"
+ "key: " + key
+ " type: " + type.toString()
+ " executeStatus: " + executionStatus
+ " task instance id: " + taskInstanceId
+ " process instance id: " + processInstanceId
+ " context: " + context
;
}
public String getKey() {
return key;
}
public void setKey(String key) {
this.key = key;
}
public void setType(StateEventType type) {
this.type = type;
}
public StateEventType getType() {
return this.type;
}
}

38
dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/FuncUtils.java → dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/StateEventType.java

@ -15,33 +15,31 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.alert.utils;
package org.apache.dolphinscheduler.common.enums;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import com.baomidou.mybatisplus.annotation.EnumValue;
public class FuncUtils {
public enum StateEventType {
private FuncUtils() {
throw new IllegalStateException(FuncUtils.class.getName());
PROCESS_STATE_CHANGE(0, "process statechange"),
TASK_STATE_CHANGE(1, "task state change"),
PROCESS_TIMEOUT(2, "process timeout"),
TASK_TIMEOUT(3, "task timeout");
StateEventType(int code, String descp) {
this.code = code;
this.descp = descp;
}
public static String mkString(Iterable<String> list, String split) {
@EnumValue
private final int code;
private final String descp;
if (null == list || StringUtils.isEmpty(split)) {
return null;
public int getCode() {
return code;
}
StringBuilder sb = new StringBuilder();
boolean first = true;
for (String item : list) {
if (first) {
first = false;
} else {
sb.append(split);
}
sb.append(item);
public String getDescp() {
return descp;
}
return sb.toString();
}
}

15
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java

@ -143,6 +143,11 @@ public class TaskNode {
*/
private String workerGroup;
/**
* environment code
*/
private Long environmentCode;
/**
* task time out
*/
@ -262,6 +267,7 @@ public class TaskNode {
&& Objects.equals(runFlag, taskNode.runFlag)
&& Objects.equals(dependence, taskNode.dependence)
&& Objects.equals(workerGroup, taskNode.workerGroup)
&& Objects.equals(environmentCode, taskNode.environmentCode)
&& Objects.equals(conditionResult, taskNode.conditionResult)
&& CollectionUtils.equalLists(depList, taskNode.depList);
}
@ -422,11 +428,20 @@ public class TaskNode {
+ ", conditionResult='" + conditionResult + '\''
+ ", taskInstancePriority=" + taskInstancePriority
+ ", workerGroup='" + workerGroup + '\''
+ ", environmentCode=" + environmentCode
+ ", timeout='" + timeout + '\''
+ ", delayTime=" + delayTime
+ '}';
}
public void setEnvironmentCode(Long environmentCode) {
this.environmentCode = environmentCode;
}
public Long getEnvironmentCode() {
return this.environmentCode;
}
public String getSwitchResult() {
return switchResult;
}

3
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/AbstractParameters.java

@ -30,6 +30,7 @@ import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
@ -152,7 +153,7 @@ public abstract class AbstractParameters implements IParameters {
ArrayNode paramsByJson = JSONUtils.parseArray(json);
Iterator<JsonNode> listIterator = paramsByJson.iterator();
while (listIterator.hasNext()) {
Map<String, String> param = JSONUtils.toMap(listIterator.next().toString(), String.class, String.class);
Map<String, String> param = JSONUtils.parseObject(listIterator.next().toString(), new TypeReference<Map<String, String>>() {});
allParams.add(param);
}
return allParams;

6
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sql/SqlParameters.java

@ -251,9 +251,9 @@ public class SqlParameters extends AbstractParameters {
sqlResultFormat.put(key, new ArrayList<>());
}
for (Map<String, String> info : sqlResult) {
for (String key : info.keySet()) {
sqlResultFormat.get(key).add(String.valueOf(info.get(key)));
}
info.forEach((key, value) -> {
sqlResultFormat.get(key).add(value);
});
}
for (Property info : outProperty) {
if (info.getType() == DataType.LIST) {

70
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Command.java

@ -14,15 +14,21 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.entity;
import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.common.enums.FailureStrategy;
import org.apache.dolphinscheduler.common.enums.Priority;
import org.apache.dolphinscheduler.common.enums.TaskDependType;
import org.apache.dolphinscheduler.common.enums.WarningType;
import java.util.Date;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableField;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
import org.apache.dolphinscheduler.common.enums.*;
import java.util.Date;
/**
* command
@ -114,6 +120,12 @@ public class Command {
@TableField("worker_group")
private String workerGroup;
/**
* environment code
*/
@TableField("environment_code")
private Long environmentCode;
public Command() {
this.taskDependType = TaskDependType.TASK_POST;
this.failureStrategy = FailureStrategy.CONTINUE;
@ -132,6 +144,7 @@ public class Command {
int warningGroupId,
Date scheduleTime,
String workerGroup,
Long environmentCode,
Priority processInstancePriority) {
this.commandType = commandType;
this.executorId = executorId;
@ -145,10 +158,10 @@ public class Command {
this.startTime = new Date();
this.updateTime = new Date();
this.workerGroup = workerGroup;
this.environmentCode = environmentCode;
this.processInstancePriority = processInstancePriority;
}
public TaskDependType getTaskDependType() {
return taskDependType;
}
@ -181,7 +194,6 @@ public class Command {
this.processDefinitionId = processDefinitionId;
}
public FailureStrategy getFailureStrategy() {
return failureStrategy;
}
@ -262,6 +274,14 @@ public class Command {
this.workerGroup = workerGroup;
}
public Long getEnvironmentCode() {
return this.environmentCode;
}
public void setEnvironmentCode(Long environmentCode) {
this.environmentCode = environmentCode;
}
@Override
public boolean equals(Object o) {
if (this == o) {
@ -285,6 +305,11 @@ public class Command {
if (workerGroup != null ? workerGroup.equals(command.workerGroup) : command.workerGroup == null) {
return false;
}
if (environmentCode != null ? environmentCode.equals(command.environmentCode) : command.environmentCode == null) {
return false;
}
if (commandType != command.commandType) {
return false;
}
@ -332,26 +357,29 @@ public class Command {
result = 31 * result + (processInstancePriority != null ? processInstancePriority.hashCode() : 0);
result = 31 * result + (updateTime != null ? updateTime.hashCode() : 0);
result = 31 * result + (workerGroup != null ? workerGroup.hashCode() : 0);
result = 31 * result + (environmentCode != null ? environmentCode.hashCode() : 0);
return result;
}
@Override
public String toString() {
return "Command{" +
"id=" + id +
", commandType=" + commandType +
", processDefinitionId=" + processDefinitionId +
", executorId=" + executorId +
", commandParam='" + commandParam + '\'' +
", taskDependType=" + taskDependType +
", failureStrategy=" + failureStrategy +
", warningType=" + warningType +
", warningGroupId=" + warningGroupId +
", scheduleTime=" + scheduleTime +
", startTime=" + startTime +
", processInstancePriority=" + processInstancePriority +
", updateTime=" + updateTime +
", workerGroup='" + workerGroup + '\'' +
'}';
return "Command{"
+ "id=" + id
+ ", commandType=" + commandType
+ ", processDefinitionId=" + processDefinitionId
+ ", executorId=" + executorId
+ ", commandParam='" + commandParam + '\''
+ ", taskDependType=" + taskDependType
+ ", failureStrategy=" + failureStrategy
+ ", warningType=" + warningType
+ ", warningGroupId=" + warningGroupId
+ ", scheduleTime=" + scheduleTime
+ ", startTime=" + startTime
+ ", processInstancePriority=" + processInstancePriority
+ ", updateTime=" + updateTime
+ ", workerGroup='" + workerGroup + '\''
+ ", environmentCode='" + environmentCode + '\''
+ '}';
}
}

142
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Environment.java

@ -0,0 +1,142 @@
/*
* 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.dao.entity;
import java.util.Date;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
import com.fasterxml.jackson.annotation.JsonFormat;
/**
* Environment
*/
@TableName("t_ds_environment")
public class Environment {
@TableId(value = "id", type = IdType.AUTO)
private int id;
/**
* environment code
*/
private Long code;
/**
* environment name
*/
private String name;
/**
* config content
*/
private String config;
private String description;
/**
* operator user id
*/
private Integer operator;
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date createTime;
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date updateTime;
public int getId() {
return id;
}
public void setId(int id) {
this.id = id;
}
public String getName() {
return name;
}
public void setName(String name) {
this.name = name;
}
public Long getCode() {
return this.code;
}
public void setCode(Long code) {
this.code = code;
}
public String getConfig() {
return this.config;
}
public void setConfig(String config) {
this.config = config;
}
public String getDescription() {
return this.description;
}
public void setDescription(String description) {
this.description = description;
}
public Integer getOperator() {
return this.operator;
}
public void setOperator(Integer operator) {
this.operator = operator;
}
public Date getCreateTime() {
return createTime;
}
public void setCreateTime(Date createTime) {
this.createTime = createTime;
}
public Date getUpdateTime() {
return updateTime;
}
public void setUpdateTime(Date updateTime) {
this.updateTime = updateTime;
}
@Override
public String toString() {
return "Environment{"
+ "id= " + id
+ ", code= " + code
+ ", name= " + name
+ ", config= " + config
+ ", description= " + description
+ ", operator= " + operator
+ ", createTime= " + createTime
+ ", updateTime= " + updateTime
+ "}";
}
}

117
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/EnvironmentWorkerGroupRelation.java

@ -0,0 +1,117 @@
/*
* 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.dao.entity;
import java.util.Date;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
import com.fasterxml.jackson.annotation.JsonFormat;
/**
* EnvironmentWorkerGroupRelation
*/
@TableName("t_ds_environment_worker_group_relation")
public class EnvironmentWorkerGroupRelation {
@TableId(value = "id", type = IdType.AUTO)
private int id;
/**
* environment code
*/
private Long environmentCode;
/**
* worker group id
*/
private String workerGroup;
/**
* operator user id
*/
private Integer operator;
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date createTime;
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date updateTime;
public int getId() {
return id;
}
public void setId(int id) {
this.id = id;
}
public String getWorkerGroup() {
return workerGroup;
}
public void setWorkerGroup(String workerGroup) {
this.workerGroup = workerGroup;
}
public Long getEnvironmentCode() {
return this.environmentCode;
}
public void setEnvironmentCode(Long environmentCode) {
this.environmentCode = environmentCode;
}
public Integer getOperator() {
return this.operator;
}
public void setOperator(Integer operator) {
this.operator = operator;
}
public Date getCreateTime() {
return createTime;
}
public void setCreateTime(Date createTime) {
this.createTime = createTime;
}
public Date getUpdateTime() {
return updateTime;
}
public void setUpdateTime(Date updateTime) {
this.updateTime = updateTime;
}
@Override
public String toString() {
return "EnvironmentWorkerGroupRelation{"
+ "id= " + id
+ ", environmentCode= " + environmentCode
+ ", workerGroup= " + workerGroup
+ ", operator= " + operator
+ ", createTime= " + createTime
+ ", updateTime= " + updateTime
+ "}";
}
}

63
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ErrorCommand.java

@ -14,15 +14,21 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.entity;
import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.common.enums.FailureStrategy;
import org.apache.dolphinscheduler.common.enums.Priority;
import org.apache.dolphinscheduler.common.enums.TaskDependType;
import org.apache.dolphinscheduler.common.enums.WarningType;
import java.util.Date;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
import com.fasterxml.jackson.annotation.JsonFormat;
import org.apache.dolphinscheduler.common.enums.*;
import java.util.Date;
/**
* command
@ -109,6 +115,11 @@ public class ErrorCommand {
*/
private String workerGroup;
/**
* environment code
*/
private Long environmentCode;
public ErrorCommand(){}
public ErrorCommand(Command command, String message) {
@ -124,6 +135,7 @@ public class ErrorCommand {
this.failureStrategy = command.getFailureStrategy();
this.startTime = command.getStartTime();
this.updateTime = command.getUpdateTime();
this.environmentCode = command.getEnvironmentCode();
this.processInstancePriority = command.getProcessInstancePriority();
this.message = message;
}
@ -155,7 +167,6 @@ public class ErrorCommand {
this.message = message;
}
public TaskDependType getTaskDependType() {
return taskDependType;
}
@ -188,7 +199,6 @@ public class ErrorCommand {
this.processDefinitionId = processDefinitionId;
}
public FailureStrategy getFailureStrategy() {
return failureStrategy;
}
@ -277,24 +287,33 @@ public class ErrorCommand {
this.message = message;
}
public Long getEnvironmentCode() {
return this.environmentCode;
}
public void setEnvironmentCode(Long environmentCode) {
this.environmentCode = environmentCode;
}
@Override
public String toString() {
return "ErrorCommand{" +
"id=" + id +
", commandType=" + commandType +
", processDefinitionId=" + processDefinitionId +
", executorId=" + executorId +
", commandParam='" + commandParam + '\'' +
", taskDependType=" + taskDependType +
", failureStrategy=" + failureStrategy +
", warningType=" + warningType +
", warningGroupId=" + warningGroupId +
", scheduleTime=" + scheduleTime +
", startTime=" + startTime +
", processInstancePriority=" + processInstancePriority +
", updateTime=" + updateTime +
", message='" + message + '\'' +
", workerGroup='" + workerGroup + '\'' +
'}';
return "ErrorCommand{"
+ "id=" + id
+ ", commandType=" + commandType
+ ", processDefinitionId=" + processDefinitionId
+ ", executorId=" + executorId
+ ", commandParam='" + commandParam + '\''
+ ", taskDependType=" + taskDependType
+ ", failureStrategy=" + failureStrategy
+ ", warningType=" + warningType
+ ", warningGroupId=" + warningGroupId
+ ", scheduleTime=" + scheduleTime
+ ", startTime=" + startTime
+ ", processInstancePriority=" + processInstancePriority
+ ", updateTime=" + updateTime
+ ", message='" + message + '\''
+ ", workerGroup='" + workerGroup + '\''
+ ", environmentCode='" + environmentCode + '\''
+ '}';
}
}

15
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java

@ -226,6 +226,11 @@ public class ProcessInstance {
*/
private String workerGroup;
/**
* environment code
*/
private Long environmentCode;
/**
* process timeout for warning
*/
@ -505,6 +510,14 @@ public class ProcessInstance {
this.executorName = executorName;
}
public Long getEnvironmentCode() {
return this.environmentCode;
}
public void setEnvironmentCode(Long environmentCode) {
this.environmentCode = environmentCode;
}
/**
* add command to history
*
@ -666,6 +679,8 @@ public class ProcessInstance {
+ ", workerGroup='"
+ workerGroup
+ '\''
+ ", environmentCode="
+ environmentCode
+ ", timeout="
+ timeout
+ ", tenantId="

14
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Schedule.java

@ -139,6 +139,11 @@ public class Schedule {
*/
private String workerGroup;
/**
* environment code
*/
private Long environmentCode;
public int getWarningGroupId() {
return warningGroupId;
}
@ -286,6 +291,14 @@ public class Schedule {
this.workerGroup = workerGroup;
}
public Long getEnvironmentCode() {
return this.environmentCode;
}
public void setEnvironmentCode(Long environmentCode) {
this.environmentCode = environmentCode;
}
@Override
public String toString() {
return "Schedule{"
@ -308,6 +321,7 @@ public class Schedule {
+ ", warningGroupId=" + warningGroupId
+ ", processInstancePriority=" + processInstancePriority
+ ", workerGroup='" + workerGroup + '\''
+ ", environmentCode='" + environmentCode + '\''
+ '}';
}

14
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinition.java

@ -128,6 +128,11 @@ public class TaskDefinition {
*/
private String workerGroup;
/**
* environment code
*/
private Long environmentCode;
/**
* fail retry times
*/
@ -395,6 +400,14 @@ public class TaskDefinition {
this.delayTime = delayTime;
}
public Long getEnvironmentCode() {
return this.environmentCode;
}
public void setEnvironmentCode(Long environmentCode) {
this.environmentCode = environmentCode;
}
@Override
public String toString() {
return "TaskDefinition{"
@ -414,6 +427,7 @@ public class TaskDefinition {
+ ", userName='" + userName + '\''
+ ", projectName='" + projectName + '\''
+ ", workerGroup='" + workerGroup + '\''
+ ", environmentCode='" + environmentCode + '\''
+ ", failRetryTimes=" + failRetryTimes
+ ", failRetryInterval=" + failRetryInterval
+ ", timeoutFlag=" + timeoutFlag

1
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinitionLog.java

@ -53,6 +53,7 @@ public class TaskDefinitionLog extends TaskDefinition {
this.setUserId(taskDefinition.getUserId());
this.setUserName(taskDefinition.getUserName());
this.setWorkerGroup(taskDefinition.getWorkerGroup());
this.setEnvironmentCode(taskDefinition.getEnvironmentCode());
this.setProjectCode(taskDefinition.getProjectCode());
this.setProjectName(taskDefinition.getProjectName());
this.setResourceIds(taskDefinition.getResourceIds());

27
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java

@ -220,6 +220,15 @@ public class TaskInstance implements Serializable {
*/
private String workerGroup;
/**
* environment code
*/
private Long environmentCode;
/**
* environment config
*/
private String environmentConfig;
/**
* executor id
@ -421,6 +430,22 @@ public class TaskInstance implements Serializable {
this.appLink = appLink;
}
public Long getEnvironmentCode() {
return this.environmentCode;
}
public void setEnvironmentCode(Long environmentCode) {
this.environmentCode = environmentCode;
}
public String getEnvironmentConfig() {
return this.environmentConfig;
}
public void setEnvironmentConfig(String environmentConfig) {
this.environmentConfig = environmentConfig;
}
public DependentParameters getDependency() {
if (this.dependency == null) {
Map<String, Object> taskParamsMap = JSONUtils.toMap(this.getTaskParams(), String.class, Object.class);
@ -623,6 +648,8 @@ public class TaskInstance implements Serializable {
+ ", processInstancePriority=" + processInstancePriority
+ ", dependentResult='" + dependentResult + '\''
+ ", workerGroup='" + workerGroup + '\''
+ ", environmentCode=" + environmentCode
+ ", environmentConfig='" + environmentConfig + '\''
+ ", executorId=" + executorId
+ ", executorName='" + executorName + '\''
+ ", delayTime=" + delayTime

8
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/CommandMapper.java

@ -17,6 +17,8 @@
package org.apache.dolphinscheduler.dao.mapper;
import com.baomidou.mybatisplus.core.mapper.BaseMapper;
import com.baomidou.mybatisplus.core.metadata.IPage;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.CommandCount;
import org.apache.ibatis.annotations.Param;
@ -50,6 +52,10 @@ public interface CommandMapper extends BaseMapper<Command> {
@Param("endTime") Date endTime,
@Param("projectCodeArray") Long[] projectCodeArray);
/**
* query command page
* @return
*/
IPage<Command> queryCommandPage(IPage<Command> page);
}

71
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/EnvironmentMapper.java

@ -0,0 +1,71 @@
/*
* 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.dao.mapper;
import org.apache.dolphinscheduler.dao.entity.Environment;
import org.apache.ibatis.annotations.Param;
import java.util.List;
import com.baomidou.mybatisplus.core.mapper.BaseMapper;
import com.baomidou.mybatisplus.core.metadata.IPage;
/**
* environment mapper interface
*/
public interface EnvironmentMapper extends BaseMapper<Environment> {
/**
* query environment by name
*
* @param name name
* @return environment
*/
Environment queryByEnvironmentName(@Param("environmentName") String name);
/**
* query environment by code
*
* @param environmentCode environmentCode
* @return environment
*/
Environment queryByEnvironmentCode(@Param("environmentCode") Long environmentCode);
/**
* query all environment list
* @return environment list
*/
List<Environment> queryAllEnvironmentList();
/**
* environment page
* @param page page
* @param searchName searchName
* @return environment IPage
*/
IPage<Environment> queryEnvironmentListPaging(IPage<Environment> page, @Param("searchName") String searchName);
/**
* delete environment by code
*
* @param code code
* @return int
*/
int deleteByCode(@Param("code") Long code);
}

57
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/EnvironmentWorkerGroupRelationMapper.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.dao.mapper;
import org.apache.dolphinscheduler.dao.entity.EnvironmentWorkerGroupRelation;
import org.apache.ibatis.annotations.Param;
import java.util.List;
import com.baomidou.mybatisplus.core.mapper.BaseMapper;
/**
* environment worker group relation mapper interface
*/
public interface EnvironmentWorkerGroupRelationMapper extends BaseMapper<EnvironmentWorkerGroupRelation> {
/**
* environment worker group relation by environmentCode
*
* @param environmentCode environmentCode
* @return EnvironmentWorkerGroupRelation list
*/
List<EnvironmentWorkerGroupRelation> queryByEnvironmentCode(@Param("environmentCode") Long environmentCode);
/**
* environment worker group relation by workerGroupName
*
* @param workerGroupName workerGroupName
* @return EnvironmentWorkerGroupRelation list
*/
List<EnvironmentWorkerGroupRelation> queryByWorkerGroupName(@Param("workerGroupName") String workerGroupName);
/**
* delete environment worker group relation by processCode
*
* @param environmentCode environmentCode
* @param workerGroupName workerGroupName
* @return int
*/
int deleteByCode(@Param("environmentCode") Long environmentCode, @Param("workerGroupName") String workerGroupName);
}

1
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/shell/CreateDolphinScheduler.java

@ -14,6 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.upgrade.shell;
import org.apache.dolphinscheduler.dao.upgrade.DolphinSchedulerManager;

7
dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/CommandMapper.xml

@ -21,7 +21,7 @@
<select id="getOneToRun" resultType="org.apache.dolphinscheduler.dao.entity.Command">
select cmd.id, cmd.command_type, cmd.process_definition_id, cmd.command_param, cmd.task_depend_type, cmd.failure_strategy,
cmd.warning_type, cmd.warning_group_id, cmd.schedule_time, cmd.start_time, cmd.executor_id, cmd.update_time,
cmd.process_instance_priority, cmd.worker_group
cmd.process_instance_priority, cmd.worker_group, cmd.environment_code
from t_ds_command cmd
join t_ds_process_definition definition on cmd.process_definition_id = definition.id
where definition.release_state = 1 AND definition.flag = 1
@ -43,4 +43,9 @@
</if>
group by cmd.command_type
</select>
<select id="queryCommandPage" resultType="org.apache.dolphinscheduler.dao.entity.Command">
select *
from t_ds_command
order by update_time asc
</select>
</mapper>

55
dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/EnvironmentMapper.xml

@ -0,0 +1,55 @@
<?xml version="1.0" encoding="UTF-8" ?>
<!--
~ Licensed to the Apache Software Foundation (ASF) under one or more
~ contributor license agreements. See the NOTICE file distributed with
~ this work for additional information regarding copyright ownership.
~ The ASF licenses this file to You under the Apache License, Version 2.0
~ (the "License"); you may not use this file except in compliance with
~ the License. You may obtain a copy of the License at
~
~ http://www.apache.org/licenses/LICENSE-2.0
~
~ Unless required by applicable law or agreed to in writing, software
~ distributed under the License is distributed on an "AS IS" BASIS,
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
~ See the License for the specific language governing permissions and
~ limitations under the License.
-->
<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" "http://mybatis.org/dtd/mybatis-3-mapper.dtd" >
<mapper namespace="org.apache.dolphinscheduler.dao.mapper.EnvironmentMapper">
<sql id="baseSql">
id, code, name, config, description, operator, create_time, update_time
</sql>
<select id="queryByEnvironmentName" resultType="org.apache.dolphinscheduler.dao.entity.Environment">
select
<include refid="baseSql"/>
from t_ds_environment
WHERE name = #{environmentName}
</select>
<select id="queryAllEnvironmentList" resultType="org.apache.dolphinscheduler.dao.entity.Environment">
select
<include refid="baseSql"/>
from t_ds_environment
order by create_time desc
</select>
<select id="queryEnvironmentListPaging" resultType="org.apache.dolphinscheduler.dao.entity.Environment">
select
<include refid="baseSql"/>
from t_ds_environment
where 1=1
<if test="searchName!=null and searchName != ''">
and name like concat('%', #{searchName}, '%')
</if>
order by create_time desc
</select>
<select id="queryByEnvironmentCode" resultType="org.apache.dolphinscheduler.dao.entity.Environment">
select
<include refid="baseSql"/>
from t_ds_environment
where code = #{environmentCode}
</select>
<delete id="deleteByCode">
delete from t_ds_environment where code = #{code}
</delete>
</mapper>

40
dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/EnvironmentWorkerGroupRelationMapper.xml

@ -0,0 +1,40 @@
<?xml version="1.0" encoding="UTF-8" ?>
<!--
~ Licensed to the Apache Software Foundation (ASF) under one or more
~ contributor license agreements. See the NOTICE file distributed with
~ this work for additional information regarding copyright ownership.
~ The ASF licenses this file to You under the Apache License, Version 2.0
~ (the "License"); you may not use this file except in compliance with
~ the License. You may obtain a copy of the License at
~
~ http://www.apache.org/licenses/LICENSE-2.0
~
~ Unless required by applicable law or agreed to in writing, software
~ distributed under the License is distributed on an "AS IS" BASIS,
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
~ See the License for the specific language governing permissions and
~ limitations under the License.
-->
<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" "http://mybatis.org/dtd/mybatis-3-mapper.dtd" >
<mapper namespace="org.apache.dolphinscheduler.dao.mapper.EnvironmentWorkerGroupRelationMapper">
<sql id="baseSql">
id, environment_code, worker_group, operator, create_time, update_time
</sql>
<select id="queryByEnvironmentCode" resultType="org.apache.dolphinscheduler.dao.entity.EnvironmentWorkerGroupRelation">
select
<include refid="baseSql"/>
from t_ds_environment_worker_group_relation
WHERE environment_code = #{environmentCode}
</select>
<select id="queryByWorkerGroupName" resultType="org.apache.dolphinscheduler.dao.entity.EnvironmentWorkerGroupRelation">
select
<include refid="baseSql"/>
from t_ds_environment_worker_group_relation
WHERE worker_group = #{workerGroupName}
</select>
<delete id="deleteByCode">
delete from t_ds_environment_worker_group_relation
WHERE environment_code = #{environmentCode} and worker_group = #{workerGroupName}
</delete>
</mapper>

2
dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.xml

@ -23,7 +23,7 @@
command_type, command_param, task_depend_type, max_try_times, failure_strategy, warning_type,
warning_group_id, schedule_time, command_start_time, global_params, flag,
update_time, is_sub_process, executor_id, history_cmd,
process_instance_priority, worker_group, timeout, tenant_id, var_pool
process_instance_priority, worker_group,environment_code, timeout, tenant_id, var_pool
</sql>
<select id="queryDetailById" resultType="org.apache.dolphinscheduler.dao.entity.ProcessInstance">
select

4
dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ScheduleMapper.xml

@ -20,11 +20,11 @@
<mapper namespace="org.apache.dolphinscheduler.dao.mapper.ScheduleMapper">
<sql id="baseSql">
id, process_definition_id, start_time, end_time, timezone_id, crontab, failure_strategy, user_id, release_state,
warning_type, warning_group_id, process_instance_priority, worker_group, create_time, update_time
warning_type, warning_group_id, process_instance_priority, worker_group, environment_code, create_time, update_time
</sql>
<sql id="baseSqlV2">
${alias}.id, ${alias}.process_definition_id, ${alias}.start_time, ${alias}.end_time, ${alias}.timezone_id, ${alias}.crontab, ${alias}.failure_strategy, ${alias}.user_id, ${alias}.release_state,
${alias}.warning_type, ${alias}.warning_group_id, ${alias}.process_instance_priority, ${alias}.worker_group, ${alias}.create_time, ${alias}.update_time
${alias}.warning_type, ${alias}.warning_group_id, ${alias}.process_instance_priority, ${alias}.worker_group, ${alias}.environment_code, ${alias}.create_time, ${alias}.update_time
</sql>
<select id="queryByProcessDefineIdPaging" resultType="org.apache.dolphinscheduler.dao.entity.Schedule">
select p_f.name as process_definition_name, p.name as project_name,u.user_name,

4
dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionLogMapper.xml

@ -20,12 +20,12 @@
<mapper namespace="org.apache.dolphinscheduler.dao.mapper.TaskDefinitionLogMapper">
<sql id="baseSql">
id, code, name, version, description, project_code, user_id, task_type, task_params, flag, task_priority,
worker_group, fail_retry_times, fail_retry_interval, timeout_flag, timeout_notify_strategy, timeout, delay_time,
worker_group, environment_code, fail_retry_times, fail_retry_interval, timeout_flag, timeout_notify_strategy, timeout, delay_time,
resource_ids, operator, operate_time, create_time, update_time
</sql>
<select id="queryByDefinitionName" resultType="org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog">
select td.id, td.code, td.name, td.version, td.description, td.project_code, td.user_id, td.task_type, td.task_params,
td.flag, td.task_priority, td.worker_group, td.fail_retry_times, td.fail_retry_interval, td.timeout_flag, td.timeout_notify_strategy,
td.flag, td.task_priority, td.worker_group, td.environment_code, td.fail_retry_times, td.fail_retry_interval, td.timeout_flag, td.timeout_notify_strategy,
td.timeout, td.delay_time, td.resource_ids, td.operator,td.operate_time, td.create_time, td.update_time,
u.user_name,p.name as project_name
from t_ds_task_definition_log td

4
dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.xml

@ -20,7 +20,7 @@
<mapper namespace="org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper">
<sql id="baseSql">
id, code, name, version, description, project_code, user_id, task_type, task_params, flag, task_priority,
worker_group, fail_retry_times, fail_retry_interval, timeout_flag, timeout_notify_strategy, timeout, delay_time,
worker_group, environment_code, fail_retry_times, fail_retry_interval, timeout_flag, timeout_notify_strategy, timeout, delay_time,
resource_ids, create_time, update_time
</sql>
<select id="queryByDefinitionName" resultType="org.apache.dolphinscheduler.dao.entity.TaskDefinition">
@ -63,7 +63,7 @@
</select>
<select id="queryByDefinitionId" resultType="org.apache.dolphinscheduler.dao.entity.TaskDefinition">
select td.id, td.code, td.name, td.version, td.description, td.project_code, td.user_id, td.task_type, td.task_params,
td.flag, td.task_priority, td.worker_group, td.fail_retry_times, td.fail_retry_interval, td.timeout_flag, td.timeout_notify_strategy,
td.flag, td.task_priority, td.worker_group, td.environment_code, td.fail_retry_times, td.fail_retry_interval, td.timeout_flag, td.timeout_notify_strategy,
td.timeout, td.delay_time, td.resource_ids, td.create_time, td.update_time, u.user_name,p.name as project_name
from t_ds_task_definition td
JOIN t_ds_user u ON td.user_id = u.id

4
dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml

@ -21,13 +21,13 @@
<sql id="baseSql">
id, name, task_type, process_instance_id, task_code, task_definition_version, state, submit_time,
start_time, end_time, host, execute_path, log_path, alert_flag, retry_times, pid, app_link,
flag, retry_interval, max_retry_times, task_instance_priority, worker_group, executor_id,
flag, retry_interval, max_retry_times, task_instance_priority, worker_group,environment_code , executor_id,
first_submit_time, delay_time, task_params, var_pool
</sql>
<sql id="baseSqlV2">
${alias}.id, ${alias}.name, ${alias}.task_type, ${alias}.task_code, ${alias}.task_definition_version, ${alias}.process_instance_id, ${alias}.state, ${alias}.submit_time,
${alias}.start_time, ${alias}.end_time, ${alias}.host, ${alias}.execute_path, ${alias}.log_path, ${alias}.alert_flag, ${alias}.retry_times, ${alias}.pid, ${alias}.app_link,
${alias}.flag, ${alias}.retry_interval, ${alias}.max_retry_times, ${alias}.task_instance_priority, ${alias}.worker_group, ${alias}.executor_id,
${alias}.flag, ${alias}.retry_interval, ${alias}.max_retry_times, ${alias}.task_instance_priority, ${alias}.worker_group,${alias}.environment_code , ${alias}.executor_id,
${alias}.first_submit_time, ${alias}.delay_time, ${alias}.task_params, ${alias}.var_pool
</sql>
<update id="setFailoverByHostAndStateArray">

199
dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/EnvironmentMapperTest.java

@ -0,0 +1,199 @@
/*
* 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.dao.mapper;
import org.apache.dolphinscheduler.dao.entity.Environment;
import java.util.Date;
import java.util.List;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.boot.test.context.SpringBootTest;
import org.springframework.test.annotation.Rollback;
import org.springframework.test.context.junit4.SpringRunner;
import org.springframework.transaction.annotation.Transactional;
import com.baomidou.mybatisplus.core.metadata.IPage;
import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
@RunWith(SpringRunner.class)
@SpringBootTest
@Transactional
@Rollback(false)
public class EnvironmentMapperTest {
@Autowired
EnvironmentMapper environmentMapper;
/**
* insert
*
* @return Environment
*/
private Environment insertOne() {
//insertOne
Environment environment = new Environment();
environment.setName("testEnv");
environment.setCode(1L);
environment.setOperator(1);
environment.setConfig(getConfig());
environment.setDescription(getDesc());
environment.setCreateTime(new Date());
environment.setUpdateTime(new Date());
environmentMapper.insert(environment);
return environment;
}
@Before
public void setUp() {
clearTestData();
}
@After
public void after() {
clearTestData();
}
public void clearTestData() {
environmentMapper.queryAllEnvironmentList().stream().forEach(environment -> {
environmentMapper.deleteByCode(environment.getCode());
});
}
/**
* test update
*/
@Test
public void testUpdate() {
//insertOne
Environment environment = insertOne();
environment.setDescription("new description info");
//update
int update = environmentMapper.updateById(environment);
Assert.assertEquals(update, 1);
}
/**
* test delete
*/
@Test
public void testDelete() {
Environment environment = insertOne();
int delete = environmentMapper.deleteById(environment.getId());
Assert.assertEquals(delete, 1);
}
/**
* test query
*/
@Test
public void testQuery() {
insertOne();
//query
List<Environment> environments = environmentMapper.selectList(null);
Assert.assertEquals(environments.size(), 1);
}
/**
* test query environment by name
*/
@Test
public void testQueryByEnvironmentName() {
Environment entity = insertOne();
Environment environment = environmentMapper.queryByEnvironmentName(entity.getName());
Assert.assertEquals(entity.toString(),environment.toString());
}
/**
* test query environment by code
*/
@Test
public void testQueryByEnvironmentCode() {
Environment entity = insertOne();
Environment environment = environmentMapper.queryByEnvironmentCode(entity.getCode());
Assert.assertEquals(entity.toString(),environment.toString());
}
/**
* test query all environments
*/
@Test
public void testQueryAllEnvironmentList() {
Environment entity = insertOne();
List<Environment> environments = environmentMapper.queryAllEnvironmentList();
Assert.assertEquals(environments.size(), 1);
Assert.assertEquals(entity.toString(),environments.get(0).toString());
}
/**
* test query environment list paging
*/
@Test
public void testQueryEnvironmentListPaging() {
Environment entity = insertOne();
Page<Environment> page = new Page<>(1, 10);
IPage<Environment> environmentIPage = environmentMapper.queryEnvironmentListPaging(page,"");
List<Environment> environmentList = environmentIPage.getRecords();
Assert.assertEquals(environmentList.size(), 1);
environmentIPage = environmentMapper.queryEnvironmentListPaging(page,"abc");
environmentList = environmentIPage.getRecords();
Assert.assertEquals(environmentList.size(), 0);
}
/**
* test query all environments
*/
@Test
public void testDeleteByCode() {
Environment entity = insertOne();
int delete = environmentMapper.deleteByCode(entity.getCode());
Assert.assertEquals(delete, 1);
}
private String getDesc() {
return "create an environment to test ";
}
/**
* create an environment config
*/
private String getConfig() {
return "export HADOOP_HOME=/opt/hadoop-2.6.5\n"
+ "export HADOOP_CONF_DIR=/etc/hadoop/conf\n"
+ "export SPARK_HOME1=/opt/soft/spark1\n"
+ "export SPARK_HOME2=/opt/soft/spark2\n"
+ "export PYTHON_HOME=/opt/soft/python\n"
+ "export JAVA_HOME=/opt/java/jdk1.8.0_181-amd64\n"
+ "export HIVE_HOME=/opt/soft/hive\n"
+ "export FLINK_HOME=/opt/soft/flink\n"
+ "export DATAX_HOME=/opt/soft/datax\n"
+ "export YARN_CONF_DIR=\"/etc/hadoop/conf\"\n"
+ "\n"
+ "export PATH=$HADOOP_HOME/bin:$SPARK_HOME1/bin:$SPARK_HOME2/bin:$PYTHON_HOME/bin:$JAVA_HOME/bin:$HIVE_HOME/bin:$FLINK_HOME/bin:$DATAX_HOME/bin:$PATH\n"
+ "\n"
+ "export HADOOP_CLASSPATH=`hadoop classpath`\n"
+ "\n"
+ "#echo \"HADOOP_CLASSPATH=\"$HADOOP_CLASSPATH";
}
}

109
dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/EnvironmentWorkerGroupRelationMapperTest.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.dao.mapper;
import org.apache.dolphinscheduler.dao.entity.EnvironmentWorkerGroupRelation;
import java.util.Date;
import java.util.List;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.boot.test.context.SpringBootTest;
import org.springframework.test.annotation.Rollback;
import org.springframework.test.context.junit4.SpringRunner;
import org.springframework.transaction.annotation.Transactional;
@RunWith(SpringRunner.class)
@SpringBootTest
@Transactional
@Rollback(true)
public class EnvironmentWorkerGroupRelationMapperTest {
@Autowired
EnvironmentWorkerGroupRelationMapper environmentWorkerGroupRelationMapper;
@Before
public void setUp() {
clearTestData();
}
@After
public void after() {
clearTestData();
}
public void clearTestData() {
environmentWorkerGroupRelationMapper.selectList(null).stream().forEach(environment -> {
environmentWorkerGroupRelationMapper.deleteById(environment.getId());
});
}
/**
* insert
*
* @return ProcessDefinition
*/
private EnvironmentWorkerGroupRelation insertOne() {
//insertOne
EnvironmentWorkerGroupRelation relation = new EnvironmentWorkerGroupRelation();
relation.setEnvironmentCode(1L);
relation.setWorkerGroup("default");
relation.setOperator(1);
relation.setUpdateTime(new Date());
relation.setCreateTime(new Date());
environmentWorkerGroupRelationMapper.insert(relation);
return relation;
}
/**
* test query
*/
@Test
public void testQuery() {
insertOne();
//query
List<EnvironmentWorkerGroupRelation> relations = environmentWorkerGroupRelationMapper.selectList(null);
Assert.assertEquals(relations.size(), 1);
}
@Test
public void testQueryByEnvironmentCode() {
EnvironmentWorkerGroupRelation relation = insertOne();
List<EnvironmentWorkerGroupRelation> environmentWorkerGroupRelations = environmentWorkerGroupRelationMapper.queryByEnvironmentCode(1L);
Assert.assertNotEquals(environmentWorkerGroupRelations.size(), 0);
}
@Test
public void testQueryByWorkerGroupName() {
EnvironmentWorkerGroupRelation relation = insertOne();
List<EnvironmentWorkerGroupRelation> environmentWorkerGroupRelations = environmentWorkerGroupRelationMapper.queryByWorkerGroupName("default");
Assert.assertNotEquals(environmentWorkerGroupRelations.size(), 0);
}
@Test
public void testDeleteByCode() {
EnvironmentWorkerGroupRelation relation = insertOne();
int i = environmentWorkerGroupRelationMapper.deleteByCode(1L, "default");
Assert.assertNotEquals(i, 0);
}
}

2
dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionLogMapperTest.java

@ -65,6 +65,8 @@ public class TaskDefinitionLogMapperTest {
taskDefinition.setProjectCode(1L);
taskDefinition.setTaskType(TaskType.SHELL.getDesc());
taskDefinition.setUserId(userId);
taskDefinition.setEnvironmentCode(1L);
taskDefinition.setWorkerGroup("default");
taskDefinition.setVersion(1);
taskDefinition.setCreateTime(new Date());
taskDefinition.setUpdateTime(new Date());

2
dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapperTest.java

@ -63,6 +63,8 @@ public class TaskDefinitionMapperTest {
taskDefinition.setTaskType(TaskType.SHELL.getDesc());
taskDefinition.setUserId(userId);
taskDefinition.setResourceIds("1");
taskDefinition.setWorkerGroup("default");
taskDefinition.setEnvironmentCode(1L);
taskDefinition.setVersion(1);
taskDefinition.setCreateTime(new Date());
taskDefinition.setUpdateTime(new Date());

10
dolphinscheduler-remote/pom.xml

@ -83,6 +83,16 @@
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
<exclusions>
<exclusion>
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.springframework</groupId>
<artifactId>spring-context</artifactId>
</dependency>
</dependencies>

17
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java

@ -121,5 +121,20 @@ public enum CommandType {
/**
* alert send response
*/
ALERT_SEND_RESPONSE;
ALERT_SEND_RESPONSE,
/**
* process host update
*/
PROCESS_HOST_UPDATE_REQUST,
/**
* process host update response
*/
PROCESS_HOST_UPDATE_RESPONSE,
/**
* state event request
*/
STATE_EVENT_REQUEST;
}

72
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/HostUpdateCommand.java

@ -0,0 +1,72 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.remote.command;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import java.io.Serializable;
/**
* process host update
*/
public class HostUpdateCommand implements Serializable {
/**
* task id
*/
private int taskInstanceId;
private String processHost;
public int getTaskInstanceId() {
return taskInstanceId;
}
public void setTaskInstanceId(int taskInstanceId) {
this.taskInstanceId = taskInstanceId;
}
public String getProcessHost() {
return processHost;
}
public void setProcessHost(String processHost) {
this.processHost = processHost;
}
/**
* package request command
*
* @return command
*/
public Command convert2Command() {
Command command = new Command();
command.setType(CommandType.PROCESS_HOST_UPDATE_REQUST);
byte[] body = JSONUtils.toJsonByteArray(this);
command.setBody(body);
return command;
}
@Override
public String toString() {
return "HostUpdateCommand{"
+ "taskInstanceId=" + taskInstanceId
+ "host=" + processHost
+ '}';
}
}

83
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/HostUpdateResponseCommand.java

@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.remote.command;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import java.io.Serializable;
public class HostUpdateResponseCommand implements Serializable {
private int taskInstanceId;
private String processHost;
private int status;
public HostUpdateResponseCommand(int taskInstanceId, String processHost, int code) {
this.taskInstanceId = taskInstanceId;
this.processHost = processHost;
this.status = code;
}
public int getTaskInstanceId() {
return this.taskInstanceId;
}
public void setTaskInstanceId(int taskInstanceId) {
this.taskInstanceId = taskInstanceId;
}
public String getProcessHost() {
return this.processHost;
}
public void setProcessHost(String processHost) {
this.processHost = processHost;
}
public int getStatus() {
return status;
}
public void setStatus(int status) {
this.status = status;
}
/**
* package request command
*
* @return command
*/
public Command convert2Command() {
Command command = new Command();
command.setType(CommandType.PROCESS_HOST_UPDATE_REQUST);
byte[] body = JSONUtils.toJsonByteArray(this);
command.setBody(body);
return command;
}
@Override
public String toString() {
return "HostUpdateResponseCommand{"
+ "taskInstanceId=" + taskInstanceId
+ "host=" + processHost
+ '}';
}
}

131
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/StateEventChangeCommand.java

@ -0,0 +1,131 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.remote.command;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import java.io.Serializable;
/**
* db task final result response command
*/
public class StateEventChangeCommand implements Serializable {
private String key;
private ExecutionStatus sourceStatus;
private int sourceProcessInstanceId;
private int sourceTaskInstanceId;
private int destProcessInstanceId;
private int destTaskInstanceId;
public StateEventChangeCommand() {
super();
}
public StateEventChangeCommand(int sourceProcessInstanceId, int sourceTaskInstanceId,
ExecutionStatus sourceStatus,
int destProcessInstanceId,
int destTaskInstanceId
) {
this.key = String.format("%d-%d-%d-%d",
sourceProcessInstanceId,
sourceTaskInstanceId,
destProcessInstanceId,
destTaskInstanceId);
this.sourceStatus = sourceStatus;
this.sourceProcessInstanceId = sourceProcessInstanceId;
this.sourceTaskInstanceId = sourceTaskInstanceId;
this.destProcessInstanceId = destProcessInstanceId;
this.destTaskInstanceId = destTaskInstanceId;
}
public String getKey() {
return key;
}
public void setKey(String key) {
this.key = key;
}
/**
* package response command
*
* @return command
*/
public Command convert2Command() {
Command command = new Command();
command.setType(CommandType.STATE_EVENT_REQUEST);
byte[] body = JSONUtils.toJsonByteArray(this);
command.setBody(body);
return command;
}
@Override
public String toString() {
return "StateEventResponseCommand{"
+ "key=" + key
+ '}';
}
public ExecutionStatus getSourceStatus() {
return sourceStatus;
}
public void setSourceStatus(ExecutionStatus sourceStatus) {
this.sourceStatus = sourceStatus;
}
public int getSourceProcessInstanceId() {
return sourceProcessInstanceId;
}
public void setSourceProcessInstanceId(int sourceProcessInstanceId) {
this.sourceProcessInstanceId = sourceProcessInstanceId;
}
public int getSourceTaskInstanceId() {
return sourceTaskInstanceId;
}
public void setSourceTaskInstanceId(int sourceTaskInstanceId) {
this.sourceTaskInstanceId = sourceTaskInstanceId;
}
public int getDestProcessInstanceId() {
return destProcessInstanceId;
}
public void setDestProcessInstanceId(int destProcessInstanceId) {
this.destProcessInstanceId = destProcessInstanceId;
}
public int getDestTaskInstanceId() {
return destTaskInstanceId;
}
public void setDestTaskInstanceId(int destTaskInstanceId) {
this.destTaskInstanceId = destTaskInstanceId;
}
}

78
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/StateEventResponseCommand.java

@ -0,0 +1,78 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.remote.command;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import java.io.Serializable;
/**
* db task final result response command
*/
public class StateEventResponseCommand implements Serializable {
private String key;
private int status;
public StateEventResponseCommand() {
super();
}
public StateEventResponseCommand(int status, String key) {
this.status = status;
this.key = key;
}
public int getStatus() {
return status;
}
public void setStatus(int status) {
this.status = status;
}
public String getKey() {
return key;
}
public void setKey(String key) {
this.key = key;
}
/**
* package response command
*
* @return command
*/
public Command convert2Command() {
Command command = new Command();
command.setType(CommandType.DB_TASK_RESPONSE);
byte[] body = JSONUtils.toJsonByteArray(this);
command.setBody(body);
return command;
}
@Override
public String toString() {
return "StateEventResponseCommand{"
+ "key=" + key
+ ", status=" + status
+ '}';
}
}

14
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskExecuteAckCommand.java

@ -34,6 +34,11 @@ public class TaskExecuteAckCommand implements Serializable {
*/
private int taskInstanceId;
/**
* process instance id
*/
private int processInstanceId;
/**
* startTime
*/
@ -130,6 +135,15 @@ public class TaskExecuteAckCommand implements Serializable {
+ ", status=" + status
+ ", logPath='" + logPath + '\''
+ ", executePath='" + executePath + '\''
+ ", processInstanceId='" + processInstanceId + '\''
+ '}';
}
public int getProcessInstanceId() {
return processInstanceId;
}
public void setProcessInstanceId(int processInstanceId) {
this.processInstanceId = processInstanceId;
}
}

16
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskExecuteResponseCommand.java

@ -32,8 +32,9 @@ public class TaskExecuteResponseCommand implements Serializable {
public TaskExecuteResponseCommand() {
}
public TaskExecuteResponseCommand(int taskInstanceId) {
public TaskExecuteResponseCommand(int taskInstanceId, int processInstanceId) {
this.taskInstanceId = taskInstanceId;
this.processInstanceId = processInstanceId;
}
/**
@ -41,6 +42,11 @@ public class TaskExecuteResponseCommand implements Serializable {
*/
private int taskInstanceId;
/**
* process instance id
*/
private int processInstanceId;
/**
* status
*/
@ -139,4 +145,12 @@ public class TaskExecuteResponseCommand implements Serializable {
+ ", appIds='" + appIds + '\''
+ '}';
}
public int getProcessInstanceId() {
return processInstanceId;
}
public void setProcessInstanceId(int processInstanceId) {
this.processInstanceId = processInstanceId;
}
}

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/processor/NettyRemoteChannel.java → dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/NettyRemoteChannel.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.worker.processor;
package org.apache.dolphinscheduler.remote.processor;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;

125
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/StateEventCallbackService.java

@ -0,0 +1,125 @@
/*
* 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.processor;
import static org.apache.dolphinscheduler.common.Constants.SLEEP_TIME_MILLIS;
import org.apache.dolphinscheduler.remote.NettyRemotingClient;
import org.apache.dolphinscheduler.remote.command.Command;
import org.apache.dolphinscheduler.remote.config.NettyClientConfig;
import org.apache.dolphinscheduler.remote.utils.Host;
import java.util.concurrent.ConcurrentHashMap;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.stereotype.Service;
import io.netty.channel.Channel;
/**
* task callback service
*/
@Service
public class StateEventCallbackService {
private final Logger logger = LoggerFactory.getLogger(StateEventCallbackService.class);
private static final int[] RETRY_BACKOFF = {1, 2, 3, 5, 10, 20, 40, 100, 100, 100, 100, 200, 200, 200};
/**
* remote channels
*/
private static final ConcurrentHashMap<String, NettyRemoteChannel> REMOTE_CHANNELS = new ConcurrentHashMap<>();
/**
* netty remoting client
*/
private final NettyRemotingClient nettyRemotingClient;
public StateEventCallbackService() {
final NettyClientConfig clientConfig = new NettyClientConfig();
this.nettyRemotingClient = new NettyRemotingClient(clientConfig);
}
/**
* add callback channel
*
* @param channel channel
*/
public void addRemoteChannel(String host, NettyRemoteChannel channel) {
REMOTE_CHANNELS.put(host, channel);
}
/**
* get callback channel
*
* @param host
* @return callback channel
*/
private NettyRemoteChannel newRemoteChannel(Host host) {
Channel newChannel;
NettyRemoteChannel nettyRemoteChannel = REMOTE_CHANNELS.get(host.getAddress());
if (nettyRemoteChannel != null) {
if (nettyRemoteChannel.isActive()) {
return nettyRemoteChannel;
}
}
newChannel = nettyRemotingClient.getChannel(host);
if (newChannel != null) {
return newRemoteChannel(newChannel, host.getAddress());
}
return null;
}
public int pause(int ntries) {
return SLEEP_TIME_MILLIS * RETRY_BACKOFF[ntries % RETRY_BACKOFF.length];
}
private NettyRemoteChannel newRemoteChannel(Channel newChannel, long opaque, String host) {
NettyRemoteChannel remoteChannel = new NettyRemoteChannel(newChannel, opaque);
addRemoteChannel(host, remoteChannel);
return remoteChannel;
}
private NettyRemoteChannel newRemoteChannel(Channel newChannel, String host) {
NettyRemoteChannel remoteChannel = new NettyRemoteChannel(newChannel);
addRemoteChannel(host, remoteChannel);
return remoteChannel;
}
/**
* remove callback channels
*/
public void remove(String host) {
REMOTE_CHANNELS.remove(host);
}
/**
* send result
*
* @param command command
*/
public void sendResult(String address, int port, Command command) {
logger.info("send result, host:{}, command:{}", address, command.toString());
Host host = new Host(address, port);
NettyRemoteChannel nettyRemoteChannel = newRemoteChannel(host);
if (nettyRemoteChannel != null) {
nettyRemoteChannel.writeAndFlush(command);
}
}
}

11
dolphinscheduler-server/pom.xml

@ -60,7 +60,16 @@
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
<exclusions>
<exclusion>
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.powermock</groupId>
<artifactId>powermock-module-junit4</artifactId>

5
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/builder/TaskExecutionContextBuilder.java

@ -56,6 +56,7 @@ public class TaskExecutionContextBuilder {
taskExecutionContext.setTaskType(taskInstance.getTaskType());
taskExecutionContext.setLogPath(taskInstance.getLogPath());
taskExecutionContext.setWorkerGroup(taskInstance.getWorkerGroup());
taskExecutionContext.setEnvironmentConfig(taskInstance.getEnvironmentConfig());
taskExecutionContext.setHost(taskInstance.getHost());
taskExecutionContext.setResources(taskInstance.getResources());
taskExecutionContext.setDelayTime(taskInstance.getDelayTime());
@ -76,7 +77,6 @@ public class TaskExecutionContextBuilder {
return this;
}
/**
* build processInstance related info
*
@ -107,7 +107,6 @@ public class TaskExecutionContextBuilder {
return this;
}
/**
* build SQLTask related info
*
@ -119,7 +118,6 @@ public class TaskExecutionContextBuilder {
return this;
}
/**
* build DataxTask related info
*
@ -153,7 +151,6 @@ public class TaskExecutionContextBuilder {
return this;
}
/**
* create
*

15
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/entity/TaskExecutionContext.java

@ -157,6 +157,12 @@ public class TaskExecutionContext implements Serializable {
*/
private String envFile;
/**
* environmentConfig
*/
private String environmentConfig;
/**
* definedParams
*/
@ -424,6 +430,14 @@ public class TaskExecutionContext implements Serializable {
this.envFile = envFile;
}
public String getEnvironmentConfig() {
return environmentConfig;
}
public void setEnvironmentConfig(String config) {
this.environmentConfig = config;
}
public Map<String, String> getDefinedParams() {
return definedParams;
}
@ -566,6 +580,7 @@ public class TaskExecutionContext implements Serializable {
+ ", taskTimeoutStrategy=" + taskTimeoutStrategy
+ ", taskTimeout=" + taskTimeout
+ ", workerGroup='" + workerGroup + '\''
+ ", environmentConfig='" + environmentConfig + '\''
+ ", delayTime=" + delayTime
+ ", resources=" + resources
+ ", sqlTaskExecutionContext=" + sqlTaskExecutionContext

26
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java

@ -24,14 +24,19 @@ import org.apache.dolphinscheduler.remote.NettyRemotingServer;
import org.apache.dolphinscheduler.remote.command.CommandType;
import org.apache.dolphinscheduler.remote.config.NettyServerConfig;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.processor.StateEventProcessor;
import org.apache.dolphinscheduler.server.master.processor.TaskAckProcessor;
import org.apache.dolphinscheduler.server.master.processor.TaskKillResponseProcessor;
import org.apache.dolphinscheduler.server.master.processor.TaskResponseProcessor;
import org.apache.dolphinscheduler.server.master.registry.MasterRegistryClient;
import org.apache.dolphinscheduler.server.master.runner.EventExecuteService;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThread;
import org.apache.dolphinscheduler.server.master.runner.MasterSchedulerService;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.quartz.QuartzExecutors;
import java.util.concurrent.ConcurrentHashMap;
import javax.annotation.PostConstruct;
import org.quartz.SchedulerException;
@ -92,6 +97,11 @@ public class MasterServer implements IStoppable {
@Autowired
private MasterSchedulerService masterSchedulerService;
@Autowired
private EventExecuteService eventExecuteService;
private ConcurrentHashMap<Integer, WorkflowExecuteThread> processInstanceExecMaps = new ConcurrentHashMap<>();
/**
* master server startup, not use web service
*
@ -111,16 +121,28 @@ public class MasterServer implements IStoppable {
NettyServerConfig serverConfig = new NettyServerConfig();
serverConfig.setListenPort(masterConfig.getListenPort());
this.nettyRemotingServer = new NettyRemotingServer(serverConfig);
this.nettyRemotingServer.registerProcessor(CommandType.TASK_EXECUTE_RESPONSE, new TaskResponseProcessor());
this.nettyRemotingServer.registerProcessor(CommandType.TASK_EXECUTE_ACK, new TaskAckProcessor());
TaskAckProcessor ackProcessor = new TaskAckProcessor();
ackProcessor.init(processInstanceExecMaps);
TaskResponseProcessor taskResponseProcessor = new TaskResponseProcessor();
taskResponseProcessor.init(processInstanceExecMaps);
StateEventProcessor stateEventProcessor = new StateEventProcessor();
stateEventProcessor.init(processInstanceExecMaps);
this.nettyRemotingServer.registerProcessor(CommandType.TASK_EXECUTE_RESPONSE, ackProcessor);
this.nettyRemotingServer.registerProcessor(CommandType.TASK_EXECUTE_ACK, taskResponseProcessor);
this.nettyRemotingServer.registerProcessor(CommandType.TASK_KILL_RESPONSE, new TaskKillResponseProcessor());
this.nettyRemotingServer.registerProcessor(CommandType.STATE_EVENT_REQUEST, stateEventProcessor);
this.nettyRemotingServer.start();
// self tolerant
this.masterRegistryClient.init(this.processInstanceExecMaps);
this.masterRegistryClient.start();
this.masterRegistryClient.setRegistryStoppable(this);
this.eventExecuteService.init(this.processInstanceExecMaps);
this.eventExecuteService.start();
// scheduler start
this.masterSchedulerService.init(this.processInstanceExecMaps);
this.masterSchedulerService.start();
// start QuartzExecutors

11
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterConfig.java

@ -45,6 +45,9 @@ public class MasterConfig {
@Value("${master.heartbeat.interval:10}")
private int masterHeartbeatInterval;
@Value("${master.state.wheel.interval:5}")
private int stateWheelInterval;
@Value("${master.task.commit.retryTimes:5}")
private int masterTaskCommitRetryTimes;
@ -139,4 +142,12 @@ public class MasterConfig {
public void setMasterDispatchTaskNumber(int masterDispatchTaskNumber) {
this.masterDispatchTaskNumber = masterDispatchTaskNumber;
}
public int getStateWheelInterval() {
return this.stateWheelInterval;
}
public void setStateWheelInterval(int stateWheelInterval) {
this.stateWheelInterval = stateWheelInterval;
}
}

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/executor/NettyExecutorManager.java

@ -150,7 +150,7 @@ public class NettyExecutorManager extends AbstractExecutorManager<Boolean>{
* @param command command
* @throws ExecuteException if error throws ExecuteException
*/
private void doExecute(final Host host, final Command command) throws ExecuteException {
public void doExecute(final Host host, final Command command) throws ExecuteException {
/**
* retry countdefault retry 3
*/

175
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/future/TaskFuture.java

@ -1,175 +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.master.future;
import org.apache.dolphinscheduler.remote.command.Command;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
/**
* task future
*/
public class TaskFuture {
private final static Logger LOGGER = LoggerFactory.getLogger(TaskFuture.class);
private final static ConcurrentHashMap<Long,TaskFuture> FUTURE_TABLE = new ConcurrentHashMap<>(256);
/**
* request unique identification
*/
private final long opaque;
/**
* timeout
*/
private final long timeoutMillis;
private final CountDownLatch latch = new CountDownLatch(1);
private final long beginTimestamp = System.currentTimeMillis();
/**
* response command
*/
private AtomicReference<Command> responseCommandReference = new AtomicReference<>();
private volatile boolean sendOk = true;
private AtomicReference<Throwable> causeReference;
public TaskFuture(long opaque, long timeoutMillis) {
this.opaque = opaque;
this.timeoutMillis = timeoutMillis;
FUTURE_TABLE.put(opaque, this);
}
/**
* wait for response
* @return command
* @throws InterruptedException if error throws InterruptedException
*/
public Command waitResponse() throws InterruptedException {
this.latch.await(timeoutMillis, TimeUnit.MILLISECONDS);
return this.responseCommandReference.get();
}
/**
* put response
*
* @param responseCommand responseCommand
*/
public void putResponse(final Command responseCommand) {
responseCommandReference.set(responseCommand);
this.latch.countDown();
FUTURE_TABLE.remove(opaque);
}
/**
* whether timeout
* @return timeout
*/
public boolean isTimeout() {
long diff = System.currentTimeMillis() - this.beginTimestamp;
return diff > this.timeoutMillis;
}
public static void notify(final Command responseCommand){
TaskFuture taskFuture = FUTURE_TABLE.remove(responseCommand.getOpaque());
if(taskFuture != null){
taskFuture.putResponse(responseCommand);
}
}
public boolean isSendOK() {
return sendOk;
}
public void setSendOk(boolean sendOk) {
this.sendOk = sendOk;
}
public void setCause(Throwable cause) {
causeReference.set(cause);
}
public Throwable getCause() {
return causeReference.get();
}
public long getOpaque() {
return opaque;
}
public long getTimeoutMillis() {
return timeoutMillis;
}
public long getBeginTimestamp() {
return beginTimestamp;
}
public Command getResponseCommand() {
return responseCommandReference.get();
}
public void setResponseCommand(Command responseCommand) {
responseCommandReference.set(responseCommand);
}
/**
* scan future table
*/
public static void scanFutureTable(){
final List<TaskFuture> futureList = new LinkedList<>();
Iterator<Map.Entry<Long, TaskFuture>> it = FUTURE_TABLE.entrySet().iterator();
while (it.hasNext()) {
Map.Entry<Long, TaskFuture> next = it.next();
TaskFuture future = next.getValue();
if ((future.getBeginTimestamp() + future.getTimeoutMillis() + 1000) <= System.currentTimeMillis()) {
futureList.add(future);
it.remove();
LOGGER.warn("remove timeout request : {}", future);
}
}
}
@Override
public String toString() {
return "TaskFuture{" +
"opaque=" + opaque +
", timeoutMillis=" + timeoutMillis +
", latch=" + latch +
", beginTimestamp=" + beginTimestamp +
", responseCommand=" + responseCommandReference.get() +
", sendOk=" + sendOk +
", cause=" + causeReference.get() +
'}';
}
}

42
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/HostUpdateResponseProcessor.java

@ -0,0 +1,42 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.processor;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.Preconditions;
import org.apache.dolphinscheduler.remote.command.Command;
import org.apache.dolphinscheduler.remote.command.CommandType;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.netty.channel.Channel;
public class HostUpdateResponseProcessor implements NettyRequestProcessor {
private final Logger logger = LoggerFactory.getLogger(HostUpdateResponseProcessor.class);
@Override
public void process(Channel channel, Command command) {
Preconditions.checkArgument(CommandType.PROCESS_HOST_UPDATE_RESPONSE == command.getType(), String.format("invalid command type : %s", command.getType()));
HostUpdateResponseProcessor responseCommand = JSONUtils.parseObject(command.getBody(), HostUpdateResponseProcessor.class);
logger.info("received process host response command : {}", responseCommand);
}
}

74
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/StateEventProcessor.java

@ -0,0 +1,74 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.processor;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.StateEvent;
import org.apache.dolphinscheduler.common.enums.StateEventType;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.Preconditions;
import org.apache.dolphinscheduler.remote.command.Command;
import org.apache.dolphinscheduler.remote.command.CommandType;
import org.apache.dolphinscheduler.remote.command.StateEventChangeCommand;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.server.master.processor.queue.StateEventResponseService;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThread;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import java.util.concurrent.ConcurrentHashMap;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.netty.channel.Channel;
/**
* handle state event received from master/api
*/
public class StateEventProcessor implements NettyRequestProcessor {
private final Logger logger = LoggerFactory.getLogger(StateEventProcessor.class);
private StateEventResponseService stateEventResponseService;
public StateEventProcessor() {
stateEventResponseService = SpringApplicationContext.getBean(StateEventResponseService.class);
}
public void init(ConcurrentHashMap<Integer, WorkflowExecuteThread> processInstanceExecMaps) {
this.stateEventResponseService.init(processInstanceExecMaps);
}
@Override
public void process(Channel channel, Command command) {
Preconditions.checkArgument(CommandType.STATE_EVENT_REQUEST == command.getType(), String.format("invalid command type: %s", command.getType()));
StateEventChangeCommand stateEventChangeCommand = JSONUtils.parseObject(command.getBody(), StateEventChangeCommand.class);
StateEvent stateEvent = new StateEvent();
stateEvent.setExecutionStatus(ExecutionStatus.RUNNING_EXECUTION);
stateEvent.setKey(stateEventChangeCommand.getKey());
stateEvent.setProcessInstanceId(stateEventChangeCommand.getDestProcessInstanceId());
stateEvent.setTaskInstanceId(stateEventChangeCommand.getDestTaskInstanceId());
StateEventType type = stateEvent.getTaskInstanceId() == 0 ? StateEventType.PROCESS_STATE_CHANGE : StateEventType.TASK_STATE_CHANGE;
stateEvent.setType(type);
logger.info("received command : {}", stateEvent.toString());
stateEventResponseService.addResponse(stateEvent);
}
}

11
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskAckProcessor.java

@ -29,8 +29,11 @@ import org.apache.dolphinscheduler.server.master.cache.TaskInstanceCacheManager;
import org.apache.dolphinscheduler.server.master.cache.impl.TaskInstanceCacheManagerImpl;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskResponseEvent;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskResponseService;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThread;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import java.util.concurrent.ConcurrentHashMap;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -58,8 +61,13 @@ public class TaskAckProcessor implements NettyRequestProcessor {
this.taskInstanceCacheManager = SpringApplicationContext.getBean(TaskInstanceCacheManagerImpl.class);
}
public void init(ConcurrentHashMap<Integer, WorkflowExecuteThread> processInstanceExecMaps) {
this.taskResponseService.init(processInstanceExecMaps);
}
/**
* task ack process
*
* @param channel channel channel
* @param command command TaskExecuteAckCommand
*/
@ -82,7 +90,8 @@ public class TaskAckProcessor implements NettyRequestProcessor {
taskAckCommand.getExecutePath(),
taskAckCommand.getLogPath(),
taskAckCommand.getTaskInstanceId(),
channel);
channel,
taskAckCommand.getProcessInstanceId());
taskResponseService.addResponse(taskResponseEvent);
}

12
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskResponseProcessor.java

@ -28,8 +28,11 @@ import org.apache.dolphinscheduler.server.master.cache.TaskInstanceCacheManager;
import org.apache.dolphinscheduler.server.master.cache.impl.TaskInstanceCacheManagerImpl;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskResponseEvent;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskResponseService;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThread;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import java.util.concurrent.ConcurrentHashMap;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -57,6 +60,10 @@ public class TaskResponseProcessor implements NettyRequestProcessor {
this.taskInstanceCacheManager = SpringApplicationContext.getBean(TaskInstanceCacheManagerImpl.class);
}
public void init(ConcurrentHashMap<Integer, WorkflowExecuteThread> processInstanceExecMaps) {
this.taskResponseService.init(processInstanceExecMaps);
}
/**
* task final result response
* need master process , state persistence
@ -80,10 +87,9 @@ public class TaskResponseProcessor implements NettyRequestProcessor {
responseCommand.getAppIds(),
responseCommand.getTaskInstanceId(),
responseCommand.getVarPool(),
channel
channel,
responseCommand.getProcessInstanceId()
);
taskResponseService.addResponse(taskResponseEvent);
}
}

149
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/StateEventResponseService.java

@ -0,0 +1,149 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.processor.queue;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.StateEvent;
import org.apache.dolphinscheduler.common.thread.Stopper;
import org.apache.dolphinscheduler.remote.command.StateEventResponseCommand;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThread;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.LinkedBlockingQueue;
import javax.annotation.PostConstruct;
import javax.annotation.PreDestroy;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.stereotype.Component;
import io.netty.channel.Channel;
/**
* task manager
*/
@Component
public class StateEventResponseService {
/**
* logger
*/
private final Logger logger = LoggerFactory.getLogger(StateEventResponseService.class);
/**
* attemptQueue
*/
private final BlockingQueue<StateEvent> eventQueue = new LinkedBlockingQueue<>(5000);
/**
* task response worker
*/
private Thread responseWorker;
private ConcurrentHashMap<Integer, WorkflowExecuteThread> processInstanceMapper;
public void init(ConcurrentHashMap<Integer, WorkflowExecuteThread> processInstanceMapper) {
if (this.processInstanceMapper == null) {
this.processInstanceMapper = processInstanceMapper;
}
}
@PostConstruct
public void start() {
this.responseWorker = new StateEventResponseWorker();
this.responseWorker.setName("StateEventResponseWorker");
this.responseWorker.start();
}
@PreDestroy
public void stop() {
this.responseWorker.interrupt();
if (!eventQueue.isEmpty()) {
List<StateEvent> remainEvents = new ArrayList<>(eventQueue.size());
eventQueue.drainTo(remainEvents);
for (StateEvent event : remainEvents) {
this.persist(event);
}
}
}
/**
* put task to attemptQueue
*/
public void addResponse(StateEvent stateEvent) {
try {
eventQueue.put(stateEvent);
} catch (InterruptedException e) {
logger.error("put state event : {} error :{}", stateEvent, e);
Thread.currentThread().interrupt();
}
}
/**
* task worker thread
*/
class StateEventResponseWorker extends Thread {
@Override
public void run() {
while (Stopper.isRunning()) {
try {
// if not task , blocking here
StateEvent stateEvent = eventQueue.take();
persist(stateEvent);
} catch (InterruptedException e) {
logger.warn("persist task error", e);
Thread.currentThread().interrupt();
}
}
logger.info("StateEventResponseWorker stopped");
}
}
private void writeResponse(StateEvent stateEvent, ExecutionStatus status) {
Channel channel = stateEvent.getChannel();
if (channel != null) {
StateEventResponseCommand command = new StateEventResponseCommand(status.getCode(), stateEvent.getKey());
channel.writeAndFlush(command.convert2Command());
}
}
private void persist(StateEvent stateEvent) {
try {
if (!this.processInstanceMapper.containsKey(stateEvent.getProcessInstanceId())) {
writeResponse(stateEvent, ExecutionStatus.FAILURE);
return;
}
WorkflowExecuteThread workflowExecuteThread = this.processInstanceMapper.get(stateEvent.getProcessInstanceId());
workflowExecuteThread.addStateEvent(stateEvent);
writeResponse(stateEvent, ExecutionStatus.SUCCESS);
} catch (Exception e) {
logger.error("persist event queue error:", stateEvent.toString(), e);
}
}
public BlockingQueue<StateEvent> getEventQueue() {
return eventQueue;
}
}

17
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskResponseEvent.java

@ -93,13 +93,16 @@ public class TaskResponseEvent {
*/
private Channel channel;
private int processInstanceId;
public static TaskResponseEvent newAck(ExecutionStatus state,
Date startTime,
String workerAddress,
String executePath,
String logPath,
int taskInstanceId,
Channel channel) {
Channel channel,
int processInstanceId) {
TaskResponseEvent event = new TaskResponseEvent();
event.setState(state);
event.setStartTime(startTime);
@ -109,6 +112,7 @@ public class TaskResponseEvent {
event.setTaskInstanceId(taskInstanceId);
event.setEvent(Event.ACK);
event.setChannel(channel);
event.setProcessInstanceId(processInstanceId);
return event;
}
@ -118,7 +122,8 @@ public class TaskResponseEvent {
String appIds,
int taskInstanceId,
String varPool,
Channel channel) {
Channel channel,
int processInstanceId) {
TaskResponseEvent event = new TaskResponseEvent();
event.setState(state);
event.setEndTime(endTime);
@ -128,6 +133,7 @@ public class TaskResponseEvent {
event.setEvent(Event.RESULT);
event.setVarPool(varPool);
event.setChannel(channel);
event.setProcessInstanceId(processInstanceId);
return event;
}
@ -227,4 +233,11 @@ public class TaskResponseEvent {
this.channel = channel;
}
public int getProcessInstanceId() {
return processInstanceId;
}
public void setProcessInstanceId(int processInstanceId) {
this.processInstanceId = processInstanceId;
}
}

35
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskResponseService.java

@ -19,15 +19,19 @@ package org.apache.dolphinscheduler.server.master.processor.queue;
import org.apache.dolphinscheduler.common.enums.Event;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.StateEvent;
import org.apache.dolphinscheduler.common.enums.StateEventType;
import org.apache.dolphinscheduler.common.thread.Stopper;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.remote.command.DBTaskAckCommand;
import org.apache.dolphinscheduler.remote.command.DBTaskResponseCommand;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThread;
import org.apache.dolphinscheduler.service.process.ProcessService;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.LinkedBlockingQueue;
import javax.annotation.PostConstruct;
@ -54,8 +58,7 @@ public class TaskResponseService {
/**
* attemptQueue
*/
private final BlockingQueue<TaskResponseEvent> eventQueue = new LinkedBlockingQueue<>(5000);
private final BlockingQueue<TaskResponseEvent> eventQueue = new LinkedBlockingQueue<>();
/**
* process service
@ -68,15 +71,24 @@ public class TaskResponseService {
*/
private Thread taskResponseWorker;
private ConcurrentHashMap<Integer, WorkflowExecuteThread> processInstanceMapper;
public void init(ConcurrentHashMap<Integer, WorkflowExecuteThread> processInstanceMapper) {
if (this.processInstanceMapper == null) {
this.processInstanceMapper = processInstanceMapper;
}
}
@PostConstruct
public void start() {
this.taskResponseWorker = new TaskResponseWorker();
this.taskResponseWorker.setName("TaskResponseWorker");
this.taskResponseWorker.setName("StateEventResponseWorker");
this.taskResponseWorker.start();
}
@PreDestroy
public void stop() {
try {
this.taskResponseWorker.interrupt();
if (!eventQueue.isEmpty()) {
List<TaskResponseEvent> remainEvents = new ArrayList<>(eventQueue.size());
@ -85,6 +97,9 @@ public class TaskResponseService {
this.persist(event);
}
}
} catch (Exception e) {
logger.error("stop error:", e);
}
}
/**
@ -121,7 +136,7 @@ public class TaskResponseService {
logger.error("persist task error", e);
}
}
logger.info("TaskResponseWorker stopped");
logger.info("StateEventResponseWorker stopped");
}
}
@ -134,10 +149,10 @@ public class TaskResponseService {
Event event = taskResponseEvent.getEvent();
Channel channel = taskResponseEvent.getChannel();
TaskInstance taskInstance = processService.findTaskInstanceById(taskResponseEvent.getTaskInstanceId());
switch (event) {
case ACK:
try {
TaskInstance taskInstance = processService.findTaskInstanceById(taskResponseEvent.getTaskInstanceId());
if (taskInstance != null) {
ExecutionStatus status = taskInstance.getState().typeIsFinished() ? taskInstance.getState() : taskResponseEvent.getState();
processService.changeTaskState(taskInstance, status,
@ -158,7 +173,6 @@ public class TaskResponseService {
break;
case RESULT:
try {
TaskInstance taskInstance = processService.findTaskInstanceById(taskResponseEvent.getTaskInstanceId());
if (taskInstance != null) {
processService.changeTaskState(taskInstance, taskResponseEvent.getState(),
taskResponseEvent.getEndTime(),
@ -180,6 +194,15 @@ public class TaskResponseService {
default:
throw new IllegalArgumentException("invalid event type : " + event);
}
WorkflowExecuteThread workflowExecuteThread = this.processInstanceMapper.get(taskResponseEvent.getProcessInstanceId());
if (workflowExecuteThread != null) {
StateEvent stateEvent = new StateEvent();
stateEvent.setProcessInstanceId(taskResponseEvent.getProcessInstanceId());
stateEvent.setTaskInstanceId(taskResponseEvent.getTaskInstanceId());
stateEvent.setExecutionStatus(taskResponseEvent.getState());
stateEvent.setType(StateEventType.TASK_STATE_CHANGE);
workflowExecuteThread.addStateEvent(stateEvent);
}
}
public BlockingQueue<TaskResponseEvent> getEventQueue() {

58
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistryClient.java

@ -25,6 +25,8 @@ 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.NodeType;
import org.apache.dolphinscheduler.common.enums.StateEvent;
import org.apache.dolphinscheduler.common.enums.StateEventType;
import org.apache.dolphinscheduler.common.model.Server;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.common.utils.DateUtils;
@ -36,6 +38,7 @@ import org.apache.dolphinscheduler.remote.utils.NamedThreadFactory;
import org.apache.dolphinscheduler.server.builder.TaskExecutionContextBuilder;
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThread;
import org.apache.dolphinscheduler.server.registry.HeartBeatTask;
import org.apache.dolphinscheduler.server.utils.ProcessUtils;
import org.apache.dolphinscheduler.service.process.ProcessService;
@ -45,12 +48,11 @@ import org.apache.dolphinscheduler.spi.register.RegistryConnectState;
import java.util.Date;
import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import javax.annotation.PostConstruct;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
@ -90,6 +92,8 @@ public class MasterRegistryClient {
*/
private ScheduledExecutorService heartBeatExecutor;
private ConcurrentHashMap<Integer, WorkflowExecuteThread> processInstanceExecMaps;
/**
* master start time
*/
@ -97,6 +101,13 @@ public class MasterRegistryClient {
private String localNodePath;
public void init(ConcurrentHashMap<Integer, WorkflowExecuteThread> processInstanceExecMaps) {
this.startTime = DateUtils.dateToString(new Date());
this.registryClient = RegistryClient.getInstance();
this.heartBeatExecutor = Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("HeartBeatExecutor"));
this.processInstanceExecMaps = processInstanceExecMaps;
}
public void start() {
String nodeLock = registryClient.getMasterStartUpLockPath();
try {
@ -182,7 +193,7 @@ public class MasterRegistryClient {
failoverMaster(serverHost);
break;
case WORKER:
failoverWorker(serverHost, true);
failoverWorker(serverHost, true, true);
break;
default:
break;
@ -265,7 +276,7 @@ public class MasterRegistryClient {
* @param workerHost worker host
* @param needCheckWorkerAlive need check worker alive
*/
private void failoverWorker(String workerHost, boolean needCheckWorkerAlive) {
private void failoverWorker(String workerHost, boolean needCheckWorkerAlive, boolean checkOwner) {
logger.info("start worker[{}] failover ...", workerHost);
List<TaskInstance> needFailoverTaskInstanceList = processService.queryNeedFailoverTaskInstances(workerHost);
for (TaskInstance taskInstance : needFailoverTaskInstanceList) {
@ -276,9 +287,17 @@ public class MasterRegistryClient {
}
ProcessInstance processInstance = processService.findProcessInstanceDetailById(taskInstance.getProcessInstanceId());
if (processInstance != null) {
taskInstance.setProcessInstance(processInstance);
if (workerHost == null
|| !checkOwner
|| processInstance.getHost().equalsIgnoreCase(workerHost)) {
// only failover the task owned myself if worker down.
// failover master need handle worker at the same time
if (processInstance == null) {
logger.error("failover error, the process {} of task {} do not exists.",
taskInstance.getProcessInstanceId(), taskInstance.getId());
continue;
}
taskInstance.setProcessInstance(processInstance);
TaskExecutionContext taskExecutionContext = TaskExecutionContextBuilder.get()
.buildTaskInstanceRelatedInfo(taskInstance)
@ -289,6 +308,18 @@ public class MasterRegistryClient {
taskInstance.setState(ExecutionStatus.NEED_FAULT_TOLERANCE);
processService.saveTaskInstance(taskInstance);
if (!processInstanceExecMaps.containsKey(processInstance.getId())) {
return;
}
WorkflowExecuteThread workflowExecuteThreadNotify = processInstanceExecMaps.get(processInstance.getId());
StateEvent stateEvent = new StateEvent();
stateEvent.setTaskInstanceId(taskInstance.getId());
stateEvent.setType(StateEventType.TASK_STATE_CHANGE);
stateEvent.setProcessInstanceId(processInstance.getId());
stateEvent.setExecutionStatus(taskInstance.getState());
workflowExecuteThreadNotify.addStateEvent(stateEvent);
}
}
logger.info("end worker[{}] failover ...", workerHost);
}
@ -312,6 +343,7 @@ public class MasterRegistryClient {
}
processService.processNeedFailoverProcessInstances(processInstance);
}
failoverWorker(masterHost, true, false);
logger.info("master failover end");
}
@ -324,12 +356,6 @@ public class MasterRegistryClient {
registryClient.releaseLock(registryClient.getMasterLockPath());
}
@PostConstruct
public void init() {
this.startTime = DateUtils.dateToString(new Date());
this.registryClient = RegistryClient.getInstance();
this.heartBeatExecutor = Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("HeartBeatExecutor"));
}
/**
* registry
@ -337,8 +363,6 @@ public class MasterRegistryClient {
public void registry() {
String address = NetUtils.getAddr(masterConfig.getListenPort());
localNodePath = getMasterPath();
registryClient.persistEphemeral(localNodePath, "");
registryClient.addConnectionStateListener(new MasterRegistryConnectStateListener());
int masterHeartbeatInterval = masterConfig.getMasterHeartbeatInterval();
HeartBeatTask heartBeatTask = new HeartBeatTask(startTime,
masterConfig.getMasterMaxCpuloadAvg(),
@ -347,6 +371,8 @@ public class MasterRegistryClient {
Constants.MASTER_TYPE,
registryClient);
registryClient.persistEphemeral(localNodePath, heartBeatTask.heartBeatInfo());
registryClient.addConnectionStateListener(new MasterRegistryConnectStateListener());
this.heartBeatExecutor.scheduleAtFixedRate(heartBeatTask, masterHeartbeatInterval, masterHeartbeatInterval, TimeUnit.SECONDS);
logger.info("master node : {} registry to ZK successfully with heartBeatInterval : {}s", address, masterHeartbeatInterval);
@ -369,6 +395,7 @@ public class MasterRegistryClient {
* remove registry info
*/
public void unRegistry() {
try {
String address = getLocalAddress();
String localNodePath = getMasterPath();
registryClient.remove(localNodePath);
@ -376,6 +403,9 @@ public class MasterRegistryClient {
heartBeatExecutor.shutdown();
logger.info("heartbeat executor shutdown");
registryClient.close();
} catch (Exception e) {
logger.error("remove registry path exception ", e);
}
}
/**

62
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/registry/ServerNodeManager.java

@ -22,17 +22,21 @@ import static org.apache.dolphinscheduler.common.Constants.REGISTRY_DOLPHINSCHED
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.NodeType;
import org.apache.dolphinscheduler.common.model.Server;
import org.apache.dolphinscheduler.common.utils.NetUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.dao.AlertDao;
import org.apache.dolphinscheduler.dao.entity.WorkerGroup;
import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper;
import org.apache.dolphinscheduler.remote.utils.NamedThreadFactory;
import org.apache.dolphinscheduler.service.queue.MasterPriorityQueue;
import org.apache.dolphinscheduler.service.registry.RegistryClient;
import org.apache.dolphinscheduler.spi.register.DataChangeEvent;
import org.apache.dolphinscheduler.spi.register.SubscribeListener;
import org.apache.commons.collections.CollectionUtils;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@ -108,12 +112,26 @@ public class ServerNodeManager implements InitializingBean {
@Autowired
private WorkerGroupMapper workerGroupMapper;
private MasterPriorityQueue masterPriorityQueue = new MasterPriorityQueue();
/**
* alert dao
*/
@Autowired
private AlertDao alertDao;
public static volatile List<Integer> SLOT_LIST = new ArrayList<>();
public static volatile Integer MASTER_SIZE = 0;
public static Integer getSlot() {
if (SLOT_LIST.size() > 0) {
return SLOT_LIST.get(0);
}
return 0;
}
/**
* init listener
*
@ -143,12 +161,11 @@ public class ServerNodeManager implements InitializingBean {
/**
* load nodes from zookeeper
*/
private void load() {
public void load() {
/**
* master nodes from zookeeper
*/
Set<String> initMasterNodes = registryClient.getMasterNodesDirectly();
syncMasterNodes(initMasterNodes);
updateMasterNodes();
/**
* worker group nodes from zookeeper
@ -241,13 +258,11 @@ public class ServerNodeManager implements InitializingBean {
try {
if (dataChangeEvent.equals(DataChangeEvent.ADD)) {
logger.info("master node : {} added.", path);
Set<String> currentNodes = registryClient.getMasterNodesDirectly();
syncMasterNodes(currentNodes);
updateMasterNodes();
}
if (dataChangeEvent.equals(DataChangeEvent.REMOVE)) {
logger.info("master node : {} down.", path);
Set<String> currentNodes = registryClient.getMasterNodesDirectly();
syncMasterNodes(currentNodes);
updateMasterNodes();
alertDao.sendServerStopedAlert(1, path, "MASTER");
}
} catch (Exception ex) {
@ -257,6 +272,23 @@ public class ServerNodeManager implements InitializingBean {
}
}
private void updateMasterNodes() {
SLOT_LIST.clear();
this.masterNodes.clear();
String nodeLock = registryClient.getMasterLockPath();
try {
registryClient.getLock(nodeLock);
Set<String> currentNodes = registryClient.getMasterNodesDirectly();
List<Server> masterNodes = registryClient.getServerList(NodeType.MASTER);
syncMasterNodes(currentNodes, masterNodes);
} catch (Exception e) {
logger.error("update master nodes error", e);
} finally {
registryClient.releaseLock(nodeLock);
}
}
/**
* get master nodes
*
@ -275,12 +307,22 @@ public class ServerNodeManager implements InitializingBean {
* sync master nodes
*
* @param nodes master nodes
* @param masterNodes
*/
private void syncMasterNodes(Set<String> nodes) {
private void syncMasterNodes(Set<String> nodes, List<Server> masterNodes) {
masterLock.lock();
try {
masterNodes.clear();
masterNodes.addAll(nodes);
this.masterNodes.addAll(nodes);
this.masterPriorityQueue.clear();
this.masterPriorityQueue.putList(masterNodes);
int index = masterPriorityQueue.getIndex(NetUtils.getHost());
if (index >= 0) {
MASTER_SIZE = nodes.size();
SLOT_LIST.add(masterPriorityQueue.getIndex(NetUtils.getHost()));
}
logger.info("update master nodes, master size: {}, slot: {}",
MASTER_SIZE, SLOT_LIST.toString()
);
} finally {
masterLock.unlock();
}

195
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/EventExecuteService.java

@ -0,0 +1,195 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.StateEvent;
import org.apache.dolphinscheduler.common.enums.StateEventType;
import org.apache.dolphinscheduler.common.thread.Stopper;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.common.utils.NetUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.remote.command.StateEventChangeCommand;
import org.apache.dolphinscheduler.remote.processor.StateEventCallbackService;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
@Service
public class EventExecuteService extends Thread {
private static final Logger logger = LoggerFactory.getLogger(EventExecuteService.class);
/**
* dolphinscheduler database interface
*/
@Autowired
private ProcessService processService;
@Autowired
private MasterConfig masterConfig;
private ExecutorService eventExecService;
/**
*
*/
private StateEventCallbackService stateEventCallbackService;
private ConcurrentHashMap<Integer, WorkflowExecuteThread> processInstanceExecMaps;
private ConcurrentHashMap<String, WorkflowExecuteThread> eventHandlerMap = new ConcurrentHashMap();
ListeningExecutorService listeningExecutorService;
public void init(ConcurrentHashMap<Integer, WorkflowExecuteThread> processInstanceExecMaps) {
eventExecService = ThreadUtils.newDaemonFixedThreadExecutor("MasterEventExecution", masterConfig.getMasterExecThreads());
this.processInstanceExecMaps = processInstanceExecMaps;
listeningExecutorService = MoreExecutors.listeningDecorator(eventExecService);
this.stateEventCallbackService = SpringApplicationContext.getBean(StateEventCallbackService.class);
}
@Override
public synchronized void start() {
super.setName("EventServiceStarted");
super.start();
}
public void close() {
eventExecService.shutdown();
logger.info("event service stopped...");
}
@Override
public void run() {
logger.info("Event service started");
while (Stopper.isRunning()) {
try {
eventHandler();
} catch (Exception e) {
logger.error("Event service thread error", e);
}
}
}
private void eventHandler() {
for (WorkflowExecuteThread workflowExecuteThread : this.processInstanceExecMaps.values()) {
if (workflowExecuteThread.eventSize() == 0
|| StringUtils.isEmpty(workflowExecuteThread.getKey())
|| eventHandlerMap.containsKey(workflowExecuteThread.getKey())) {
continue;
}
int processInstanceId = workflowExecuteThread.getProcessInstance().getId();
logger.info("handle process instance : {} events, count:{}",
processInstanceId,
workflowExecuteThread.eventSize());
logger.info("already exists handler process size:{}", this.eventHandlerMap.size());
eventHandlerMap.put(workflowExecuteThread.getKey(), workflowExecuteThread);
ListenableFuture future = this.listeningExecutorService.submit(workflowExecuteThread);
FutureCallback futureCallback = new FutureCallback() {
@Override
public void onSuccess(Object o) {
if (workflowExecuteThread.workFlowFinish()) {
processInstanceExecMaps.remove(processInstanceId);
notifyProcessChanged();
logger.info("process instance {} finished.", processInstanceId);
}
if (workflowExecuteThread.getProcessInstance().getId() != processInstanceId) {
processInstanceExecMaps.remove(processInstanceId);
processInstanceExecMaps.put(workflowExecuteThread.getProcessInstance().getId(), workflowExecuteThread);
}
eventHandlerMap.remove(workflowExecuteThread.getKey());
}
private void notifyProcessChanged() {
Map<ProcessInstance, TaskInstance> fatherMaps
= processService.notifyProcessList(processInstanceId, 0);
for (ProcessInstance processInstance : fatherMaps.keySet()) {
String address = NetUtils.getAddr(masterConfig.getListenPort());
if (processInstance.getHost().equalsIgnoreCase(address)) {
notifyMyself(processInstance, fatherMaps.get(processInstance));
} else {
notifyProcess(processInstance, fatherMaps.get(processInstance));
}
}
}
private void notifyMyself(ProcessInstance processInstance, TaskInstance taskInstance) {
logger.info("notify process {} task {} state change", processInstance.getId(), taskInstance.getId());
if (!processInstanceExecMaps.containsKey(processInstance.getId())) {
return;
}
WorkflowExecuteThread workflowExecuteThreadNotify = processInstanceExecMaps.get(processInstance.getId());
StateEvent stateEvent = new StateEvent();
stateEvent.setTaskInstanceId(taskInstance.getId());
stateEvent.setType(StateEventType.TASK_STATE_CHANGE);
stateEvent.setProcessInstanceId(processInstance.getId());
stateEvent.setExecutionStatus(ExecutionStatus.RUNNING_EXECUTION);
workflowExecuteThreadNotify.addStateEvent(stateEvent);
}
private void notifyProcess(ProcessInstance processInstance, TaskInstance taskInstance) {
String host = processInstance.getHost();
if (StringUtils.isEmpty(host)) {
logger.info("process {} host is empty, cannot notify task {} now.",
processInstance.getId(), taskInstance.getId());
return;
}
String address = host.split(":")[0];
int port = Integer.parseInt(host.split(":")[1]);
logger.info("notify process {} task {} state change, host:{}",
processInstance.getId(), taskInstance.getId(), host);
StateEventChangeCommand stateEventChangeCommand = new StateEventChangeCommand(
processInstanceId, 0, workflowExecuteThread.getProcessInstance().getState(), processInstance.getId(), taskInstance.getId()
);
stateEventCallbackService.sendResult(address, port, stateEventChangeCommand.convert2Command());
}
@Override
public void onFailure(Throwable throwable) {
}
};
Futures.addCallback(future, futureCallback, this.listeningExecutorService);
}
}
}

92
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java

@ -24,18 +24,21 @@ import org.apache.dolphinscheduler.common.utils.NetUtils;
import org.apache.dolphinscheduler.common.utils.OSUtils;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.remote.NettyRemotingClient;
import org.apache.dolphinscheduler.remote.config.NettyClientConfig;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.dispatch.executor.NettyExecutorManager;
import org.apache.dolphinscheduler.server.master.registry.MasterRegistryClient;
import org.apache.dolphinscheduler.server.master.registry.ServerNodeManager;
import org.apache.dolphinscheduler.service.alert.ProcessAlertManager;
import org.apache.dolphinscheduler.service.process.ProcessService;
import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import javax.annotation.PostConstruct;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
@ -81,26 +84,42 @@ public class MasterSchedulerService extends Thread {
*/
private NettyRemotingClient nettyRemotingClient;
@Autowired
NettyExecutorManager nettyExecutorManager;
/**
* master exec service
*/
private ThreadPoolExecutor masterExecService;
private ConcurrentHashMap<Integer, WorkflowExecuteThread> processInstanceExecMaps;
ConcurrentHashMap<Integer, ProcessInstance> processTimeoutCheckList = new ConcurrentHashMap<>();
ConcurrentHashMap<Integer, TaskInstance> taskTimeoutCheckList = new ConcurrentHashMap<>();
private StateWheelExecuteThread stateWheelExecuteThread;
/**
* constructor of MasterSchedulerService
*/
@PostConstruct
public void init() {
public void init(ConcurrentHashMap<Integer, WorkflowExecuteThread> processInstanceExecMaps) {
this.processInstanceExecMaps = processInstanceExecMaps;
this.masterExecService = (ThreadPoolExecutor) ThreadUtils.newDaemonFixedThreadExecutor("Master-Exec-Thread", masterConfig.getMasterExecThreads());
NettyClientConfig clientConfig = new NettyClientConfig();
this.nettyRemotingClient = new NettyRemotingClient(clientConfig);
stateWheelExecuteThread = new StateWheelExecuteThread(processTimeoutCheckList,
taskTimeoutCheckList,
this.processInstanceExecMaps,
masterConfig.getStateWheelInterval() * Constants.SLEEP_TIME_MILLIS);
}
@Override
public synchronized void start() {
super.setName("MasterSchedulerService");
super.start();
this.stateWheelExecuteThread.start();
}
public void close() {
@ -131,10 +150,6 @@ public class MasterSchedulerService extends Thread {
Thread.sleep(Constants.SLEEP_TIME_MILLIS);
continue;
}
// todo 串行执行 为何还需要判断状态?
/* if (zkMasterClient.getZkClient().getState() == CuratorFrameworkState.STARTED) {
scheduleProcess();
}*/
scheduleProcess();
} catch (Exception e) {
logger.error("master scheduler thread error", e);
@ -142,31 +157,39 @@ public class MasterSchedulerService extends Thread {
}
}
/**
* 1. get command by slot
* 2. donot handle command if slot is empty
*
* @throws Exception
*/
private void scheduleProcess() throws Exception {
try {
masterRegistryClient.blockAcquireMutex();
int activeCount = masterExecService.getActiveCount();
// make sure to scan and delete command table in one transaction
Command command = processService.findOneCommand();
Command command = findOneCommand();
if (command != null) {
logger.info("find one command: id: {}, type: {}", command.getId(), command.getCommandType());
try {
ProcessInstance processInstance = processService.handleCommand(logger,
getLocalAddress(),
this.masterConfig.getMasterExecThreads() - activeCount, command);
if (processInstance != null) {
logger.info("start master exec thread , split DAG ...");
masterExecService.execute(
new MasterExecThread(
WorkflowExecuteThread workflowExecuteThread = new WorkflowExecuteThread(
processInstance
, processService
, nettyRemotingClient
, nettyExecutorManager
, processAlertManager
, masterConfig));
, masterConfig
, taskTimeoutCheckList);
this.processInstanceExecMaps.put(processInstance.getId(), workflowExecuteThread);
if (processInstance.getTimeout() > 0) {
this.processTimeoutCheckList.put(processInstance.getId(), processInstance);
}
logger.info("command {} process {} start...",
command.getId(), processInstance.getId());
masterExecService.execute(workflowExecuteThread);
}
} catch (Exception e) {
logger.error("scan command error ", e);
@ -176,9 +199,36 @@ public class MasterSchedulerService extends Thread {
//indicate that no command ,sleep for 1s
Thread.sleep(Constants.SLEEP_TIME_MILLIS);
}
} finally {
masterRegistryClient.releaseLock();
}
private Command findOneCommand() {
int pageNumber = 0;
Command result = null;
while (Stopper.isRunning()) {
if (ServerNodeManager.MASTER_SIZE == 0) {
return null;
}
List<Command> commandList = processService.findCommandPage(ServerNodeManager.MASTER_SIZE, pageNumber);
if (commandList.size() == 0) {
return null;
}
for (Command command : commandList) {
int slot = ServerNodeManager.getSlot();
if (ServerNodeManager.MASTER_SIZE != 0
&& command.getId() % ServerNodeManager.MASTER_SIZE == slot) {
result = command;
break;
}
}
if (result != null) {
logger.info("find command {}, slot:{} :",
result.getId(),
ServerNodeManager.getSlot());
break;
}
pageNumber += 1;
}
return result;
}
private String getLocalAddress() {

154
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/StateWheelExecuteThread.java

@ -0,0 +1,154 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.StateEvent;
import org.apache.dolphinscheduler.common.enums.StateEventType;
import org.apache.dolphinscheduler.common.enums.TimeoutFlag;
import org.apache.dolphinscheduler.common.thread.Stopper;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.hadoop.util.ThreadUtil;
import java.util.concurrent.ConcurrentHashMap;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* 1. timeout check wheel
* 2. dependent task check wheel
*/
public class StateWheelExecuteThread extends Thread {
private static final Logger logger = LoggerFactory.getLogger(StateWheelExecuteThread.class);
ConcurrentHashMap<Integer, ProcessInstance> processInstanceCheckList;
ConcurrentHashMap<Integer, TaskInstance> taskInstanceCheckList;
private ConcurrentHashMap<Integer, WorkflowExecuteThread> processInstanceExecMaps;
private int stateCheckIntervalSecs;
public StateWheelExecuteThread(ConcurrentHashMap<Integer, ProcessInstance> processInstances,
ConcurrentHashMap<Integer, TaskInstance> taskInstances,
ConcurrentHashMap<Integer, WorkflowExecuteThread> processInstanceExecMaps,
int stateCheckIntervalSecs) {
this.processInstanceCheckList = processInstances;
this.taskInstanceCheckList = taskInstances;
this.processInstanceExecMaps = processInstanceExecMaps;
this.stateCheckIntervalSecs = stateCheckIntervalSecs;
}
@Override
public void run() {
logger.info("state wheel thread start");
while (Stopper.isRunning()) {
try {
checkProcess();
checkTask();
} catch (Exception e) {
logger.error("state wheel thread check error:", e);
}
ThreadUtil.sleepAtLeastIgnoreInterrupts(stateCheckIntervalSecs);
}
}
public boolean addProcess(ProcessInstance processInstance) {
this.processInstanceCheckList.put(processInstance.getId(), processInstance);
return true;
}
public boolean addTask(TaskInstance taskInstance) {
this.taskInstanceCheckList.put(taskInstance.getId(), taskInstance);
return true;
}
private void checkTask() {
if (taskInstanceCheckList.isEmpty()) {
return;
}
for (TaskInstance taskInstance : this.taskInstanceCheckList.values()) {
if (TimeoutFlag.OPEN == taskInstance.getTaskDefine().getTimeoutFlag()) {
long timeRemain = DateUtils.getRemainTime(taskInstance.getStartTime(), taskInstance.getTaskDefine().getTimeout() * Constants.SEC_2_MINUTES_TIME_UNIT);
if (0 <= timeRemain && processTimeout(taskInstance)) {
taskInstanceCheckList.remove(taskInstance.getId());
return;
}
}
if (taskInstance.isSubProcess() || taskInstance.isDependTask()) {
processDependCheck(taskInstance);
}
}
}
private void checkProcess() {
if (processInstanceCheckList.isEmpty()) {
return;
}
for (ProcessInstance processInstance : this.processInstanceCheckList.values()) {
long timeRemain = DateUtils.getRemainTime(processInstance.getStartTime(), processInstance.getTimeout() * Constants.SEC_2_MINUTES_TIME_UNIT);
if (0 <= timeRemain && processTimeout(processInstance)) {
processInstanceCheckList.remove(processInstance.getId());
}
}
}
private void putEvent(StateEvent stateEvent) {
if (!processInstanceExecMaps.containsKey(stateEvent.getProcessInstanceId())) {
return;
}
WorkflowExecuteThread workflowExecuteThread = this.processInstanceExecMaps.get(stateEvent.getProcessInstanceId());
workflowExecuteThread.addStateEvent(stateEvent);
}
private boolean processDependCheck(TaskInstance taskInstance) {
StateEvent stateEvent = new StateEvent();
stateEvent.setType(StateEventType.TASK_STATE_CHANGE);
stateEvent.setProcessInstanceId(taskInstance.getProcessInstanceId());
stateEvent.setTaskInstanceId(taskInstance.getId());
stateEvent.setExecutionStatus(ExecutionStatus.RUNNING_EXECUTION);
putEvent(stateEvent);
return true;
}
private boolean processTimeout(TaskInstance taskInstance) {
StateEvent stateEvent = new StateEvent();
stateEvent.setType(StateEventType.TASK_TIMEOUT);
stateEvent.setProcessInstanceId(taskInstance.getProcessInstanceId());
stateEvent.setTaskInstanceId(taskInstance.getId());
putEvent(stateEvent);
return true;
}
private boolean processTimeout(ProcessInstance processInstance) {
StateEvent stateEvent = new StateEvent();
stateEvent.setType(StateEventType.PROCESS_TIMEOUT);
stateEvent.setProcessInstanceId(processInstance.getId());
putEvent(stateEvent);
return true;
}
}

676
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java → dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThread.java

@ -32,28 +32,39 @@ import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.FailureStrategy;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.Priority;
import org.apache.dolphinscheduler.common.enums.StateEvent;
import org.apache.dolphinscheduler.common.enums.StateEventType;
import org.apache.dolphinscheduler.common.enums.TaskDependType;
import org.apache.dolphinscheduler.common.enums.TaskTimeoutStrategy;
import org.apache.dolphinscheduler.common.enums.TimeoutFlag;
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.ProcessDag;
import org.apache.dolphinscheduler.common.process.Property;
import org.apache.dolphinscheduler.common.thread.Stopper;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
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.NetUtils;
import org.apache.dolphinscheduler.common.utils.OSUtils;
import org.apache.dolphinscheduler.common.utils.ParameterUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.dao.entity.Environment;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.ProjectUser;
import org.apache.dolphinscheduler.dao.entity.Schedule;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.utils.DagHelper;
import org.apache.dolphinscheduler.remote.NettyRemotingClient;
import org.apache.dolphinscheduler.remote.command.HostUpdateCommand;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.dispatch.executor.NettyExecutorManager;
import org.apache.dolphinscheduler.server.master.runner.task.ITaskProcessor;
import org.apache.dolphinscheduler.server.master.runner.task.TaskAction;
import org.apache.dolphinscheduler.server.master.runner.task.TaskProcessorFactory;
import org.apache.dolphinscheduler.service.alert.ProcessAlertManager;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.service.quartz.cron.CronUtils;
@ -67,29 +78,32 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.collect.HashBasedTable;
import com.google.common.collect.Lists;
import com.google.common.collect.Table;
/**
* master exec thread,split dag
*/
public class MasterExecThread implements Runnable {
public class WorkflowExecuteThread implements Runnable {
/**
* logger of MasterExecThread
* logger of WorkflowExecuteThread
*/
private static final Logger logger = LoggerFactory.getLogger(MasterExecThread.class);
private static final Logger logger = LoggerFactory.getLogger(WorkflowExecuteThread.class);
/**
* runing TaskNode
*/
private final Map<MasterBaseTaskExecThread, Future<Boolean>> activeTaskNode = new ConcurrentHashMap<>();
private final Map<Integer, ITaskProcessor> activeTaskProcessorMaps = new ConcurrentHashMap<>();
/**
* task exec service
*/
@ -166,7 +180,8 @@ public class MasterExecThread implements Runnable {
/**
*
*/
private NettyRemotingClient nettyRemotingClient;
private NettyExecutorManager nettyExecutorManager;
/**
* submit post node
*
@ -174,18 +189,29 @@ public class MasterExecThread implements Runnable {
*/
private Map<String, Object> propToValue = new ConcurrentHashMap<>();
private ConcurrentLinkedQueue<StateEvent> stateEvents = new ConcurrentLinkedQueue<>();
private List<Date> complementListDate = Lists.newLinkedList();
private Table<Integer, Long, TaskInstance> taskInstanceHashMap = HashBasedTable.create();
private ProcessDefinition processDefinition;
private String key;
private ConcurrentHashMap<Integer, TaskInstance> taskTimeoutCheckList;
/**
* constructor of MasterExecThread
* constructor of WorkflowExecuteThread
*
* @param processInstance processInstance
* @param processService processService
* @param nettyRemotingClient nettyRemotingClient
* @param nettyExecutorManager nettyExecutorManager
*/
public MasterExecThread(ProcessInstance processInstance
public WorkflowExecuteThread(ProcessInstance processInstance
, ProcessService processService
, NettyRemotingClient nettyRemotingClient
, NettyExecutorManager nettyExecutorManager
, ProcessAlertManager processAlertManager
, MasterConfig masterConfig) {
, MasterConfig masterConfig
, ConcurrentHashMap<Integer, TaskInstance> taskTimeoutCheckList) {
this.processService = processService;
this.processInstance = processInstance;
@ -193,135 +219,226 @@ public class MasterExecThread implements Runnable {
int masterTaskExecNum = masterConfig.getMasterExecTaskNum();
this.taskExecService = ThreadUtils.newDaemonFixedThreadExecutor("Master-Task-Exec-Thread",
masterTaskExecNum);
this.nettyRemotingClient = nettyRemotingClient;
this.nettyExecutorManager = nettyExecutorManager;
this.processAlertManager = processAlertManager;
this.taskTimeoutCheckList = taskTimeoutCheckList;
}
@Override
public void run() {
// process instance is null
if (processInstance == null) {
logger.info("process instance is not exists");
return;
try {
startProcess();
handleEvents();
} catch (Exception e) {
logger.error("handler error:", e);
}
// check to see if it's done
if (processInstance.getState().typeIsFinished()) {
logger.info("process instance is done : {}", processInstance.getId());
return;
}
private void handleEvents() {
while (this.stateEvents.size() > 0) {
try {
if (processInstance.isComplementData() && Flag.NO == processInstance.getIsSubProcess()) {
// sub process complement data
executeComplementProcess();
} else {
// execute flow
executeProcess();
StateEvent stateEvent = this.stateEvents.peek();
if (stateEventHandler(stateEvent)) {
this.stateEvents.remove(stateEvent);
}
} catch (Exception e) {
logger.error("master exec thread exception", e);
logger.error("process execute failed, process id:{}", processInstance.getId());
processInstance.setState(ExecutionStatus.FAILURE);
processInstance.setEndTime(new Date());
processService.updateProcessInstance(processInstance);
} finally {
taskExecService.shutdown();
logger.error("state handle error:", e);
}
}
}
/**
* execute process
*
* @throws Exception exception
*/
private void executeProcess() throws Exception {
prepareProcess();
runProcess();
endProcess();
public String getKey() {
if (StringUtils.isNotEmpty(key)
|| this.processDefinition == null) {
return key;
}
/**
* execute complement process
*
* @throws Exception exception
*/
private void executeComplementProcess() throws Exception {
key = String.format("{}_{}_{}",
this.processDefinition.getCode(),
this.processDefinition.getVersion(),
this.processInstance.getId());
return key;
}
Map<String, String> cmdParam = JSONUtils.toMap(processInstance.getCommandParam());
public boolean addStateEvent(StateEvent stateEvent) {
if (processInstance.getId() != stateEvent.getProcessInstanceId()) {
logger.info("state event would be abounded :{}", stateEvent.toString());
return false;
}
this.stateEvents.add(stateEvent);
return true;
}
Date startDate = DateUtils.getScheduleDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_START_DATE));
Date endDate = DateUtils.getScheduleDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_END_DATE));
processService.saveProcessInstance(processInstance);
public int eventSize() {
return this.stateEvents.size();
}
// get schedules
int processDefinitionId = processInstance.getProcessDefinition().getId();
List<Schedule> schedules = processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId);
List<Date> listDate = Lists.newLinkedList();
if (!CollectionUtils.isEmpty(schedules)) {
for (Schedule schedule : schedules) {
listDate.addAll(CronUtils.getSelfFireDateList(startDate, endDate, schedule.getCrontab()));
}
}
// get first fire date
Iterator<Date> iterator = null;
Date scheduleDate;
if (!CollectionUtils.isEmpty(listDate)) {
iterator = listDate.iterator();
scheduleDate = iterator.next();
processInstance.setScheduleTime(scheduleDate);
processService.updateProcessInstance(processInstance);
public ProcessInstance getProcessInstance() {
return this.processInstance;
}
private boolean stateEventHandler(StateEvent stateEvent) {
logger.info("process event: {}", stateEvent.toString());
if (!checkStateEvent(stateEvent)) {
return false;
}
boolean result = false;
switch (stateEvent.getType()) {
case PROCESS_STATE_CHANGE:
result = processStateChangeHandler(stateEvent);
break;
case TASK_STATE_CHANGE:
result = taskStateChangeHandler(stateEvent);
break;
case PROCESS_TIMEOUT:
result = processTimeout();
break;
case TASK_TIMEOUT:
result = taskTimeout(stateEvent);
break;
default:
break;
}
if (result) {
this.stateEvents.remove(stateEvent);
}
return result;
}
private boolean taskTimeout(StateEvent stateEvent) {
if (taskInstanceHashMap.containsRow(stateEvent.getTaskInstanceId())) {
return true;
}
TaskInstance taskInstance = taskInstanceHashMap
.row(stateEvent.getTaskInstanceId())
.values()
.iterator().next();
if (TimeoutFlag.CLOSE == taskInstance.getTaskDefine().getTimeoutFlag()) {
return true;
}
TaskTimeoutStrategy taskTimeoutStrategy = taskInstance.getTaskDefine().getTimeoutNotifyStrategy();
if (TaskTimeoutStrategy.FAILED == taskTimeoutStrategy) {
ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(stateEvent.getTaskInstanceId());
taskProcessor.action(TaskAction.TIMEOUT);
return false;
} else {
scheduleDate = processInstance.getScheduleTime();
if (scheduleDate == null) {
scheduleDate = startDate;
processAlertManager.sendTaskTimeoutAlert(processInstance, taskInstance, taskInstance.getTaskDefine());
return true;
}
}
private boolean processTimeout() {
this.processAlertManager.sendProcessTimeoutAlert(this.processInstance, this.processDefinition);
return true;
}
while (Stopper.isRunning()) {
logger.info("process {} start to complement {} data", processInstance.getId(), DateUtils.dateToString(scheduleDate));
// prepare dag and other info
prepareProcess();
private boolean taskStateChangeHandler(StateEvent stateEvent) {
TaskInstance task = processService.findTaskInstanceById(stateEvent.getTaskInstanceId());
if (stateEvent.getExecutionStatus().typeIsFinished()) {
taskFinished(task);
} else if (activeTaskProcessorMaps.containsKey(stateEvent.getTaskInstanceId())) {
ITaskProcessor iTaskProcessor = activeTaskProcessorMaps.get(stateEvent.getTaskInstanceId());
iTaskProcessor.run();
if (dag == null) {
logger.error("process {} dag is null, please check out parameters",
processInstance.getId());
processInstance.setState(ExecutionStatus.SUCCESS);
processService.updateProcessInstance(processInstance);
if (iTaskProcessor.taskState().typeIsFinished()) {
task = processService.findTaskInstanceById(stateEvent.getTaskInstanceId());
taskFinished(task);
}
} else {
logger.error("state handler error: {}", stateEvent.toString());
}
return true;
}
private void taskFinished(TaskInstance task) {
logger.info("work flow {} task {} state:{} ",
processInstance.getId(),
task.getId(),
task.getState());
if (task.taskCanRetry()) {
addTaskToStandByList(task);
return;
}
ProcessInstance processInstance = processService.findProcessInstanceById(this.processInstance.getId());
completeTaskList.put(task.getName(), task);
activeTaskProcessorMaps.remove(task.getId());
taskTimeoutCheckList.remove(task.getId());
if (task.getState().typeIsSuccess()) {
processInstance.setVarPool(task.getVarPool());
processService.saveProcessInstance(processInstance);
submitPostNode(task.getName());
} else if (task.getState().typeIsFailure()) {
if (task.isConditionsTask()
|| DagHelper.haveConditionsAfterNode(task.getName(), dag)) {
submitPostNode(task.getName());
} else {
errorTaskList.put(task.getName(), task);
if (processInstance.getFailureStrategy() == FailureStrategy.END) {
killAllTasks();
}
}
}
this.updateProcessInstanceState();
}
// execute process ,waiting for end
runProcess();
private boolean checkStateEvent(StateEvent stateEvent) {
if (this.processInstance.getId() != stateEvent.getProcessInstanceId()) {
logger.error("mismatch process instance id: {}, state event:{}",
this.processInstance.getId(),
stateEvent.toString());
return false;
}
return true;
}
private boolean processStateChangeHandler(StateEvent stateEvent) {
try {
logger.info("process:{} state {} change to {}", processInstance.getId(), processInstance.getState(), stateEvent.getExecutionStatus());
processInstance = processService.findProcessInstanceById(this.processInstance.getId());
if (processComplementData()) {
return true;
}
if (stateEvent.getExecutionStatus().typeIsFinished()) {
endProcess();
// process instance failure ,no more complements
if (!processInstance.getState().typeIsSuccess()) {
logger.info("process {} state {}, complement not completely!", processInstance.getId(), processInstance.getState());
break;
}
// current process instance success ,next execute
if (null == iterator) {
// loop by day
scheduleDate = DateUtils.getSomeDay(scheduleDate, 1);
if (scheduleDate.after(endDate)) {
// all success
logger.info("process {} complement completely!", processInstance.getId());
break;
if (stateEvent.getExecutionStatus() == ExecutionStatus.READY_STOP) {
killAllTasks();
}
} else {
// loop by schedule date
if (!iterator.hasNext()) {
// all success
logger.info("process {} complement completely!", processInstance.getId());
break;
return true;
} catch (Exception e) {
logger.error("process state change error:", e);
}
return true;
}
private boolean processComplementData() throws Exception {
if (!needComplementProcess()) {
return false;
}
Date scheduleDate = processInstance.getScheduleTime();
if (scheduleDate == null) {
scheduleDate = complementListDate.get(0);
} else if (processInstance.getState().typeIsFinished()) {
endProcess();
int index = complementListDate.indexOf(scheduleDate);
if (index >= complementListDate.size() - 1 || !processInstance.getState().typeIsSuccess()) {
// complement data ends || no success
return false;
}
scheduleDate = iterator.next();
scheduleDate = complementListDate.get(index + 1);
//the next process complement
processInstance.setId(0);
}
// flow end
// execute next process instance complement data
processInstance.setScheduleTime(scheduleDate);
Map<String, String> cmdParam = JSONUtils.toMap(processInstance.getCommandParam());
if (cmdParam.containsKey(Constants.CMD_PARAM_RECOVERY_START_NODE_STRING)) {
cmdParam.remove(Constants.CMD_PARAM_RECOVERY_START_NODE_STRING);
processInstance.setCommandParam(JSONUtils.toJsonString(cmdParam));
@ -329,13 +446,30 @@ public class MasterExecThread implements Runnable {
processInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
processInstance.setGlobalParams(ParameterUtils.curingGlobalParams(
processInstance.getProcessDefinition().getGlobalParamMap(),
processInstance.getProcessDefinition().getGlobalParamList(),
processDefinition.getGlobalParamMap(),
processDefinition.getGlobalParamList(),
CommandType.COMPLEMENT_DATA, processInstance.getScheduleTime()));
processInstance.setId(0);
processInstance.setStartTime(new Date());
processInstance.setEndTime(null);
processService.saveProcessInstance(processInstance);
this.taskInstanceHashMap.clear();
startProcess();
return true;
}
private boolean needComplementProcess() {
if (processInstance.isComplementData()
&& Flag.NO == processInstance.getIsSubProcess()) {
return true;
}
return false;
}
private void startProcess() throws Exception {
buildFlowDag();
if (this.taskInstanceHashMap.size() == 0) {
initTaskQueue();
submitPostNode(null);
}
}
@ -358,6 +492,7 @@ public class MasterExecThread implements Runnable {
* process end handle
*/
private void endProcess() {
this.stateEvents.clear();
processInstance.setEndTime(new Date());
processService.updateProcessInstance(processInstance);
if (processInstance.getState().typeIsWaitingThread()) {
@ -374,6 +509,11 @@ public class MasterExecThread implements Runnable {
* @throws Exception exception
*/
private void buildFlowDag() throws Exception {
if (this.dag != null) {
return;
}
processDefinition = processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
processInstance.getProcessDefinitionVersion());
recoverNodeIdList = getStartTaskInstanceList(processInstance.getCommandParam());
List<TaskNode> taskNodeList =
processService.genTaskNodeList(processInstance.getProcessDefinitionCode(), processInstance.getProcessDefinitionVersion(), new HashMap<>());
@ -402,7 +542,7 @@ public class MasterExecThread implements Runnable {
private void initTaskQueue() {
taskFailedSubmit = false;
activeTaskNode.clear();
activeTaskProcessorMaps.clear();
dependFailedTask.clear();
completeTaskList.clear();
errorTaskList.clear();
@ -418,6 +558,24 @@ public class MasterExecThread implements Runnable {
errorTaskList.put(task.getName(), task);
}
}
if (complementListDate.size() == 0 && needComplementProcess()) {
Map<String, String> cmdParam = JSONUtils.toMap(processInstance.getCommandParam());
Date startDate = DateUtils.getScheduleDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_START_DATE));
Date endDate = DateUtils.getScheduleDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_END_DATE));
if (startDate.after(endDate)) {
Date tmp = startDate;
startDate = endDate;
endDate = tmp;
}
ProcessDefinition processDefinition = processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
processInstance.getProcessDefinitionVersion());
List<Schedule> schedules = processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinition.getId());
complementListDate.addAll(CronUtils.getSelfFireDateList(startDate, endDate, schedules));
logger.info(" process definition id:{} complement data: {}",
processDefinition.getId(), complementListDate.toString());
}
}
/**
@ -427,21 +585,73 @@ public class MasterExecThread implements Runnable {
* @return TaskInstance
*/
private TaskInstance submitTaskExec(TaskInstance taskInstance) {
MasterBaseTaskExecThread abstractExecThread = null;
if (taskInstance.isSubProcess()) {
abstractExecThread = new SubProcessTaskExecThread(taskInstance);
} else if (taskInstance.isDependTask()) {
abstractExecThread = new DependentTaskExecThread(taskInstance);
} else if (taskInstance.isConditionsTask()) {
abstractExecThread = new ConditionsTaskExecThread(taskInstance);
} else if (taskInstance.isSwitchTask()) {
abstractExecThread = new SwitchTaskExecThread(taskInstance);
try {
ITaskProcessor taskProcessor = TaskProcessorFactory.getTaskProcessor(taskInstance.getTaskType());
if (taskInstance.getState() == ExecutionStatus.RUNNING_EXECUTION
&& taskProcessor.getType().equalsIgnoreCase(Constants.COMMON_TASK_TYPE)) {
notifyProcessHostUpdate(taskInstance);
}
boolean submit = taskProcessor.submit(taskInstance, processInstance, masterConfig.getMasterTaskCommitRetryTimes(), masterConfig.getMasterTaskCommitInterval());
if (submit) {
this.taskInstanceHashMap.put(taskInstance.getId(), taskInstance.getTaskCode(), taskInstance);
activeTaskProcessorMaps.put(taskInstance.getId(), taskProcessor);
taskProcessor.run();
addTimeoutCheck(taskInstance);
TaskDefinition taskDefinition = processService.findTaskDefinition(
taskInstance.getTaskCode(),
taskInstance.getTaskDefinitionVersion());
taskInstance.setTaskDefine(taskDefinition);
if (taskProcessor.taskState().typeIsFinished()) {
StateEvent stateEvent = new StateEvent();
stateEvent.setProcessInstanceId(this.processInstance.getId());
stateEvent.setTaskInstanceId(taskInstance.getId());
stateEvent.setExecutionStatus(taskProcessor.taskState());
stateEvent.setType(StateEventType.TASK_STATE_CHANGE);
this.stateEvents.add(stateEvent);
}
return taskInstance;
} else {
abstractExecThread = new MasterTaskExecThread(taskInstance);
logger.error("process id:{} name:{} submit standby task id:{} name:{} failed!",
processInstance.getId(), processInstance.getName(),
taskInstance.getId(), taskInstance.getName());
return null;
}
} catch (Exception e) {
logger.error("submit standby task error", e);
return null;
}
}
private void notifyProcessHostUpdate(TaskInstance taskInstance) {
if (StringUtils.isEmpty(taskInstance.getHost())) {
return;
}
try {
HostUpdateCommand hostUpdateCommand = new HostUpdateCommand();
hostUpdateCommand.setProcessHost(NetUtils.getAddr(masterConfig.getListenPort()));
hostUpdateCommand.setTaskInstanceId(taskInstance.getId());
Host host = new Host(taskInstance.getHost());
nettyExecutorManager.doExecute(host, hostUpdateCommand.convert2Command());
} catch (Exception e) {
logger.error("notify process host update", e);
}
}
private void addTimeoutCheck(TaskInstance taskInstance) {
TaskDefinition taskDefinition = processService.findTaskDefinition(
taskInstance.getTaskCode(),
taskInstance.getTaskDefinitionVersion()
);
taskInstance.setTaskDefine(taskDefinition);
if (TimeoutFlag.OPEN == taskDefinition.getTimeoutFlag()) {
this.taskTimeoutCheckList.put(taskInstance.getId(), taskInstance);
return;
}
if (taskInstance.isDependTask() || taskInstance.isSubProcess()) {
this.taskTimeoutCheckList.put(taskInstance.getId(), taskInstance);
}
Future<Boolean> future = taskExecService.submit(abstractExecThread);
activeTaskNode.putIfAbsent(abstractExecThread, future);
return abstractExecThread.getTaskInstance();
}
/**
@ -514,10 +724,23 @@ public class MasterExecThread implements Runnable {
String processWorkerGroup = processInstance.getWorkerGroup();
processWorkerGroup = StringUtils.isBlank(processWorkerGroup) ? DEFAULT_WORKER_GROUP : processWorkerGroup;
String taskWorkerGroup = StringUtils.isBlank(taskNode.getWorkerGroup()) ? processWorkerGroup : taskNode.getWorkerGroup();
Long processEnvironmentCode = Objects.isNull(processInstance.getEnvironmentCode()) ? -1 : processInstance.getEnvironmentCode();
Long taskEnvironmentCode = Objects.isNull(taskNode.getEnvironmentCode()) ? processEnvironmentCode : taskNode.getEnvironmentCode();
if (!processWorkerGroup.equals(DEFAULT_WORKER_GROUP) && taskWorkerGroup.equals(DEFAULT_WORKER_GROUP)) {
taskInstance.setWorkerGroup(processWorkerGroup);
taskInstance.setEnvironmentCode(processEnvironmentCode);
} else {
taskInstance.setWorkerGroup(taskWorkerGroup);
taskInstance.setEnvironmentCode(taskEnvironmentCode);
}
if (!taskInstance.getEnvironmentCode().equals(-1L)) {
Environment environment = processService.findEnvironmentByCode(taskInstance.getEnvironmentCode());
if (Objects.nonNull(environment) && StringUtils.isNotEmpty(environment.getConfig())) {
taskInstance.setEnvironmentConfig(environment.getConfig());
}
}
// delay execution time
taskInstance.setDelayTime(taskNode.getDelayTime());
@ -585,7 +808,11 @@ public class MasterExecThread implements Runnable {
List<TaskInstance> taskInstances = new ArrayList<>();
for (String taskNode : submitTaskNodeList) {
TaskNode taskNodeObject = dag.getNode(taskNode);
taskInstances.add(createTaskInstance(processInstance, taskNodeObject));
if (taskInstanceHashMap.containsColumn(taskNodeObject.getCode())) {
continue;
}
TaskInstance task = createTaskInstance(processInstance, taskNodeObject);
taskInstances.add(task);
}
// if previous node success , post node submit
@ -605,6 +832,8 @@ public class MasterExecThread implements Runnable {
addTaskToStandByList(task);
}
}
submitStandByTask();
updateProcessInstanceState();
}
/**
@ -727,7 +956,7 @@ public class MasterExecThread implements Runnable {
return true;
}
if (processInstance.getFailureStrategy() == FailureStrategy.CONTINUE) {
return readyToSubmitTaskQueue.size() == 0 || activeTaskNode.size() == 0;
return readyToSubmitTaskQueue.size() == 0 || activeTaskProcessorMaps.size() == 0;
}
}
return false;
@ -771,11 +1000,10 @@ public class MasterExecThread implements Runnable {
*
* @return process instance execution status
*/
private ExecutionStatus getProcessInstanceState() {
ProcessInstance instance = processService.findProcessInstanceById(processInstance.getId());
private ExecutionStatus getProcessInstanceState(ProcessInstance instance) {
ExecutionStatus state = instance.getState();
if (activeTaskNode.size() > 0 || hasRetryTaskInStandBy()) {
if (activeTaskProcessorMaps.size() > 0 || hasRetryTaskInStandBy()) {
// active task and retry task exists
return runningState(state);
}
@ -867,7 +1095,8 @@ public class MasterExecThread implements Runnable {
* after each batch of tasks is executed, the status of the process instance is updated
*/
private void updateProcessInstanceState() {
ExecutionStatus state = getProcessInstanceState();
ProcessInstance instance = processService.findProcessInstanceById(processInstance.getId());
ExecutionStatus state = getProcessInstanceState(instance);
if (processInstance.getState() != state) {
logger.info(
"work flow process instance [id: {}, name:{}], state change from {} to {}, cmd type: {}",
@ -875,11 +1104,14 @@ public class MasterExecThread implements Runnable {
processInstance.getState(), state,
processInstance.getCommandType());
ProcessInstance instance = processService.findProcessInstanceById(processInstance.getId());
instance.setState(state);
instance.setProcessDefinition(processInstance.getProcessDefinition());
processService.updateProcessInstance(instance);
processInstance = instance;
StateEvent stateEvent = new StateEvent();
stateEvent.setExecutionStatus(processInstance.getState());
stateEvent.setProcessInstanceId(this.processInstance.getId());
stateEvent.setType(StateEventType.PROCESS_STATE_CHANGE);
this.processStateChangeHandler(stateEvent);
}
}
@ -913,11 +1145,15 @@ public class MasterExecThread implements Runnable {
* @param taskInstance task instance
*/
private void removeTaskFromStandbyList(TaskInstance taskInstance) {
logger.info("remove task from stand by list: {}", taskInstance.getName());
logger.info("remove task from stand by list, id: {} name:{}",
taskInstance.getId(),
taskInstance.getName());
try {
readyToSubmitTaskQueue.remove(taskInstance);
} catch (Exception e) {
logger.error("remove task instance from readyToSubmitTaskQueue error, taskName: {}", taskInstance.getName(), e);
logger.error("remove task instance from readyToSubmitTaskQueue error, task id:{}, Name: {}",
taskInstance.getId(),
taskInstance.getName(), e);
}
}
@ -935,131 +1171,6 @@ public class MasterExecThread implements Runnable {
return false;
}
/**
* submit and watch the tasks, until the work flow stop
*/
private void runProcess() {
// submit start node
submitPostNode(null);
boolean sendTimeWarning = false;
while (!processInstance.isProcessInstanceStop() && Stopper.isRunning()) {
// send warning email if process time out.
if (!sendTimeWarning && checkProcessTimeOut(processInstance)) {
processAlertManager.sendProcessTimeoutAlert(processInstance,
processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
processInstance.getProcessDefinitionVersion()));
sendTimeWarning = true;
}
for (Map.Entry<MasterBaseTaskExecThread, Future<Boolean>> entry : activeTaskNode.entrySet()) {
Future<Boolean> future = entry.getValue();
TaskInstance task = entry.getKey().getTaskInstance();
if (!future.isDone()) {
continue;
}
// node monitor thread complete
task = this.processService.findTaskInstanceById(task.getId());
if (task == null) {
this.taskFailedSubmit = true;
activeTaskNode.remove(entry.getKey());
continue;
}
// node monitor thread complete
if (task.getState().typeIsFinished()) {
activeTaskNode.remove(entry.getKey());
}
logger.info("task :{}, id:{} complete, state is {} ",
task.getName(), task.getId(), task.getState());
// node success , post node submit
if (task.getState() == ExecutionStatus.SUCCESS) {
ProcessDefinition relatedProcessDefinition = processInstance.getProcessDefinition();
processInstance = processService.findProcessInstanceById(processInstance.getId());
processInstance.setProcessDefinition(relatedProcessDefinition);
processInstance.setVarPool(task.getVarPool());
processService.updateProcessInstance(processInstance);
completeTaskList.put(task.getName(), task);
submitPostNode(task.getName());
continue;
}
// node fails, retry first, and then execute the failure process
if (task.getState().typeIsFailure()) {
if (task.getState() == ExecutionStatus.NEED_FAULT_TOLERANCE) {
this.recoverToleranceFaultTaskList.add(task);
}
if (task.taskCanRetry()) {
addTaskToStandByList(task);
} else {
completeTaskList.put(task.getName(), task);
if (task.isConditionsTask()
|| DagHelper.haveConditionsAfterNode(task.getName(), dag)) {
submitPostNode(task.getName());
} else {
errorTaskList.put(task.getName(), task);
if (processInstance.getFailureStrategy() == FailureStrategy.END) {
killTheOtherTasks();
}
}
}
continue;
}
// other status stop/pause
completeTaskList.put(task.getName(), task);
}
// send alert
if (CollectionUtils.isNotEmpty(this.recoverToleranceFaultTaskList)) {
processAlertManager.sendAlertWorkerToleranceFault(processInstance, recoverToleranceFaultTaskList);
this.recoverToleranceFaultTaskList.clear();
}
// updateProcessInstance completed task status
// failure priority is higher than pause
// if a task fails, other suspended tasks need to be reset kill
// check if there exists forced success nodes in errorTaskList
if (errorTaskList.size() > 0) {
for (Map.Entry<String, TaskInstance> entry : completeTaskList.entrySet()) {
TaskInstance completeTask = entry.getValue();
if (completeTask.getState() == ExecutionStatus.PAUSE) {
completeTask.setState(ExecutionStatus.KILL);
completeTaskList.put(entry.getKey(), completeTask);
processService.updateTaskInstance(completeTask);
}
}
for (Map.Entry<String, TaskInstance> entry : errorTaskList.entrySet()) {
TaskInstance errorTask = entry.getValue();
TaskInstance currentTask = processService.findTaskInstanceById(errorTask.getId());
if (currentTask == null) {
continue;
}
// for nodes that have been forced success
if (errorTask.getState().typeIsFailure() && currentTask.getState().equals(ExecutionStatus.FORCED_SUCCESS)) {
// update state in this thread and remove from errorTaskList
errorTask.setState(currentTask.getState());
logger.info("task: {} has been forced success, remove it from error task list", errorTask.getName());
errorTaskList.remove(errorTask.getName());
// submit post nodes
submitPostNode(errorTask.getName());
}
}
}
if (canSubmitTaskToQueue()) {
submitStandByTask();
}
try {
Thread.sleep(Constants.SLEEP_TIME_MILLIS);
} catch (InterruptedException e) {
logger.error(e.getMessage(), e);
Thread.currentThread().interrupt();
}
updateProcessInstanceState();
}
logger.info("process:{} end, state :{}", processInstance.getId(), processInstance.getState());
}
/**
* whether check process time out
*
@ -1089,28 +1200,30 @@ public class MasterExecThread implements Runnable {
/**
* close the on going tasks
*/
private void killTheOtherTasks() {
private void killAllTasks() {
logger.info("kill called on process instance id: {}, num: {}", processInstance.getId(),
activeTaskNode.size());
for (Map.Entry<MasterBaseTaskExecThread, Future<Boolean>> entry : activeTaskNode.entrySet()) {
MasterBaseTaskExecThread taskExecThread = entry.getKey();
Future<Boolean> future = entry.getValue();
TaskInstance taskInstance = taskExecThread.getTaskInstance();
taskInstance = processService.findTaskInstanceById(taskInstance.getId());
if (taskInstance != null && taskInstance.getState().typeIsFinished()) {
activeTaskProcessorMaps.size());
for (int taskId : activeTaskProcessorMaps.keySet()) {
TaskInstance taskInstance = processService.findTaskInstanceById(taskId);
if (taskInstance == null || taskInstance.getState().typeIsFinished()) {
continue;
}
if (!future.isDone()) {
// record kill info
logger.info("kill process instance, id: {}, task: {}", processInstance.getId(), taskExecThread.getTaskInstance().getId());
// kill node
taskExecThread.kill();
ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskId);
taskProcessor.action(TaskAction.STOP);
if (taskProcessor.taskState().typeIsFinished()) {
StateEvent stateEvent = new StateEvent();
stateEvent.setType(StateEventType.TASK_STATE_CHANGE);
stateEvent.setProcessInstanceId(this.processInstance.getId());
stateEvent.setTaskInstanceId(taskInstance.getId());
stateEvent.setExecutionStatus(taskProcessor.taskState());
this.addStateEvent(stateEvent);
}
}
}
public boolean workFlowFinish() {
return this.processInstance.getState().typeIsFinished();
}
/**
@ -1144,6 +1257,9 @@ public class MasterExecThread implements Runnable {
int length = readyToSubmitTaskQueue.size();
for (int i = 0; i < length; i++) {
TaskInstance task = readyToSubmitTaskQueue.peek();
if (task == null) {
continue;
}
// stop tasks which is retrying if forced success happens
if (task.taskCanRetry()) {
TaskInstance retryTask = processService.findTaskInstanceById(task.getId());
@ -1165,9 +1281,13 @@ public class MasterExecThread implements Runnable {
DependResult dependResult = getDependResultForTask(task);
if (DependResult.SUCCESS == dependResult) {
if (retryTaskIntervalOverTime(task)) {
submitTaskExec(task);
TaskInstance taskInstance = submitTaskExec(task);
if (taskInstance == null) {
this.taskFailedSubmit = true;
} else {
removeTaskFromStandbyList(task);
}
}
} else if (DependResult.FAILED == dependResult) {
// if the dependency fails, the current node is not submitted and the state changes to failure.
dependFailedTask.put(task.getName(), task);

112
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseTaskProcessor.java

@ -0,0 +1,112 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public abstract class BaseTaskProcessor implements ITaskProcessor {
protected Logger logger = LoggerFactory.getLogger(getClass());
protected boolean killed = false;
protected boolean paused = false;
protected boolean timeout = false;
protected TaskInstance taskInstance = null;
protected ProcessInstance processInstance;
/**
* pause task, common tasks donot need this.
*
* @return
*/
protected abstract boolean pauseTask();
/**
* kill task, all tasks need to realize this function
*
* @return
*/
protected abstract boolean killTask();
/**
* task timeout process
* @return
*/
protected abstract boolean taskTimeout();
@Override
public void run() {
}
@Override
public boolean action(TaskAction taskAction) {
switch (taskAction) {
case STOP:
return stop();
case PAUSE:
return pause();
case TIMEOUT:
return timeout();
default:
logger.error("unknown task action: {}", taskAction.toString());
}
return false;
}
protected boolean timeout() {
if (timeout) {
return true;
}
timeout = taskTimeout();
return timeout;
}
/**
* @return
*/
protected boolean pause() {
if (paused) {
return true;
}
paused = pauseTask();
return paused;
}
protected boolean stop() {
if (killed) {
return true;
}
killed = killTask();
return killed;
}
@Override
public String getType() {
return null;
}
}

33
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessFactory.java

@ -0,0 +1,33 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task;
import org.apache.dolphinscheduler.common.Constants;
public class CommonTaskProcessFactory implements ITaskProcessFactory {
@Override
public String type() {
return Constants.COMMON_TASK_TYPE;
}
@Override
public ITaskProcessor create() {
return new CommonTaskProcessor();
}
}

175
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessor.java

@ -0,0 +1,175 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.remote.command.TaskKillRequestCommand;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext;
import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType;
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException;
import org.apache.dolphinscheduler.server.master.dispatch.executor.NettyExecutorManager;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.service.queue.TaskPriority;
import org.apache.dolphinscheduler.service.queue.TaskPriorityQueue;
import org.apache.dolphinscheduler.service.queue.TaskPriorityQueueImpl;
import java.util.Date;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
/**
* common task processor
*/
public class CommonTaskProcessor extends BaseTaskProcessor {
@Autowired
private TaskPriorityQueue taskUpdateQueue;
@Autowired
MasterConfig masterConfig;
@Autowired
NettyExecutorManager nettyExecutorManager;
/**
* logger of MasterBaseTaskExecThread
*/
protected Logger logger = LoggerFactory.getLogger(getClass());
protected ProcessService processService = SpringApplicationContext.getBean(ProcessService.class);
@Override
public boolean submit(TaskInstance task, ProcessInstance processInstance, int maxRetryTimes, int commitInterval) {
this.processInstance = processInstance;
this.taskInstance = processService.submitTask(task, maxRetryTimes, commitInterval);
if (this.taskInstance == null) {
return false;
}
dispatchTask(taskInstance, processInstance);
return true;
}
@Override
public ExecutionStatus taskState() {
return this.taskInstance.getState();
}
@Override
public void run() {
}
@Override
protected boolean taskTimeout() {
return true;
}
/**
* common task cannot be paused
*/
@Override
protected boolean pauseTask() {
return true;
}
@Override
public String getType() {
return Constants.COMMON_TASK_TYPE;
}
private boolean dispatchTask(TaskInstance taskInstance, ProcessInstance processInstance) {
try {
if (taskUpdateQueue == null) {
this.initQueue();
}
if (taskInstance.getState().typeIsFinished()) {
logger.info(String.format("submit task , but task [%s] state [%s] is already finished. ", taskInstance.getName(), taskInstance.getState().toString()));
return true;
}
// task cannot be submitted because its execution state is RUNNING or DELAY.
if (taskInstance.getState() == ExecutionStatus.RUNNING_EXECUTION
|| taskInstance.getState() == ExecutionStatus.DELAY_EXECUTION) {
logger.info("submit task, but the status of the task {} is already running or delayed.", taskInstance.getName());
return true;
}
logger.info("task ready to submit: {}", taskInstance);
TaskPriority taskPriority = new TaskPriority(processInstance.getProcessInstancePriority().getCode(),
processInstance.getId(), taskInstance.getProcessInstancePriority().getCode(),
taskInstance.getId(), org.apache.dolphinscheduler.common.Constants.DEFAULT_WORKER_GROUP);
taskUpdateQueue.put(taskPriority);
logger.info(String.format("master submit success, task : %s", taskInstance.getName()));
return true;
} catch (Exception e) {
logger.error("submit task Exception: ", e);
logger.error("task error : %s", JSONUtils.toJsonString(taskInstance));
return false;
}
}
public void initQueue() {
this.taskUpdateQueue = SpringApplicationContext.getBean(TaskPriorityQueueImpl.class);
}
@Override
public boolean killTask() {
try {
taskInstance = processService.findTaskInstanceById(taskInstance.getId());
if (taskInstance == null) {
return true;
}
if (taskInstance.getState().typeIsFinished()) {
return true;
}
if (null == taskInstance.getHost() || taskInstance.getHost().isEmpty()) {
taskInstance.setState(ExecutionStatus.KILL);
taskInstance.setEndTime(new Date());
processService.updateTaskInstance(taskInstance);
return true;
}
TaskKillRequestCommand killCommand = new TaskKillRequestCommand();
killCommand.setTaskInstanceId(taskInstance.getId());
ExecutionContext executionContext = new ExecutionContext(killCommand.convert2Command(), ExecutorType.WORKER);
Host host = Host.of(taskInstance.getHost());
executionContext.setHost(host);
nettyExecutorManager.executeDirectly(executionContext);
} catch (ExecuteException e) {
logger.error("kill task error:", e);
return false;
}
logger.info("master kill taskInstance name :{} taskInstance id:{}",
taskInstance.getName(), taskInstance.getId());
return true;
}
}

32
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ConditionTaskProcessFactory.java

@ -0,0 +1,32 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task;
import org.apache.dolphinscheduler.common.enums.TaskType;
public class ConditionTaskProcessFactory implements ITaskProcessFactory {
@Override
public String type() {
return TaskType.CONDITIONS.getDesc();
}
@Override
public ITaskProcessor create() {
return new ConditionTaskProcessor();
}
}

210
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ConditionTaskProcessor.java

@ -0,0 +1,210 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.DependResult;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.TaskTimeoutStrategy;
import org.apache.dolphinscheduler.common.enums.TaskType;
import org.apache.dolphinscheduler.common.model.DependentItem;
import org.apache.dolphinscheduler.common.model.DependentTaskModel;
import org.apache.dolphinscheduler.common.task.dependent.DependentParameters;
import org.apache.dolphinscheduler.common.utils.DependentUtils;
import org.apache.dolphinscheduler.common.utils.LoggerUtils;
import org.apache.dolphinscheduler.common.utils.NetUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.utils.LogUtils;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService;
import java.util.ArrayList;
import java.util.Date;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.slf4j.LoggerFactory;
/**
* condition task processor
*/
public class ConditionTaskProcessor extends BaseTaskProcessor {
/**
* dependent parameters
*/
private DependentParameters dependentParameters;
ProcessInstance processInstance;
/**
* condition result
*/
private DependResult conditionResult = DependResult.WAITING;
/**
* complete task map
*/
private Map<String, ExecutionStatus> completeTaskList = new ConcurrentHashMap<>();
protected ProcessService processService = SpringApplicationContext.getBean(ProcessService.class);
MasterConfig masterConfig = SpringApplicationContext.getBean(MasterConfig.class);
private TaskDefinition taskDefinition;
@Override
public boolean submit(TaskInstance task, ProcessInstance processInstance, int masterTaskCommitRetryTimes, int masterTaskCommitInterval) {
this.processInstance = processInstance;
this.taskInstance = processService.submitTask(task, masterTaskCommitRetryTimes, masterTaskCommitInterval);
if (this.taskInstance == null) {
return false;
}
taskDefinition = processService.findTaskDefinition(
taskInstance.getTaskCode(), taskInstance.getTaskDefinitionVersion()
);
logger = LoggerFactory.getLogger(LoggerUtils.buildTaskId(LoggerUtils.TASK_LOGGER_INFO_PREFIX,
processInstance.getProcessDefinitionCode(),
processInstance.getProcessDefinitionVersion(),
taskInstance.getProcessInstanceId(),
taskInstance.getId()));
String threadLoggerInfoName = String.format(Constants.TASK_LOG_INFO_FORMAT, processService.formatTaskAppId(this.taskInstance));
Thread.currentThread().setName(threadLoggerInfoName);
initTaskParameters();
logger.info("dependent task start");
endTask();
return true;
}
@Override
public ExecutionStatus taskState() {
return this.taskInstance.getState();
}
@Override
public void run() {
if (conditionResult.equals(DependResult.WAITING)) {
setConditionResult();
} else {
endTask();
}
}
@Override
protected boolean pauseTask() {
this.taskInstance.setState(ExecutionStatus.PAUSE);
this.taskInstance.setEndTime(new Date());
processService.saveTaskInstance(taskInstance);
return true;
}
@Override
protected boolean taskTimeout() {
TaskTimeoutStrategy taskTimeoutStrategy =
taskDefinition.getTimeoutNotifyStrategy();
if (taskTimeoutStrategy == TaskTimeoutStrategy.WARN) {
return true;
}
logger.info("condition task {} timeout, strategy {} ",
taskInstance.getId(), taskTimeoutStrategy.getDescp());
conditionResult = DependResult.FAILED;
endTask();
return true;
}
@Override
protected boolean killTask() {
this.taskInstance.setState(ExecutionStatus.KILL);
this.taskInstance.setEndTime(new Date());
processService.saveTaskInstance(taskInstance);
return true;
}
@Override
public String getType() {
return TaskType.CONDITIONS.getDesc();
}
private void initTaskParameters() {
taskInstance.setLogPath(LogUtils.getTaskLogPath(processInstance.getProcessDefinitionCode(),
processInstance.getProcessDefinitionVersion(),
taskInstance.getProcessInstanceId(),
taskInstance.getId()));
this.taskInstance.setHost(NetUtils.getAddr(masterConfig.getListenPort()));
taskInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
taskInstance.setStartTime(new Date());
this.processService.saveTaskInstance(taskInstance);
this.dependentParameters = taskInstance.getDependency();
}
private void setConditionResult() {
List<TaskInstance> taskInstances = processService.findValidTaskListByProcessId(taskInstance.getProcessInstanceId());
for (TaskInstance task : taskInstances) {
completeTaskList.putIfAbsent(task.getName(), task.getState());
}
List<DependResult> modelResultList = new ArrayList<>();
for (DependentTaskModel dependentTaskModel : dependentParameters.getDependTaskList()) {
List<DependResult> itemDependResult = new ArrayList<>();
for (DependentItem item : dependentTaskModel.getDependItemList()) {
itemDependResult.add(getDependResultForItem(item));
}
DependResult modelResult = DependentUtils.getDependResultForRelation(dependentTaskModel.getRelation(), itemDependResult);
modelResultList.add(modelResult);
}
conditionResult = DependentUtils.getDependResultForRelation(dependentParameters.getRelation(), modelResultList);
logger.info("the conditions task depend result : {}", conditionResult);
}
/**
* depend result for depend item
*/
private DependResult getDependResultForItem(DependentItem item) {
DependResult dependResult = DependResult.SUCCESS;
if (!completeTaskList.containsKey(item.getDepTasks())) {
logger.info("depend item: {} have not completed yet.", item.getDepTasks());
dependResult = DependResult.FAILED;
return dependResult;
}
ExecutionStatus executionStatus = completeTaskList.get(item.getDepTasks());
if (executionStatus != item.getStatus()) {
logger.info("depend item : {} expect status: {}, actual status: {}", item.getDepTasks(), item.getStatus(), executionStatus);
dependResult = DependResult.FAILED;
}
logger.info("dependent item complete {} {},{}",
Constants.DEPENDENT_SPLIT, item.getDepTasks(), dependResult);
return dependResult;
}
/**
*
*/
private void endTask() {
ExecutionStatus status = (conditionResult == DependResult.SUCCESS) ? ExecutionStatus.SUCCESS : ExecutionStatus.FAILURE;
taskInstance.setState(status);
taskInstance.setEndTime(new Date());
processService.updateTaskInstance(taskInstance);
}
}

33
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/DependentTaskProcessFactory.java

@ -0,0 +1,33 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task;
import org.apache.dolphinscheduler.common.enums.TaskType;
public class DependentTaskProcessFactory implements ITaskProcessFactory {
@Override
public String type() {
return TaskType.DEPENDENT.getDesc();
}
@Override
public ITaskProcessor create() {
return new DependentTaskProcessor();
}
}

220
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/DependentTaskProcessor.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.server.master.runner.task;
import static org.apache.dolphinscheduler.common.Constants.DEPENDENT_SPLIT;
import org.apache.dolphinscheduler.common.enums.DependResult;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.TaskTimeoutStrategy;
import org.apache.dolphinscheduler.common.enums.TaskType;
import org.apache.dolphinscheduler.common.model.DependentTaskModel;
import org.apache.dolphinscheduler.common.task.dependent.DependentParameters;
import org.apache.dolphinscheduler.common.utils.DependentUtils;
import org.apache.dolphinscheduler.common.utils.NetUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.utils.DependentExecute;
import org.apache.dolphinscheduler.server.utils.LogUtils;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService;
import java.util.ArrayList;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import com.fasterxml.jackson.annotation.JsonFormat;
/**
* dependent task processor
*/
public class DependentTaskProcessor extends BaseTaskProcessor {
private DependentParameters dependentParameters;
/**
* dependent task list
*/
private List<DependentExecute> dependentTaskList = new ArrayList<>();
/**
* depend item result map
* save the result to log file
*/
private Map<String, DependResult> dependResultMap = new HashMap<>();
/**
* dependent date
*/
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date dependentDate;
DependResult result;
ProcessInstance processInstance;
TaskDefinition taskDefinition;
protected ProcessService processService = SpringApplicationContext.getBean(ProcessService.class);
MasterConfig masterConfig = SpringApplicationContext.getBean(MasterConfig.class);
boolean allDependentItemFinished;
@Override
public boolean submit(TaskInstance task, ProcessInstance processInstance, int masterTaskCommitRetryTimes, int masterTaskCommitInterval) {
this.processInstance = processInstance;
this.taskInstance = task;
this.taskInstance = processService.submitTask(task, masterTaskCommitRetryTimes, masterTaskCommitInterval);
if (this.taskInstance == null) {
return false;
}
taskDefinition = processService.findTaskDefinition(
taskInstance.getTaskCode(), taskInstance.getTaskDefinitionVersion()
);
taskInstance.setLogPath(LogUtils.getTaskLogPath(processInstance.getProcessDefinitionCode(),
processInstance.getProcessDefinitionVersion(),
taskInstance.getProcessInstanceId(),
taskInstance.getId()));
taskInstance.setHost(NetUtils.getAddr(masterConfig.getListenPort()));
taskInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
taskInstance.setStartTime(new Date());
processService.updateTaskInstance(taskInstance);
initDependParameters();
return true;
}
@Override
public ExecutionStatus taskState() {
return this.taskInstance.getState();
}
@Override
public void run() {
if (!allDependentItemFinished) {
allDependentItemFinished = allDependentTaskFinish();
}
if (allDependentItemFinished) {
getTaskDependResult();
endTask();
}
}
@Override
protected boolean taskTimeout() {
TaskTimeoutStrategy taskTimeoutStrategy =
taskDefinition.getTimeoutNotifyStrategy();
if (TaskTimeoutStrategy.FAILED != taskTimeoutStrategy
&& TaskTimeoutStrategy.WARNFAILED != taskTimeoutStrategy) {
return true;
}
logger.info("dependent task {} timeout, strategy {} ",
taskInstance.getId(), taskTimeoutStrategy.getDescp());
result = DependResult.FAILED;
endTask();
return true;
}
/**
* init dependent parameters
*/
private void initDependParameters() {
this.dependentParameters = taskInstance.getDependency();
for (DependentTaskModel taskModel : dependentParameters.getDependTaskList()) {
this.dependentTaskList.add(new DependentExecute(taskModel.getDependItemList(), taskModel.getRelation()));
}
if (processInstance.getScheduleTime() != null) {
this.dependentDate = this.processInstance.getScheduleTime();
} else {
this.dependentDate = new Date();
}
}
@Override
protected boolean pauseTask() {
this.taskInstance.setState(ExecutionStatus.PAUSE);
this.taskInstance.setEndTime(new Date());
processService.saveTaskInstance(taskInstance);
return true;
}
@Override
protected boolean killTask() {
this.taskInstance.setState(ExecutionStatus.KILL);
this.taskInstance.setEndTime(new Date());
processService.saveTaskInstance(taskInstance);
return true;
}
/**
* judge all dependent tasks finish
*
* @return whether all dependent tasks finish
*/
private boolean allDependentTaskFinish() {
boolean finish = true;
for (DependentExecute dependentExecute : dependentTaskList) {
for (Map.Entry<String, DependResult> entry : dependentExecute.getDependResultMap().entrySet()) {
if (!dependResultMap.containsKey(entry.getKey())) {
dependResultMap.put(entry.getKey(), entry.getValue());
//save depend result to log
logger.info("dependent item complete {} {},{}", DEPENDENT_SPLIT, entry.getKey(), entry.getValue());
}
}
if (!dependentExecute.finish(dependentDate)) {
finish = false;
}
}
return finish;
}
/**
* get dependent result
*
* @return DependResult
*/
private DependResult getTaskDependResult() {
List<DependResult> dependResultList = new ArrayList<>();
for (DependentExecute dependentExecute : dependentTaskList) {
DependResult dependResult = dependentExecute.getModelDependResult(dependentDate);
dependResultList.add(dependResult);
}
result = DependentUtils.getDependResultForRelation(this.dependentParameters.getRelation(), dependResultList);
logger.info("dependent task completed, dependent result:{}", result);
return result;
}
/**
*
*/
private void endTask() {
ExecutionStatus status;
status = (result == DependResult.SUCCESS) ? ExecutionStatus.SUCCESS : ExecutionStatus.FAILURE;
taskInstance.setState(status);
taskInstance.setEndTime(new Date());
processService.saveTaskInstance(taskInstance);
}
@Override
public String getType() {
return TaskType.DEPENDENT.getDesc();
}
}

25
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ITaskProcessFactory.java

@ -0,0 +1,25 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task;
public interface ITaskProcessFactory {
String type();
ITaskProcessor create();
}

39
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/ITaskProcessor.java

@ -0,0 +1,39 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
/**
* interface of task processor in master
*/
public interface ITaskProcessor {
void run();
boolean action(TaskAction taskAction);
String getType();
boolean submit(TaskInstance taskInstance, ProcessInstance processInstance, int masterTaskCommitRetryTimes, int masterTaskCommitInterval);
ExecutionStatus taskState();
}

32
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/SubTaskProcessFactory.java

@ -0,0 +1,32 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task;
import org.apache.dolphinscheduler.common.enums.TaskType;
public class SubTaskProcessFactory implements ITaskProcessFactory {
@Override
public String type() {
return TaskType.SUB_PROCESS.getDesc();
}
@Override
public ITaskProcessor create() {
return new SubTaskProcessor();
}
}

Some files were not shown because too many files have changed in this diff Show More

Loading…
Cancel
Save