Browse Source

Merge remote-tracking branch 'upstream/dev' into dev

pull/3/MERGE
dailidong 4 years ago
parent
commit
f430fed8f9
  1. 2
      .github/workflows/ci_e2e.yml
  2. 31
      docker/build/Dockerfile
  3. 6
      dolphinscheduler-api/pom.xml
  4. 4
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskRecordController.java
  5. 4
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java
  6. 5
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ResourcesService.java
  7. 44
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProcessDefinitionServiceImpl.java
  8. 26
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TenantServiceImpl.java
  9. 47
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/RegexUtils.java
  10. 4
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ProcessDefinitionControllerTest.java
  11. 90
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java
  12. 39
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/RegexUtilsTest.java
  13. 9
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java
  14. 78
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/TaskParams.java
  15. 10
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/DateUtils.java
  16. 45
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/HadoopUtils.java
  17. 121
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/NetUtils.java
  18. 72
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtils.java
  19. 124
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/VarPoolUtils.java
  20. 3
      dolphinscheduler-common/src/main/resources/common.properties
  21. 19
      dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/DateUtilsTest.java
  22. 73
      dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/VarPoolUtilsTest.java
  23. 213
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java
  24. 12
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java
  25. 9
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionMapper.java
  26. 6
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionMapper.xml
  27. 3
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.xml
  28. 41
      dolphinscheduler-dist/release-docs/LICENSE
  29. 4
      dolphinscheduler-dist/release-docs/NOTICE
  30. 19
      dolphinscheduler-dist/release-docs/licenses/ui-licenses/LICENSE-dagre
  31. 6
      dolphinscheduler-dist/src/main/assembly/dolphinscheduler-binary.xml
  32. 51
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java
  33. 12
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java
  34. 106
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java
  35. 12
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/TaskExecuteResponseCommand.java
  36. 75
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java
  37. 67
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java
  38. 10
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java
  39. 48
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Host.java
  40. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java
  41. 44
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/HostWeight.java
  42. 13
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/LowerWeightRoundRobin.java
  43. 3
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskResponseProcessor.java
  44. 18
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskResponseEvent.java
  45. 3
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskResponseService.java
  46. 15
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java
  47. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/LogUtils.java
  48. 44
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistry.java
  49. 1
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskExecuteThread.java
  50. 15
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java
  51. 13
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractTask.java
  52. 78
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskManager.java
  53. 189
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/python/PythonTask.java
  54. 604
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java
  55. 51
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/LowerWeightRoundRobinTest.java
  56. 16
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RandomSelectorTest.java
  57. 70
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RoundRobinSelectorTest.java
  58. 43
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/HostTest.java
  59. 114
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/TaskManagerTest.java
  60. 575
      dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java
  61. 116
      dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/process/ProcessServiceTest.java
  62. 1
      dolphinscheduler-ui/package.json
  63. 7
      dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/config.js
  64. 214
      dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.js
  65. 218
      dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/_source/dependentTimeout.vue
  66. 104
      dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue
  67. 108
      dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/pre_tasks.vue
  68. 3
      dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/_source/instanceConditions/common.js
  69. 19
      dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/_source/list.vue
  70. 2
      dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/_source/util.js
  71. 11
      dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue
  72. 23
      dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/taskInstance/_source/list.vue
  73. 3
      dolphinscheduler-ui/src/js/conf/home/pages/user/pages/password/_source/info.vue
  74. 12
      dolphinscheduler-ui/src/js/conf/home/store/dag/actions.js
  75. 9
      dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js
  76. 9
      dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js
  77. 10
      e2e/pom.xml
  78. 45
      e2e/src/main/java/org/apache/dolphinscheduler/util/YmlReader.java
  79. 1
      e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java
  80. 3
      e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java
  81. 12
      e2e/src/test/java/org/apache/dolphinscheduler/data/project/ProcessInstanceData.java
  82. 16
      e2e/src/test/java/org/apache/dolphinscheduler/data/project/ProjectData.java
  83. 19
      e2e/src/test/java/org/apache/dolphinscheduler/data/project/RunWorkflowData.java
  84. 25
      e2e/src/test/java/org/apache/dolphinscheduler/data/project/TimingData.java
  85. 76
      e2e/src/test/java/org/apache/dolphinscheduler/data/project/WorkflowDefineData.java
  86. 18
      e2e/src/test/java/org/apache/dolphinscheduler/data/security/AlertManageData.java
  87. 31
      e2e/src/test/java/org/apache/dolphinscheduler/data/security/QueueManageData.java
  88. 30
      e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java
  89. 15
      e2e/src/test/java/org/apache/dolphinscheduler/data/security/TokenManageData.java
  90. 29
      e2e/src/test/java/org/apache/dolphinscheduler/data/security/UserManageData.java
  91. 2
      e2e/src/test/java/org/apache/dolphinscheduler/locator/project/WorkflowDefineLocator.java
  92. 6
      e2e/src/test/java/org/apache/dolphinscheduler/locator/security/TokenManageLocator.java
  93. 2
      e2e/src/test/java/org/apache/dolphinscheduler/locator/security/UserManageLocator.java
  94. 8
      e2e/src/test/java/org/apache/dolphinscheduler/page/LoginPage.java
  95. 4
      e2e/src/test/java/org/apache/dolphinscheduler/page/project/ProcessInstancePage.java
  96. 15
      e2e/src/test/java/org/apache/dolphinscheduler/page/project/ProjectPage.java
  97. 11
      e2e/src/test/java/org/apache/dolphinscheduler/page/project/RunWorkflowPage.java
  98. 31
      e2e/src/test/java/org/apache/dolphinscheduler/page/project/TimingPage.java
  99. 57
      e2e/src/test/java/org/apache/dolphinscheduler/page/project/WorkflowDefinePage.java
  100. 14
      e2e/src/test/java/org/apache/dolphinscheduler/page/security/AlertManagePage.java
  101. Some files were not shown because too many files have changed in this diff Show More

2
.github/workflows/ci_e2e.yml

@ -59,7 +59,7 @@ jobs:
sudo dpkg -i google-chrome*.deb
sudo apt-get install -f -y
google-chrome -version
googleVersion=`google-chrome -version | awk '{print $3}'`
googleVersion=$(curl -s https://chromedriver.storage.googleapis.com/LATEST_RELEASE)
wget -N https://chromedriver.storage.googleapis.com/${googleVersion}/chromedriver_linux64.zip
unzip chromedriver_linux64.zip
sudo mv -f chromedriver /usr/local/share/chromedriver

31
docker/build/Dockerfile

@ -27,13 +27,13 @@ ENV DEBIAN_FRONTEND noninteractive
#If install slowly, you can replcae alpine's mirror with aliyun's mirror, Example:
#RUN sed -i "s/dl-cdn.alpinelinux.org/mirrors.aliyun.com/g" /etc/apk/repositories
RUN apk update && \
apk --update add --no-cache dos2unix shadow bash openrc python2 python3 sudo vim wget iputils net-tools openssh-server py-pip tini && \
apk add --update procps && \
apk add --update --no-cache dos2unix shadow bash openrc python2 python3 sudo vim wget iputils net-tools openssh-server py-pip tini && \
apk add --update --no-cache procps && \
openrc boot && \
pip install kazoo
#2. install jdk
RUN apk add openjdk8
RUN apk add --update --no-cache openjdk8
ENV JAVA_HOME /usr/lib/jvm/java-1.8-openjdk
ENV PATH $JAVA_HOME/bin:$PATH
@ -43,19 +43,20 @@ RUN mv /opt/apache-dolphinscheduler-incubating-${VERSION}-dolphinscheduler-bin/
ENV DOLPHINSCHEDULER_HOME /opt/dolphinscheduler
#4. install database, if use mysql as your backend database, the `mysql-client` package should be installed
RUN apk add postgresql postgresql-contrib
RUN apk add --update --no-cache postgresql postgresql-contrib
#5. modify nginx
RUN echo "daemon off;" >> /etc/nginx/nginx.conf && \
rm -rf /etc/nginx/conf.d/*
ADD ./conf/nginx/dolphinscheduler.conf /etc/nginx/conf.d
COPY ./conf/nginx/dolphinscheduler.conf /etc/nginx/conf.d
#6. add configuration and modify permissions and set soft links
ADD ./checkpoint.sh /root/checkpoint.sh
ADD ./startup-init-conf.sh /root/startup-init-conf.sh
ADD ./startup.sh /root/startup.sh
ADD ./conf/dolphinscheduler/*.tpl /opt/dolphinscheduler/conf/
ADD ./conf/dolphinscheduler/logback/* /opt/dolphinscheduler/conf/
ADD conf/dolphinscheduler/env/dolphinscheduler_env.sh /opt/dolphinscheduler/conf/env/
COPY ./checkpoint.sh /root/checkpoint.sh
COPY ./startup-init-conf.sh /root/startup-init-conf.sh
COPY ./startup.sh /root/startup.sh
COPY ./conf/dolphinscheduler/*.tpl /opt/dolphinscheduler/conf/
COPY ./conf/dolphinscheduler/logback/* /opt/dolphinscheduler/conf/
COPY conf/dolphinscheduler/env/dolphinscheduler_env.sh /opt/dolphinscheduler/conf/env/
RUN chmod +x /root/checkpoint.sh && \
chmod +x /root/startup-init-conf.sh && \
chmod +x /root/startup.sh && \
@ -70,12 +71,12 @@ RUN chmod +x /root/checkpoint.sh && \
dos2unix /opt/dolphinscheduler/bin/*.sh && \
rm -rf /bin/sh && \
ln -s /bin/bash /bin/sh && \
mkdir -p /tmp/xls
#7. remove apk index cache and disable coredup for sudo
RUN rm -rf /var/cache/apk/* && \
mkdir -p /tmp/xls && \
#7. remove apk index cache and disable coredup for sudo
rm -rf /var/cache/apk/* && \
echo "Set disable_coredump false" >> /etc/sudo.conf
#8. expose port
EXPOSE 2181 2888 3888 5432 5678 1234 12345 50051 8888

6
dolphinscheduler-api/pom.xml

@ -162,6 +162,12 @@
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<exclusions>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>

4
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskRecordController.java

@ -22,11 +22,13 @@ import org.apache.dolphinscheduler.api.service.TaskRecordService;
import org.apache.dolphinscheduler.api.utils.Result;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.dao.entity.User;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.http.HttpStatus;
import org.springframework.web.bind.annotation.*;
import springfox.documentation.annotations.ApiIgnore;
import java.util.Map;
@ -59,7 +61,7 @@ public class TaskRecordController extends BaseController {
* @param taskDate task date
* @param startTime start time
* @param endTime end time
* @param pageNo page numbere
* @param pageNo page number
* @param pageSize page size
* @return task record list
*/

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

@ -193,7 +193,9 @@ public enum Status {
BATCH_COPY_PROCESS_DEFINITION_ERROR(10159, "batch copy process definition error", "复制工作流错误"),
BATCH_MOVE_PROCESS_DEFINITION_ERROR(10160, "batch move process definition error", "移动工作流错误"),
QUERY_WORKFLOW_LINEAGE_ERROR(10161, "query workflow lineage error", "查询血缘失败"),
DELETE_PROCESS_DEFINITION_BY_ID_FAIL(10162,"delete process definition by id fail, for there are {0} process instances in executing using it", "删除工作流定义失败,有[{0}]个运行中的工作流实例正在使用"),
DELETE_PROCESS_DEFINITION_BY_ID_FAIL(10162, "delete process definition by id fail, for there are {0} process instances in executing using it", "删除工作流定义失败,有[{0}]个运行中的工作流实例正在使用"),
CHECK_TENANT_CODE_ERROR(10163, "Please enter the English tenant code", "请输入英文租户编码"),
UDF_FUNCTION_NOT_EXIST(20001, "UDF function not found", "UDF函数不存在"),
UDF_FUNCTION_EXISTS(20002, "UDF function already exists", "UDF函数已存在"),

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

@ -37,6 +37,7 @@ import org.apache.dolphinscheduler.dao.utils.ResourceProcessDefinitionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.dao.DuplicateKeyException;
import org.springframework.stereotype.Service;
import org.springframework.transaction.annotation.Transactional;
import org.springframework.web.multipart.MultipartFile;
@ -139,6 +140,10 @@ public class ResourcesService extends BaseService {
}
}
result.setData(resultMap);
} catch (DuplicateKeyException e) {
logger.error("resource directory {} has exist, can't recreate", fullName);
putMsg(result, Status.RESOURCE_EXIST);
return result;
} catch (Exception e) {
logger.error("resource already exists, can't recreate ", e);
throw new RuntimeException("resource already exists, can't recreate");

44
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProcessDefinitionServiceImpl.java

@ -48,7 +48,7 @@ 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.task.AbstractParameters;
import org.apache.dolphinscheduler.common.process.ResourceInfo;
import org.apache.dolphinscheduler.common.thread.Stopper;
import org.apache.dolphinscheduler.common.utils.CollectionUtils;
import org.apache.dolphinscheduler.common.utils.DateUtils;
@ -77,6 +77,7 @@ import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
@ -84,6 +85,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;
@ -221,7 +223,7 @@ public class ProcessDefinitionServiceImpl extends BaseService implements
// return processDefinition object with ID
result.put(Constants.DATA_LIST, processDefineMapper.selectById(processDefine.getId()));
putMsg(result, Status.SUCCESS);
result.put("processDefinitionId", processDefine.getId());
result.put(PROCESSDEFINITIONID, processDefine.getId());
return result;
}
@ -232,25 +234,16 @@ public class ProcessDefinitionServiceImpl extends BaseService implements
* @return resource ids
*/
private String getResourceIds(ProcessData processData) {
List<TaskNode> tasks = processData.getTasks();
Set<Integer> resourceIds = new HashSet<>();
for (TaskNode taskNode : tasks) {
String taskParameter = taskNode.getParams();
AbstractParameters params = TaskParametersUtils.getParameters(taskNode.getType(), taskParameter);
if (CollectionUtils.isNotEmpty(params.getResourceFilesList())) {
Set<Integer> tempSet = params.getResourceFilesList().stream().map(t -> t.getId()).collect(Collectors.toSet());
resourceIds.addAll(tempSet);
}
}
StringBuilder sb = new StringBuilder();
for (int i : resourceIds) {
if (sb.length() > 0) {
sb.append(",");
}
sb.append(i);
}
return sb.toString();
return Optional.ofNullable(processData.getTasks())
.orElse(Collections.emptyList())
.stream()
.map(taskNode -> TaskParametersUtils.getParameters(taskNode.getType(), taskNode.getParams()))
.filter(Objects::nonNull)
.flatMap(parameters -> parameters.getResourceFilesList().stream())
.map(ResourceInfo::getId)
.distinct()
.map(Objects::toString)
.collect(Collectors.joining(","));
}
/**
@ -441,11 +434,11 @@ public class ProcessDefinitionServiceImpl extends BaseService implements
if (resultEnum != Status.SUCCESS) {
return checkResult;
}
ProcessDefinition processDefinition = processDefineMapper.queryByDefineName(project.getId(), name);
ProcessDefinition processDefinition = processDefineMapper.verifyByDefineName(project.getId(), name);
if (processDefinition == null) {
putMsg(result, Status.SUCCESS);
} else {
putMsg(result, Status.PROCESS_INSTANCE_EXIST, name);
putMsg(result, Status.VERIFY_PROCESS_DEFINITION_NAME_UNIQUE_ERROR, name);
}
return result;
}
@ -690,6 +683,7 @@ public class ProcessDefinitionServiceImpl extends BaseService implements
exportProcessMeta.setProjectName(processDefinition.getProjectName());
exportProcessMeta.setProcessDefinitionName(processDefinition.getName());
exportProcessMeta.setProcessDefinitionJson(processDefinition.getProcessDefinitionJson());
exportProcessMeta.setProcessDefinitionDescription(processDefinition.getDescription());
exportProcessMeta.setProcessDefinitionLocations(processDefinition.getLocations());
exportProcessMeta.setProcessDefinitionConnects(processDefinition.getConnects());
@ -845,7 +839,7 @@ public class ProcessDefinitionServiceImpl extends BaseService implements
try {
createProcessResult = createProcessDefinition(loginUser
, currentProjectName,
processDefinitionName + "_import_" + System.currentTimeMillis(),
processDefinitionName + "_import_" + DateUtils.getCurrentTimeStamp(),
importProcessParam,
processMeta.getProcessDefinitionDescription(),
processMeta.getProcessDefinitionLocations(),
@ -1433,7 +1427,7 @@ public class ProcessDefinitionServiceImpl extends BaseService implements
return createProcessDefinition(
loginUser,
targetProject.getName(),
processDefinition.getName() + "_copy_" + System.currentTimeMillis(),
processDefinition.getName() + "_copy_" + DateUtils.getCurrentTimeStamp(),
processDefinition.getProcessDefinitionJson(),
processDefinition.getDescription(),
processDefinition.getLocations(),

26
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TenantServiceImpl.java

@ -21,6 +21,7 @@ import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.service.BaseService;
import org.apache.dolphinscheduler.api.service.TenantService;
import org.apache.dolphinscheduler.api.utils.PageInfo;
import org.apache.dolphinscheduler.api.utils.RegexUtils;
import org.apache.dolphinscheduler.api.utils.Result;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.utils.CollectionUtils;
@ -73,11 +74,11 @@ public class TenantServiceImpl extends BaseService implements TenantService {
/**
* create tenant
*
* @param loginUser login user
* @param loginUser login user
* @param tenantCode tenant code
* @param tenantName tenant name
* @param queueId queue id
* @param desc description
* @param queueId queue id
* @param desc description
* @return create result code
* @throws Exception exception
*/
@ -94,6 +95,11 @@ public class TenantServiceImpl extends BaseService implements TenantService {
return result;
}
if (RegexUtils.isNumeric(tenantCode)) {
putMsg(result, Status.CHECK_TENANT_CODE_ERROR);
return result;
}
if (checkTenantExists(tenantCode)) {
putMsg(result, Status.REQUEST_PARAMS_NOT_VALID_ERROR, tenantCode);
return result;
@ -131,8 +137,8 @@ public class TenantServiceImpl extends BaseService implements TenantService {
*
* @param loginUser login user
* @param searchVal search value
* @param pageNo page number
* @param pageSize page size
* @param pageNo page number
* @param pageSize page size
* @return tenant list page
*/
public Map<String, Object> queryTenantList(User loginUser, String searchVal, Integer pageNo, Integer pageSize) {
@ -157,12 +163,12 @@ public class TenantServiceImpl extends BaseService implements TenantService {
/**
* updateProcessInstance tenant
*
* @param loginUser login user
* @param id tennat id
* @param loginUser login user
* @param id tennat id
* @param tenantCode tennat code
* @param tenantName tennat name
* @param queueId queue id
* @param desc description
* @param queueId queue id
* @param desc description
* @return update result code
* @throws Exception exception
*/
@ -229,7 +235,7 @@ public class TenantServiceImpl extends BaseService implements TenantService {
* delete tenant
*
* @param loginUser login user
* @param id tenant id
* @param id tenant id
* @return delete result code
* @throws Exception exception
*/

47
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/RegexUtils.java

@ -0,0 +1,47 @@
/*
* 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.utils;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
/**
* This is Regex expression utils.
*/
public class RegexUtils {
/**
* check number regex expression
*/
private static final String CHECK_NUMBER = "^-?\\d+(\\.\\d+)?$";
private RegexUtils() {
}
/**
* check if the input is number
*
* @param str input
* @return
*/
public static boolean isNumeric(String str) {
Pattern pattern = Pattern.compile(CHECK_NUMBER);
Matcher isNum = pattern.matcher(str);
return isNum.matches();
}
}

4
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ProcessDefinitionControllerTest.java

@ -117,14 +117,14 @@ public class ProcessDefinitionControllerTest {
public void testVerifyProcessDefinitionName() throws Exception {
Map<String, Object> result = new HashMap<>();
putMsg(result, Status.PROCESS_INSTANCE_EXIST);
putMsg(result, Status.VERIFY_PROCESS_DEFINITION_NAME_UNIQUE_ERROR);
String projectName = "test";
String name = "dag_test";
Mockito.when(processDefinitionService.verifyProcessDefinitionName(user, projectName, name)).thenReturn(result);
Result response = processDefinitionController.verifyProcessDefinitionName(user, projectName, name);
Assert.assertEquals(Status.PROCESS_INSTANCE_EXIST.getCode(), response.getCode().intValue());
Assert.assertEquals(Status.VERIFY_PROCESS_DEFINITION_NAME_UNIQUE_ERROR.getCode(), response.getCode().intValue());
}

90
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java

@ -17,6 +17,8 @@
package org.apache.dolphinscheduler.api.service;
import static org.assertj.core.api.Assertions.assertThat;
import org.apache.dolphinscheduler.api.dto.ProcessMeta;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.service.impl.ProcessDefinitionServiceImpl;
@ -28,6 +30,9 @@ import org.apache.dolphinscheduler.common.enums.Priority;
import org.apache.dolphinscheduler.common.enums.ReleaseState;
import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.common.enums.WarningType;
import org.apache.dolphinscheduler.common.model.TaskNode;
import org.apache.dolphinscheduler.common.process.ResourceInfo;
import org.apache.dolphinscheduler.common.task.shell.ShellParameters;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.FileUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
@ -51,8 +56,11 @@ import org.apache.http.entity.ContentType;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.lang.reflect.Method;
import java.text.MessageFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
@ -66,6 +74,7 @@ import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
import org.springframework.mock.web.MockMultipartFile;
import org.springframework.util.ReflectionUtils;
import org.springframework.web.multipart.MultipartFile;
import com.baomidou.mybatisplus.core.metadata.IPage;
@ -609,16 +618,16 @@ public class ProcessDefinitionServiceTest {
//project check auth success, process not exist
putMsg(result, Status.SUCCESS, projectName);
Mockito.when(processDefineMapper.queryByDefineName(project.getId(), "test_pdf")).thenReturn(null);
Mockito.when(processDefineMapper.verifyByDefineName(project.getId(), "test_pdf")).thenReturn(null);
Map<String, Object> processNotExistRes = processDefinitionService.verifyProcessDefinitionName(loginUser,
"project_test1", "test_pdf");
Assert.assertEquals(Status.SUCCESS, processNotExistRes.get(Constants.STATUS));
//process exist
Mockito.when(processDefineMapper.queryByDefineName(project.getId(), "test_pdf")).thenReturn(getProcessDefinition());
Mockito.when(processDefineMapper.verifyByDefineName(project.getId(), "test_pdf")).thenReturn(getProcessDefinition());
Map<String, Object> processExistRes = processDefinitionService.verifyProcessDefinitionName(loginUser,
"project_test1", "test_pdf");
Assert.assertEquals(Status.PROCESS_INSTANCE_EXIST, processExistRes.get(Constants.STATUS));
Assert.assertEquals(Status.VERIFY_PROCESS_DEFINITION_NAME_UNIQUE_ERROR, processExistRes.get(Constants.STATUS));
}
@Test
@ -984,6 +993,81 @@ public class ProcessDefinitionServiceTest {
loginUser, projectName, "1", null);
}
@Test
public void testGetResourceIds() throws Exception {
// set up
Method testMethod = ReflectionUtils.findMethod(ProcessDefinitionServiceImpl.class, "getResourceIds", ProcessData.class);
assertThat(testMethod).isNotNull();
testMethod.setAccessible(true);
// when processData has empty task, then return empty string
ProcessData input1 = new ProcessData();
input1.setTasks(Collections.emptyList());
String output1 = (String) testMethod.invoke(processDefinitionService, input1);
assertThat(output1).isEmpty();
// when task is null, then return empty string
ProcessData input2 = new ProcessData();
input2.setTasks(null);
String output2 = (String) testMethod.invoke(processDefinitionService, input2);
assertThat(output2).isEmpty();
// when task type is incorrect mapping, then return empty string
ProcessData input3 = new ProcessData();
TaskNode taskNode3 = new TaskNode();
taskNode3.setType("notExistType");
input3.setTasks(Collections.singletonList(taskNode3));
String output3 = (String) testMethod.invoke(processDefinitionService, input3);
assertThat(output3).isEmpty();
// when task parameter list is null, then return empty string
ProcessData input4 = new ProcessData();
TaskNode taskNode4 = new TaskNode();
taskNode4.setType("SHELL");
taskNode4.setParams(null);
input4.setTasks(Collections.singletonList(taskNode4));
String output4 = (String) testMethod.invoke(processDefinitionService, input4);
assertThat(output4).isEmpty();
// when resource id list is 0 1, then return 0,1
ProcessData input5 = new ProcessData();
TaskNode taskNode5 = new TaskNode();
taskNode5.setType("SHELL");
ShellParameters shellParameters5 = new ShellParameters();
ResourceInfo resourceInfo5A = new ResourceInfo();
resourceInfo5A.setId(0);
ResourceInfo resourceInfo5B = new ResourceInfo();
resourceInfo5B.setId(1);
shellParameters5.setResourceList(Arrays.asList(resourceInfo5A, resourceInfo5B));
taskNode5.setParams(JSONUtils.toJsonString(shellParameters5));
input5.setTasks(Collections.singletonList(taskNode5));
String output5 = (String) testMethod.invoke(processDefinitionService, input5);
assertThat(output5.split(",")).hasSize(2)
.containsExactlyInAnyOrder("0", "1");
// when resource id list is 0 1 1 2, then return 0,1,2
ProcessData input6 = new ProcessData();
TaskNode taskNode6 = new TaskNode();
taskNode6.setType("SHELL");
ShellParameters shellParameters6 = new ShellParameters();
ResourceInfo resourceInfo6A = new ResourceInfo();
resourceInfo6A.setId(0);
ResourceInfo resourceInfo6B = new ResourceInfo();
resourceInfo6B.setId(1);
ResourceInfo resourceInfo6C = new ResourceInfo();
resourceInfo6C.setId(1);
ResourceInfo resourceInfo6D = new ResourceInfo();
resourceInfo6D.setId(2);
shellParameters6.setResourceList(Arrays.asList(resourceInfo6A, resourceInfo6B, resourceInfo6C, resourceInfo6D));
taskNode6.setParams(JSONUtils.toJsonString(shellParameters6));
input6.setTasks(Collections.singletonList(taskNode6));
String output6 = (String) testMethod.invoke(processDefinitionService, input6);
assertThat(output6.split(",")).hasSize(3)
.containsExactlyInAnyOrder("0", "1", "2");
}
/**
* get mock datasource
*

39
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/RegexUtilsTest.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.api.utils;
import org.junit.Assert;
import org.junit.Test;
/**
* RegexUtils test case
*/
public class RegexUtilsTest {
@Test
public void testIsNumeric() {
String num1 = "123467854678";
boolean numeric = RegexUtils.isNumeric(num1);
Assert.assertTrue(numeric);
String num2 = "0.0.01";
boolean numeric2 = RegexUtils.isNumeric(num2);
Assert.assertFalse(numeric2);
}
}

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

@ -266,6 +266,10 @@ public final class Constants {
*/
public static final String YYYYMMDDHHMMSS = "yyyyMMddHHmmss";
/**
* date format of yyyyMMddHHmmssSSS
*/
public static final String YYYYMMDDHHMMSSSSS = "yyyyMMddHHmmssSSS";
/**
* http connect time out
*/
@ -1000,4 +1004,9 @@ public final class Constants {
public static final String DATASOURCE_ENCRYPTION_SALT_DEFAULT = "!@#$%^&*";
public static final String DATASOURCE_ENCRYPTION_ENABLE = "datasource.encryption.enable";
public static final String DATASOURCE_ENCRYPTION_SALT = "datasource.encryption.salt";
/**
* Network IP gets priority, default inner outer
*/
public static final String NETWORK_PRIORITY_STRATEGY = "dolphin.scheduler.network.priority.strategy";
}

78
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/TaskParams.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.common.task;
import java.util.Map;
public class TaskParams {
private String rawScript;
private Map<String, String>[] localParams;
public void setRawScript(String rawScript) {
this.rawScript = rawScript;
}
public void setLocalParams(Map<String, String>[] localParams) {
this.localParams = localParams;
}
public String getRawScript() {
return rawScript;
}
public void setLocalParamValue(String prop, Object value) {
if (localParams == null || value == null) {
return;
}
for (int i = 0; i < localParams.length; i++) {
if (localParams[i].get("prop").equals(prop)) {
localParams[i].put("value", (String)value);
}
}
}
public void setLocalParamValue(Map<String, Object> propToValue) {
if (localParams == null || propToValue == null) {
return;
}
for (int i = 0; i < localParams.length; i++) {
String prop = localParams[i].get("prop");
if (propToValue.containsKey(prop)) {
localParams[i].put("value",(String)propToValue.get(prop));
}
}
}
public String getLocalParamValue(String prop) {
if (localParams == null) {
return null;
}
for (int i = 0; i < localParams.length; i++) {
String tmpProp = localParams[i].get("prop");
if (tmpProp.equals(prop)) {
return localParams[i].get("value");
}
}
return null;
}
public Map<String, String>[] getLocalParams() {
return localParams;
}
}

10
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/DateUtils.java

@ -444,4 +444,14 @@ public class DateUtils {
long usedTime = (System.currentTimeMillis() - baseTime.getTime()) / 1000;
return intervalSeconds - usedTime;
}
/**
* get current time stamp : yyyyMMddHHmmssSSS
*
* @return date string
*/
public static String getCurrentTimeStamp() {
return getCurrentTime(Constants.YYYYMMDDHHMMSSSSS);
}
}

45
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/HadoopUtils.java

@ -14,26 +14,31 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.common.utils;
import com.fasterxml.jackson.databind.node.ObjectNode;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import org.apache.commons.io.IOUtils;
import static org.apache.dolphinscheduler.common.Constants.RESOURCE_UPLOAD_PATH;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.ResUploadType;
import org.apache.dolphinscheduler.common.enums.ResourceType;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.*;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.client.cli.RMAdminCLI;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.*;
import java.io.BufferedReader;
import java.io.Closeable;
import java.io.File;
import java.io.IOException;
import java.io.InputStreamReader;
import java.nio.file.Files;
import java.security.PrivilegedExceptionAction;
import java.util.Collections;
@ -43,7 +48,13 @@ import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.dolphinscheduler.common.Constants.RESOURCE_UPLOAD_PATH;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.node.ObjectNode;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
/**
* hadoop utils
@ -102,7 +113,6 @@ public class HadoopUtils implements Closeable {
}
}
/**
* init hadoop configuration
*/
@ -167,7 +177,6 @@ public class HadoopUtils implements Closeable {
fs = FileSystem.get(configuration);
}
} catch (Exception e) {
logger.error(e.getMessage(), e);
}
@ -232,11 +241,11 @@ public class HadoopUtils implements Closeable {
return new byte[0];
}
FSDataInputStream fsDataInputStream = fs.open(new Path(hdfsFilePath));
return IOUtils.toByteArray(fsDataInputStream);
try (FSDataInputStream fsDataInputStream = fs.open(new Path(hdfsFilePath))) {
return IOUtils.toByteArray(fsDataInputStream);
}
}
/**
* cat file on hdfs
*
@ -527,7 +536,6 @@ public class HadoopUtils implements Closeable {
return String.format("%s/udfs", getHdfsTenantDir(tenantCode));
}
/**
* get hdfs file name
*
@ -579,7 +587,6 @@ public class HadoopUtils implements Closeable {
return String.format("%s/%s", getHdfsDataBasePath(), tenantCode);
}
/**
* getAppAddress
*
@ -610,7 +617,6 @@ public class HadoopUtils implements Closeable {
return start + activeRM + end;
}
@Override
public void close() throws IOException {
if (fs != null) {
@ -623,7 +629,6 @@ public class HadoopUtils implements Closeable {
}
}
/**
* yarn ha admin utils
*/
@ -669,7 +674,6 @@ public class HadoopUtils implements Closeable {
return null;
}
/**
* get ResourceManager state
*
@ -694,4 +698,5 @@ public class HadoopUtils implements Closeable {
}
}
}

121
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/NetUtils.java

@ -21,6 +21,8 @@ import static org.apache.dolphinscheduler.common.Constants.DOLPHIN_SCHEDULER_PRE
import static java.util.Collections.emptyList;
import org.apache.dolphinscheduler.common.Constants;
import java.io.IOException;
import java.net.Inet6Address;
import java.net.InetAddress;
@ -42,22 +44,20 @@ import org.slf4j.LoggerFactory;
*/
public class NetUtils {
private NetUtils() {
throw new UnsupportedOperationException("Construct NetUtils");
}
private static Logger logger = LoggerFactory.getLogger(NetUtils.class);
private static final Pattern IP_PATTERN = Pattern.compile("\\d{1,3}(\\.\\d{1,3}){3,5}$");
private static String ANY_HOST_VALUE = "0.0.0.0";
private static String LOCAL_HOST_VALUE = "127.0.0.1";
private static final String NETWORK_PRIORITY_DEFAULT = "default";
private static final String NETWORK_PRIORITY_INNER = "inner";
private static final String NETWORK_PRIORITY_OUTER = "outer";
private static final Logger logger = LoggerFactory.getLogger(NetUtils.class);
private static final String ANY_HOST_VALUE = "0.0.0.0";
private static final String LOCAL_HOST_VALUE = "127.0.0.1";
private static InetAddress LOCAL_ADDRESS = null;
private static volatile String HOST_ADDRESS;
private NetUtils() {
throw new UnsupportedOperationException("Construct NetUtils");
}
public static String getHost() {
if (HOST_ADDRESS != null) {
return HOST_ADDRESS;
@ -87,24 +87,27 @@ public class NetUtils {
if (null != LOCAL_ADDRESS) {
return LOCAL_ADDRESS;
}
InetAddress localAddress = null;
NetworkInterface networkInterface = findNetworkInterface();
Enumeration<InetAddress> addresses = networkInterface.getInetAddresses();
while (addresses.hasMoreElements()) {
Optional<InetAddress> addressOp = toValidAddress(addresses.nextElement());
if (addressOp.isPresent()) {
try {
if (addressOp.get().isReachable(100)) {
LOCAL_ADDRESS = addressOp.get();
return LOCAL_ADDRESS;
try {
NetworkInterface networkInterface = findNetworkInterface();
if (networkInterface != null) {
Enumeration<InetAddress> addresses = networkInterface.getInetAddresses();
while (addresses.hasMoreElements()) {
Optional<InetAddress> addressOp = toValidAddress(addresses.nextElement());
if (addressOp.isPresent()) {
try {
if (addressOp.get().isReachable(100)) {
LOCAL_ADDRESS = addressOp.get();
return LOCAL_ADDRESS;
}
} catch (IOException e) {
logger.warn("test address id reachable io exception", e);
}
}
} catch (IOException e) {
logger.warn("test address id reachable io exception", e);
}
}
}
try {
localAddress = InetAddress.getLocalHost();
} catch (UnknownHostException e) {
logger.warn("InetAddress get LocalHost exception", e);
@ -190,7 +193,7 @@ public class NetUtils {
if (null != result) {
return result;
}
return validNetworkInterfaces.get(0);
return findAddress(validNetworkInterfaces);
}
/**
@ -227,4 +230,70 @@ public class NetUtils {
String preferredNetworkInterface = System.getProperty(DOLPHIN_SCHEDULER_PREFERRED_NETWORK_INTERFACE);
return Objects.equals(networkInterface.getDisplayName(), preferredNetworkInterface);
}
private static NetworkInterface findAddress(List<NetworkInterface> validNetworkInterfaces) {
if (validNetworkInterfaces.isEmpty()) {
return null;
}
String networkPriority = PropertyUtils.getString(Constants.NETWORK_PRIORITY_STRATEGY, NETWORK_PRIORITY_DEFAULT);
if (NETWORK_PRIORITY_DEFAULT.equalsIgnoreCase(networkPriority)) {
return findAddressByDefaultPolicy(validNetworkInterfaces);
} else if (NETWORK_PRIORITY_INNER.equalsIgnoreCase(networkPriority)) {
return findInnerAddress(validNetworkInterfaces);
} else if (NETWORK_PRIORITY_OUTER.equalsIgnoreCase(networkPriority)) {
return findOuterAddress(validNetworkInterfaces);
} else {
logger.error("There is no matching network card acquisition policy!");
return null;
}
}
private static NetworkInterface findAddressByDefaultPolicy(List<NetworkInterface> validNetworkInterfaces) {
NetworkInterface networkInterface;
networkInterface = findInnerAddress(validNetworkInterfaces);
if (networkInterface == null) {
networkInterface = findOuterAddress(validNetworkInterfaces);
if (networkInterface == null) {
networkInterface = validNetworkInterfaces.get(0);
}
}
return networkInterface;
}
/**
* Get the Intranet IP
*
* @return If no {@link NetworkInterface} is available , return <code>null</code>
*/
private static NetworkInterface findInnerAddress(List<NetworkInterface> validNetworkInterfaces) {
NetworkInterface networkInterface = null;
for (NetworkInterface ni : validNetworkInterfaces) {
Enumeration<InetAddress> address = ni.getInetAddresses();
while (address.hasMoreElements()) {
InetAddress ip = address.nextElement();
if (ip.isSiteLocalAddress()
&& !ip.isLoopbackAddress()) {
networkInterface = ni;
}
}
}
return networkInterface;
}
private static NetworkInterface findOuterAddress(List<NetworkInterface> validNetworkInterfaces) {
NetworkInterface networkInterface = null;
for (NetworkInterface ni : validNetworkInterfaces) {
Enumeration<InetAddress> address = ni.getInetAddresses();
while (address.hasMoreElements()) {
InetAddress ip = address.nextElement();
if (!ip.isSiteLocalAddress()
&& !ip.isLoopbackAddress()) {
networkInterface = ni;
}
}
}
return networkInterface;
}
}

72
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtils.java

@ -55,42 +55,42 @@ public class TaskParametersUtils {
* @return task parameters
*/
public static AbstractParameters getParameters(String taskType, String parameter) {
try {
switch (EnumUtils.getEnum(TaskType.class, taskType)) {
case SUB_PROCESS:
return JSONUtils.parseObject(parameter, SubProcessParameters.class);
case WATERDROP:
return JSONUtils.parseObject(parameter, ShellParameters.class);
case SHELL:
return JSONUtils.parseObject(parameter, ShellParameters.class);
case PROCEDURE:
return JSONUtils.parseObject(parameter, ProcedureParameters.class);
case SQL:
return JSONUtils.parseObject(parameter, SqlParameters.class);
case MR:
return JSONUtils.parseObject(parameter, MapreduceParameters.class);
case SPARK:
return JSONUtils.parseObject(parameter, SparkParameters.class);
case PYTHON:
return JSONUtils.parseObject(parameter, PythonParameters.class);
case DEPENDENT:
return JSONUtils.parseObject(parameter, DependentParameters.class);
case FLINK:
return JSONUtils.parseObject(parameter, FlinkParameters.class);
case HTTP:
return JSONUtils.parseObject(parameter, HttpParameters.class);
case DATAX:
return JSONUtils.parseObject(parameter, DataxParameters.class);
case CONDITIONS:
return JSONUtils.parseObject(parameter, ConditionsParameters.class);
case SQOOP:
return JSONUtils.parseObject(parameter, SqoopParameters.class);
default:
return null;
}
} catch (Exception e) {
logger.error(e.getMessage(), e);
TaskType anEnum = EnumUtils.getEnum(TaskType.class, taskType);
if (anEnum == null) {
logger.error("not support task type: {}", taskType);
return null;
}
return null;
switch (anEnum) {
case SUB_PROCESS:
return JSONUtils.parseObject(parameter, SubProcessParameters.class);
case SHELL:
case WATERDROP:
return JSONUtils.parseObject(parameter, ShellParameters.class);
case PROCEDURE:
return JSONUtils.parseObject(parameter, ProcedureParameters.class);
case SQL:
return JSONUtils.parseObject(parameter, SqlParameters.class);
case MR:
return JSONUtils.parseObject(parameter, MapreduceParameters.class);
case SPARK:
return JSONUtils.parseObject(parameter, SparkParameters.class);
case PYTHON:
return JSONUtils.parseObject(parameter, PythonParameters.class);
case DEPENDENT:
return JSONUtils.parseObject(parameter, DependentParameters.class);
case FLINK:
return JSONUtils.parseObject(parameter, FlinkParameters.class);
case HTTP:
return JSONUtils.parseObject(parameter, HttpParameters.class);
case DATAX:
return JSONUtils.parseObject(parameter, DataxParameters.class);
case CONDITIONS:
return JSONUtils.parseObject(parameter, ConditionsParameters.class);
case SQOOP:
return JSONUtils.parseObject(parameter, SqoopParameters.class);
default:
return null;
}
}
}

124
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/VarPoolUtils.java

@ -0,0 +1,124 @@
/*
* 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.utils;
import org.apache.dolphinscheduler.common.model.TaskNode;
import org.apache.dolphinscheduler.common.task.TaskParams;
import java.text.ParseException;
import java.util.Map;
public class VarPoolUtils {
/**
* getTaskNodeLocalParam
* @param taskNode taskNode
* @param prop prop
* @return localParamForProp
*/
public static Object getTaskNodeLocalParam(TaskNode taskNode, String prop) {
String taskParamsJson = taskNode.getParams();
TaskParams taskParams = JSONUtils.parseObject(taskParamsJson, TaskParams.class);
if (taskParams == null) {
return null;
}
return taskParams.getLocalParamValue(prop);
}
/**
* setTaskNodeLocalParams
* @param taskNode taskNode
* @param prop LocalParamName
* @param value LocalParamValue
*/
public static void setTaskNodeLocalParams(TaskNode taskNode, String prop, Object value) {
String taskParamsJson = taskNode.getParams();
TaskParams taskParams = JSONUtils.parseObject(taskParamsJson, TaskParams.class);
if (taskParams == null) {
return;
}
taskParams.setLocalParamValue(prop, value);
taskNode.setParams(JSONUtils.toJsonString(taskParams));
}
/**
* setTaskNodeLocalParams
* @param taskNode taskNode
* @param propToValue propToValue
*/
public static void setTaskNodeLocalParams(TaskNode taskNode, Map<String, Object> propToValue) {
String taskParamsJson = taskNode.getParams();
TaskParams taskParams = JSONUtils.parseObject(taskParamsJson, TaskParams.class);
if (taskParams == null) {
return;
}
taskParams.setLocalParamValue(propToValue);
taskNode.setParams(JSONUtils.toJsonString(taskParams));
}
/**
* convertVarPoolToMap
* @param propToValue propToValue
* @param varPool varPool
* @throws ParseException ParseException
*/
public static void convertVarPoolToMap(Map<String, Object> propToValue, String varPool) throws ParseException {
if (varPool == null || propToValue == null) {
return;
}
String[] splits = varPool.split("\\$VarPool\\$");
for (String kv : splits) {
String[] kvs = kv.split(",");
if (kvs.length == 2) {
propToValue.put(kvs[0], kvs[1]);
} else {
throw new ParseException(kv, 2);
}
}
}
/**
* convertPythonScriptPlaceholders
* @param rawScript rawScript
* @return String
* @throws StringIndexOutOfBoundsException StringIndexOutOfBoundsException
*/
public static String convertPythonScriptPlaceholders(String rawScript) throws StringIndexOutOfBoundsException {
int len = "${setShareVar(${".length();
int scriptStart = 0;
while ((scriptStart = rawScript.indexOf("${setShareVar(${", scriptStart)) != -1) {
int start = -1;
int end = rawScript.indexOf('}', scriptStart + len);
String prop = rawScript.substring(scriptStart + len, end);
start = rawScript.indexOf(',', end);
end = rawScript.indexOf(')', start);
String value = rawScript.substring(start + 1, end);
start = rawScript.indexOf('}', start) + 1;
end = rawScript.length();
String replaceScript = String.format("print(\"${{setValue({},{})}}\".format(\"%s\",%s))", prop, value);
rawScript = rawScript.substring(0, scriptStart) + replaceScript + rawScript.substring(start, end);
scriptStart += replaceScript.length();
}
return rawScript;
}
}

3
dolphinscheduler-common/src/main/resources/common.properties

@ -72,3 +72,6 @@ kerberos.expire.time=2
# datasource encryption salt
datasource.encryption.enable=false
datasource.encryption.salt=!@#$%^&*
# Network IP gets priority, default inner outer
#dolphin.scheduler.network.priority.strategy=default

19
dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/DateUtilsTest.java

@ -14,14 +14,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.common.utils;
import org.junit.Assert;
import org.junit.Test;
import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.Date;
import org.junit.Assert;
import org.junit.Test;
public class DateUtilsTest {
@Test
public void format2Readable() throws ParseException {
@ -38,10 +40,8 @@ public class DateUtilsTest {
Assert.assertEquals("01 09:23:08", readableDate);
}
@Test
public void testWeek(){
public void testWeek() {
Date curr = DateUtils.stringToDate("2019-02-01 00:00:00");
Date monday1 = DateUtils.stringToDate("2019-01-28 00:00:00");
Date sunday1 = DateUtils.stringToDate("2019-02-03 00:00:00");
@ -54,7 +54,7 @@ public class DateUtilsTest {
}
@Test
public void diffHours(){
public void diffHours() {
Date d1 = DateUtils.stringToDate("2019-01-28 00:00:00");
Date d2 = DateUtils.stringToDate("2019-01-28 20:00:00");
Assert.assertEquals(DateUtils.diffHours(d1, d2), 20);
@ -150,4 +150,11 @@ public class DateUtilsTest {
Date curr = DateUtils.getEndOfHour(d1);
Assert.assertEquals(DateUtils.dateToString(curr), "2019-01-31 11:59:59");
}
@Test
public void getCurrentTimeStamp() {
String timeStamp = DateUtils.getCurrentTimeStamp();
Assert.assertNotNull(timeStamp);
}
}

73
dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/VarPoolUtilsTest.java

@ -0,0 +1,73 @@
/*
* 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.utils;
import org.apache.dolphinscheduler.common.model.TaskNode;
import java.util.concurrent.ConcurrentHashMap;
import org.junit.Assert;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class VarPoolUtilsTest {
private static final Logger logger = LoggerFactory.getLogger(VarPoolUtilsTest.class);
@Test
public void testSetTaskNodeLocalParams() {
String taskJson = "{\"conditionResult\":\"{\\\"successNode\\\":[\\\"\\\"],\\\"failedNode\\\":[\\\"\\\"]}\","
+ "\"conditionsTask\":false,\"depList\":[],\"dependence\":\"{}\",\"forbidden\":false,\"id\":\"tasks-75298\",\"maxRetryTimes\":0,\"name\":\"a1\","
+ "\"params\":\"{\\\"rawScript\\\":\\\"print(\\\\\\\"this is python task \\\\\\\",${p0})\\\","
+ "\\\"localParams\\\":[{\\\"prop\\\":\\\"p1\\\",\\\"direct\\\":\\\"IN\\\",\\\"type\\\":\\\"VARCHAR\\\",\\\"value\\\":\\\"1\\\"}],"
+ "\\\"resourceList\\\":[]}\",\"preTasks\":\"[]\",\"retryInterval\":1,\"runFlag\":\"NORMAL\",\"taskInstancePriority\":\"MEDIUM\","
+ "\"taskTimeoutParameter\":{\"enable\":false,\"interval\":0},\"timeout\":\"{\\\"enable\\\":false,\\\"strategy\\\":\\\"\\\"}\","
+ "\"type\":\"PYTHON\",\"workerGroup\":\"default\"}";
TaskNode taskNode = JSONUtils.parseObject(taskJson, TaskNode.class);
VarPoolUtils.setTaskNodeLocalParams(taskNode, "p1", "test1");
Assert.assertEquals(VarPoolUtils.getTaskNodeLocalParam(taskNode, "p1"), "test1");
ConcurrentHashMap<String, Object> propToValue = new ConcurrentHashMap<String, Object>();
propToValue.put("p1", "test2");
VarPoolUtils.setTaskNodeLocalParams(taskNode, propToValue);
Assert.assertEquals(VarPoolUtils.getTaskNodeLocalParam(taskNode, "p1"), "test2");
}
@Test
public void testConvertVarPoolToMap() throws Exception {
String varPool = "p1,66$VarPool$p2,69$VarPool$";
ConcurrentHashMap<String, Object> propToValue = new ConcurrentHashMap<String, Object>();
VarPoolUtils.convertVarPoolToMap(propToValue, varPool);
Assert.assertEquals((String)propToValue.get("p1"), "66");
Assert.assertEquals((String)propToValue.get("p2"), "69");
logger.info(propToValue.toString());
}
@Test
public void testConvertPythonScriptPlaceholders() throws Exception {
String rawScript = "print(${p1});\n${setShareVar(${p1},3)};\n${setShareVar(${p2},4)};";
rawScript = VarPoolUtils.convertPythonScriptPlaceholders(rawScript);
Assert.assertEquals(rawScript, "print(${p1});\n"
+ "print(\"${{setValue({},{})}}\".format(\"p1\",3));\n"
+ "print(\"${{setValue({},{})}}\".format(\"p2\",4));");
logger.info(rawScript);
}
}

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

@ -14,18 +14,27 @@
* 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.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.TaskDependType;
import org.apache.dolphinscheduler.common.enums.WarningType;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import java.util.Date;
import java.util.Objects;
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 com.baomidou.mybatisplus.core.toolkit.StringUtils;
import com.fasterxml.jackson.annotation.JsonFormat;
import org.apache.dolphinscheduler.common.enums.*;
import java.util.Date;
import java.util.Objects;
/**
* process instance
@ -36,7 +45,7 @@ public class ProcessInstance {
/**
* id
*/
@TableId(value="id", type=IdType.AUTO)
@TableId(value = "id", type = IdType.AUTO)
private int id;
/**
* process definition id
@ -53,13 +62,13 @@ public class ProcessInstance {
/**
* start time
*/
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss",timezone="GMT+8")
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date startTime;
/**
* end time
*/
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss",timezone="GMT+8")
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date endTime;
/**
@ -108,7 +117,7 @@ public class ProcessInstance {
private FailureStrategy failureStrategy;
/**
* warning type
* warning type
*/
private WarningType warningType;
@ -120,13 +129,13 @@ public class ProcessInstance {
/**
* schedule time
*/
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss",timezone="GMT+8")
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date scheduleTime;
/**
* command start time
*/
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss",timezone="GMT+8")
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date commandStartTime;
/**
@ -189,6 +198,7 @@ public class ProcessInstance {
/**
* process duration
*
* @return
*/
@TableField(exist = false)
@ -214,6 +224,11 @@ public class ProcessInstance {
*/
private int tenantId;
/**
* varPool string
*/
private String varPool;
/**
* receivers for api
*/
@ -226,21 +241,34 @@ public class ProcessInstance {
@TableField(exist = false)
private String receiversCc;
public ProcessInstance(){
public ProcessInstance() {
}
/**
* set the process name with process define version and timestamp
*
* @param processDefinition processDefinition
*/
public ProcessInstance(ProcessDefinition processDefinition){
public ProcessInstance(ProcessDefinition processDefinition) {
this.processDefinition = processDefinition;
this.name = processDefinition.getName() + "-" +
processDefinition.getVersion() + "-" +
System.currentTimeMillis();
this.name = processDefinition.getName()
+ "-"
+
processDefinition.getVersion()
+ "-"
+
DateUtils.getCurrentTimeStamp();
}
public String getVarPool() {
return varPool;
}
public void setVarPool(String varPool) {
this.varPool = varPool;
}
public ProcessDefinition getProcessDefinition() {
return processDefinition;
}
@ -313,7 +341,6 @@ public class ProcessInstance {
this.name = name;
}
public String getHost() {
return host;
}
@ -322,7 +349,6 @@ public class ProcessInstance {
this.host = host;
}
public CommandType getCommandType() {
return commandType;
}
@ -347,7 +373,6 @@ public class ProcessInstance {
this.taskDependType = taskDependType;
}
public int getMaxTryTimes() {
return maxTryTimes;
}
@ -364,8 +389,7 @@ public class ProcessInstance {
this.failureStrategy = failureStrategy;
}
public boolean isProcessInstanceStop(){
public boolean isProcessInstanceStop() {
return this.state.typeIsFinished();
}
@ -441,7 +465,6 @@ public class ProcessInstance {
this.executorId = executorId;
}
public Flag getIsSubProcess() {
return isSubProcess;
}
@ -457,6 +480,7 @@ public class ProcessInstance {
public void setProcessInstancePriority(Priority processInstancePriority) {
this.processInstancePriority = processInstancePriority;
}
public String getLocations() {
return locations;
}
@ -477,6 +501,10 @@ public class ProcessInstance {
return historyCmd;
}
public void setHistoryCmd(String historyCmd) {
this.historyCmd = historyCmd;
}
public String getExecutorName() {
return executorName;
}
@ -485,28 +513,26 @@ public class ProcessInstance {
this.executorName = executorName;
}
public void setHistoryCmd(String historyCmd) {
this.historyCmd = historyCmd;
}
/**
* add command to history
*
* @param cmd cmd
*/
public void addHistoryCmd(CommandType cmd){
if(StringUtils.isNotEmpty(this.historyCmd)){
public void addHistoryCmd(CommandType cmd) {
if (StringUtils.isNotEmpty(this.historyCmd)) {
this.historyCmd = String.format("%s,%s", this.historyCmd, cmd.toString());
}else{
} else {
this.historyCmd = cmd.toString();
}
}
/**
* check this process is start complement data
*
* @return whether complement data
*/
public boolean isComplementData(){
if(StringUtils.isEmpty(this.historyCmd)){
public boolean isComplementData() {
if (StringUtils.isEmpty(this.historyCmd)) {
return false;
}
return historyCmd.startsWith(CommandType.COMPLEMENT_DATA.toString());
@ -515,10 +541,11 @@ public class ProcessInstance {
/**
* get current command type,
* if start with complement data,return complement
*
* @return CommandType
*/
public CommandType getCmdTypeIfComplement(){
if(isComplementData()){
public CommandType getCmdTypeIfComplement() {
if (isComplementData()) {
return CommandType.COMPLEMENT_DATA;
}
return commandType;
@ -556,15 +583,14 @@ public class ProcessInstance {
this.timeout = timeout;
}
public int getTenantId() {
return this.tenantId;
}
public void setTenantId(int tenantId) {
this.tenantId = tenantId;
}
public int getTenantId() {
return this.tenantId ;
}
public String getReceivers() {
return receivers;
}
@ -583,44 +609,83 @@ public class ProcessInstance {
@Override
public String toString() {
return "ProcessInstance{" +
"id=" + id +
", processDefinitionId=" + processDefinitionId +
", state=" + state +
", recovery=" + recovery +
", startTime=" + startTime +
", endTime=" + endTime +
", runTimes=" + runTimes +
", name='" + name + '\'' +
", host='" + host + '\'' +
", processDefinition=" + processDefinition +
", commandType=" + commandType +
", commandParam='" + commandParam + '\'' +
", taskDependType=" + taskDependType +
", maxTryTimes=" + maxTryTimes +
", failureStrategy=" + failureStrategy +
", warningType=" + warningType +
", warningGroupId=" + warningGroupId +
", scheduleTime=" + scheduleTime +
", commandStartTime=" + commandStartTime +
", globalParams='" + globalParams + '\'' +
", processInstanceJson='" + processInstanceJson + '\'' +
", executorId=" + executorId +
", tenantCode='" + tenantCode + '\'' +
", queue='" + queue + '\'' +
", isSubProcess=" + isSubProcess +
", locations='" + locations + '\'' +
", connects='" + connects + '\'' +
", historyCmd='" + historyCmd + '\'' +
", dependenceScheduleTimes='" + dependenceScheduleTimes + '\'' +
", duration=" + duration +
", processInstancePriority=" + processInstancePriority +
", workerGroup='" + workerGroup + '\'' +
", timeout=" + timeout +
", tenantId=" + tenantId +
", receivers='" + receivers + '\'' +
", receiversCc='" + receiversCc + '\'' +
'}';
return "ProcessInstance{"
+ "id=" + id
+ ", processDefinitionId=" + processDefinitionId
+ ", state=" + state
+ ", recovery=" + recovery
+ ", startTime=" + startTime
+ ", endTime=" + endTime
+ ", runTimes=" + runTimes
+ ", name='" + name + '\''
+ ", host='" + host + '\''
+ ", processDefinition="
+ processDefinition
+ ", commandType="
+ commandType
+ ", commandParam='"
+ commandParam
+ '\''
+ ", taskDependType="
+ taskDependType
+ ", maxTryTimes="
+ maxTryTimes
+ ", failureStrategy="
+ failureStrategy
+ ", warningType="
+ warningType
+ ", warningGroupId="
+ warningGroupId
+ ", scheduleTime="
+ scheduleTime
+ ", commandStartTime="
+ commandStartTime
+ ", globalParams='"
+ globalParams
+ '\''
+ ", processInstanceJson='"
+ processInstanceJson
+ '\''
+ ", executorId="
+ executorId
+ ", tenantCode='"
+ tenantCode
+ '\''
+ ", queue='"
+ queue
+ '\''
+ ", isSubProcess="
+ isSubProcess
+ ", locations='"
+ locations
+ '\''
+ ", connects='"
+ connects
+ '\''
+ ", historyCmd='"
+ historyCmd
+ '\''
+ ", dependenceScheduleTimes='"
+ dependenceScheduleTimes
+ '\''
+ ", duration="
+ duration
+ ", processInstancePriority="
+ processInstancePriority
+ ", workerGroup='"
+ workerGroup
+ '\''
+ ", timeout="
+ timeout
+ ", tenantId="
+ tenantId
+ ", receivers='"
+ receivers
+ '\''
+ ", receiversCc='"
+ receiversCc
+ '\''
+ '}';
}
@Override

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

@ -211,6 +211,11 @@ public class TaskInstance implements Serializable {
*/
private int executorId;
/**
* varPool string
*/
private String varPool;
/**
* executor name
*/
@ -232,7 +237,14 @@ public class TaskInstance implements Serializable {
this.executePath = executePath;
}
public String getVarPool() {
return varPool;
}
public void setVarPool(String varPool) {
this.varPool = varPool;
}
public ProcessInstance getProcessInstance() {
return processInstance;
}

9
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionMapper.java

@ -34,6 +34,15 @@ import com.baomidou.mybatisplus.core.metadata.IPage;
*/
public interface ProcessDefinitionMapper extends BaseMapper<ProcessDefinition> {
/**
* verify process definition by name
*
* @param projectId projectId
* @param name name
* @return process definition
*/
ProcessDefinition verifyByDefineName(@Param("projectId") int projectId,
@Param("processDefinitionName") String name);
/**
* query process definition by name

6
dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionMapper.xml

@ -18,6 +18,12 @@
<!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.ProcessDefinitionMapper">
<select id="verifyByDefineName" resultType="org.apache.dolphinscheduler.dao.entity.ProcessDefinition">
select pd.*
from t_ds_process_definition pd
WHERE pd.project_id = #{projectId}
and pd.name = #{processDefinitionName}
</select>
<select id="queryByDefineName" resultType="org.apache.dolphinscheduler.dao.entity.ProcessDefinition">
select pd.*,u.user_name,p.name as project_name,t.tenant_code,t.tenant_name,q.queue,q.queue_name
from t_ds_process_definition pd

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

@ -167,7 +167,7 @@
<select id="queryLastRunningProcess" resultType="org.apache.dolphinscheduler.dao.entity.ProcessInstance">
select *
from t_ds_process_instance
where 1=1
where process_definition_id=#{processDefinitionId}
<if test="states !=null and states.length != 0">
and state in
<foreach collection="states" item="i" index="index" open="(" separator="," close=")">
@ -175,7 +175,6 @@
</foreach>
</if>
<if test="startTime!=null and endTime != null ">
and process_definition_id=#{processDefinitionId}
and (schedule_time <![CDATA[ >= ]]> #{startTime} and schedule_time <![CDATA[ <= ]]> #{endTime}
or start_time <![CDATA[ >= ]]> #{startTime} and start_time <![CDATA[ <= ]]> #{endTime})
</if>

41
dolphinscheduler-dist/release-docs/LICENSE vendored

@ -230,7 +230,7 @@ The text of each license is also included at licenses/LICENSE-[project].txt.
clickhouse-jdbc 0.1.52: https://mvnrepository.com/artifact/ru.yandex.clickhouse/clickhouse-jdbc/0.1.52, Apache 2.0
commons-beanutils 1.7.0 https://mvnrepository.com/artifact/commons-beanutils/commons-beanutils/1.7.0, Apache 2.0
commons-cli 1.2: https://mvnrepository.com/artifact/commons-cli/commons-cli/1.2, Apache 2.0
commons-codec 1.6: https://mvnrepository.com/artifact/commons-codec/commons-codec/1.6, Apache 2.0
commons-codec 1.11: https://mvnrepository.com/artifact/commons-codec/commons-codec/1.11, Apache 2.0
commons-collections 3.2.2: https://mvnrepository.com/artifact/commons-collections/commons-collections/3.2.2, Apache 2.0
commons-collections4 4.1: https://mvnrepository.com/artifact/org.apache.commons/commons-collections4/4.1, Apache 2.0
commons-compress 1.4.1: https://mvnrepository.com/artifact/org.apache.commons/commons-compress/1.4.1, Apache 2.0
@ -348,20 +348,20 @@ The text of each license is also included at licenses/LICENSE-[project].txt.
snakeyaml 1.23: https://mvnrepository.com/artifact/org.yaml/snakeyaml/1.23, Apache 2.0
snappy 0.2: https://mvnrepository.com/artifact/org.iq80.snappy/snappy/0.2, Apache 2.0
snappy-java 1.0.4.1: https://github.com/xerial/snappy-java, Apache 2.0
spring-aop 5.1.5.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-aop/5.1.5.RELEASE, Apache 2.0
spring-beans 5.1.5.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-beans/5.1.5.RELEASE, Apache 2.0
spring-boot 2.1.3.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot/2.1.3.RELEASE, Apache 2.0
spring-boot-autoconfigure 2.1.3.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-autoconfigure/2.1.3.RELEASE, Apache 2.0
spring-boot-starter 2.1.3.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-starter/2.1.3.RELEASE, Apache 2.0
spring-boot-starter-aop 2.1.3.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-starter-aop/2.1.3.RELEASE, Apache 2.0
spring-boot-starter-jdbc 2.1.3.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-starter-jdbc/2.1.3.RELEASE, Apache 2.0
spring-boot-starter-jetty 2.1.3.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-starter-jetty/2.1.3.RELEASE, Apache 2.0
spring-boot-starter-json 2.1.3.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-starter-json/2.1.3.RELEASE, Apache 2.0
spring-boot-starter-logging 2.1.3.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-starter-logging/2.1.3.RELEASE, Apache 2.0
spring-boot-starter-web 2.1.3.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-starter-web/2.1.3.RELEASE, Apache 2.0
spring-context 5.1.5.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-context/5.1.5.RELEASE, Apache 2.0
spring-core 5.1.5.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-core, Apache 2.0
spring-expression 5.1.5.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-expression, Apache 2.0
spring-aop 5.1.18.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-aop/5.1.18.RELEASE, Apache 2.0
spring-beans 5.1.18.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-beans/5.1.18.RELEASE, Apache 2.0
spring-boot 2.1.17.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot/2.1.17.RELEASE, Apache 2.0
spring-boot-autoconfigure 2.1.17.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-autoconfigure/2.1.17.RELEASE, Apache 2.0
spring-boot-starter 2.1.17.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-starter/2.1.17.RELEASE, Apache 2.0
spring-boot-starter-aop 2.1.17.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-starter-aop/2.1.17.RELEASE, Apache 2.0
spring-boot-starter-jdbc 2.1.17.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-starter-jdbc/2.1.17.RELEASE, Apache 2.0
spring-boot-starter-jetty 2.1.17.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-starter-jetty/2.1.17.RELEASE, Apache 2.0
spring-boot-starter-json 2.1.17.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-starter-json/2.1.17.RELEASE, Apache 2.0
spring-boot-starter-logging 2.1.17.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-starter-logging/2.1.17.RELEASE, Apache 2.0
spring-boot-starter-web 2.1.17.RELEASE: https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-starter-web/2.1.17.RELEASE, Apache 2.0
spring-context 5.1.18.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-context/5.1.18.RELEASE, Apache 2.0
spring-core 5.1.18.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-core/5.1.18.RELEASE, Apache 2.0
spring-expression 5.1.18.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-expression/5.1.18.RELEASE, Apache 2.0
springfox-core 2.9.2: https://mvnrepository.com/artifact/io.springfox/springfox-core, Apache 2.0
springfox-schema 2.9.2: https://mvnrepository.com/artifact/io.springfox/springfox-schema, Apache 2.0
springfox-spi 2.9.2: https://mvnrepository.com/artifact/io.springfox/springfox-spi, Apache 2.0
@ -369,13 +369,13 @@ The text of each license is also included at licenses/LICENSE-[project].txt.
springfox-swagger2 2.9.2: https://mvnrepository.com/artifact/io.springfox/springfox-swagger2/2.9.2, Apache 2.0
springfox-swagger-common 2.9.2: https://mvnrepository.com/artifact/io.springfox/springfox-swagger-common/2.9.2, Apache 2.0
springfox-swagger-ui 2.9.2: https://mvnrepository.com/artifact/io.springfox/springfox-swagger-ui/2.9.2, Apache 2.0
spring-jcl 5.1.5.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-jcl/5.1.5.RELEASE, Apache 2.0
spring-jdbc 5.1.5.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-jdbc/5.1.5.RELEASE, Apache 2.0
spring-jcl 5.1.18.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-jcl/5.1.18.RELEASE, Apache 2.0
spring-jdbc 5.1.18.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-jdbc/5.1.18.RELEASE, Apache 2.0
spring-plugin-core 1.2.0.RELEASE: https://mvnrepository.com/artifact/org.springframework.plugin/spring-plugin-core/1.2.0.RELEASE, Apache 2.0
spring-plugin-metadata 1.2.0.RELEASE: https://mvnrepository.com/artifact/org.springframework.plugin/spring-plugin-metadata/1.2.0.RELEASE, Apache 2.0
spring-tx 5.1.5.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-tx/5.1.5.RELEASE, Apache 2.0
spring-web 5.1.5.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-web/5.1.5.RELEASE, Apache 2.0
spring-webmvc 5.1.5.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-webmvc/5.1.5.RELEASE, Apache 2.0
spring-tx 5.1.18.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-tx/5.1.18.RELEASE, Apache 2.0
spring-web 5.1.18.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-web/5.1.18.RELEASE, Apache 2.0
spring-webmvc 5.1.18.RELEASE: https://mvnrepository.com/artifact/org.springframework/spring-webmvc/5.1.18.RELEASE, Apache 2.0
swagger-annotations 1.5.20: https://mvnrepository.com/artifact/io.swagger/swagger-annotations/1.5.20, Apache 2.0
swagger-bootstrap-ui 1.9.3: https://mvnrepository.com/artifact/com.github.xiaoymin/swagger-bootstrap-ui/1.9.3, Apache 2.0
swagger-models 1.5.20: https://mvnrepository.com/artifact/io.swagger/swagger-models/1.5.20, Apache 2.0
@ -502,6 +502,7 @@ MIT licenses
vue-router 2.7.0: https://github.com/vuejs/vue-router MIT
vuex 3.0.0: https://github.com/vuejs/vuex MIT
vuex-router-sync 4.1.2: https://github.com/vuejs/vuex-router-sync MIT
dagre 0.8.5: https://github.com/dagrejs/dagre MIT
========================================
Apache 2.0 licenses

4
dolphinscheduler-dist/release-docs/NOTICE vendored

@ -384,8 +384,8 @@ This product contains the Maven wrapper scripts from 'Maven Wrapper', that provi
Spring Framework NOTICE
========================================================================
Spring Framework 5.1.5.RELEASE
Copyright (c) 2002-2019 Pivotal, Inc.
Spring Framework 5.1.18.RELEASE
Copyright (c) 2002-2020 Pivotal, Inc.
This product is licensed to you under the Apache License, Version 2.0
(the "License"). You may not use this product except in compliance with

19
dolphinscheduler-dist/release-docs/licenses/ui-licenses/LICENSE-dagre vendored

@ -0,0 +1,19 @@
Copyright (c) 2012-2014 Chris Pettitt
Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
in the Software without restriction, including without limitation the rights
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the Software is
furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in
all copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
THE SOFTWARE.

6
dolphinscheduler-dist/src/main/assembly/dolphinscheduler-binary.xml vendored

@ -112,8 +112,8 @@
</fileSet>
<!--server end-->
<!--service end-->
<!--service start-->
<fileSet>
<directory>${basedir}/../dolphinscheduler-service/src/main/resources</directory>
<includes>
@ -226,4 +226,4 @@
</excludes>
</dependencySet>
</dependencySets>
</assembly>
</assembly>

51
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java

@ -17,17 +17,6 @@
package org.apache.dolphinscheduler.remote;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.epoll.EpollEventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.SocketChannel;
import org.apache.dolphinscheduler.remote.codec.NettyDecoder;
import org.apache.dolphinscheduler.remote.codec.NettyEncoder;
import org.apache.dolphinscheduler.remote.command.Command;
@ -41,19 +30,40 @@ import org.apache.dolphinscheduler.remote.future.ReleaseSemaphore;
import org.apache.dolphinscheduler.remote.future.ResponseFuture;
import org.apache.dolphinscheduler.remote.handler.NettyClientHandler;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.remote.utils.CallerThreadExecutePolicy;
import org.apache.dolphinscheduler.remote.utils.Constants;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.remote.utils.NamedThreadFactory;
import org.apache.dolphinscheduler.remote.utils.NettyUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.net.InetSocketAddress;
import java.util.concurrent.*;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.Semaphore;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.epoll.EpollEventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.SocketChannel;
import io.netty.handler.timeout.IdleStateHandler;
/**
* remoting netty client
*/
@ -162,11 +172,10 @@ public class NettyRemotingClient {
.option(ChannelOption.CONNECT_TIMEOUT_MILLIS, clientConfig.getConnectTimeoutMillis())
.handler(new ChannelInitializer<SocketChannel>() {
@Override
public void initChannel(SocketChannel ch) throws Exception {
ch.pipeline().addLast(
new NettyDecoder(),
clientHandler,
encoder);
public void initChannel(SocketChannel ch) {
ch.pipeline()
.addLast("client-idle-handler", new IdleStateHandler(Constants.NETTY_CLIENT_HEART_BEAT_TIME, 0, 0, TimeUnit.MILLISECONDS))
.addLast(new NettyDecoder(), clientHandler, encoder);
}
});
this.responseFutureExecutor.scheduleAtFixedRate(new Runnable() {

12
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java

@ -29,6 +29,7 @@ import org.apache.dolphinscheduler.remote.utils.NettyUtils;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
@ -39,11 +40,11 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.epoll.EpollEventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.SocketChannel;
import io.netty.handler.timeout.IdleStateHandler;
/**
* remoting netty server
@ -183,10 +184,11 @@ public class NettyRemotingServer {
* @param ch socket channel
*/
private void initNettyChannel(SocketChannel ch) {
ChannelPipeline pipeline = ch.pipeline();
pipeline.addLast("encoder", encoder);
pipeline.addLast("decoder", new NettyDecoder());
pipeline.addLast("handler", serverHandler);
ch.pipeline()
.addLast("encoder", encoder)
.addLast("decoder", new NettyDecoder())
.addLast("server-idle-handle", new IdleStateHandler(0, 0, Constants.NETTY_SERVER_HEART_BEAT_TIME, TimeUnit.MILLISECONDS))
.addLast("handler", serverHandler);
}
/**

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

@ -1 +1,105 @@
/* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package org.apache.dolphinscheduler.remote.command; public enum CommandType { /** * remove task log request, */ REMOVE_TAK_LOG_REQUEST, /** * remove task log response */ REMOVE_TAK_LOG_RESPONSE, /** * roll view log request */ ROLL_VIEW_LOG_REQUEST, /** * roll view log response */ ROLL_VIEW_LOG_RESPONSE, /** * view whole log request */ VIEW_WHOLE_LOG_REQUEST, /** * view whole log response */ VIEW_WHOLE_LOG_RESPONSE, /** * get log bytes request */ GET_LOG_BYTES_REQUEST, /** * get log bytes response */ GET_LOG_BYTES_RESPONSE, WORKER_REQUEST, MASTER_RESPONSE, /** * execute task request */ TASK_EXECUTE_REQUEST, /** * execute task ack */ TASK_EXECUTE_ACK, /** * execute task response */ TASK_EXECUTE_RESPONSE, /** * kill task */ TASK_KILL_REQUEST, /** * kill task response */ TASK_KILL_RESPONSE, /** * ping */ PING, /** * pong */ PONG; }
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.remote.command;
public enum CommandType {
/**
* remove task log request,
*/
REMOVE_TAK_LOG_REQUEST,
/**
* remove task log response
*/
REMOVE_TAK_LOG_RESPONSE,
/**
* roll view log request
*/
ROLL_VIEW_LOG_REQUEST,
/**
* roll view log response
*/
ROLL_VIEW_LOG_RESPONSE,
/**
* view whole log request
*/
VIEW_WHOLE_LOG_REQUEST,
/**
* view whole log response
*/
VIEW_WHOLE_LOG_RESPONSE,
/**
* get log bytes request
*/
GET_LOG_BYTES_REQUEST,
/**
* get log bytes response
*/
GET_LOG_BYTES_RESPONSE,
WORKER_REQUEST,
MASTER_RESPONSE,
/**
* execute task request
*/
TASK_EXECUTE_REQUEST,
/**
* execute task ack
*/
TASK_EXECUTE_ACK,
/**
* execute task response
*/
TASK_EXECUTE_RESPONSE,
/**
* kill task
*/
TASK_KILL_REQUEST,
/**
* kill task response
*/
TASK_KILL_RESPONSE,
/**
* HEART_BEAT
*/
HEART_BEAT,
/**
* ping
*/
PING,
/**
* pong
*/
PONG;
}

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

@ -63,7 +63,19 @@ public class TaskExecuteResponseCommand implements Serializable {
*/
private String appIds;
/**
* varPool string
*/
private String varPool;
public void setVarPool(String varPool) {
this.varPool = varPool;
}
public String getVarPool() {
return varPool;
}
public int getTaskInstanceId() {
return taskInstanceId;
}

75
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java

@ -14,9 +14,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.remote.handler;
import io.netty.channel.*;
import org.apache.dolphinscheduler.remote.NettyRemotingClient;
import org.apache.dolphinscheduler.remote.command.Command;
import org.apache.dolphinscheduler.remote.command.CommandType;
@ -25,16 +25,24 @@ import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.utils.ChannelUtils;
import org.apache.dolphinscheduler.remote.utils.Constants;
import org.apache.dolphinscheduler.remote.utils.Pair;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.RejectedExecutionException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.handler.timeout.IdleStateEvent;
/**
* netty client request handler
* netty client request handler
*/
@ChannelHandler.Sharable
public class NettyClientHandler extends ChannelInboundHandlerAdapter {
@ -42,12 +50,14 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter {
private final Logger logger = LoggerFactory.getLogger(NettyClientHandler.class);
/**
* netty client
* netty client
*/
private final NettyRemotingClient nettyRemotingClient;
private static byte[] heartBeatData = "heart_beat".getBytes();
/**
* callback thread executor
* callback thread executor
*/
private final ExecutorService callbackExecutor;
@ -57,19 +67,19 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter {
private final ConcurrentHashMap<CommandType, Pair<NettyRequestProcessor, ExecutorService>> processors;
/**
* default executor
* default executor
*/
private final ExecutorService defaultExecutor = Executors.newFixedThreadPool(Constants.CPUS);
public NettyClientHandler(NettyRemotingClient nettyRemotingClient, ExecutorService callbackExecutor){
public NettyClientHandler(NettyRemotingClient nettyRemotingClient, ExecutorService callbackExecutor) {
this.nettyRemotingClient = nettyRemotingClient;
this.callbackExecutor = callbackExecutor;
this.processors = new ConcurrentHashMap();
}
/**
* When the current channel is not active,
* the current channel has reached the end of its life cycle
* When the current channel is not active,
* the current channel has reached the end of its life cycle
*
* @param ctx channel handler context
* @throws Exception
@ -81,7 +91,7 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter {
}
/**
* The current channel reads data from the remote
* The current channel reads data from the remote
*
* @param ctx channel handler context
* @param msg message
@ -89,55 +99,55 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter {
*/
@Override
public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
processReceived(ctx.channel(), (Command)msg);
processReceived(ctx.channel(), (Command) msg);
}
/**
* register processor
*
* @param commandType command type
* @param processor processor
* @param processor processor
*/
public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) {
this.registerProcessor(commandType, processor, null);
}
/**
* register processor
* register processor
*
* @param commandType command type
* @param processor processor
* @param executor thread executor
* @param processor processor
* @param executor thread executor
*/
public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) {
ExecutorService executorRef = executor;
if(executorRef == null){
if (executorRef == null) {
executorRef = defaultExecutor;
}
this.processors.putIfAbsent(commandType, new Pair<>(processor, executorRef));
}
/**
* process received logic
* process received logic
*
* @param command command
*/
private void processReceived(final Channel channel, final Command command) {
ResponseFuture future = ResponseFuture.getFuture(command.getOpaque());
if(future != null){
if (future != null) {
future.setResponseCommand(command);
future.release();
if(future.getInvokeCallback() != null){
if (future.getInvokeCallback() != null) {
this.callbackExecutor.submit(new Runnable() {
@Override
public void run() {
future.executeInvokeCallback();
}
});
} else{
} else {
future.putResponse(command);
}
} else{
} else {
processByCommandType(channel, command);
}
}
@ -163,9 +173,10 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter {
}
/**
* caught exception
* @param ctx channel handler context
* @param cause cause
* caught exception
*
* @param ctx channel handler context
* @param cause cause
* @throws Exception
*/
@Override
@ -175,4 +186,18 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter {
ctx.channel().close();
}
@Override
public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {
if (evt instanceof IdleStateEvent) {
Command heartBeat = new Command();
heartBeat.setType(CommandType.HEART_BEAT);
heartBeat.setBody(heartBeatData);
ctx.writeAndFlush(heartBeat)
.addListener(ChannelFutureListener.CLOSE_ON_FAILURE);
} else {
super.userEventTriggered(ctx, evt);
}
}
}

67
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java

@ -17,22 +17,30 @@
package org.apache.dolphinscheduler.remote.handler;
import io.netty.channel.*;
import org.apache.dolphinscheduler.remote.NettyRemotingServer;
import org.apache.dolphinscheduler.remote.command.Command;
import org.apache.dolphinscheduler.remote.command.CommandType;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.utils.ChannelUtils;
import org.apache.dolphinscheduler.remote.utils.Pair;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.RejectedExecutionException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.netty.channel.Channel;
import io.netty.channel.ChannelConfig;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.handler.timeout.IdleStateEvent;
/**
* netty server request handler
* netty server request handler
*/
@ChannelHandler.Sharable
public class NettyServerHandler extends ChannelInboundHandlerAdapter {
@ -40,22 +48,23 @@ public class NettyServerHandler extends ChannelInboundHandlerAdapter {
private final Logger logger = LoggerFactory.getLogger(NettyServerHandler.class);
/**
* netty remote server
* netty remote server
*/
private final NettyRemotingServer nettyRemotingServer;
/**
* server processors queue
* server processors queue
*/
private final ConcurrentHashMap<CommandType, Pair<NettyRequestProcessor, ExecutorService>> processors = new ConcurrentHashMap();
public NettyServerHandler(NettyRemotingServer nettyRemotingServer){
public NettyServerHandler(NettyRemotingServer nettyRemotingServer) {
this.nettyRemotingServer = nettyRemotingServer;
}
/**
* When the current channel is not active,
* the current channel has reached the end of its life cycle
* When the current channel is not active,
* the current channel has reached the end of its life cycle
*
* @param ctx channel handler context
* @throws Exception
*/
@ -73,38 +82,39 @@ public class NettyServerHandler extends ChannelInboundHandlerAdapter {
*/
@Override
public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
processReceived(ctx.channel(), (Command)msg);
processReceived(ctx.channel(), (Command) msg);
}
/**
* register processor
*
* @param commandType command type
* @param processor processor
* @param processor processor
*/
public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) {
this.registerProcessor(commandType, processor, null);
}
/**
* register processor
* register processor
*
* @param commandType command type
* @param processor processor
* @param executor thread executor
* @param processor processor
* @param executor thread executor
*/
public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) {
ExecutorService executorRef = executor;
if(executorRef == null){
if (executorRef == null) {
executorRef = nettyRemotingServer.getDefaultExecutor();
}
this.processors.putIfAbsent(commandType, new Pair<>(processor, executorRef));
}
/**
* process received logic
* process received logic
*
* @param channel channel
* @param msg message
* @param msg message
*/
private void processReceived(final Channel channel, final Command msg) {
final CommandType commandType = msg.getType();
@ -132,22 +142,22 @@ public class NettyServerHandler extends ChannelInboundHandlerAdapter {
}
/**
* caught exception
* caught exception
*
* @param ctx channel handler context
* @param ctx channel handler context
* @param cause cause
* @throws Exception
*/
@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
logger.error("exceptionCaught : {}",cause.getMessage(), cause);
logger.error("exceptionCaught : {}", cause.getMessage(), cause);
ctx.channel().close();
}
/**
* channel write changed
* channel write changed
*
* @param ctx channel handler context
* @param ctx channel handler context
* @throws Exception
*/
@Override
@ -158,16 +168,25 @@ public class NettyServerHandler extends ChannelInboundHandlerAdapter {
if (!ch.isWritable()) {
if (logger.isWarnEnabled()) {
logger.warn("{} is not writable, over high water level : {}",
ch, config.getWriteBufferHighWaterMark());
ch, config.getWriteBufferHighWaterMark());
}
config.setAutoRead(false);
} else {
if (logger.isWarnEnabled()) {
logger.warn("{} is writable, to low water : {}",
ch, config.getWriteBufferLowWaterMark());
ch, config.getWriteBufferLowWaterMark());
}
config.setAutoRead(true);
}
}
@Override
public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {
if (evt instanceof IdleStateEvent) {
ctx.channel().close();
} else {
super.userEventTriggered(ctx, evt);
}
}
}

10
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java

@ -20,7 +20,6 @@ package org.apache.dolphinscheduler.remote.utils;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
/**
* constant
*/
@ -30,6 +29,10 @@ public class Constants {
public static final String SLASH = "/";
public static final int NETTY_SERVER_HEART_BEAT_TIME = 1000 * 60 * 3 + 1000;
public static final int NETTY_CLIENT_HEART_BEAT_TIME = 1000 * 60;
/**
* charset
*/
@ -53,4 +56,9 @@ public class Constants {
*/
public static final String OS_NAME = System.getProperty("os.name");
/**
* warm up time
*/
public static final int WARM_UP_TIME = 10 * 60 * 1000;
}

48
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Host.java

@ -14,6 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.remote.utils;
import java.io.Serializable;
@ -44,6 +45,11 @@ public class Host implements Serializable {
*/
private int weight;
/**
* startTime
*/
private long startTime;
/**
* workGroup
*/
@ -58,19 +64,21 @@ public class Host implements Serializable {
this.address = ip + ":" + port;
}
public Host(String ip, int port, int weight) {
public Host(String ip, int port, int weight, long startTime) {
this.ip = ip;
this.port = port;
this.address = ip + ":" + port;
this.weight = weight;
this.weight = getWarmUpWeight(weight, startTime);
this.startTime = startTime;
}
public Host(String ip, int port, int weight,String workGroup) {
public Host(String ip, int port, int weight, long startTime, String workGroup) {
this.ip = ip;
this.port = port;
this.address = ip + ":" + port;
this.weight = weight;
this.workGroup=workGroup;
this.weight = getWarmUpWeight(weight, startTime);
this.workGroup = workGroup;
this.startTime = startTime;
}
public String getAddress() {
@ -98,6 +106,14 @@ public class Host implements Serializable {
this.weight = weight;
}
public long getStartTime() {
return startTime;
}
public void setStartTime(long startTime) {
this.startTime = startTime;
}
public int getPort() {
return port;
}
@ -133,8 +149,8 @@ public class Host implements Serializable {
if (parts.length == 2) {
host = new Host(parts[0], Integer.parseInt(parts[1]));
}
if (parts.length == 3) {
host = new Host(parts[0], Integer.parseInt(parts[1]), Integer.parseInt(parts[2]));
if (parts.length == 4) {
host = new Host(parts[0], Integer.parseInt(parts[1]), Integer.parseInt(parts[2]), Long.parseLong(parts[3]));
}
return host;
}
@ -169,8 +185,20 @@ public class Host implements Serializable {
@Override
public String toString() {
return "Host{" +
"address='" + address + '\'' +
'}';
return "Host{"
+ "address='" + address + '\''
+ '}';
}
/**
* warm up
*/
private int getWarmUpWeight(int weight, long startTime) {
long uptime = System.currentTimeMillis() - startTime;
//If the warm-up is not over, reduce the weight
if (uptime > 0 && uptime < Constants.WARM_UP_TIME) {
return (int) (weight * ((float) uptime / Constants.WARM_UP_TIME));
}
return weight;
}
}

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java

@ -326,7 +326,7 @@ public class TaskPriorityQueueConsumer extends Thread{
udfFunIdsArray[i]=Integer.parseInt(udfFunIds[i]);
}
List<UdfFunc> udfFuncList = processService.queryUdfFunListByids(udfFunIdsArray);
List<UdfFunc> udfFuncList = processService.queryUdfFunListByIds(udfFunIdsArray);
Map<UdfFunc,String> udfFuncMap = new HashMap<>();
for(UdfFunc udfFunc : udfFuncList) {
String tenantCode = processService.queryTenantCodeByResName(udfFunc.getResourceName(), ResourceType.UDF);

44
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/HostWeight.java

@ -17,6 +17,7 @@
package org.apache.dolphinscheduler.server.master.dispatch.host.assign;
import org.apache.dolphinscheduler.remote.utils.Constants;
import org.apache.dolphinscheduler.remote.utils.Host;
/**
@ -32,25 +33,25 @@ public class HostWeight {
private final Host host;
private final int weight;
private final double weight;
private int currentWeight;
private double currentWeight;
public HostWeight(Host host, double cpu, double memory, double loadAverage) {
this.weight = calculateWeight(cpu, memory, loadAverage);
this.host = host ;
this.currentWeight = weight ;
this.weight = getWeight(cpu, memory, loadAverage, host);
this.host = host;
this.currentWeight = weight;
}
public int getCurrentWeight() {
public double getCurrentWeight() {
return currentWeight;
}
public int getWeight() {
public double getWeight() {
return weight;
}
public void setCurrentWeight(int currentWeight) {
public void setCurrentWeight(double currentWeight) {
this.currentWeight = currentWeight;
}
@ -60,14 +61,27 @@ public class HostWeight {
@Override
public String toString() {
return "HostWeight{" +
"host=" + host +
", weight=" + weight +
", currentWeight=" + currentWeight +
'}';
return "HostWeight{"
+ "host=" + host
+ ", weight=" + weight
+ ", currentWeight=" + currentWeight
+ '}';
}
private int calculateWeight(double cpu, double memory, double loadAverage){
return (int)(cpu * CPU_FACTOR + memory * MEMORY_FACTOR + loadAverage * LOAD_AVERAGE_FACTOR);
private double getWeight(double cpu, double memory, double loadAverage, Host host) {
double calculateWeight = cpu * CPU_FACTOR + memory * MEMORY_FACTOR + loadAverage * LOAD_AVERAGE_FACTOR;
return getWarmUpWeight(host, calculateWeight);
}
/**
* If the warm-up is not over, add the weight
*/
private double getWarmUpWeight(Host host, double weight) {
long startTime = host.getStartTime();
long uptime = System.currentTimeMillis() - startTime;
if (uptime > 0 && uptime < Constants.WARM_UP_TIME) {
return weight * Constants.WARM_UP_TIME / uptime;
}
return weight;
}
}

13
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/LowerWeightRoundRobin.java

@ -20,24 +20,25 @@ package org.apache.dolphinscheduler.server.master.dispatch.host.assign;
import java.util.Collection;
/**
* lower weight round robin
* lower weight round robin
*/
public class LowerWeightRoundRobin extends AbstractSelector<HostWeight>{
public class LowerWeightRoundRobin extends AbstractSelector<HostWeight> {
/**
* select
*
* @param sources sources
* @return HostWeight
*/
@Override
public HostWeight doSelect(Collection<HostWeight> sources){
int totalWeight = 0;
int lowWeight = 0;
public HostWeight doSelect(Collection<HostWeight> sources) {
double totalWeight = 0;
double lowWeight = 0;
HostWeight lowerNode = null;
for (HostWeight hostWeight : sources) {
totalWeight += hostWeight.getWeight();
hostWeight.setCurrentWeight(hostWeight.getCurrentWeight() + hostWeight.getWeight());
if (lowerNode == null || lowWeight > hostWeight.getCurrentWeight() ) {
if (lowerNode == null || lowWeight > hostWeight.getCurrentWeight()) {
lowerNode = hostWeight;
lowWeight = hostWeight.getCurrentWeight();
}

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

@ -90,7 +90,8 @@ public class TaskResponseProcessor implements NettyRequestProcessor {
responseCommand.getEndTime(),
responseCommand.getProcessId(),
responseCommand.getAppIds(),
responseCommand.getTaskInstanceId());
responseCommand.getTaskInstanceId(),
responseCommand.getVarPool());
taskResponseService.addResponse(taskResponseEvent);

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

@ -79,7 +79,12 @@ public class TaskResponseEvent {
*/
private Event event;
public static TaskResponseEvent newAck(ExecutionStatus state, Date startTime, String workerAddress, String executePath, String logPath, int taskInstanceId){
/**
* varPool
*/
private String varPool;
public static TaskResponseEvent newAck(ExecutionStatus state, Date startTime, String workerAddress, String executePath, String logPath, int taskInstanceId) {
TaskResponseEvent event = new TaskResponseEvent();
event.setState(state);
event.setStartTime(startTime);
@ -91,7 +96,7 @@ public class TaskResponseEvent {
return event;
}
public static TaskResponseEvent newResult(ExecutionStatus state, Date endTime, int processId, String appIds, int taskInstanceId){
public static TaskResponseEvent newResult(ExecutionStatus state, Date endTime, int processId, String appIds, int taskInstanceId, String varPool) {
TaskResponseEvent event = new TaskResponseEvent();
event.setState(state);
event.setEndTime(endTime);
@ -99,9 +104,18 @@ public class TaskResponseEvent {
event.setAppIds(appIds);
event.setTaskInstanceId(taskInstanceId);
event.setEvent(Event.RESULT);
event.setVarPool(varPool);
return event;
}
public String getVarPool() {
return varPool;
}
public void setVarPool(String varPool) {
this.varPool = varPool;
}
public int getTaskInstanceId() {
return taskInstanceId;
}

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

@ -139,7 +139,8 @@ public class TaskResponseService {
taskResponseEvent.getEndTime(),
taskResponseEvent.getProcessId(),
taskResponseEvent.getAppIds(),
taskResponseEvent.getTaskInstanceId());
taskResponseEvent.getTaskInstanceId(),
taskResponseEvent.getVarPool());
break;
default:
throw new IllegalArgumentException("invalid event type : " + event);

15
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java

@ -45,6 +45,7 @@ import org.apache.dolphinscheduler.common.utils.JSONUtils;
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.common.utils.VarPoolUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.Schedule;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
@ -59,6 +60,7 @@ import org.apache.commons.io.FileUtils;
import java.io.File;
import java.io.IOException;
import java.text.ParseException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@ -651,14 +653,23 @@ public class MasterExecThread implements Runnable {
* submit post node
* @param parentNodeName parent node name
*/
private Map<String,Object> propToValue = new ConcurrentHashMap<String, Object>();
private void submitPostNode(String parentNodeName){
List<String> submitTaskNodeList = parsePostNodeList(parentNodeName);
List<TaskInstance> taskInstances = new ArrayList<>();
for(String taskNode : submitTaskNodeList){
try {
VarPoolUtils.convertVarPoolToMap(propToValue, processInstance.getVarPool());
} catch (ParseException e) {
logger.error("parse {} exception", processInstance.getVarPool(), e);
throw new RuntimeException();
}
TaskNode taskNodeObject = dag.getNode(taskNode);
VarPoolUtils.setTaskNodeLocalParams(taskNodeObject, propToValue);
taskInstances.add(createTaskInstance(processInstance, taskNode,
dag.getNode(taskNode)));
taskNodeObject));
}
// if previous node success , post node submit
@ -999,6 +1010,8 @@ public class MasterExecThread implements Runnable {
task.getName(), task.getId(), task.getState());
// node success , post node submit
if(task.getState() == ExecutionStatus.SUCCESS){
processInstance.setVarPool(task.getVarPool());
processService.updateProcessInstance(processInstance);
completeTaskList.put(task.getName(), task);
submitPostNode(task.getName());
continue;

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/LogUtils.java

@ -70,7 +70,7 @@ public class LogUtils {
* get task log path by TaskExecutionContext
*/
public static String getTaskLogPath(TaskExecutionContext taskExecutionContext) {
return getTaskLogPath(taskExecutionContext.getProcessId(), taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId());
return getTaskLogPath(taskExecutionContext.getProcessDefineId(), taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId());
}
}

44
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistry.java

@ -14,19 +14,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.worker.registry;
import java.util.Date;
import java.util.Set;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import static org.apache.dolphinscheduler.common.Constants.COLON;
import static org.apache.dolphinscheduler.common.Constants.DEFAULT_WORKER_GROUP;
import static org.apache.dolphinscheduler.common.Constants.SLASH;
import javax.annotation.PostConstruct;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.state.ConnectionState;
import org.apache.curator.framework.state.ConnectionStateListener;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.NetUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
@ -34,6 +28,19 @@ import org.apache.dolphinscheduler.remote.utils.NamedThreadFactory;
import org.apache.dolphinscheduler.server.registry.HeartBeatTask;
import org.apache.dolphinscheduler.server.registry.ZookeeperRegistryCenter;
import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.state.ConnectionState;
import org.apache.curator.framework.state.ConnectionStateListener;
import java.util.Date;
import java.util.Set;
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;
@ -41,8 +48,6 @@ import org.springframework.stereotype.Service;
import com.google.common.collect.Sets;
import static org.apache.dolphinscheduler.common.Constants.*;
/**
* worker registry
@ -111,10 +116,10 @@ public class WorkerRegistry {
}
HeartBeatTask heartBeatTask = new HeartBeatTask(this.startTime,
this.workerConfig.getWorkerReservedMemory(),
this.workerConfig.getWorkerMaxCpuloadAvg(),
workerZkPaths,
this.zookeeperRegistryCenter);
this.workerConfig.getWorkerReservedMemory(),
this.workerConfig.getWorkerMaxCpuloadAvg(),
workerZkPaths,
this.zookeeperRegistryCenter);
this.heartBeatExecutor.scheduleAtFixedRate(heartBeatTask, workerHeartbeatInterval, workerHeartbeatInterval, TimeUnit.SECONDS);
logger.info("worker node : {} heartbeat interval {} s", address, workerHeartbeatInterval);
@ -142,6 +147,7 @@ public class WorkerRegistry {
String address = getLocalAddress();
String workerZkPathPrefix = this.zookeeperRegistryCenter.getWorkerPath();
String weight = getWorkerWeight();
String workerStartTime = COLON + System.currentTimeMillis();
for (String workGroup : this.workerGroups) {
StringBuilder workerZkPathBuilder = new StringBuilder(100);
@ -153,6 +159,7 @@ public class WorkerRegistry {
workerZkPathBuilder.append(workGroup.trim().toLowerCase()).append(SLASH);
workerZkPathBuilder.append(address);
workerZkPathBuilder.append(weight);
workerZkPathBuilder.append(workerStartTime);
workerZkPaths.add(workerZkPathBuilder.toString());
}
return workerZkPaths;
@ -162,13 +169,14 @@ public class WorkerRegistry {
* get local address
*/
private String getLocalAddress() {
return NetUtils.getHost() + ":" + workerConfig.getListenPort();
return NetUtils.getHost() + COLON + workerConfig.getListenPort();
}
/**
* get Worker Weight
*/
private String getWorkerWeight() {
return ":" + workerConfig.getWeight();
return COLON + workerConfig.getWeight();
}
}

1
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskExecuteThread.java

@ -155,6 +155,7 @@ public class TaskExecuteThread implements Runnable {
responseCommand.setEndTime(new Date());
responseCommand.setProcessId(task.getProcessId());
responseCommand.setAppIds(task.getAppIds());
responseCommand.setVarPool(task.getVarPool());
logger.info("task instance id : {},task final status : {}", taskExecutionContext.getTaskInstanceId(), task.getExitStatus());
} catch (Exception e) {
logger.error("task scheduler failure", e);

15
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java

@ -66,6 +66,7 @@ public abstract class AbstractCommandExecutor {
*/
protected static final Pattern APPLICATION_REGEX = Pattern.compile(Constants.APPLICATION_REGEX);
protected StringBuilder varPool = new StringBuilder();
/**
* process
*/
@ -234,7 +235,10 @@ public abstract class AbstractCommandExecutor {
return result;
}
public String getVarPool() {
return varPool.toString();
}
/**
* cancel application
* @throws Exception exception
@ -347,8 +351,13 @@ public abstract class AbstractCommandExecutor {
long lastFlushTime = System.currentTimeMillis();
while ((line = inReader.readLine()) != null) {
logBuffer.add(line);
lastFlushTime = flush(lastFlushTime);
if (line.startsWith("${setValue(")) {
varPool.append(line.substring("${setValue(".length(), line.length() - 2));
varPool.append("$VarPool$");
} else {
logBuffer.add(line);
lastFlushTime = flush(lastFlushTime);
}
}
} catch (Exception e) {
logger.error(e.getMessage(),e);

13
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractTask.java

@ -47,6 +47,11 @@ import java.util.Map;
*/
public abstract class AbstractTask {
/**
* varPool string
*/
protected String varPool;
/**
* taskExecutionContext
**/
@ -121,6 +126,14 @@ public abstract class AbstractTask {
logger.info(" -> {}", String.join("\n\t", logs));
}
public void setVarPool(String varPool) {
this.varPool = varPool;
}
public String getVarPool() {
return varPool;
}
/**
* get exit status code
* @return exit status code

78
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskManager.java

@ -14,8 +14,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.worker.task;
package org.apache.dolphinscheduler.server.worker.task;
import org.apache.dolphinscheduler.common.enums.TaskType;
import org.apache.dolphinscheduler.common.utils.EnumUtils;
@ -37,42 +37,44 @@ import org.slf4j.Logger;
*/
public class TaskManager {
/**
* create new task
* @param taskExecutionContext taskExecutionContext
* @param logger logger
* @return AbstractTask
* @throws IllegalArgumentException illegal argument exception
*/
public static AbstractTask newTask(TaskExecutionContext taskExecutionContext,
Logger logger)
throws IllegalArgumentException {
switch (EnumUtils.getEnum(TaskType.class,taskExecutionContext.getTaskType())) {
case SHELL:
return new ShellTask(taskExecutionContext, logger);
case WATERDROP:
return new ShellTask(taskExecutionContext, logger);
case PROCEDURE:
return new ProcedureTask(taskExecutionContext, logger);
case SQL:
return new SqlTask(taskExecutionContext, logger);
case MR:
return new MapReduceTask(taskExecutionContext, logger);
case SPARK:
return new SparkTask(taskExecutionContext, logger);
case FLINK:
return new FlinkTask(taskExecutionContext, logger);
case PYTHON:
return new PythonTask(taskExecutionContext, logger);
case HTTP:
return new HttpTask(taskExecutionContext, logger);
case DATAX:
return new DataxTask(taskExecutionContext, logger);
case SQOOP:
return new SqoopTask(taskExecutionContext, logger);
default:
logger.error("unsupport task type: {}", taskExecutionContext.getTaskType());
throw new IllegalArgumentException("not support task type");
/**
* create new task
* @param taskExecutionContext taskExecutionContext
* @param logger logger
* @return AbstractTask
* @throws IllegalArgumentException illegal argument exception
*/
public static AbstractTask newTask(TaskExecutionContext taskExecutionContext, Logger logger) throws IllegalArgumentException {
TaskType anEnum = EnumUtils.getEnum(TaskType.class, taskExecutionContext.getTaskType());
if (anEnum == null) {
logger.error("not support task type: {}", taskExecutionContext.getTaskType());
throw new IllegalArgumentException("not support task type");
}
switch (anEnum) {
case SHELL:
case WATERDROP:
return new ShellTask(taskExecutionContext, logger);
case PROCEDURE:
return new ProcedureTask(taskExecutionContext, logger);
case SQL:
return new SqlTask(taskExecutionContext, logger);
case MR:
return new MapReduceTask(taskExecutionContext, logger);
case SPARK:
return new SparkTask(taskExecutionContext, logger);
case FLINK:
return new FlinkTask(taskExecutionContext, logger);
case PYTHON:
return new PythonTask(taskExecutionContext, logger);
case HTTP:
return new HttpTask(taskExecutionContext, logger);
case DATAX:
return new DataxTask(taskExecutionContext, logger);
case SQOOP:
return new SqoopTask(taskExecutionContext, logger);
default:
logger.error("not support task type: {}", taskExecutionContext.getTaskType());
throw new IllegalArgumentException("not support task type");
}
}
}
}

189
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/python/PythonTask.java

@ -38,103 +38,110 @@ import java.util.Map;
*/
public class PythonTask extends AbstractTask {
/**
* python parameters
*/
private PythonParameters pythonParameters;
/**
* task dir
*/
private String taskDir;
/**
* python command executor
*/
private PythonCommandExecutor pythonCommandExecutor;
/**
* taskExecutionContext
*/
private TaskExecutionContext taskExecutionContext;
/**
* constructor
* @param taskExecutionContext taskExecutionContext
* @param logger logger
*/
public PythonTask(TaskExecutionContext taskExecutionContext, Logger logger) {
super(taskExecutionContext, logger);
this.taskExecutionContext = taskExecutionContext;
this.pythonCommandExecutor = new PythonCommandExecutor(this::logHandle,
taskExecutionContext,
logger);
}
@Override
public void init() {
logger.info("python task params {}", taskExecutionContext.getTaskParams());
pythonParameters = JSONUtils.parseObject(taskExecutionContext.getTaskParams(), PythonParameters.class);
if (!pythonParameters.checkParameters()) {
throw new RuntimeException("python task params is not valid");
/**
* python parameters
*/
private PythonParameters pythonParameters;
/**
* task dir
*/
private String taskDir;
/**
* python command executor
*/
private PythonCommandExecutor pythonCommandExecutor;
/**
* taskExecutionContext
*/
private TaskExecutionContext taskExecutionContext;
/**
* constructor
* @param taskExecutionContext taskExecutionContext
* @param logger logger
*/
public PythonTask(TaskExecutionContext taskExecutionContext, Logger logger) {
super(taskExecutionContext, logger);
this.taskExecutionContext = taskExecutionContext;
this.pythonCommandExecutor = new PythonCommandExecutor(this::logHandle,
taskExecutionContext,
logger);
}
}
@Override
public void handle() throws Exception {
try {
// construct process
CommandExecuteResult commandExecuteResult = pythonCommandExecutor.run(buildCommand());
@Override
public void init() {
logger.info("python task params {}", taskExecutionContext.getTaskParams());
setExitStatusCode(commandExecuteResult.getExitStatusCode());
setAppIds(commandExecuteResult.getAppIds());
setProcessId(commandExecuteResult.getProcessId());
}
catch (Exception e) {
logger.error("python task failure", e);
setExitStatusCode(Constants.EXIT_CODE_FAILURE);
throw e;
}
}
@Override
public void cancelApplication(boolean cancelApplication) throws Exception {
// cancel process
pythonCommandExecutor.cancelApplication();
}
/**
* build command
* @return raw python script
* @throws Exception exception
*/
private String buildCommand() throws Exception {
String rawPythonScript = pythonParameters.getRawScript().replaceAll("\\r\\n", "\n");
// replace placeholder
Map<String, Property> paramsMap = ParamUtils.convert(ParamUtils.getUserDefParamsMap(taskExecutionContext.getDefinedParams()),
taskExecutionContext.getDefinedParams(),
pythonParameters.getLocalParametersMap(),
CommandType.of(taskExecutionContext.getCmdTypeIfComplement()),
taskExecutionContext.getScheduleTime());
if (paramsMap != null){
rawPythonScript = ParameterUtils.convertParameterPlaceholders(rawPythonScript, ParamUtils.convert(paramsMap));
}
pythonParameters = JSONUtils.parseObject(taskExecutionContext.getTaskParams(), PythonParameters.class);
logger.info("raw python script : {}", pythonParameters.getRawScript());
logger.info("task dir : {}", taskDir);
return rawPythonScript;
}
if (!pythonParameters.checkParameters()) {
throw new RuntimeException("python task params is not valid");
}
}
@Override
public AbstractParameters getParameters() {
return pythonParameters;
}
@Override
public void handle() throws Exception {
try {
// construct process
CommandExecuteResult commandExecuteResult = pythonCommandExecutor.run(buildCommand());
setExitStatusCode(commandExecuteResult.getExitStatusCode());
setAppIds(commandExecuteResult.getAppIds());
setProcessId(commandExecuteResult.getProcessId());
setVarPool(pythonCommandExecutor.getVarPool());
}
catch (Exception e) {
logger.error("python task failure", e);
setExitStatusCode(Constants.EXIT_CODE_FAILURE);
throw e;
}
}
@Override
public void cancelApplication(boolean cancelApplication) throws Exception {
// cancel process
pythonCommandExecutor.cancelApplication();
}
/**
* build command
* @return raw python script
* @throws Exception exception
*/
private String buildCommand() throws Exception {
String rawPythonScript = pythonParameters.getRawScript().replaceAll("\\r\\n", "\n");
// replace placeholder
Map<String, Property> paramsMap = ParamUtils.convert(ParamUtils.getUserDefParamsMap(taskExecutionContext.getDefinedParams()),
taskExecutionContext.getDefinedParams(),
pythonParameters.getLocalParametersMap(),
CommandType.of(taskExecutionContext.getCmdTypeIfComplement()),
taskExecutionContext.getScheduleTime());
try {
rawPythonScript = VarPoolUtils.convertPythonScriptPlaceholders(rawPythonScript);
}
catch (StringIndexOutOfBoundsException e) {
logger.error("setShareVar field format error, raw python script : {}", rawPythonScript);
}
if (paramsMap != null) {
rawPythonScript = ParameterUtils.convertParameterPlaceholders(rawPythonScript, ParamUtils.convert(paramsMap));
}
logger.info("raw python script : {}", pythonParameters.getRawScript());
logger.info("task dir : {}", taskDir);
return rawPythonScript;
}
@Override
public AbstractParameters getParameters() {
return pythonParameters;
}
}

604
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java

@ -20,6 +20,7 @@ import org.apache.commons.lang.StringUtils;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.TreeCacheEvent;
import org.apache.curator.framework.recipes.locks.InterProcessMutex;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.ZKNodeType;
@ -33,6 +34,7 @@ import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.utils.ProcessUtils;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.service.zk.AbstractZKClient;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
@ -45,309 +47,309 @@ import static org.apache.dolphinscheduler.common.Constants.SLEEP_TIME_MILLIS;
/**
* zookeeper master client
*
* single instance
* zookeeper master client
* <p>
* single instance
*/
@Component
public class ZKMasterClient extends AbstractZKClient {
/**
* logger
*/
private static final Logger logger = LoggerFactory.getLogger(ZKMasterClient.class);
/**
* process service
*/
@Autowired
private ProcessService processService;
public void start() {
InterProcessMutex mutex = null;
try {
// create distributed lock with the root node path of the lock space as /dolphinscheduler/lock/failover/master
String znodeLock = getMasterStartUpLockPath();
mutex = new InterProcessMutex(getZkClient(), znodeLock);
mutex.acquire();
// init system znode
this.initSystemZNode();
while (!checkZKNodeExists(NetUtils.getHost(), ZKNodeType.MASTER)){
ThreadUtils.sleep(SLEEP_TIME_MILLIS);
}
// self tolerant
if (getActiveMasterNum() == 1) {
failoverWorker(null, true);
failoverMaster(null);
}
}catch (Exception e){
logger.error("master start up exception",e);
}finally {
releaseMutex(mutex);
}
}
@Override
public void close(){
super.close();
}
/**
* handle path events that this class cares about
* @param client zkClient
* @param event path event
* @param path zk path
*/
@Override
protected void dataChanged(CuratorFramework client, TreeCacheEvent event, String path) {
//monitor master
if(path.startsWith(getZNodeParentPath(ZKNodeType.MASTER)+Constants.SINGLE_SLASH)){
handleMasterEvent(event,path);
}else if(path.startsWith(getZNodeParentPath(ZKNodeType.WORKER)+Constants.SINGLE_SLASH)){
//monitor worker
handleWorkerEvent(event,path);
}
}
/**
* remove zookeeper node path
*
* @param path zookeeper node path
* @param zkNodeType zookeeper node type
* @param failover is failover
*/
private void removeZKNodePath(String path, ZKNodeType zkNodeType, boolean failover) {
logger.info("{} node deleted : {}", zkNodeType.toString(), path);
InterProcessMutex mutex = null;
try {
String failoverPath = getFailoverLockPath(zkNodeType);
// create a distributed lock
mutex = new InterProcessMutex(getZkClient(), failoverPath);
mutex.acquire();
String serverHost = getHostByEventDataPath(path);
// handle dead server
handleDeadServer(path, zkNodeType, Constants.ADD_ZK_OP);
//failover server
if(failover){
failoverServerWhenDown(serverHost, zkNodeType);
}
}catch (Exception e){
logger.error("{} server failover failed.", zkNodeType.toString());
logger.error("failover exception ",e);
}
finally {
releaseMutex(mutex);
}
}
/**
* failover server when server down
*
* @param serverHost server host
* @param zkNodeType zookeeper node type
* @throws Exception exception
*/
private void failoverServerWhenDown(String serverHost, ZKNodeType zkNodeType) throws Exception {
if(StringUtils.isEmpty(serverHost) || serverHost.startsWith(NetUtils.getHost())){
return ;
}
switch (zkNodeType){
case MASTER:
failoverMaster(serverHost);
break;
case WORKER:
failoverWorker(serverHost, true);
default:
break;
}
}
/**
* get failover lock path
*
* @param zkNodeType zookeeper node type
* @return fail over lock path
*/
private String getFailoverLockPath(ZKNodeType zkNodeType){
switch (zkNodeType){
case MASTER:
return getMasterFailoverLockPath();
case WORKER:
return getWorkerFailoverLockPath();
default:
return "";
}
}
/**
* monitor master
* @param event event
* @param path path
*/
public void handleMasterEvent(TreeCacheEvent event, String path){
switch (event.getType()) {
case NODE_ADDED:
logger.info("master node added : {}", path);
break;
case NODE_REMOVED:
removeZKNodePath(path, ZKNodeType.MASTER, true);
break;
default:
break;
}
}
/**
* monitor worker
* @param event event
* @param path path
*/
public void handleWorkerEvent(TreeCacheEvent event, String path){
switch (event.getType()) {
case NODE_ADDED:
logger.info("worker node added : {}", path);
break;
case NODE_REMOVED:
logger.info("worker node deleted : {}", path);
removeZKNodePath(path, ZKNodeType.WORKER, true);
break;
default:
break;
}
}
/**
* task needs failover if task start before worker starts
*
* @param taskInstance task instance
* @return true if task instance need fail over
*/
private boolean checkTaskInstanceNeedFailover(TaskInstance taskInstance) throws Exception {
boolean taskNeedFailover = true;
//now no host will execute this task instance,so no need to failover the task
if(taskInstance.getHost() == null){
return false;
}
// if the worker node exists in zookeeper, we must check the task starts after the worker
if(checkZKNodeExists(taskInstance.getHost(), ZKNodeType.WORKER)){
//if task start after worker starts, there is no need to failover the task.
if(checkTaskAfterWorkerStart(taskInstance)){
taskNeedFailover = false;
}
}
return taskNeedFailover;
}
/**
* check task start after the worker server starts.
*
* @param taskInstance task instance
* @return true if task instance start time after worker server start date
*/
private boolean checkTaskAfterWorkerStart(TaskInstance taskInstance) {
if(StringUtils.isEmpty(taskInstance.getHost())){
return false;
}
Date workerServerStartDate = null;
List<Server> workerServers = getServersList(ZKNodeType.WORKER);
for(Server workerServer : workerServers){
if(taskInstance.getHost().equals(workerServer.getHost() + Constants.COLON + workerServer.getPort())){
workerServerStartDate = workerServer.getCreateTime();
break;
}
}
if(workerServerStartDate != null){
return taskInstance.getStartTime().after(workerServerStartDate);
}else{
return false;
}
}
/**
* failover worker tasks
*
* 1. kill yarn job if there are yarn jobs in tasks.
* 2. change task state from running to need failover.
* 3. failover all tasks when workerHost is null
* @param workerHost worker host
*/
/**
* failover worker tasks
*
* 1. kill yarn job if there are yarn jobs in tasks.
* 2. change task state from running to need failover.
* 3. failover all tasks when workerHost is null
* @param workerHost worker host
* @param needCheckWorkerAlive need check worker alive
* @throws Exception exception
*/
private void failoverWorker(String workerHost, boolean needCheckWorkerAlive) throws Exception {
logger.info("start worker[{}] failover ...", workerHost);
List<TaskInstance> needFailoverTaskInstanceList = processService.queryNeedFailoverTaskInstances(workerHost);
for(TaskInstance taskInstance : needFailoverTaskInstanceList){
if(needCheckWorkerAlive){
if(!checkTaskInstanceNeedFailover(taskInstance)){
continue;
}
}
ProcessInstance processInstance = processService.findProcessInstanceDetailById(taskInstance.getProcessInstanceId());
if(processInstance != null){
taskInstance.setProcessInstance(processInstance);
}
TaskExecutionContext taskExecutionContext = TaskExecutionContextBuilder.get()
.buildTaskInstanceRelatedInfo(taskInstance)
.buildProcessInstanceRelatedInfo(processInstance)
.create();
// only kill yarn job if exists , the local thread has exited
ProcessUtils.killYarnJob(taskExecutionContext);
taskInstance.setState(ExecutionStatus.NEED_FAULT_TOLERANCE);
processService.saveTaskInstance(taskInstance);
}
logger.info("end worker[{}] failover ...", workerHost);
}
/**
* failover master tasks
*
* @param masterHost master host
*/
private void failoverMaster(String masterHost) {
logger.info("start master failover ...");
List<ProcessInstance> needFailoverProcessInstanceList = processService.queryNeedFailoverProcessInstances(masterHost);
//updateProcessInstance host is null and insert into command
for(ProcessInstance processInstance : needFailoverProcessInstanceList){
if(Constants.NULL.equals(processInstance.getHost()) ){
continue;
}
processService.processNeedFailoverProcessInstances(processInstance);
}
logger.info("master failover end");
}
public InterProcessMutex blockAcquireMutex() throws Exception {
InterProcessMutex mutex = new InterProcessMutex(getZkClient(), getMasterLockPath());
mutex.acquire();
return mutex;
}
/**
* logger
*/
private static final Logger logger = LoggerFactory.getLogger(ZKMasterClient.class);
/**
* process service
*/
@Autowired
private ProcessService processService;
public void start() {
InterProcessMutex mutex = null;
try {
// create distributed lock with the root node path of the lock space as /dolphinscheduler/lock/failover/master
String znodeLock = getMasterStartUpLockPath();
mutex = new InterProcessMutex(getZkClient(), znodeLock);
mutex.acquire();
// init system znode
this.initSystemZNode();
while (!checkZKNodeExists(NetUtils.getHost(), ZKNodeType.MASTER)) {
ThreadUtils.sleep(SLEEP_TIME_MILLIS);
}
// self tolerant
if (getActiveMasterNum() == 1) {
failoverWorker(null, true);
failoverMaster(null);
}
} catch (Exception e) {
logger.error("master start up exception", e);
} finally {
releaseMutex(mutex);
}
}
@Override
public void close() {
super.close();
}
/**
* handle path events that this class cares about
*
* @param client zkClient
* @param event path event
* @param path zk path
*/
@Override
protected void dataChanged(CuratorFramework client, TreeCacheEvent event, String path) {
//monitor master
if (path.startsWith(getZNodeParentPath(ZKNodeType.MASTER) + Constants.SINGLE_SLASH)) {
handleMasterEvent(event, path);
} else if (path.startsWith(getZNodeParentPath(ZKNodeType.WORKER) + Constants.SINGLE_SLASH)) {
//monitor worker
handleWorkerEvent(event, path);
}
}
/**
* remove zookeeper node path
*
* @param path zookeeper node path
* @param zkNodeType zookeeper node type
* @param failover is failover
*/
private void removeZKNodePath(String path, ZKNodeType zkNodeType, boolean failover) {
logger.info("{} node deleted : {}", zkNodeType.toString(), path);
InterProcessMutex mutex = null;
try {
String failoverPath = getFailoverLockPath(zkNodeType);
// create a distributed lock
mutex = new InterProcessMutex(getZkClient(), failoverPath);
mutex.acquire();
String serverHost = getHostByEventDataPath(path);
// handle dead server
handleDeadServer(path, zkNodeType, Constants.ADD_ZK_OP);
//failover server
if (failover) {
failoverServerWhenDown(serverHost, zkNodeType);
}
} catch (Exception e) {
logger.error("{} server failover failed.", zkNodeType.toString());
logger.error("failover exception ", e);
} finally {
releaseMutex(mutex);
}
}
/**
* failover server when server down
*
* @param serverHost server host
* @param zkNodeType zookeeper node type
* @throws Exception exception
*/
private void failoverServerWhenDown(String serverHost, ZKNodeType zkNodeType) throws Exception {
if (StringUtils.isEmpty(serverHost)) {
return;
}
switch (zkNodeType) {
case MASTER:
failoverMaster(serverHost);
break;
case WORKER:
failoverWorker(serverHost, true);
break;
default:
break;
}
}
/**
* get failover lock path
*
* @param zkNodeType zookeeper node type
* @return fail over lock path
*/
private String getFailoverLockPath(ZKNodeType zkNodeType) {
switch (zkNodeType) {
case MASTER:
return getMasterFailoverLockPath();
case WORKER:
return getWorkerFailoverLockPath();
default:
return "";
}
}
/**
* monitor master
*
* @param event event
* @param path path
*/
public void handleMasterEvent(TreeCacheEvent event, String path) {
switch (event.getType()) {
case NODE_ADDED:
logger.info("master node added : {}", path);
break;
case NODE_REMOVED:
removeZKNodePath(path, ZKNodeType.MASTER, true);
break;
default:
break;
}
}
/**
* monitor worker
*
* @param event event
* @param path path
*/
public void handleWorkerEvent(TreeCacheEvent event, String path) {
switch (event.getType()) {
case NODE_ADDED:
logger.info("worker node added : {}", path);
break;
case NODE_REMOVED:
logger.info("worker node deleted : {}", path);
removeZKNodePath(path, ZKNodeType.WORKER, true);
break;
default:
break;
}
}
/**
* task needs failover if task start before worker starts
*
* @param taskInstance task instance
* @return true if task instance need fail over
*/
private boolean checkTaskInstanceNeedFailover(TaskInstance taskInstance) throws Exception {
boolean taskNeedFailover = true;
//now no host will execute this task instance,so no need to failover the task
if (taskInstance.getHost() == null) {
return false;
}
// if the worker node exists in zookeeper, we must check the task starts after the worker
if (checkZKNodeExists(taskInstance.getHost(), ZKNodeType.WORKER)) {
//if task start after worker starts, there is no need to failover the task.
if (checkTaskAfterWorkerStart(taskInstance)) {
taskNeedFailover = false;
}
}
return taskNeedFailover;
}
/**
* check task start after the worker server starts.
*
* @param taskInstance task instance
* @return true if task instance start time after worker server start date
*/
private boolean checkTaskAfterWorkerStart(TaskInstance taskInstance) {
if (StringUtils.isEmpty(taskInstance.getHost())) {
return false;
}
Date workerServerStartDate = null;
List<Server> workerServers = getServersList(ZKNodeType.WORKER);
for (Server workerServer : workerServers) {
if (taskInstance.getHost().equals(workerServer.getHost() + Constants.COLON + workerServer.getPort())) {
workerServerStartDate = workerServer.getCreateTime();
break;
}
}
if (workerServerStartDate != null) {
return taskInstance.getStartTime().after(workerServerStartDate);
}
return false;
}
/**
* failover worker tasks
*
* 1. kill yarn job if there are yarn jobs in tasks.
* 2. change task state from running to need failover.
* 3. failover all tasks when workerHost is null
* @param workerHost worker host
*/
/**
* failover worker tasks
* <p>
* 1. kill yarn job if there are yarn jobs in tasks.
* 2. change task state from running to need failover.
* 3. failover all tasks when workerHost is null
*
* @param workerHost worker host
* @param needCheckWorkerAlive need check worker alive
* @throws Exception exception
*/
private void failoverWorker(String workerHost, boolean needCheckWorkerAlive) throws Exception {
logger.info("start worker[{}] failover ...", workerHost);
List<TaskInstance> needFailoverTaskInstanceList = processService.queryNeedFailoverTaskInstances(workerHost);
for (TaskInstance taskInstance : needFailoverTaskInstanceList) {
if (needCheckWorkerAlive) {
if (!checkTaskInstanceNeedFailover(taskInstance)) {
continue;
}
}
ProcessInstance processInstance = processService.findProcessInstanceDetailById(taskInstance.getProcessInstanceId());
if (processInstance != null) {
taskInstance.setProcessInstance(processInstance);
}
TaskExecutionContext taskExecutionContext = TaskExecutionContextBuilder.get()
.buildTaskInstanceRelatedInfo(taskInstance)
.buildProcessInstanceRelatedInfo(processInstance)
.create();
// only kill yarn job if exists , the local thread has exited
ProcessUtils.killYarnJob(taskExecutionContext);
taskInstance.setState(ExecutionStatus.NEED_FAULT_TOLERANCE);
processService.saveTaskInstance(taskInstance);
}
logger.info("end worker[{}] failover ...", workerHost);
}
/**
* failover master tasks
*
* @param masterHost master host
*/
private void failoverMaster(String masterHost) {
logger.info("start master failover ...");
List<ProcessInstance> needFailoverProcessInstanceList = processService.queryNeedFailoverProcessInstances(masterHost);
//updateProcessInstance host is null and insert into command
for (ProcessInstance processInstance : needFailoverProcessInstanceList) {
if (Constants.NULL.equals(processInstance.getHost())) {
continue;
}
processService.processNeedFailoverProcessInstances(processInstance);
}
logger.info("master failover end");
}
public InterProcessMutex blockAcquireMutex() throws Exception {
InterProcessMutex mutex = new InterProcessMutex(getZkClient(), getMasterLockPath());
mutex.acquire();
return mutex;
}
}

51
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/LowerWeightRoundRobinTest.java

@ -14,9 +14,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.dispatch.host.assign;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.junit.Assert;
import org.junit.Test;
import java.util.ArrayList;
@ -27,15 +30,47 @@ public class LowerWeightRoundRobinTest {
@Test
public void testSelect(){
public void testSelect() {
Collection<HostWeight> sources = new ArrayList<>();
sources.add(new HostWeight(Host.of("192.158.2.1:11"), 0.06, 0.44, 3.84));
sources.add(new HostWeight(Host.of("192.158.2.1:22"), 0.06, 0.56, 3.24));
sources.add(new HostWeight(Host.of("192.158.2.1:33"), 0.06, 0.80, 3.15));
System.out.println(sources);
sources.add(new HostWeight(Host.of("192.158.2.1:11:100:" + (System.currentTimeMillis() - 60 * 8 * 1000)), 0.06, 0.44, 3.84));
sources.add(new HostWeight(Host.of("192.158.2.2:22:100:" + (System.currentTimeMillis() - 60 * 5 * 1000)), 0.06, 0.56, 3.24));
sources.add(new HostWeight(Host.of("192.158.2.3:33:100:" + (System.currentTimeMillis() - 60 * 2 * 1000)), 0.06, 0.80, 3.15));
LowerWeightRoundRobin roundRobin = new LowerWeightRoundRobin();
HostWeight result;
result = roundRobin.select(sources);
Assert.assertEquals("192.158.2.1", result.getHost().getIp());
result = roundRobin.select(sources);
Assert.assertEquals("192.158.2.2", result.getHost().getIp());
result = roundRobin.select(sources);
Assert.assertEquals("192.158.2.1", result.getHost().getIp());
result = roundRobin.select(sources);
Assert.assertEquals("192.158.2.2", result.getHost().getIp());
result = roundRobin.select(sources);
Assert.assertEquals("192.158.2.1", result.getHost().getIp());
result = roundRobin.select(sources);
Assert.assertEquals("192.158.2.2", result.getHost().getIp());
}
@Test
public void testWarmUpSelect() {
Collection<HostWeight> sources = new ArrayList<>();
sources.add(new HostWeight(Host.of("192.158.2.1:11:100:" + (System.currentTimeMillis() - 60 * 8 * 1000)), 0.06, 0.44, 3.84));
sources.add(new HostWeight(Host.of("192.158.2.2:22:100:" + (System.currentTimeMillis() - 60 * 5 * 1000)), 0.06, 0.44, 3.84));
sources.add(new HostWeight(Host.of("192.158.2.3:33:100:" + (System.currentTimeMillis() - 60 * 3 * 1000)), 0.06, 0.44, 3.84));
sources.add(new HostWeight(Host.of("192.158.2.4:33:100:" + (System.currentTimeMillis() - 60 * 11 * 1000)), 0.06, 0.44, 3.84));
LowerWeightRoundRobin roundRobin = new LowerWeightRoundRobin();
for(int i = 0; i < 100; i ++){
System.out.println(roundRobin.select(sources));
}
HostWeight result;
result = roundRobin.select(sources);
Assert.assertEquals("192.158.2.4", result.getHost().getIp());
result = roundRobin.select(sources);
Assert.assertEquals("192.158.2.1", result.getHost().getIp());
result = roundRobin.select(sources);
Assert.assertEquals("192.158.2.2", result.getHost().getIp());
result = roundRobin.select(sources);
Assert.assertEquals("192.158.2.4", result.getHost().getIp());
result = roundRobin.select(sources);
Assert.assertEquals("192.158.2.1", result.getHost().getIp());
}
}

16
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RandomSelectorTest.java

@ -14,11 +14,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.dispatch.host.assign;
import org.apache.commons.lang.ObjectUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.junit.Assert;
import org.junit.Test;
@ -31,22 +31,22 @@ import java.util.Collections;
public class RandomSelectorTest {
@Test(expected = IllegalArgumentException.class)
public void testSelectWithIllegalArgumentException(){
public void testSelectWithIllegalArgumentException() {
RandomSelector selector = new RandomSelector();
selector.select(Collections.EMPTY_LIST);
selector.select(null);
}
@Test
public void testSelect1(){
public void testSelect1() {
RandomSelector selector = new RandomSelector();
Host result = selector.select(Arrays.asList(new Host("192.168.1.1",80,100),new Host("192.168.1.2",80,20)));
Host result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 100, System.currentTimeMillis()), new Host("192.168.1.2", 80, 20, System.currentTimeMillis())));
Assert.assertNotNull(result);
}
@Test
public void testSelect(){
public void testSelect() {
RandomSelector selector = new RandomSelector();
Host result = selector.select(Arrays.asList(new Host("192.168.1.1",80,100),new Host("192.168.1.1",80,20)));
Host result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 100, System.currentTimeMillis()), new Host("192.168.1.1", 80, 20, System.currentTimeMillis())));
Assert.assertNotNull(result);
}

70
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/RoundRobinSelectorTest.java

@ -14,16 +14,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.dispatch.host.assign;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.junit.Assert;
import org.junit.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
/**
* round robin selector
@ -33,43 +33,59 @@ public class RoundRobinSelectorTest {
@Test(expected = IllegalArgumentException.class)
public void testSelectWithIllegalArgumentException() {
RoundRobinSelector selector = new RoundRobinSelector();
selector.select(Collections.EMPTY_LIST);
selector.select(null);
}
@Test
public void testSelect1() {
RoundRobinSelector selector = new RoundRobinSelector();
Host result = null;
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris")));
Host result;
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.1", result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris")));
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.2", result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris")));
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.1", result.getIp());
// add new host
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris")));
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.1", result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.2", result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris"),
new Host("192.168.1.3", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.1", result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris")));
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris"),
new Host("192.168.1.3", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.3", result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris"),
new Host("192.168.1.3", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.1", result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris"),
new Host("192.168.1.3", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.2", result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris"), new Host("192.168.1.3", 80, 10, "kris")));
Assert.assertEquals("192.168.1.1",result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris"), new Host("192.168.1.3", 80, 10, "kris")));
Assert.assertEquals("192.168.1.3",result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris"), new Host("192.168.1.3", 80, 10, "kris")));
Assert.assertEquals("192.168.1.1",result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris"), new Host("192.168.1.3", 80, 10, "kris")));
Assert.assertEquals("192.168.1.2",result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris"), new Host("192.168.1.3", 80, 10, "kris")));
Assert.assertEquals("192.168.1.1",result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris"), new Host("192.168.1.3", 80, 10, "kris")));
Assert.assertEquals("192.168.1.3",result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris"),
new Host("192.168.1.3", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.1", result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris"),
new Host("192.168.1.3", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.3", result.getIp());
// remove host3
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris")));
Assert.assertEquals("192.168.1.1",result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris")));
Assert.assertEquals("192.168.1.2",result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, "kris"), new Host("192.168.1.2", 80, 10, "kris")));
Assert.assertEquals("192.168.1.1",result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.1", result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.2", result.getIp());
result = selector.select(Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis(), "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis(), "kris")));
Assert.assertEquals("192.168.1.1", result.getIp());
}
@Test
public void testWarmUpRoundRobinSelector() {
RoundRobinSelector selector = new RoundRobinSelector();
Host result;
result = selector.select(
Arrays.asList(new Host("192.168.1.1", 80, 20, System.currentTimeMillis() - 60 * 1000 * 2, "kris"), new Host("192.168.1.2", 80, 10, System.currentTimeMillis() - 60 * 1000 * 10, "kris")));
Assert.assertEquals("192.168.1.2", result.getIp());
}

43
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/HostTest.java

@ -0,0 +1,43 @@
/*
* 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.utils;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.junit.Assert;
import org.junit.Test;
/**
* host test
*/
public class HostTest {
@Test
public void testHostWarmUp() {
Host host = Host.of(("192.158.2.2:22:100:" + (System.currentTimeMillis() - 60 * 5 * 1000)));
Assert.assertEquals(50, host.getWeight());
host = Host.of(("192.158.2.2:22:100:" + (System.currentTimeMillis() - 60 * 10 * 1000)));
Assert.assertEquals(100, host.getWeight());
}
@Test
public void testHost() {
Host host = Host.of("192.158.2.2:22");
Assert.assertEquals(22, host.getPort());
}
}

114
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/TaskManagerTest.java

@ -0,0 +1,114 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.worker.task;
import org.apache.dolphinscheduler.common.utils.LoggerUtils;
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.worker.cache.impl.TaskExecutionContextCacheManagerImpl;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import java.util.Date;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.powermock.api.mockito.PowerMockito;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@RunWith(PowerMockRunner.class)
@PrepareForTest({SpringApplicationContext.class})
public class TaskManagerTest {
private static Logger logger = LoggerFactory.getLogger(TaskManagerTest.class);
private TaskExecutionContext taskExecutionContext;
private Logger taskLogger;
private TaskExecutionContextCacheManagerImpl taskExecutionContextCacheManager;
@Before
public void before() {
// init task execution context, logger
taskExecutionContext = new TaskExecutionContext();
taskExecutionContext.setProcessId(12345);
taskExecutionContext.setProcessDefineId(1);
taskExecutionContext.setProcessInstanceId(1);
taskExecutionContext.setTaskInstanceId(1);
taskExecutionContext.setTaskType("");
taskExecutionContext.setFirstSubmitTime(new Date());
taskExecutionContext.setDelayTime(0);
taskExecutionContext.setLogPath("/tmp/test.log");
taskExecutionContext.setHost("localhost");
taskExecutionContext.setExecutePath("/tmp/dolphinscheduler/exec/process/1/2/3/4");
taskLogger = LoggerFactory.getLogger(LoggerUtils.buildTaskId(
LoggerUtils.TASK_LOGGER_INFO_PREFIX,
taskExecutionContext.getProcessDefineId(),
taskExecutionContext.getProcessInstanceId(),
taskExecutionContext.getTaskInstanceId()
));
taskExecutionContextCacheManager = new TaskExecutionContextCacheManagerImpl();
taskExecutionContextCacheManager.cacheTaskExecutionContext(taskExecutionContext);
PowerMockito.mockStatic(SpringApplicationContext.class);
PowerMockito.when(SpringApplicationContext.getBean(TaskExecutionContextCacheManagerImpl.class))
.thenReturn(taskExecutionContextCacheManager);
}
@Test
public void testNewTask() {
taskExecutionContext.setTaskType("SHELL");
Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
taskExecutionContext.setTaskType("WATERDROP");
Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
taskExecutionContext.setTaskType("HTTP");
Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
taskExecutionContext.setTaskType("MR");
Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
taskExecutionContext.setTaskType("SPARK");
Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
taskExecutionContext.setTaskType("FLINK");
Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
taskExecutionContext.setTaskType("PYTHON");
Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
taskExecutionContext.setTaskType("DATAX");
Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
taskExecutionContext.setTaskType("SQOOP");
Assert.assertNotNull(TaskManager.newTask(taskExecutionContext,taskLogger));
}
@Test(expected = IllegalArgumentException.class)
public void testNewTaskIsNull() {
taskExecutionContext.setTaskType(null);
TaskManager.newTask(taskExecutionContext,taskLogger);
}
@Test(expected = IllegalArgumentException.class)
public void testNewTaskIsNotExists() {
taskExecutionContext.setTaskType("XXX");
TaskManager.newTask(taskExecutionContext,taskLogger);
}
}

575
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java

File diff suppressed because it is too large Load Diff

116
dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/process/ProcessServiceTest.java

@ -0,0 +1,116 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.service.process;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.common.enums.WarningType;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.ProcessInstanceMap;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import java.util.Date;
import java.util.HashMap;
import java.util.Map;
import org.junit.Assert;
import org.junit.Test;
import com.fasterxml.jackson.databind.JsonNode;
/**
* process service test
*/
public class ProcessServiceTest {
@Test
public void testCreateSubCommand() {
ProcessService processService = new ProcessService();
ProcessInstance parentInstance = new ProcessInstance();
parentInstance.setProcessDefinitionId(1);
parentInstance.setWarningType(WarningType.SUCCESS);
parentInstance.setWarningGroupId(0);
TaskInstance task = new TaskInstance();
task.setTaskJson("{\"params\":{\"processDefinitionId\":100}}");
task.setId(10);
ProcessInstance childInstance = null;
ProcessInstanceMap instanceMap = new ProcessInstanceMap();
instanceMap.setParentProcessInstanceId(1);
instanceMap.setParentTaskInstanceId(10);
Command command = null;
//father history: start; child null == command type: start
parentInstance.setHistoryCmd("START_PROCESS");
parentInstance.setCommandType(CommandType.START_PROCESS);
command = processService.createSubProcessCommand(
parentInstance, childInstance, instanceMap, task
);
Assert.assertEquals(CommandType.START_PROCESS, command.getCommandType());
//father history: start,start failure; child null == command type: start
parentInstance.setCommandType(CommandType.START_FAILURE_TASK_PROCESS);
parentInstance.setHistoryCmd("START_PROCESS,START_FAILURE_TASK_PROCESS");
command = processService.createSubProcessCommand(
parentInstance, childInstance, instanceMap, task
);
Assert.assertEquals(CommandType.START_PROCESS, command.getCommandType());
//father history: scheduler,start failure; child null == command type: scheduler
parentInstance.setCommandType(CommandType.START_FAILURE_TASK_PROCESS);
parentInstance.setHistoryCmd("SCHEDULER,START_FAILURE_TASK_PROCESS");
command = processService.createSubProcessCommand(
parentInstance, childInstance, instanceMap, task
);
Assert.assertEquals(CommandType.SCHEDULER, command.getCommandType());
//father history: complement,start failure; child null == command type: complement
String startString = "2020-01-01 00:00:00";
String endString = "2020-01-10 00:00:00";
parentInstance.setCommandType(CommandType.START_FAILURE_TASK_PROCESS);
parentInstance.setHistoryCmd("COMPLEMENT_DATA,START_FAILURE_TASK_PROCESS");
Map<String,String> complementMap = new HashMap<>();
complementMap.put(Constants.CMDPARAM_COMPLEMENT_DATA_START_DATE, startString);
complementMap.put(Constants.CMDPARAM_COMPLEMENT_DATA_END_DATE, endString);
parentInstance.setCommandParam(JSONUtils.toJsonString(complementMap));
command = processService.createSubProcessCommand(
parentInstance, childInstance, instanceMap, task
);
Assert.assertEquals(CommandType.COMPLEMENT_DATA, command.getCommandType());
JsonNode complementDate = JSONUtils.parseObject(command.getCommandParam());
Date start = DateUtils.stringToDate(complementDate.get(Constants.CMDPARAM_COMPLEMENT_DATA_START_DATE).asText());
Date end = DateUtils.stringToDate(complementDate.get(Constants.CMDPARAM_COMPLEMENT_DATA_END_DATE).asText());
Assert.assertEquals(startString, DateUtils.dateToString(start));
Assert.assertEquals(endString, DateUtils.dateToString(end));
//father history: start,failure,start failure; child not null == command type: start failure
childInstance = new ProcessInstance();
parentInstance.setCommandType(CommandType.START_FAILURE_TASK_PROCESS);
parentInstance.setHistoryCmd("START_PROCESS,START_FAILURE_TASK_PROCESS");
command = processService.createSubProcessCommand(
parentInstance, childInstance, instanceMap, task
);
Assert.assertEquals(CommandType.START_FAILURE_TASK_PROCESS, command.getCommandType());
}
}

1
dolphinscheduler-ui/package.json

@ -20,6 +20,7 @@
"clipboard": "^2.0.1",
"codemirror": "^5.43.0",
"d3": "^3.5.17",
"dagre": "^0.8.5",
"dayjs": "^1.7.8",
"echarts": "4.1.0",
"html2canvas": "^0.5.0-beta4",

7
dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/config.js

@ -236,6 +236,13 @@ const tasksState = {
color: '#5102ce',
icoUnicode: 'ans-icon-coin',
isSpin: false
},
FORCED_SUCCESS: {
id: 13,
desc: `${i18n.$t('Forced success')}`,
color: '#5102ce',
icoUnicode: 'ans-icon-success-solid',
isSpin: false
}
}

214
dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.js

@ -22,6 +22,7 @@ import { jsPlumb } from 'jsplumb'
import JSP from './plugIn/jsPlumbHandle'
import DownChart from './plugIn/downChart'
import store from '@/conf/home/store'
import dagre from "dagre"
/**
* Prototype method
@ -115,202 +116,40 @@ Dag.prototype.toolbarEvent = function ({ item, code, is }) {
*/
Dag.prototype.backfill = function (arg) {
if (arg) {
let locationsValue = store.state.dag.locations
const locationsValue1 = store.state.dag.locations
const locationsValue2 = store.state.dag.locations
const arr = []
for (const i in locationsValue1) {
const objs = {}
objs.id = i
arr.push(Object.assign(objs, locationsValue1[i])) // Attributes
}
const tmp = []
for (const i in locationsValue2) {
if (locationsValue2[i].targetarr !== '' && locationsValue2[i].targetarr.split(',').length > 1) {
tmp.push(locationsValue2[i])
}
}
const copy = function (array) {
const newArray = []
for (const item of array) {
newArray.push(item)
}
return newArray
}
const newArr = copy(arr)
const getNewArr = function () {
for (let i = 0; i < newArr.length; i++) {
if (newArr[i].targetarr !== '' && newArr[i].targetarr.split(',').length > 1) {
newArr[i].targetarr = newArr[i].targetarr.split(',').shift()
}
}
return newArr
}
getNewArr()
/**
* @description Transform flat data into a tree structure
* @param {Array} arr Flat data
* @param {String} pidStr targetarr key name
* @param {String} idStr id key name
* @param {String} childrenStr children key name
*/
const fommat = function ({ arrayList, pidStr = 'targetarr', idStr = 'id', childrenStr = 'children' }) {
const listOjb = {} // Used to store objects of the form {key: obj}
const treeList = [] // An array to store the final tree structure data
// Transform the data into {key: obj} format, which is convenient for the following data processing
for (let i = 0; i < arrayList.length; i++) {
listOjb[arrayList[i][idStr]] = arrayList[i]
}
// Format data based on pid
for (let j = 0; j < arrayList.length; j++) {
// Determine if the parent exists
// let haveParent = arrayList[j].targetarr.split(',').length>1?listOjb[arrayList[j].targetarr.split(',')[0]]:listOjb[arrayList[j][pidStr]]
const haveParent = listOjb[arrayList[j][pidStr]]
if (haveParent) {
// If there is no parent children field, create a children field
!haveParent[childrenStr] && (haveParent[childrenStr] = [])
// Insert child in parent
haveParent[childrenStr].push(arrayList[j])
} else {
// If there is no parent, insert directly into the outermost layer
treeList.push(arrayList[j])
}
}
return treeList
}
const datas = fommat({ arrayList: newArr, pidStr: 'targetarr' })
// Count the number of leaf nodes
const getLeafCountTree = function (json) {
if (!json.children) {
json.colspan = 1
return 1
} else {
let leafCount = 0
for (let i = 0; i < json.children.length; i++) {
leafCount = leafCount + getLeafCountTree(json.children[i])
}
json.colspan = leafCount
return leafCount
}
}
// Number of tree node levels
const countTree = getLeafCountTree(datas[0])
const getMaxFloor = function (treeData) {
let max = 0
function each (data, floor) {
data.forEach(e => {
e.floor = floor
e.x = floor * 170
if (floor > max) {
max = floor
}
if (e.children) {
each(e.children, floor + 1)
}
})
}
each(treeData, 1)
return max
}
getMaxFloor(datas)
// The last child of each node
let lastchildren = []
const forxh = function (list) {
for (let i = 0; i < list.length; i++) {
const chlist = list[i]
if (chlist.children) {
forxh(chlist.children)
} else {
lastchildren.push(chlist)
}
}
}
forxh(datas)
// Get all parent nodes above the leaf node
const treeFindPath = function (tree, func, path, n) {
if (!tree) return []
for (const data of tree) {
path.push(data.name)
if (func(data)) return path
if (data.children) {
const findChildren = treeFindPath(data.children, func, path, n)
if (findChildren.length) return findChildren
}
path.pop()
}
return []
}
const toLine = function (data) {
return data.reduce((arrData, { id, name, targetarr, x, y, children = [] }) =>
arrData.concat([{ id, name, targetarr, x, y }], toLine(children)), [])
}
const listarr = toLine(datas)
const listarrs = toLine(datas)
const dataObject = {}
for (let i = 0; i < listarrs.length; i++) {
delete (listarrs[i].id)
}
const marginX = 100
const g = new dagre.graphlib.Graph()
g.setGraph({})
g.setDefaultEdgeLabel(function () { return {} })
for (let a = 0; a < listarr.length; a++) {
dataObject[listarr[a].id] = listarrs[a]
for (const i in store.state.dag.locations) {
const location = store.state.dag.locations[i]
g.setNode(i, { label: i, width: Math.min(location.name.length * 7, 170), height: 150 })
}
// Comparison function
const createComparisonFunction = function (propertyName) {
return function (object1, object2) {
const value1 = object1[propertyName]
const value2 = object2[propertyName]
if (value1 < value2) {
return -1
} else if (value1 > value2) {
return 1
} else {
return 0
}
}
for (const i in store.state.dag.connects) {
const connect = store.state.dag.connects[i]
g.setEdge(connect['endPointSourceId'], connect['endPointTargetId'])
}
dagre.layout(g)
lastchildren = lastchildren.sort(createComparisonFunction('x'))
// Coordinate value of each leaf node
for (let a = 0; a < lastchildren.length; a++) {
dataObject[lastchildren[a].id].y = (a + 1) * 120
}
for (let i = 0; i < lastchildren.length; i++) {
const node = treeFindPath(datas, data => data.targetarr === lastchildren[i].targetarr, [], i + 1)
for (let j = 0; j < node.length; j++) {
for (let k = 0; k < listarrs.length; k++) {
if (node[j] === listarrs[k].name) {
listarrs[k].y = (i + 1) * 120
}
}
}
}
for (let i = 0; i < tmp.length; i++) {
for (const objs in dataObject) {
if (tmp[i].name === dataObject[objs].name) {
dataObject[objs].targetarr = tmp[i].targetarr
}
}
}
for (let a = 0; a < lastchildren.length; a++) {
dataObject[lastchildren[a].id].y = (a + 1) * 120
}
if (countTree > 1) {
dataObject[Object.keys(locationsValue1)[0]].y = (countTree / 2) * 120 + 50
}
locationsValue = dataObject
const self = this
const dataObject = {}
g.nodes().forEach(function (v) {
const node = g.node(v)
const location = store.state.dag.locations[node.label]
const obj = {}
obj.name = location.name
obj.x = node.x + marginX
obj.y = node.y
obj.targetarr = location.targetarr
dataObject[node.label] = obj
})
jsPlumb.ready(() => {
JSP.init({
dag: this.dag,
instance: this.instance,
options: {
onRemoveNodes ($id) {
self.dag.removeEventModelById($id)
this.dag.removeEventModelById($id)
}
}
})
@ -319,20 +158,19 @@ Dag.prototype.backfill = function (arg) {
// connects
connects: _.cloneDeep(store.state.dag.connects),
// Node location information
locations: _.cloneDeep(locationsValue),
locations: _.cloneDeep(dataObject),
// Node data
largeJson: _.cloneDeep(store.state.dag.tasks)
})
})
} else {
const self = this
jsPlumb.ready(() => {
JSP.init({
dag: this.dag,
instance: this.instance,
options: {
onRemoveNodes ($id) {
self.dag.removeEventModelById($id)
this.dag.removeEventModelById($id)
}
}
})

218
dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/_source/dependentTimeout.vue

@ -0,0 +1,218 @@
/*
* 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.
*/
<template>
<div class="timeout-alarm-model">
<div class="clearfix list">
<div class="text-box">
<span>{{$t('Timeout alarm')}}</span>
</div>
<div class="cont-box">
<label class="label-box">
<div style="padding-top: 5px;">
<x-switch v-model="enable" @on-click="_onSwitch(0, $event)" :disabled="isDetails"></x-switch>
</div>
</label>
</div>
</div>
<div class="clearfix list" v-if="enable">
<div class="text-box">
<span>{{$t('Waiting Dependent start')}}</span>
</div>
<div class="cont-box">
<label class="label-box">
<div style="padding: 5px 0;">
<x-switch v-model="waitStartTimeout.enable" @on-click="_onSwitch(1, $event)" :disabled="isDetails"></x-switch>
</div>
</label>
</div>
</div>
<div class="clearfix list" v-if="enable && waitStartTimeout.enable">
<div class="cont-box">
<label class="label-box">
<span class="text-box">
<span>{{$t('Timeout period')}}</span>
</span>
<x-input v-model="waitStartTimeout.interval" style="width: 100px;" :disabled="isDetails" maxlength="9">
<span slot="append">{{$t('Minute')}}</span>
</x-input>
<span class="text-box">
<span>{{$t('Check interval')}}</span>
</span>
<x-input v-model="waitStartTimeout.checkInterval" style="width: 100px;" :disabled="isDetails" maxlength="9">
<span slot="append">{{$t('Minute')}}</span>
</x-input>
<span class="text-box">
<span>{{$t('Timeout strategy')}}</span>
</span>
<div style="padding-top: 6px;">
<x-checkbox-group v-model="waitStartTimeout.strategy">
<x-checkbox label="FAILED" :disabled="true">{{$t('Timeout failure')}}</x-checkbox>
</x-checkbox-group>
</div>
</label>
</div>
</div>
<div class="clearfix list" v-if="enable">
<div class="text-box">
<span>{{$t('Waiting Dependent complete')}}</span>
</div>
<div class="cont-box">
<label class="label-box">
<div style="padding: 5px 0;">
<x-switch v-model="waitCompleteTimeout.enable" @on-click="_onSwitch(2, $event)" :disabled="isDetails"></x-switch>
</div>
</label>
</div>
</div>
<div class="clearfix list" v-if="enable && waitCompleteTimeout.enable">
<div class="cont-box">
<label class="label-box">
<span class="text-box">
<span>{{$t('Timeout period')}}</span>
</span>
<x-input v-model="waitCompleteTimeout.interval" style="width: 100px;" :disabled="isDetails" maxlength="9">
<span slot="append">{{$t('Minute')}}</span>
</x-input>
<span class="text-box">
<span>{{$t('Timeout strategy')}}</span>
</span>
<div style="padding-top: 6px;">
<x-checkbox-group v-model="waitCompleteTimeout.strategy">
<x-checkbox label="WARN" :disabled="isDetails">{{$t('Timeout alarm')}}</x-checkbox>
<x-checkbox label="FAILED" :disabled="isDetails">{{$t('Timeout failure')}}</x-checkbox>
</x-checkbox-group>
</div>
</label>
</div>
</div>
</div>
</template>
<script>
import _ from 'lodash'
import disabledState from '@/module/mixin/disabledState'
export default {
name: 'form-dependent-timeout',
data () {
return {
// Timeout display hiding
enable: false,
waitStartTimeout: {
enable: false,
// Timeout strategy
strategy: ['FAILED'],
// Timeout period
interval: null,
checkInterval: null
},
waitCompleteTimeout: {
enable: false,
// Timeout strategy
strategy: [],
// Timeout period
interval: null
}
}
},
mixins: [disabledState],
props: {
backfillItem: Object
},
methods: {
_onSwitch (p, is) {
// reset timeout setting when switch timeout on/off.
// p = 0 for timeout switch; p = 1 for wait start timeout switch; p = 2 for wait complete timeout switch.
if (p === 1 || p === 0) {
this.waitStartTimeout.interval = is ? 30 : null
this.waitStartTimeout.checkInterval = is ? 1 : null
}
if (p === 2 || p === 0) {
this.waitCompleteTimeout.strategy = is ? ['WARN'] : []
this.waitCompleteTimeout.interval = is ? 30 : null
}
},
_verification () {
// Verification timeout policy
if (this.enable
&& (this.waitCompleteTimeout.enable && !this.waitCompleteTimeout.strategy.length)
|| (this.waitStartTimeout.enable && !this.waitStartTimeout.strategy.length)) {
this.$message.warning(`${this.$t('Timeout strategy must be selected')}`)
return false
}
// Verify timeout duration Non 0 positive integer
const reg = /^[1-9]\d*$/
if (this.enable
&& (this.waitCompleteTimeout.enable && !reg.test(this.waitCompleteTimeout.interval))
|| (this.waitStartTimeout.enable && (!reg.test(this.waitStartTimeout.interval || !reg.test(this.waitStartTimeout.checkInterval))))) {
this.$message.warning(`${this.$t('Timeout must be a positive integer')}`)
return false
}
// Verify timeout duration longer than check interval
if (this.enable && this.waitStartTimeout.enable && this.waitStartTimeout.checkInterval >= this.waitStartTimeout.interval) {
this.$message.warning(`${this.$t('Timeout must be longer than check interval')}`)
return false
}
this.$emit('on-timeout', {
waitStartTimeout: {
strategy: 'FAILED',
interval: parseInt(this.waitStartTimeout.interval),
checkInterval: parseInt(this.waitStartTimeout.checkInterval),
enable: this.waitStartTimeout.enable
},
waitCompleteTimeout: {
strategy: (() => {
// Handling checkout sequence
let strategy = this.waitCompleteTimeout.strategy
if (strategy.length === 2 && strategy[0] === 'FAILED') {
return [strategy[1], strategy[0]].join(',')
} else {
return strategy.join(',')
}
})(),
interval: parseInt(this.waitCompleteTimeout.interval),
enable: this.waitCompleteTimeout.enable
}
})
return true
}
},
watch: {
},
created () {
let o = this.backfillItem
// Non-null objects represent backfill
if (!_.isEmpty(o)) {
if (o.timeout) {
this.enable = true
this.waitCompleteTimeout.enable = o.timeout.enable || false
this.waitCompleteTimeout.strategy = _.split(o.timeout.strategy, ',') || ['WARN']
this.waitCompleteTimeout.interval = o.timeout.interval || null
}
if (o.waitStartTimeout) {
this.enable = true
this.waitStartTimeout.enable = o.waitStartTimeout.enable || false
this.waitStartTimeout.strategy = ['FAILED']
this.waitStartTimeout.interval = o.waitStartTimeout.interval || null
this.waitStartTimeout.checkInterval = o.waitStartTimeout.checkInterval || null
}
}
},
mounted () {
},
components: {}
}
</script>

104
dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue

@ -162,10 +162,18 @@
<!-- Task timeout alarm -->
<m-timeout-alarm
v-if="taskType !== 'DEPENDENT'"
ref="timeout"
:backfill-item="backfillItem"
@on-timeout="_onTimeout">
</m-timeout-alarm>
<!-- Dependent timeout alarm -->
<m-dependent-timeout
v-if="taskType === 'DEPENDENT'"
ref="dependentTimeout"
:backfill-item="backfillItem"
@on-timeout="_onDependentTimeout">
</m-dependent-timeout>
<!-- shell node -->
<m-shell
@ -277,6 +285,12 @@
:backfill-item="backfillItem"
:pre-node="preNode">
</m-conditions>
<!-- Pre-tasks in workflow -->
<m-pre-tasks
v-if="['SHELL', 'SUB_PROCESS'].indexOf(taskType) > -1"
@on-pre-tasks="_onPreTasks"
ref="PRE_TASK"
:backfill-item="backfillItem"></m-pre-tasks>
</div>
</div>
<div class="bottom-box">
@ -309,7 +323,9 @@
import mSubProcess from './tasks/sub_process'
import mSelectInput from './_source/selectInput'
import mTimeoutAlarm from './_source/timeoutAlarm'
import mDependentTimeout from './_source/dependentTimeout'
import mWorkerGroups from './_source/workerGroups'
import mPreTasks from './tasks/pre_tasks'
import clickoutside from '@/module/util/clickoutside'
import disabledState from '@/module/mixin/disabledState'
import { isNameExDag, rtBantpl } from './../plugIn/util'
@ -356,6 +372,8 @@
delayTime: '0',
// Task timeout alarm
timeout: {},
// (For Dependent nodes) Wait start timeout alarm
waitStartTimeout: {},
// Task priority
taskInstancePriority: 'MEDIUM',
// worker group id
@ -369,7 +387,11 @@
value: 'failed',
label: `${i18n.$t('failed')}`
}
]
],
// preTasks
preTaskIdsInWorkflow: [],
preTasksToAdd: [], // pre-taskIds to add, used in jsplumb connects
preTasksToDelete: [], // pre-taskIds to delete, used in jsplumb connects
}
},
/**
@ -393,6 +415,14 @@
_onDependent (o) {
this.dependence = Object.assign(this.dependence, {}, o)
},
/**
* Pre-tasks in workflow
*/
_onPreTasks (o) {
this.preTaskIdsInWorkflow = o.preTasks
this.preTasksToAdd = o.preTasksToAdd
this.preTasksToDelete = o.preTasksToDelete
},
/**
* cache dependent
*/
@ -405,6 +435,13 @@
_onTimeout (o) {
this.timeout = Object.assign(this.timeout, {}, o)
},
/**
* Dependent timeout alarm
*/
_onDependentTimeout (o) {
this.timeout = Object.assign(this.timeout, {}, o.waitCompleteTimeout)
this.waitStartTimeout = Object.assign(this.waitStartTimeout, {}, o.waitStartTimeout)
},
/**
* Click external to close the current component
*/
@ -483,6 +520,7 @@
retryInterval: this.retryInterval,
delayTime: this.delayTime,
timeout: this.timeout,
waitStartTimeout: this.waitStartTimeout,
taskInstancePriority: this.taskInstancePriority,
workerGroup: this.workerGroup,
status: this.status,
@ -536,13 +574,57 @@
return
}
// Verify task alarm parameters
if (!this.$refs['timeout']._verification()) {
return
if (this.taskType === 'DEPENDENT') {
if (!this.$refs['dependentTimeout']._verification()) {
return
}
} else {
if (!this.$refs['timeout']._verification()) {
return
}
}
// Verify node parameters
if (!this.$refs[this.taskType]._verification()) {
return
}
// Verify preTasks and update dag-things
if (this.$refs['PRE_TASK']) {
if (!this.$refs['PRE_TASK']._verification()) {
return
}
else {
// Sync data-targetarr
$(`#${this.id}`).attr(
'data-targetarr', this.preTaskIdsInWorkflow ? this.preTaskIdsInWorkflow.join(',') : '')
// Update JSP connections
let plumbIns = JSP.JspInstance
var targetId = this.id
// Update new connections
this.preTasksToAdd.map(sourceId => {
plumbIns.connect({
source: sourceId,
target: targetId,
type: 'basic',
paintStyle: { strokeWidth: 2, stroke: '#2d8cf0' },
HoverPaintStyle: {stroke: '#ccc', strokeWidth: 3}
})
})
// Update remove connections
let currentConnects = plumbIns.getAllConnections()
let len = currentConnects.length
for (let i = 0; i < len; i++) {
if (this.preTasksToDelete.indexOf(currentConnects[i].sourceId) > -1 && currentConnects[i].targetId == targetId) {
plumbIns.deleteConnection(currentConnects[i])
i -= 1
len -= 1
}
}
}
}
$(`#${this.id}`).find('span').text(this.name)
this.conditionResult.successNode[0] = this.successBranch
@ -562,6 +644,7 @@
retryInterval: this.retryInterval,
delayTime: this.delayTime,
timeout: this.timeout,
waitStartTimeout: this.waitStartTimeout,
taskInstancePriority: this.taskInstancePriority,
workerGroup: this.workerGroup,
status: this.status,
@ -684,6 +767,16 @@
}
this.cacheBackfillItem = JSON.parse(JSON.stringify(o))
this.isContentBox = true
// Init value of preTask selector
let preTaskIds = $(`#${this.id}`).attr('data-targetarr')
if (!_.isEmpty(this.backfillItem)) {
if (preTaskIds && preTaskIds.length) {
this.backfillItem.preTasks = preTaskIds.split(',')
} else {
this.backfillItem.preTasks = []
}
}
},
mounted () {
let self = this
@ -719,6 +812,7 @@
retryInterval: this.retryInterval,
delayTime: this.delayTime,
timeout: this.timeout,
waitStartTimeout: this.waitStartTimeout,
taskInstancePriority: this.taskInstancePriority,
workerGroup: this.workerGroup,
successBranch: this.successBranch,
@ -744,8 +838,10 @@
mConditions,
mSelectInput,
mTimeoutAlarm,
mDependentTimeout,
mPriority,
mWorkerGroups
mWorkerGroups,
mPreTasks,
}
}
</script>

108
dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/pre_tasks.vue

@ -0,0 +1,108 @@
/*
* 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.
*/
<template>
<div class="pre_tasks-model">
<div class="clearfix list">
<div class="text-box">
<span>{{$t('Pre tasks')}}</span>
</div>
<div class="cont-box">
<div class="label-box">
<x-select
ref="preTasksSelector"
style="width: 100%;"
filterable
multiple
v-model="preTasks"
:disabled="isDetails"
:id="preTasksSelectorId">
<x-option
v-for="task in preTaskList"
:key="task.id"
:value="task.id"
:label="task.name">
</x-option>
</x-select>
</div>
</div>
</div>
</div>
</template>
<script>
import disabledState from '@/module/mixin/disabledState'
export default {
name: 'pre_tasks',
mixins: [disabledState],
props: {
backfillItem: Object
},
data () {
return {
preTasksSelectorId: '_preTasksSelectorId', // Refresh target vue-component by changing id
preTasks: [],
preTasksOld: [],
}
},
mounted () {
this.preTasks = this.backfillItem['preTasks'] || this.preTasks
this.preTasksOld = this.preTasks
// Refresh target vue-component by changing id
this.$nextTick(() => {
this.preTasksSelectorId = 'preTasksSelectorId'
})
},
computed: {
preTaskList: function () {
let currentTaskId = this.backfillItem['id'] || this.id
let cacheTasks = Object.assign({}, this.store.state.dag.tasks)
let keys = Object.keys(cacheTasks)
for (let i = 0; i < keys.length; i++) {
let key = keys[i]
if ((!cacheTasks[key].id || !cacheTasks[key].name) || (currentTaskId && cacheTasks[key].id === currentTaskId)) {
// Clean undefined and current task data
delete cacheTasks[key]
}
}
return cacheTasks
},
// preTaskIds used to create new connection
preTasksToAdd: function () {
let toAddTasks = this.preTasks.filter(taskId => {
return (this.preTasksOld.indexOf(taskId) === -1)
})
return toAddTasks
},
// preTaskIds used to delete connection
preTasksToDelete: function () {
return this.preTasksOld.filter(taskId => this.preTasks.indexOf(taskId) === -1)
},
},
methods: {
// Pass data to parent-level to process dag
_verification () {
this.$emit('on-pre-tasks', {
preTasks: this.preTasks,
preTasksToAdd: this.preTasksToAdd,
preTasksToDelete: this.preTasksToDelete,
})
return true
}
}
}
</script>

3
dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/_source/instanceConditions/common.js

@ -63,6 +63,9 @@ const stateType = [
}, {
code: 'DELAY_EXECUTION',
label: `${i18n.$t('Delay execution')}`
}, {
code: 'FORCED_SUCCESS',
label: `${i18n.$t('Forced success')}`
}
]

19
dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/_source/list.vue

@ -124,7 +124,6 @@
</table>
</div>
<x-poptip
v-show="strSelectIds !== ''"
ref="poptipDeleteAll"
placement="bottom-start"
width="90">
@ -134,14 +133,12 @@
<x-button type="primary" size="xsmall" shape="circle" @click="_delete({},-1)">{{$t('Confirm')}}</x-button>
</div>
<template slot="reference">
<x-button size="xsmall" style="position: absolute; bottom: -48px; left: 22px;" >{{$t('Delete')}}</x-button>
<x-button size="xsmall" :disabled="!strSelectIds" style="position: absolute; bottom: -48px; left: 22px;" >{{$t('Delete')}}</x-button>
</template>
</x-poptip>
<template v-if="strSelectIds !== ''">
<x-button size="xsmall" style="position: absolute; bottom: -48px; left: 80px;" @click="_batchExport(item)" >{{$t('Export')}}</x-button>
<x-button size="xsmall" style="position: absolute; bottom: -48px; left: 140px;" @click="_batchCopy(item)" >{{$t('Batch copy')}}</x-button>
<x-button size="xsmall" style="position: absolute; bottom: -48px; left: 225px;" @click="_batchMove(item)" >{{$t('Batch move')}}</x-button>
</template>
<x-button size="xsmall" :disabled="!strSelectIds" style="position: absolute; bottom: -48px; left: 80px;" @click="_batchExport(item)" >{{$t('Export')}}</x-button>
<x-button size="xsmall" :disabled="!strSelectIds" style="position: absolute; bottom: -48px; left: 140px;" @click="_batchCopy(item)" >{{$t('Batch copy')}}</x-button>
<x-button size="xsmall" :disabled="!strSelectIds" style="position: absolute; bottom: -48px; left: 225px;" @click="_batchMove(item)" >{{$t('Batch move')}}</x-button>
</div>
</template>
@ -268,6 +265,12 @@
* Close the delete layer
*/
_closeDelete (i) {
// close batch
if (i < 0) {
this.$refs['poptipDeleteAll'].doClose()
return
}
// close one
this.$refs[`poptip-delete-${i}`][0].doClose()
},
/**
@ -596,8 +599,10 @@
}).then(res => {
this._onUpdate()
this.checkAll = false
this.strSelectIds = ''
this.$message.success(res.msg)
}).catch(e => {
this.strSelectIds = ''
this.checkAll = false
this.$message.error(e.msg || '')
})

2
dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/_source/util.js

@ -37,7 +37,7 @@ const warningTypeList = [
]
const isEmial = (val) => {
let regEmail = /^([a-zA-Z0-9]+[_|\-|\.]?)*[a-zA-Z0-9]+@([a-zA-Z0-9]+[_|\-|\.]?)*[a-zA-Z0-9]+\.[a-zA-Z]{2,3}$/ // eslint-disable-line
let regEmail = /^([a-zA-Z0-9]+[_|\-|\.]?)*[a-zA-Z0-9]+@([a-zA-Z0-9]+[_|\-|\.]?)*[a-zA-Z0-9]+\.[a-zA-Z]{2,}$/ // eslint-disable-line
return regEmail.test(val)
}

11
dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue

@ -288,7 +288,6 @@
</table>
</div>
<x-poptip
v-show="strDelete !== ''"
ref="poptipDeleteAll"
placement="bottom-start"
width="90">
@ -298,7 +297,7 @@
<x-button type="primary" size="xsmall" shape="circle" @click="_delete({},-1)">{{$t('Confirm')}}</x-button>
</div>
<template slot="reference">
<x-button size="xsmall" style="position: absolute; bottom: -48px; left: 22px;" >{{$t('Delete')}}</x-button>
<x-button size="xsmall" :disabled="!strDelete" style="position: absolute; bottom: -48px; left: 22px;" >{{$t('Delete')}}</x-button>
</template>
</x-poptip>
</div>
@ -344,6 +343,12 @@
* Close the delete layer
*/
_closeDelete (i) {
// close batch
if (i < 0) {
this.$refs['poptipDeleteAll'].doClose()
return
}
// close one
this.$refs[`poptip-delete-${i}`][0].doClose()
},
/**
@ -539,9 +544,11 @@
}).then(res => {
this._onUpdate()
this.checkAll = false
this.strDelete = ''
this.$message.success(res.msg)
}).catch(e => {
this.checkAll = false
this.strDelete = ''
this.$message.error(e.msg || '')
})
}

23
dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/taskInstance/_source/list.vue

@ -91,6 +91,16 @@
<td><span>{{item.duration}}</span></td>
<td><span>{{item.retryTimes}}</span></td>
<td>
<x-button
v-if="item.state === 'FAILURE' || item.state === 'NEED_FAULT_TOLERANCE' || item.state === 'KILL'"
type="error"
shape="circle"
size="xsmall"
data-toggle="tooltip"
:title="$t('Force success')"
icon="ans-icon-success-solid"
@click="_forceSuccess(item)">
</x-button>
<x-button
type="info"
shape="circle"
@ -110,6 +120,7 @@
import Permissions from '@/module/permissions'
import mLog from '@/conf/home/pages/dag/_source/formModel/log'
import { tasksState } from '@/conf/home/pages/dag/_source/config'
import { mapActions } from 'vuex'
export default {
name: 'list',
@ -126,6 +137,7 @@
pageSize: Number
},
methods: {
...mapActions('dag', ['forceTaskSuccess']),
_rtState (code) {
let o = tasksState[code]
return `<em class="${o.icoUnicode} ${o.isSpin ? 'as as-spin' : ''}" style="color:${o.color}" data-toggle="tooltip" data-container="body" title="${o.desc}"></em>`
@ -156,6 +168,17 @@
}
})
},
_forceSuccess (item) {
this.forceTaskSuccess({taskInstanceId: item.id}).then(res => {
if (res.code === 0) {
this.$message.success(res.msg)
} else {
this.$message.error(res.msg)
}
}).catch(e => {
this.$message.error(e.msg)
})
},
_go (item) {
this.$router.push({ path: `/projects/instance/list/${item.processInstanceId}` })
},

3
dolphinscheduler-ui/src/js/conf/home/pages/user/pages/password/_source/info.vue

@ -85,7 +85,8 @@
userPassword: this.userPassword,
tenantId: this.userInfo.tenantId,
email: this.userInfo.email,
phone: this.userInfo.phone
phone: this.userInfo.phone,
state: this.userInfo.state
}
this.spinnerLoading = true
this.updateUser(param).then(res => {

12
dolphinscheduler-ui/src/js/conf/home/store/dag/actions.js

@ -734,6 +734,18 @@ export default {
})
})
},
/**
* Force fail/kill/need_fault_tolerance task success
*/
forceTaskSuccess ({ state }, payload) {
return new Promise((resolve, reject) => {
io.post(`projects/${state.projectName}/task-instance/force-success`, payload, res => {
resolve(res)
}).catch(e => {
reject(e)
})
})
},
/**
* Query task record list
*/

9
dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js

@ -32,6 +32,7 @@ export default {
'Current node settings': 'Current node settings',
'View history': 'View history',
'View log': 'View log',
'Force success': 'Force success',
'Enter this child node': 'Enter this child node',
'Node name': 'Node name',
'Run flag': 'Run flag',
@ -426,8 +427,13 @@ export default {
'Timeout alarm': 'Timeout alarm',
'Timeout failure': 'Timeout failure',
'Timeout period': 'Timeout period',
'Waiting Dependent complete': 'Waiting Dependent complete',
'Waiting Dependent start': 'Waiting Dependent start',
'Check interval': 'Check interval',
'Timeout must be longer than check interval': 'Timeout must be longer than check interval',
'Timeout strategy must be selected': 'Timeout strategy must be selected',
'Timeout must be a positive integer': 'Timeout must be a positive integer',
'Forced success': 'Forced success',
'Add dependency': 'Add dependency',
and: 'and',
or: 'or',
@ -642,5 +648,6 @@ export default {
'Related items': 'Related items',
'Project name is required': 'Project name is required',
'Batch move': 'Batch move',
Version: 'Version'
Version: 'Version',
'Pre tasks': 'Pre tasks',
}

9
dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js

@ -32,6 +32,7 @@ export default {
'Current node settings': '当前节点设置',
'View history': '查看历史',
'View log': '查看日志',
'Force success': '强制成功',
'Enter this child node': '进入该子节点',
'Node name': '节点名称',
'Please enter name (required)': '请输入名称(必填)',
@ -420,8 +421,12 @@ export default {
'Timeout alarm': '超时告警',
'Timeout failure': '超时失败',
'Timeout period': '超时时长',
'Waiting Dependent complete': '等待依赖完成',
'Waiting Dependent start': '等待依赖启动',
'Check interval': '检查间隔',
'Timeout strategy must be selected': '超时策略必须选一个',
'Timeout must be a positive integer': '超时时长必须为正整数',
'Timeout must be longer than check interval': '超时时间必须比检查间隔长',
'Add dependency': '添加依赖',
and: '且',
or: '或',
@ -432,6 +437,7 @@ export default {
Running: '正在运行',
'Waiting for dependency to complete': '等待依赖完成',
'Delay execution': '延时执行',
'Forced success': '强制成功过',
Selected: '已选',
CurrentHour: '当前小时',
Last1Hour: '前1小时',
@ -642,5 +648,6 @@ export default {
'Related items': '关联项目',
'Project name is required': '项目名称必填',
'Batch move': '批量移动',
Version: '版本'
Version: '版本',
'Pre tasks': '前置任务',
}

10
e2e/pom.xml

@ -87,6 +87,16 @@
<artifactId>commons-pool2</artifactId>
<version>${commons-pool2.version}</version>
</dependency>
<dependency>
<groupId>org.jyaml</groupId>
<artifactId>jyaml</artifactId>
<version>1.3</version>
</dependency>
<dependency>
<groupId>org.springframework</groupId>
<artifactId>spring-core</artifactId>
<version>5.1.5.RELEASE</version>
</dependency>
</dependencies>
<build>

45
e2e/src/main/java/org/apache/dolphinscheduler/util/YmlReader.java

@ -0,0 +1,45 @@
/*
* 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.util;
import java.io.IOException;
import java.io.InputStream;
import java.util.HashMap;
import org.ho.yaml.Yaml;
import org.springframework.core.io.DefaultResourceLoader;
import org.springframework.core.io.Resource;
/**
* read yml file
*/
public class YmlReader {
public static HashMap<String,HashMap<String, String>> map;
public String getDataYml(String filePath, String key1, String key2) {
Yaml yaml = new Yaml();
Resource resource = new DefaultResourceLoader().getResource("classpath:" + filePath + ".yml");
try {
InputStream inputStream = resource.getInputStream();
map = yaml.loadType(inputStream, HashMap.class);
} catch (IOException e) {
e.printStackTrace();
}
String data = map.get(key1).get(key2);
return data;
}
}

1
e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java

@ -23,7 +23,6 @@ import org.openqa.selenium.PageLoadStrategy;
import org.openqa.selenium.WebDriver;
import org.openqa.selenium.chrome.ChromeDriver;
import org.openqa.selenium.chrome.ChromeOptions;
import org.openqa.selenium.remote.CapabilityType;
import java.io.IOException;
import java.util.concurrent.TimeUnit;

3
e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java

@ -28,7 +28,6 @@ public class LoginData {
*/
public static final String URL = PropertiesReader.getKey("LOGIN_URL");
/**
* Login username
*/
@ -38,6 +37,4 @@ public class LoginData {
* Login password
*/
public static final String PASSWORD = PropertiesReader.getKey("PASSWORD");
public static final String TENANT = "租户管理 - DolphinScheduler";
}

12
e2e/src/test/java/org/apache/dolphinscheduler/data/project/ProcessInstanceData.java

@ -16,10 +16,12 @@
*/
package org.apache.dolphinscheduler.data.project;
public class ProcessInstanceData {
//Process Instance page title
public static final String PROCESS_INSTANCE_TITLE = "工作流实例 - DolphinScheduler";
public static final String RERUN_TYPE= "重跑";
import org.apache.dolphinscheduler.util.YmlReader;
public class ProcessInstanceData {
public String getProcessInstanceData(String param) {
YmlReader ymlReader = new YmlReader();
String processInstanceData = ymlReader.getDataYml("testData/workflow_zh_cn", "processInstance", param);
return processInstanceData;
}
}

16
e2e/src/test/java/org/apache/dolphinscheduler/data/project/ProjectData.java

@ -16,14 +16,12 @@
*/
package org.apache.dolphinscheduler.data.project;
public class ProjectData {
public static final String CREATE_PROJECT_BUTTON = "创建项目";
import org.apache.dolphinscheduler.util.YmlReader;
// create project name
public static final String PROJECT_NAME = "selenium_project_1";
// create project description
public static final String DESCRIPTION = "test create project description";
// project page title
public static final String PROJECT_TITLE = "项目 - DolphinScheduler";
public class ProjectData {
public String getProjectData(String param) {
YmlReader ymlReader = new YmlReader();
String projectData = ymlReader.getDataYml("testData/workflow_zh_cn", "projectManage", param);
return projectData;
}
}

19
e2e/src/test/java/org/apache/dolphinscheduler/data/project/RunWorkflowData.java

@ -16,17 +16,12 @@
*/
package org.apache.dolphinscheduler.data.project;
public class RunWorkflowData {
/**
* run workflow data
*/
//input shell task name
public static final String RECIPIENT = "123456789@qq.com";
//input shell task description
public static final String Cc = "qwe12312sds@qq.com";
public static final String RUN_WORKFLOW_TITLE = "工作流定义 - DolphinScheduler";
import org.apache.dolphinscheduler.util.YmlReader;
public class RunWorkflowData {
public String getRunWorkflowData(String param) {
YmlReader ymlReader = new YmlReader();
String runWorkflowData = ymlReader.getDataYml("testData/workflow_zh_cn", "runWorkflow", param);
return runWorkflowData;
}
}

25
e2e/src/test/java/org/apache/dolphinscheduler/data/project/TimingData.java

@ -16,23 +16,12 @@
*/
package org.apache.dolphinscheduler.data.project;
public class TimingData {
/**
* timing data
*/
//input shell task name
public static final String RECIPIENT = "test123456@qq.com";
//input shell task description
public static final String Cc = "test.123qwe@qq.com";
public static final String EDIT_RECIPIENT = "test.edit123456@qq.com";
public static final String EDIT_Cc = "test.edit123qwe@qq.com";
import org.apache.dolphinscheduler.util.YmlReader;
public static final String TIMING_OFFLINE_STATE = "下线";
public static final String TIMING_ONLINE_STATE = "上线";
public static final String TIMING_TITLE = "定时任务列表 - DolphinScheduler";
public class TimingData {
public String getTimingData(String param) {
YmlReader ymlReader = new YmlReader();
String timingData = ymlReader.getDataYml("testData/workflow_zh_cn", "timing", param);
return timingData;
}
}

76
e2e/src/test/java/org/apache/dolphinscheduler/data/project/WorkflowDefineData.java

@ -16,74 +16,12 @@
*/
package org.apache.dolphinscheduler.data.project;
public class WorkflowDefineData {
/**
* create workflow data
*/
public static final String workflow_define = "工作流定义";
//input shell task name
public static final String SHELL_TASK_NAME = "shell_task_selenium_1";
//input shell task description
public static final String SHELL_TASK_DESCRIPTION = "shell task description test";
//input timeout
public static final String INPUT_TIMEOUT = "60";
//input shell script
public static final String SHELL_SCRIPT = "echo 1111111";
//input custom parameters
public static final String INPUT_CUSTOM_PARAMETERS = "selenium_parameter";
//input custom parameters value
public static final String INPUT_CUSTOM_PARAMETERS_VALUE = "selenium_parameter_123";
//input add custom parameters
public static final String INPUT_ADD_CUSTOM_PARAMETERS = "selenium_parameter_delete";
//input add custom parameters value
public static final String INPUT_ADD_CUSTOM_PARAMETERS_VALUE = "selenium_parameter_delete_456";
//workflow define title
public static final String WORKFLOW_TITLE = "工作流定义 - DolphinScheduler";
//create workflow title
public static final String CREATE_WORKFLOW_TITLE = "创建流程定义 - DolphinScheduler";
/**
* save workflow data
*/
//input workflow name
public static final String INPUT_WORKFLOW_NAME = "selenium_shell_1";
//input workflow description
public static final String INPUT_WORKFLOW_DESCRIPTION = "test selenium_shell_1 description";
//input workflow timeout
public static final String INPUT_WORKFLOW_TIMEOUT = "30";
//input workflow global parameters
public static final String INPUT_WORKFLOW_GLOBAL_PARAMETERS = "selenium_global_parameters_1";
//input workflow global parameters value
public static final String INPUT_WORKFLOW_GLOBAL_PARAMETERS_VALUES = "selenium_global_parameters_value_1";
//input to add workflow global parameters
public static final String INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS = "selenium_global_parameters_2";
//input to add workflow global parameters value
public static final String INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS_VALUES = "selenium_global_parameters_value_2";
/**
* workflowDefine list
*/
public static final String WORKFLOW_ONLINE_STATE = "上线";
public static final String WORKFLOW_OFFLINE_STATE = "下线";
import org.apache.dolphinscheduler.util.YmlReader;
public class WorkflowDefineData {
public String getWorkflowDefineData(String param) {
YmlReader ymlReader = new YmlReader();
String workflowDefineData = ymlReader.getDataYml("testData/workflow_zh_cn", "workflowDefine", param);
return workflowDefineData;
}
}

18
e2e/src/test/java/org/apache/dolphinscheduler/data/security/AlertManageData.java

@ -16,17 +16,13 @@
*/
package org.apache.dolphinscheduler.data.security;
public class AlertManageData {
//Alert Name
public static final String ALERT_NAME = "selenium_alert_Name";
public static final String CREATE_ALERT = "创建告警组";
// Alert Type
public static final String ALERT_TYPE = "邮件";
import org.apache.dolphinscheduler.util.YmlReader;
//Alert Description
public static final String DESCRIPTION = "create alert test";
public static final String ALERT_MANAGE = "告警组管理 - DolphinScheduler";
public class AlertManageData {
public String getAlertData(String param) {
YmlReader ymlReader = new YmlReader();
String alertData = ymlReader.getDataYml("testData/security_zh_cn", "alertManage", param);
return alertData;
}
}

31
e2e/src/test/java/org/apache/dolphinscheduler/data/security/QueueManageData.java

@ -16,29 +16,12 @@
*/
package org.apache.dolphinscheduler.data.security;
public class QueueManageData {
/**
* Create Queue Name
*/
public static final String QUEUE_NAME = "selenium_queue_name";
public static final String CREATE_QUEUE = "创建队列";
/**
* Create Queue Value
*/
public static final String QUEUE_VALUE = "selenium_queue_value";
/**
* Edit Queue Name
*/
public static final String EDIT_QUEUE_NAME = "selenium_queue_value_edit";
/**
* Edit Queue Value
*/
public static final String EDIT_QUEUE_VALUE = "selenium_queue_value_edit";
public static final String QUEUE_MANAGE = "队列管理 - DolphinScheduler";
import org.apache.dolphinscheduler.util.YmlReader;
public class QueueManageData {
public String getQueueData(String param) {
YmlReader ymlReader = new YmlReader();
String queueData = ymlReader.getDataYml("testData/security_zh_cn", "queueManage", param);
return queueData;
}
}

30
e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java

@ -20,30 +20,12 @@
*/
package org.apache.dolphinscheduler.data.security;
import org.apache.dolphinscheduler.util.YmlReader;
public class TenantManageData {
/**
* Tenant Code
*/
public static final String TENANT_CODE = "selenium_tenant_code_1";
/**
* Tenant Name
*/
public static final String TENANT_NAME = "selenium_tenant_Name";
/**
* Queue
*/
public static final String QUEUE = "default";
/**
* Description
*/
public static final String DESCRIPTION = "create tenant test";
public static final String TENANT_MANAGE = "租户管理 - DolphinScheduler";
public String getTenantData(String param) {
YmlReader ymlReader = new YmlReader();
String tenantData = ymlReader.getDataYml("testData/security_zh_cn", "tenantManage", param);
return tenantData;
}
}

15
e2e/src/test/java/org/apache/dolphinscheduler/data/security/TokenManageData.java

@ -16,13 +16,12 @@
*/
package org.apache.dolphinscheduler.data.security;
public class TokenManageData {
public static final String TOKEN_MANAGE = "令牌管理 - DolphinScheduler";
public static final String CREATE_TOKEN = "创建令牌";
public static final String DATE = "2038-06-10 00:00:00";
public static final String NAME = "admin";
import org.apache.dolphinscheduler.util.YmlReader;
public class TokenManageData {
public String getTokenData(String param) {
YmlReader ymlReader = new YmlReader();
String tokenData = ymlReader.getDataYml("testData/security_zh_cn", "tokenManage", param);
return tokenData;
}
}

29
e2e/src/test/java/org/apache/dolphinscheduler/data/security/UserManageData.java

@ -16,29 +16,12 @@
*/
package org.apache.dolphinscheduler.data.security;
import org.apache.dolphinscheduler.util.YmlReader;
public class UserManageData {
/**
* create user
*/
public static final String USERNAME = "selenium_user_1";
public static final String PASSWORD = "123456qwe";
public static final String EMAIL = "123456789@qq.com";
public static final String PHONE = "15811112222";
public static final String USER_MANAGE = "用户管理 - DolphinScheduler";
public static final String CREATE_USER_BUTTON = "创建用户";
/**
* edit user
*/
public static final String EDIT_USERNAME = "selenium_user_edit";
public static final String EDIT_PASSWORD = "123456qwe";
public static final String EDIT_EMAIL = "123456_edit@qq.com";
public static final String EDIT_PHONE = "15800001111";
public String getUserData(String param) {
YmlReader ymlReader = new YmlReader();
String userData = ymlReader.getDataYml("testData/security_zh_cn", "userManage", param);
return userData;
}
}

2
e2e/src/test/java/org/apache/dolphinscheduler/locator/project/WorkflowDefineLocator.java

@ -179,6 +179,8 @@ public class WorkflowDefineLocator {
//scroll to element bottom
public static final By SCROLL_BOTTOM = By.xpath("//span/a/em");
public static final By WORKFLOW_NAME = By.xpath("//table/tr[2]/td[3]/span/a");
/**
* online workflow
*/

6
e2e/src/test/java/org/apache/dolphinscheduler/locator/security/TokenManageLocator.java

@ -24,18 +24,18 @@ public class TokenManageLocator {
public static final By CLICK_CREATE_TOKEN = By.xpath("//div[2]/div/div[2]/div[2]/div/div[1]/button/span");
public static final By CREATE_TOKEN_POPUP = By.xpath("//div[5]/div/div[2]/div/div[1]/span");
public static final By SELECT_USER = By.xpath("//div[2]/div[2]/div/div/div/span/i");
public static final By CLICK_GENERATE_TOKEN_BUTTON = By.xpath("//div[3]/div[2]/button/span");
public static final By CLICK_SUBMIT_BUTTON = By.xpath("//div[3]/button[2]/span");
public static final By EDIT_TOKEN_BUTTON = By.xpath("//table/tr[2]/td[7]/button");
//edit token
public static final By TOKEN = By.xpath("//table/tr[2]/td[1]/span");
public static final By EDIT_TOKEN_BUTTON = By.xpath("//table/tr[2]/td[7]/button");
public static final By CLICK_EDIT_BUTTON = By.xpath("//div[3]/div[1]/div/table/tr[2]/td[7]/button/i");
//delete token
public static final By CLICK_DELETE_BUTTON = By.xpath("//div[3]/div[1]/div/table/tr[2]/td[7]/span/button");

2
e2e/src/test/java/org/apache/dolphinscheduler/locator/security/UserManageLocator.java

@ -49,6 +49,8 @@ public class UserManageLocator {
public static final By SUBMIT = By.xpath("//div[3]/button[2]/span");
public static final By USERNAME = By.xpath("//table/tr[2]/td[2]/span");
/**
* edit user
*/

8
e2e/src/test/java/org/apache/dolphinscheduler/page/LoginPage.java

@ -18,13 +18,14 @@ package org.apache.dolphinscheduler.page;
import org.apache.dolphinscheduler.common.PageCommon;
import org.apache.dolphinscheduler.data.LoginData;
import org.apache.dolphinscheduler.data.security.TenantManageData;
import org.apache.dolphinscheduler.locator.LoginLocator;
import org.openqa.selenium.Cookie;
import org.openqa.selenium.WebDriver;
public class LoginPage extends PageCommon {
TenantManageData tenantManageData = new TenantManageData();
/**
* Unique constructor
* @param driver driver
@ -33,7 +34,6 @@ public class LoginPage extends PageCommon {
super(driver);
}
/**
* jump page
*/
@ -65,6 +65,6 @@ public class LoginPage extends PageCommon {
moveToElement(LoginLocator.LOGIN_BUTTON_MOVE);
// Whether to enter the specified page after login
return ifTitleContains(LoginData.TENANT);
return ifTitleContains(tenantManageData.getTenantData("tenantTitle"));
}
}

4
e2e/src/test/java/org/apache/dolphinscheduler/page/project/ProcessInstancePage.java

@ -23,6 +23,8 @@ import org.apache.dolphinscheduler.data.project.ProcessInstanceData;
public class ProcessInstancePage extends PageCommon {
ProcessInstanceData processInstanceData = new ProcessInstanceData();
public ProcessInstancePage(WebDriver driver) {
super(driver);
}
@ -36,7 +38,7 @@ public class ProcessInstancePage extends PageCommon {
clickTopElement(ProcessInstanceLocator.CLICK_PROCESS_INSTANCE_NAME);
locateElement(ProcessInstanceLocator.PROCESS_INSTANCE_SUCCESS_STATE);
clickTopElement(ProcessInstanceLocator.CLICK_RERUN_BUTTON);
return ifTitleContains(ProcessInstanceData.PROCESS_INSTANCE_TITLE);
return ifTitleContains(processInstanceData.getProcessInstanceData("processInstanceTitle"));
}
}

15
e2e/src/test/java/org/apache/dolphinscheduler/page/project/ProjectPage.java

@ -17,12 +17,13 @@
package org.apache.dolphinscheduler.page.project;
import org.apache.dolphinscheduler.common.PageCommon;
import org.apache.dolphinscheduler.constant.TestConstant;
import org.apache.dolphinscheduler.data.project.ProjectData;
import org.apache.dolphinscheduler.locator.project.ProjectLocator;
import org.openqa.selenium.WebDriver;
public class ProjectPage extends PageCommon {
ProjectData projectData = new ProjectData();
public ProjectPage(WebDriver driver) {
super(driver);
}
@ -32,7 +33,7 @@ public class ProjectPage extends PageCommon {
*/
public boolean jumpProjectManagePage() throws InterruptedException {
clickTopElement(ProjectLocator.PROJECT_MANAGE);
return ifTitleContains(ProjectData.PROJECT_TITLE);
return ifTitleContains(projectData.getProjectData("projectTitle"));
}
/**
@ -41,18 +42,18 @@ public class ProjectPage extends PageCommon {
* @return Whether to enter the specified page after create project
*/
public boolean createProject() throws InterruptedException {
ifTextExists(ProjectLocator.CREATE_PROJECT_BUTTON,ProjectData.CREATE_PROJECT_BUTTON);
ifTextExists(ProjectLocator.CREATE_PROJECT_BUTTON, projectData.getProjectData("createProjectButton"));
clickElement(ProjectLocator.CREATE_PROJECT_BUTTON);
// input create project data
sendInput(ProjectLocator.PROJECT_NAME, ProjectData.PROJECT_NAME);
sendInput(ProjectLocator.PROJECT_DESCRIPTION, ProjectData.DESCRIPTION);
sendInput(ProjectLocator.PROJECT_NAME, projectData.getProjectData("projectName"));
sendInput(ProjectLocator.PROJECT_DESCRIPTION, projectData.getProjectData("description"));
// click submit button
clickButton(ProjectLocator.SUBMIT_BUTTON);
// Whether to enter the specified page after submit
return ifTextExists(ProjectLocator.LIST_PROJECT_NAME,ProjectData.PROJECT_NAME);
return ifTextExists(ProjectLocator.LIST_PROJECT_NAME, projectData.getProjectData("projectName"));
}
/**
@ -68,6 +69,6 @@ public class ProjectPage extends PageCommon {
clickElement(ProjectLocator.CONFIRM_DELETE_PROJECT_BUTTON);
// Whether to enter the specified page after submit
return ifTitleContains(ProjectData.PROJECT_TITLE);
return ifTitleContains(projectData.getProjectData("projectTitle"));
}
}

11
e2e/src/test/java/org/apache/dolphinscheduler/page/project/RunWorkflowPage.java

@ -24,13 +24,16 @@ import org.apache.dolphinscheduler.locator.project.WorkflowDefineLocator;
import org.openqa.selenium.WebDriver;
public class RunWorkflowPage extends PageCommon {
RunWorkflowData runWorkflowData = new RunWorkflowData();
WorkflowDefineData workflowDefineData = new WorkflowDefineData();
public RunWorkflowPage(WebDriver driver) {
super(driver);
}
public boolean runWorkflow() throws InterruptedException {
// Determine whether the workflow status is online
ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, WorkflowDefineData.WORKFLOW_ONLINE_STATE);
ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, runWorkflowData.getRunWorkflowData("online"));
// click run workflow button
System.out.println("Click run workflow button");
@ -46,10 +49,10 @@ public class RunWorkflowPage extends PageCommon {
clickElement(RunWorkflowLocator.SELECT_WORKER_GROUP);
clickElement(RunWorkflowLocator.CLICK_NOTICE_GROUP);
clickElement(RunWorkflowLocator.SELECT_NOTICE_GROUP);
sendInput(RunWorkflowLocator.INPUT_RECIPIENT, RunWorkflowData.RECIPIENT);
sendInput(RunWorkflowLocator.INPUT_Cc,RunWorkflowData.Cc);
sendInput(RunWorkflowLocator.INPUT_RECIPIENT, runWorkflowData.getRunWorkflowData("recipient"));
sendInput(RunWorkflowLocator.INPUT_Cc, runWorkflowData.getRunWorkflowData("Cc"));
clickButton(RunWorkflowLocator.CLICK_RUNNING_BUTTON);
return ifTitleContains(RunWorkflowData.RUN_WORKFLOW_TITLE);
return ifTitleContains(workflowDefineData.getWorkflowDefineData("workflowDefineTitle"));
}
}

31
e2e/src/test/java/org/apache/dolphinscheduler/page/project/TimingPage.java

@ -18,23 +18,24 @@ package org.apache.dolphinscheduler.page.project;
import org.apache.dolphinscheduler.common.PageCommon;
import org.apache.dolphinscheduler.data.project.TimingData;
import org.apache.dolphinscheduler.data.project.WorkflowDefineData;
import org.apache.dolphinscheduler.locator.project.TimingLocator;
import org.apache.dolphinscheduler.locator.project.WorkflowDefineLocator;
import org.openqa.selenium.WebDriver;
public class TimingPage extends PageCommon {
TimingData timingData = new TimingData();
public TimingPage(WebDriver driver) {
super(driver);
}
/**
* create timing
*/
public boolean createTiming() throws InterruptedException {
flushPage();
// Determine whether the workflow status is online
ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, WorkflowDefineData.WORKFLOW_ONLINE_STATE);
ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, timingData.getTimingData("online"));
// click timing button
System.out.println("Click timing button");
@ -52,11 +53,11 @@ public class TimingPage extends PageCommon {
clickElement(TimingLocator.SELECT_WORKER_GROUP);
clickElement(TimingLocator.CLICK_NOTICE_GROUP);
clickElement(TimingLocator.SELECT_NOTICE_GROUP);
sendInput(TimingLocator.INPUT_RECIPIENT, TimingData.RECIPIENT);
sendInput(TimingLocator.INPUT_Cc,TimingData.Cc);
sendInput(TimingLocator.INPUT_RECIPIENT, timingData.getTimingData("recipient"));
sendInput(TimingLocator.INPUT_Cc, timingData.getTimingData("Cc"));
clickButton(TimingLocator.CLICK_CREATE_BUTTON);
return ifTextExists(TimingLocator.TIMING_STATE, TimingData.TIMING_OFFLINE_STATE);
return ifTextExists(TimingLocator.TIMING_STATE, timingData.getTimingData("offline"));
}
/**
@ -68,7 +69,7 @@ public class TimingPage extends PageCommon {
clickButton(TimingLocator.CLICK_TIMING_MANAGEMENT_BUTTON);
// Determine whether the workflow name exists
ifTextExists(TimingLocator.WORKFLOW_NAME, WorkflowDefineData.INPUT_WORKFLOW_NAME);
ifTextExists(TimingLocator.WORKFLOW_NAME, timingData.getTimingData("offline"));
System.out.println("Click edit timing button");
clickButton(TimingLocator.CLICK_EDIT_TIMING_BUTTON);
@ -85,11 +86,11 @@ public class TimingPage extends PageCommon {
clickElement(TimingLocator.SELECT_WORKER_GROUP);
clickElement(TimingLocator.CLICK_NOTICE_GROUP);
clickElement(TimingLocator.SELECT_NOTICE_GROUP);
sendInput(TimingLocator.INPUT_RECIPIENT, TimingData.EDIT_RECIPIENT);
sendInput(TimingLocator.INPUT_Cc,TimingData.EDIT_Cc);
sendInput(TimingLocator.INPUT_RECIPIENT, timingData.getTimingData("editRecipient"));
sendInput(TimingLocator.INPUT_Cc, timingData.getTimingData("editCc"));
clickButton(TimingLocator.CLICK_CREATE_BUTTON);
return ifTitleContains(TimingData.TIMING_TITLE );
return ifTitleContains(timingData.getTimingData("timingTitle"));
}
@ -99,13 +100,13 @@ public class TimingPage extends PageCommon {
public boolean onlineTiming() throws InterruptedException {
flushPage();
// Determine whether the timing is offline
ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, TimingData.TIMING_OFFLINE_STATE);
ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, timingData.getTimingData("offline"));
// click online timing button
System.out.println("Click online timing button");
clickElement(TimingLocator.CLICK_ONLINE_TIMING_BUTTON);
return ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, TimingData.TIMING_ONLINE_STATE);
return ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, timingData.getTimingData("online"));
}
@ -115,13 +116,13 @@ public class TimingPage extends PageCommon {
public boolean offlineTiming() throws InterruptedException {
flushPage();
// Determine whether the timing is online
ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, TimingData.TIMING_ONLINE_STATE);
ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, timingData.getTimingData("online"));
// click offline timing button
System.out.println("Click offline timing button");
clickElement(TimingLocator.CLICK_OFFLINE_TIMING_BUTTON);
return ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, TimingData.TIMING_OFFLINE_STATE);
return ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, timingData.getTimingData("offline"));
}
@ -131,7 +132,7 @@ public class TimingPage extends PageCommon {
*/
public boolean deleteTiming() throws InterruptedException {
// Determine whether the timing is offline
ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, TimingData.TIMING_OFFLINE_STATE);
ifTextExists(TimingLocator.TIMING_MANAGEMENT_TIMING_STATE, timingData.getTimingData("offline"));
// click offline timing button
System.out.println("Click delete timing button");

57
e2e/src/test/java/org/apache/dolphinscheduler/page/project/WorkflowDefinePage.java

@ -17,7 +17,6 @@
package org.apache.dolphinscheduler.page.project;
import org.apache.dolphinscheduler.common.PageCommon;
import org.apache.dolphinscheduler.constant.TestConstant;
import org.apache.dolphinscheduler.data.project.ProjectData;
import org.apache.dolphinscheduler.data.project.WorkflowDefineData;
import org.apache.dolphinscheduler.locator.project.ProjectLocator;
@ -25,6 +24,9 @@ import org.apache.dolphinscheduler.locator.project.WorkflowDefineLocator;
import org.openqa.selenium.WebDriver;
public class WorkflowDefinePage extends PageCommon {
WorkflowDefineData workflowDefineData = new WorkflowDefineData();
ProjectData projectData = new ProjectData();
public WorkflowDefinePage(WebDriver driver) {
super(driver);
}
@ -34,18 +36,18 @@ public class WorkflowDefinePage extends PageCommon {
*/
public boolean jumpWorkflowPage() throws InterruptedException {
ifTextExists(ProjectLocator.LIST_PROJECT_NAME, ProjectData.PROJECT_NAME);
ifTextExists(ProjectLocator.LIST_PROJECT_NAME, projectData.getProjectData("projectName"));
// click project name
clickElement(WorkflowDefineLocator.CLICK_PROJECT_NAME);
ifTextExists(WorkflowDefineLocator.CLICK_WORKFLOW_DEFINE,WorkflowDefineData.workflow_define);
ifTextExists(WorkflowDefineLocator.CLICK_WORKFLOW_DEFINE,workflowDefineData.getWorkflowDefineData("workflowDefine"));
System.out.println("Click on workflow define to jump to workflow define page");
// click workflow define
clickElement(WorkflowDefineLocator.CLICK_WORKFLOW_DEFINE);
return ifTitleContains(WorkflowDefineData.WORKFLOW_TITLE);
return ifTitleContains(workflowDefineData.getWorkflowDefineData("workflowDefineTitle"));
}
public boolean createWorkflow() throws InterruptedException {
@ -58,8 +60,7 @@ public class WorkflowDefinePage extends PageCommon {
dragAndDrop(WorkflowDefineLocator.MOUSE_DOWN_AT_SHELL, WorkflowDefineLocator.MOUSE_MOVE_SHELL_AT_DAG);
//input shell task _name
sendInput(WorkflowDefineLocator.INPUT_SHELL_TASK_NAME , WorkflowDefineData.SHELL_TASK_NAME);
sendInput(WorkflowDefineLocator.INPUT_SHELL_TASK_NAME, workflowDefineData.getWorkflowDefineData("shellTaskName"));
//click stop run type
clickElement(WorkflowDefineLocator.CLICK_STOP_RUN_TYPE);
@ -67,7 +68,7 @@ public class WorkflowDefinePage extends PageCommon {
clickElement(WorkflowDefineLocator.CLICK_NORMAL_RUN_TYPE);
//input shell task description
sendInput(WorkflowDefineLocator.INPUT_SHELL_TASK_DESCRIPTION , WorkflowDefineData.SHELL_TASK_DESCRIPTION);
sendInput(WorkflowDefineLocator.INPUT_SHELL_TASK_DESCRIPTION, workflowDefineData.getWorkflowDefineData("shellTaskDescription"));
//select task priority
clickElement(WorkflowDefineLocator.CLICK_TASK_PRIORITY);
@ -100,20 +101,20 @@ public class WorkflowDefinePage extends PageCommon {
clearInput(WorkflowDefineLocator.SELECT_TIMEOUT);
//input timeout
sendInput(WorkflowDefineLocator.SELECT_TIMEOUT, WorkflowDefineData.INPUT_TIMEOUT);
sendInput(WorkflowDefineLocator.SELECT_TIMEOUT, workflowDefineData.getWorkflowDefineData("taskTimeout"));
//click codeMirror and input script
inputCodeMirror(WorkflowDefineLocator.CLICK_CODE_MIRROR, WorkflowDefineLocator.INPUT_SCRIPT, WorkflowDefineData.SHELL_SCRIPT);
inputCodeMirror(WorkflowDefineLocator.CLICK_CODE_MIRROR, WorkflowDefineLocator.INPUT_SCRIPT, workflowDefineData.getWorkflowDefineData("shellScript"));
scrollToElementBottom(WorkflowDefineLocator.SCROLL_BOTTOM);
//click custom parameters
clickElement(WorkflowDefineLocator.CLICK_CUSTOM_PARAMETERS);
//input custom parameters
sendInput(WorkflowDefineLocator.INPUT_CUSTOM_PARAMETERS, WorkflowDefineData.INPUT_CUSTOM_PARAMETERS);
sendInput(WorkflowDefineLocator.INPUT_CUSTOM_PARAMETERS, workflowDefineData.getWorkflowDefineData("customParameter1"));
//input custom parameters value
sendInput(WorkflowDefineLocator.INPUT_CUSTOM_PARAMETERS_VALUE, WorkflowDefineData.INPUT_CUSTOM_PARAMETERS_VALUE);
sendInput(WorkflowDefineLocator.INPUT_CUSTOM_PARAMETERS_VALUE, workflowDefineData.getWorkflowDefineData("customParameterValue1"));
//click add custom parameters
clickElement(WorkflowDefineLocator.CLICK_ADD_CUSTOM_PARAMETERS);
@ -121,10 +122,10 @@ public class WorkflowDefinePage extends PageCommon {
scrollToElementBottom(WorkflowDefineLocator.SCROLL_BOTTOM);
//input add custom parameters
sendInput(WorkflowDefineLocator.INPUT_ADD_CUSTOM_PARAMETERS, WorkflowDefineData.INPUT_ADD_CUSTOM_PARAMETERS);
sendInput(WorkflowDefineLocator.INPUT_ADD_CUSTOM_PARAMETERS, workflowDefineData.getWorkflowDefineData("customParameter2"));
//input add custom parameters value
sendInput(WorkflowDefineLocator.INPUT_ADD_CUSTOM_PARAMETERS_VALUE, WorkflowDefineData.INPUT_ADD_CUSTOM_PARAMETERS_VALUE);
sendInput(WorkflowDefineLocator.INPUT_ADD_CUSTOM_PARAMETERS_VALUE, workflowDefineData.getWorkflowDefineData("customParameterValue2"));
//click delete custom parameters
clickElement(WorkflowDefineLocator.CLICK_DELETE_CUSTOM_PARAMETERS);
@ -140,7 +141,7 @@ public class WorkflowDefinePage extends PageCommon {
clickButton(WorkflowDefineLocator.COPY_TASK);
clickButton(WorkflowDefineLocator.CLICK_LINE);
mouseMovePosition(WorkflowDefineLocator.LINE_SOURCES_TASK,WorkflowDefineLocator.LINE_TARGET_TASK);
return ifTitleContains(WorkflowDefineData.CREATE_WORKFLOW_TITLE);
return ifTitleContains(workflowDefineData.getWorkflowDefineData("createWorkflowTitle"));
}
/**
@ -153,10 +154,10 @@ public class WorkflowDefinePage extends PageCommon {
clickElement(WorkflowDefineLocator.CLICK_SAVE_WORKFLOW_BUTTON);
//input workflow name
sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_NAME, WorkflowDefineData.INPUT_WORKFLOW_NAME);
sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_NAME, workflowDefineData.getWorkflowDefineData("workflowName"));
//input workflow description
sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_DESCRIPTION, WorkflowDefineData.INPUT_WORKFLOW_DESCRIPTION);
sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_DESCRIPTION, workflowDefineData.getWorkflowDefineData("workflowDescription"));
//select tenant
clickElement(WorkflowDefineLocator.CLICK_TENANT);
@ -167,25 +168,25 @@ public class WorkflowDefinePage extends PageCommon {
clearInput(WorkflowDefineLocator.INPUT_WORKFLOW_TIMEOUT);
//input workflow timeout
sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_TIMEOUT, WorkflowDefineData.INPUT_WORKFLOW_TIMEOUT);
sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_TIMEOUT, workflowDefineData.getWorkflowDefineData("workflowTimeout"));
//click workflow global parameters
clickElement(WorkflowDefineLocator.CLICK_WORKFLOW_GLOBAL_PARAMETERS);
//input workflow global parameters
sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_GLOBAL_PARAMETERS, WorkflowDefineData.INPUT_WORKFLOW_GLOBAL_PARAMETERS);
sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_GLOBAL_PARAMETERS, workflowDefineData.getWorkflowDefineData("globalParameter1"));
//input workflow global parameters value
sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_GLOBAL_PARAMETERS_VALUES, WorkflowDefineData.INPUT_WORKFLOW_GLOBAL_PARAMETERS_VALUES);
sendInput(WorkflowDefineLocator.INPUT_WORKFLOW_GLOBAL_PARAMETERS_VALUES, workflowDefineData.getWorkflowDefineData("globalParameterValue1"));
//click to add workflow global parameters
clickElement(WorkflowDefineLocator.CLICK_ADD_WORKFLOW_GLOBAL_PARAMETERS);
//input to add workflow global parameters
sendInput(WorkflowDefineLocator.INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS, WorkflowDefineData.INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS);
sendInput(WorkflowDefineLocator.INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS, workflowDefineData.getWorkflowDefineData("globalParameter2"));
//input to add workflow global parameters value
sendInput(WorkflowDefineLocator.INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS_VALUES, WorkflowDefineData.INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS_VALUES);
sendInput(WorkflowDefineLocator.INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS_VALUES, workflowDefineData.getWorkflowDefineData("globalParameterValue2"));
//delete workflow global parameters value
clickElement(WorkflowDefineLocator.CLICK_DELETE_WORKFLOW_GLOBAL_PARAMETERS);
@ -194,33 +195,33 @@ public class WorkflowDefinePage extends PageCommon {
System.out.println("submit workflow");
clickButton(WorkflowDefineLocator.CLICK_ADD_BUTTON);
return ifTitleContains(WorkflowDefineData.CREATE_WORKFLOW_TITLE);
return ifTitleContains(workflowDefineData.getWorkflowDefineData("createWorkflowTitle"));
}
public boolean onlineWorkflow() throws InterruptedException {
clickElement(WorkflowDefineLocator.CLICK_WORKFLOW_DEFINE);
// Determine whether the workflow status is offline
ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE,WorkflowDefineData.WORKFLOW_OFFLINE_STATE);
ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, workflowDefineData.getWorkflowDefineData("offline"));
// click online button
System.out.println("Click online workflow button");
clickButton(WorkflowDefineLocator.CLICK_ONLINE_WORKFLOW_BUTTON);
return ifTitleContains(WorkflowDefineData.WORKFLOW_TITLE);
return ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, workflowDefineData.getWorkflowDefineData("online"));
}
public boolean offlineWorkflow() throws InterruptedException {
clickElement(WorkflowDefineLocator.CLICK_WORKFLOW_DEFINE);
// Determine whether the workflow status is online
ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE,WorkflowDefineData.WORKFLOW_ONLINE_STATE);
ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, workflowDefineData.getWorkflowDefineData("online"));
// click offline button
System.out.println("offline workflow");
clickButton(WorkflowDefineLocator.CLICK_OFFLINE_WORKFLOW_BUTTON);
return ifTitleContains(WorkflowDefineData.WORKFLOW_TITLE);
return ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, workflowDefineData.getWorkflowDefineData("offline"));
}
@ -229,7 +230,7 @@ public class WorkflowDefinePage extends PageCommon {
clickElement(WorkflowDefineLocator.CLICK_WORKFLOW_DEFINE);
// Determine whether the workflow status is offline
ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE,WorkflowDefineData.WORKFLOW_OFFLINE_STATE);
ifTextExists(WorkflowDefineLocator.WORKFLOW_STATE, workflowDefineData.getWorkflowDefineData("offline"));
clickButton(WorkflowDefineLocator.DELETE_WORKFLOW_BOTTOM);
@ -237,6 +238,6 @@ public class WorkflowDefinePage extends PageCommon {
clickButton(WorkflowDefineLocator.CONFIRM_DELETE_WORKFLOW_BOTTOM);
// Whether to enter the specified page after submit
return ifTitleContains(WorkflowDefineData.WORKFLOW_TITLE);
return ifTitleContains(workflowDefineData.getWorkflowDefineData("workflowDefineTitle"));
}
}

14
e2e/src/test/java/org/apache/dolphinscheduler/page/security/AlertManagePage.java

@ -22,6 +22,8 @@ import org.apache.dolphinscheduler.locator.security.AlertManageLocator;
import org.openqa.selenium.WebDriver;
public class AlertManagePage extends PageCommon {
AlertManageData alertManageData = new AlertManageData();
/**
* Unique constructor
* @param driver driver
@ -41,26 +43,26 @@ public class AlertManagePage extends PageCommon {
clickElement(AlertManageLocator.CLICK_ALERT_MANAGE);
//determine whether the create alert button exists
ifTextExists(AlertManageLocator.CLICK_CREATE_ALERT,AlertManageData.CREATE_ALERT);
ifTextExists(AlertManageLocator.CLICK_CREATE_ALERT,alertManageData.getAlertData("createAlert"));
// click create alert button
System.out.println("start click create alert button");
clickElement(AlertManageLocator.CLICK_CREATE_ALERT);
// input alert data
System.out.println("start input alert ");
sendInput(AlertManageLocator.INPUT_ALERT_NAME, AlertManageData.ALERT_NAME);
sendInput(AlertManageLocator.INPUT_ALERT_NAME, alertManageData.getAlertData("alertName"));
clickElement(AlertManageLocator.CLICK_ALERT_TYPE);
clickElement(AlertManageLocator.SELECT_ALERT_EMAIL);
sendInput(AlertManageLocator.INPUT_ALERT_DESCRIPTION, AlertManageData.DESCRIPTION);
sendInput(AlertManageLocator.INPUT_ALERT_DESCRIPTION, alertManageData.getAlertData("description"));
// click button
clickButton(AlertManageLocator.SUBMIT_ALERT);
// Whether to enter the specified page after submit
return ifTextExists(AlertManageLocator.ALERT_NAME, AlertManageData.ALERT_NAME);
return ifTextExists(AlertManageLocator.ALERT_NAME, alertManageData.getAlertData("alertName"));
}
public boolean deleteAlert() throws InterruptedException {
@ -68,7 +70,7 @@ public class AlertManagePage extends PageCommon {
// click alert manage
clickElement(AlertManageLocator.CLICK_ALERT_MANAGE);
ifTextExists(AlertManageLocator.ALERT_NAME, AlertManageData.ALERT_NAME);
ifTextExists(AlertManageLocator.ALERT_NAME, alertManageData.getAlertData("alertName"));
// click delete alert button
clickButton(AlertManageLocator.DELETE_ALERT_BUTTON);
@ -77,6 +79,6 @@ public class AlertManagePage extends PageCommon {
clickButton(AlertManageLocator.CONFIRM_DELETE_ALERT_BUTTON);
// Whether to enter the specified page after submit
return ifTitleContains(AlertManageData.ALERT_MANAGE);
return ifTitleContains(alertManageData.getAlertData("alertTitle"));
}
}

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

Loading…
Cancel
Save