Browse Source

Merge pull request #27 from apache/dev

update
pull/2/head
samz406 5 years ago committed by GitHub
parent
commit
ec13974eed
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 4
      .github/workflows/ci_ut.yml
  2. 2
      README.md
  3. 2
      dockerfile/conf/dolphinscheduler/conf/common/common.properties
  4. 2
      dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/AlertServer.java
  5. 2
      dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/manager/EnterpriseWeChatManager.java
  6. 27
      dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/runner/AlertSender.java
  7. 3
      dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/template/AlertTemplate.java
  8. 2
      dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/Constants.java
  9. 24
      dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/EnterpriseWeChatUtils.java
  10. 3
      dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/ExcelUtils.java
  11. 15
      dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/MailUtils.java
  12. 3
      dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/PropertyUtils.java
  13. 49
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/CombinedApplicationServer.java
  14. 2
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProjectController.java
  15. 247
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/dto/ProcessMeta.java
  16. 603
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionService.java
  17. 10
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ResourcesService.java
  18. 2
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/UdfFuncService.java
  19. 30
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/FileUtils.java
  20. 58
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/ZooKeeperState.java
  21. 83
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/exportprocess/DataSourceParam.java
  22. 112
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/exportprocess/DependentParam.java
  23. 39
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/exportprocess/ProcessAddTaskParam.java
  24. 38
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/exportprocess/TaskNodeParamFactory.java
  25. 211
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java
  26. 3
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ResourcesServiceTest.java
  27. 31
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/FileUtilsTest.java
  28. 86
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/exportprocess/DataSourceParamTest.java
  29. 112
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/exportprocess/DependentParamTest.java
  30. 2
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java
  31. 7
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/ITaskQueue.java
  32. 15
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/TaskQueueZkImpl.java
  33. 2
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/thread/Stopper.java
  34. 84
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/AbstractZKClient.java
  35. 1
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperCachedOperator.java
  36. 11
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperOperator.java
  37. 2
      dolphinscheduler-common/src/main/resources/common.properties
  38. 6
      dolphinscheduler-common/src/main/resources/quartz.properties
  39. 11
      dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/queue/TaskQueueZKImplTest.java
  40. 2
      dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/threadutils/ThreadUtilsTest.java
  41. 6
      dolphinscheduler-dao/pom.xml
  42. 5
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/ProcessDao.java
  43. 82
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/AccessToken.java
  44. 79
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Alert.java
  45. 25
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/AlertGroup.java
  46. 84
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java
  47. 2
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProjectUser.java
  48. 22
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Queue.java
  49. 2
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Resource.java
  50. 2
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ResourcesUser.java
  51. 24
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Session.java
  52. 18
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Tenant.java
  53. 2
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/UDFUser.java
  54. 2
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/UdfFunc.java
  55. 39
      dolphinscheduler-dao/src/main/resources/application.properties
  56. 39
      dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/AccessTokenMapperTest.java
  57. 6
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java
  58. 14
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterConfig.java
  59. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/log/MasterLogFilter.java
  60. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java
  61. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/monitor/RunConfig.java
  62. 11
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java
  63. 10
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/config/WorkerConfig.java
  64. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/log/SensitiveDataConverter.java
  65. 47
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/FetchTaskThread.java
  66. 6
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java
  67. 9
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java
  68. 3
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKWorkerClient.java
  69. 4
      dolphinscheduler-server/src/main/resources/config/install_config.conf
  70. 22
      dolphinscheduler-server/src/main/resources/config/run_config.conf
  71. 4
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/EnvFileTest.java
  72. 18
      dolphinscheduler-ui/.editorconfig
  73. 15
      dolphinscheduler-ui/.env
  74. 8
      dolphinscheduler-ui/_test_/.babelrc
  75. 55
      dolphinscheduler-ui/_test_/Counter.spec.js
  76. 43
      dolphinscheduler-ui/_test_/package.json
  77. 34
      dolphinscheduler-ui/_test_/test.spec.js
  78. 100
      dolphinscheduler-ui/build/webpack.config.test.js
  79. 53
      dolphinscheduler-ui/src/components/Counter.vue
  80. 28
      dolphinscheduler-ui/src/components/Message.vue
  81. 3
      dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss
  82. 7
      dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue
  83. 17
      dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/datasource.vue
  84. 50
      dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/dependItemList.vue
  85. 7
      dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/resources.vue
  86. 3
      dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/sqlType.vue
  87. 11
      dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/udfs.vue
  88. 35
      dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/flink.vue
  89. 36
      dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/spark.vue
  90. 35
      dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sql.vue
  91. 4
      dolphinscheduler-ui/src/js/conf/home/pages/datasource/pages/list/_source/createDataSource.vue
  92. 2
      dolphinscheduler-ui/src/js/conf/home/pages/datasource/pages/list/_source/list.vue
  93. 23
      dolphinscheduler-ui/src/js/conf/home/pages/datasource/pages/list/index.vue
  94. 4
      dolphinscheduler-ui/src/js/conf/home/pages/monitor/pages/servers/servers.scss
  95. 4
      dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/_source/instanceConditions/index.vue
  96. 4
      dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/_source/taskRecordList/index.vue
  97. 16
      dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/index.vue
  98. 16
      dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/index.vue
  99. 1
      dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/list/_source/createProject.vue
  100. 5
      dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/list/_source/list.vue
  101. Some files were not shown because too many files have changed in this diff Show More

4
.github/workflows/ci_ut.yml

@ -15,7 +15,7 @@
# limitations under the License.
#
on: [push, "pull_request"]
on: ["push", "pull_request"]
env:
DOCKER_DIR: ./docker
LOG_DIR: /tmp/dolphinscheduler
@ -47,7 +47,7 @@ jobs:
- name: Compile
run: |
export MAVEN_OPTS='-Dmaven.repo.local=.m2/repository -XX:+TieredCompilation -XX:TieredStopAtLevel=1 -XX:+CMSClassUnloadingEnabled -XX:+UseConcMarkSweepGC -XX:-UseGCOverheadLimit -Xmx3g'
mvn test -Dmaven.test.skip=false cobertura:cobertura
mvn test -B -Dmaven.test.skip=false cobertura:cobertura
CODECOV_TOKEN="09c2663f-b091-4258-8a47-c981827eb29a" bash <(curl -s https://codecov.io/bash)
- name: Run SonarCloud Analysis
run: >

2
README.md

@ -99,7 +99,7 @@ It is because of the shoulders of these open source projects that the birth of t
### Get Help
1. Submit an issue
1. Subscribe the mail list : https://dolphinscheduler.apache.org/en-us/docs/user_doc/subscribe.html. then send mail to dev@dolphinscheduler.apache.org
1. Subscribe the mail list : https://dolphinscheduler.apache.org/en-us/docs/1.2.0/user_doc/subscribe.html. then send mail to dev@dolphinscheduler.apache.org
1. Contact WeChat group manager, ID 510570367. This is for Mandarin(CN) discussion.
### License

2
dockerfile/conf/dolphinscheduler/conf/common/common.properties

@ -49,7 +49,7 @@ login.user.keytab.username=hdfs-mycluster@ESZ.COM
login.user.keytab.path=/opt/hdfs.headless.keytab
# system env path. self configuration, please make sure the directory and file exists and have read write execute permissions
dolphinscheduler.env.path=/opt/dolphinscheduler/conf/env/.dolphinscheduler_env.sh
dolphinscheduler.env.path=/opt/dolphinscheduler/conf/env/dolphinscheduler_env.sh
#resource.view.suffixs
resource.view.suffixs=txt,log,sh,conf,cfg,py,java,sql,hql,xml

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

@ -56,7 +56,7 @@ public class AlertServer {
logger.info("alert server ready start ");
while (Stopper.isRunning()){
try {
Thread.sleep(Constants.ALERT_SCAN_INTERVEL);
Thread.sleep(Constants.ALERT_SCAN_INTERVAL);
} catch (InterruptedException e) {
logger.error(e.getMessage(),e);
Thread.currentThread().interrupt();

2
dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/manager/EnterpriseWeChatManager.java

@ -32,7 +32,7 @@ import java.util.Map;
* Enterprise WeChat Manager
*/
public class EnterpriseWeChatManager {
private static final Logger logger = LoggerFactory.getLogger(MsgManager.class);
private static final Logger logger = LoggerFactory.getLogger(EnterpriseWeChatManager.class);
/**
* Enterprise We Chat send
* @param alert the alert

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

@ -31,6 +31,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
@ -64,7 +65,7 @@ public class AlertSender{
users = alertDao.listUserByAlertgroupId(alert.getAlertGroupId());
// receiving group list
List<String> receviersList = new ArrayList<String>();
List<String> receviersList = new ArrayList<>();
for(User user:users){
receviersList.add(user.getEmail());
}
@ -72,13 +73,11 @@ public class AlertSender{
String receivers = alert.getReceivers();
if (StringUtils.isNotEmpty(receivers)){
String[] splits = receivers.split(",");
for (String receiver : splits){
receviersList.add(receiver);
}
receviersList.addAll(Arrays.asList(splits));
}
// copy list
List<String> receviersCcList = new ArrayList<String>();
List<String> receviersCcList = new ArrayList<>();
// Custom Copier
@ -86,9 +85,7 @@ public class AlertSender{
if (StringUtils.isNotEmpty(receiversCc)){
String[] splits = receiversCc.split(",");
for (String receiverCc : splits){
receviersCcList.add(receiverCc);
}
receviersCcList.addAll(Arrays.asList(splits));
}
if (CollectionUtils.isEmpty(receviersList) && CollectionUtils.isEmpty(receviersCcList)) {
@ -106,7 +103,13 @@ public class AlertSender{
alert.setInfo(retMaps);
}
boolean flag = Boolean.parseBoolean(String.valueOf(retMaps.get(Constants.STATUS)));
//send flag
boolean flag = false;
if (null != retMaps) {
flag = Boolean.parseBoolean(String.valueOf(retMaps.get(Constants.STATUS)));
}
if (flag) {
alertDao.updateAlert(AlertStatus.EXECUTION_SUCCESS, "execution success", alert.getId());
logger.info("alert send success");
@ -121,8 +124,10 @@ public class AlertSender{
}
} else {
alertDao.updateAlert(AlertStatus.EXECUTION_FAILURE, String.valueOf(retMaps.get(Constants.MESSAGE)), alert.getId());
logger.info("alert send error : {}", String.valueOf(retMaps.get(Constants.MESSAGE)));
if (null != retMaps) {
alertDao.updateAlert(AlertStatus.EXECUTION_FAILURE, String.valueOf(retMaps.get(Constants.MESSAGE)), alert.getId());
logger.info("alert send error : {}", retMaps.get(Constants.MESSAGE));
}
}
}

3
dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/template/AlertTemplate.java

@ -34,6 +34,9 @@ public interface AlertTemplate {
/**
* default showAll is true
* @param content alert message content
* @param showType show type
* @return a message from a specified alert template
*/
default String getMessageFromTemplate(String content,ShowType showType){
return getMessageFromTemplate(content,showType,true);

2
dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/Constants.java

@ -133,7 +133,7 @@ public class Constants {
public static final String TH_END = "</th>";
public static final int ALERT_SCAN_INTERVEL = 5000;
public static final int ALERT_SCAN_INTERVAL = 5000;
public static final String MARKDOWN_QUOTE = ">";

24
dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/EnterpriseWeChatUtils.java

@ -201,22 +201,22 @@ public class EnterpriseWeChatUtils {
public static String markdownTable(String title,String content){
List<LinkedHashMap> mapItemsList = JSONUtils.toList(content, LinkedHashMap.class);
StringBuilder contents = new StringBuilder(200);
for (LinkedHashMap mapItems : mapItemsList){
Set<Map.Entry<String, String>> entries = mapItems.entrySet();
if (null != mapItemsList) {
for (LinkedHashMap mapItems : mapItemsList){
Set<Map.Entry<String, String>> entries = mapItems.entrySet();
Iterator<Map.Entry<String, String>> iterator = entries.iterator();
StringBuilder t = new StringBuilder(String.format("`%s`%s",title,Constants.MARKDOWN_ENTER));
Iterator<Map.Entry<String, String>> iterator = entries.iterator();
while (iterator.hasNext()){
StringBuilder t = new StringBuilder(String.format("`%s`%s",title,Constants.MARKDOWN_ENTER));
while (iterator.hasNext()){
Map.Entry<String, String> entry = iterator.next();
t.append(Constants.MARKDOWN_QUOTE);
t.append(entry.getKey()).append(":").append(entry.getValue());
t.append(Constants.MARKDOWN_ENTER);
Map.Entry<String, String> entry = iterator.next();
t.append(Constants.MARKDOWN_QUOTE);
t.append(entry.getKey()).append(":").append(entry.getValue());
t.append(Constants.MARKDOWN_ENTER);
}
contents.append(t);
}
contents.append(t);
}
return contents.toString();
}

3
dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/ExcelUtils.java

@ -16,6 +16,7 @@
*/
package org.apache.dolphinscheduler.alert.utils;
import org.apache.dolphinscheduler.common.utils.CollectionUtils;
import org.apache.poi.hssf.usermodel.HSSFCell;
import org.apache.poi.hssf.usermodel.HSSFRow;
import org.apache.poi.hssf.usermodel.HSSFSheet;
@ -47,7 +48,7 @@ public class ExcelUtils {
//The JSONUtils.toList has been try catch ex
itemsList = JSONUtils.toList(content, LinkedHashMap.class);
if (itemsList == null || itemsList.size() == 0){
if (CollectionUtils.isEmpty(itemsList)){
logger.error("itemsList is null");
throw new RuntimeException("itemsList is null");
}

15
dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/MailUtils.java

@ -25,7 +25,6 @@ import org.apache.dolphinscheduler.common.utils.CollectionUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.util.ResourceUtils;
import javax.mail.*;
import javax.mail.internet.*;
@ -33,7 +32,6 @@ import java.io.*;
import java.util.*;
/**
* mail utils
*/
@ -98,7 +96,7 @@ public class MailUtils {
return retMap;
}
receivers.removeIf((from) -> (StringUtils.isEmpty(from)));
receivers.removeIf(StringUtils::isEmpty);
if (showType == ShowType.TABLE || showType == ShowType.TEXT){
// send email
@ -187,7 +185,7 @@ public class MailUtils {
/**
* get MimeMessage
* @param receivers
* @param receivers receivers
* @return the MimeMessage
* @throws MessagingException
*/
@ -231,8 +229,7 @@ public class MailUtils {
}
};
Session session = Session.getInstance(props, auth);
return session;
return Session.getInstance(props, auth);
}
/**
@ -320,12 +317,12 @@ public class MailUtils {
public static void deleteFile(File file){
if(file.exists()){
if(file.delete()){
logger.info("delete success:"+file.getAbsolutePath()+file.getName());
logger.info("delete success: {}",file.getAbsolutePath() + file.getName());
}else{
logger.info("delete fail"+file.getAbsolutePath()+file.getName());
logger.info("delete fail: {}", file.getAbsolutePath() + file.getName());
}
}else{
logger.info("file not exists:"+file.getAbsolutePath()+file.getName());
logger.info("file not exists: {}", file.getAbsolutePath() + file.getName());
}
}

3
dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/PropertyUtils.java

@ -205,8 +205,7 @@ public class PropertyUtils {
return null;
}
try {
String[] propertyArray = value.split(splitStr);
return propertyArray;
return value.split(splitStr);
} catch (PatternSyntaxException e) {
logger.info(e.getMessage(),e);
}

49
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/CombinedApplicationServer.java

@ -1,49 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.api;
import org.apache.dolphinscheduler.alert.AlertServer;
import org.apache.dolphinscheduler.server.master.MasterServer;
import org.apache.dolphinscheduler.server.rpc.LoggerServer;
import org.apache.dolphinscheduler.server.worker.WorkerServer;
import org.springframework.boot.autoconfigure.SpringBootApplication;
import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty;
import org.springframework.boot.web.servlet.ServletComponentScan;
import org.springframework.boot.web.servlet.support.SpringBootServletInitializer;
import org.springframework.context.annotation.ComponentScan;
import org.springframework.context.annotation.Import;
import springfox.documentation.swagger2.annotations.EnableSwagger2;
@SpringBootApplication
@ConditionalOnProperty(prefix = "server", name = "is-combined-server", havingValue = "true")
@ServletComponentScan
@ComponentScan("org.apache.dolphinscheduler")
@Import({MasterServer.class, WorkerServer.class})
@EnableSwagger2
public class CombinedApplicationServer extends SpringBootServletInitializer {
public static void main(String[] args) throws Exception {
ApiApplicationServer.main(args);
LoggerServer server = new LoggerServer();
server.start();
AlertServer alertServer = AlertServer.getInstance();
alertServer.start();
}
}

2
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProjectController.java

@ -256,9 +256,9 @@ public class ProjectController extends BaseController {
/**
* import process definition
*
* @param loginUser login user
* @param file resource file
* @param projectName project name
* @return import result code
*/
@ApiOperation(value = "importProcessDefinition", notes= "EXPORT_PROCCESS_DEFINITION_NOTES")

247
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/dto/ProcessMeta.java

@ -0,0 +1,247 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.api.dto;
/**
* ProcessMeta
*/
public class ProcessMeta {
/**
* project name
*/
private String projectName;
/**
* process definition name
*/
private String processDefinitionName;
/**
* processs definition json
*/
private String processDefinitionJson;
/**
* process definition desc
*/
private String processDefinitionDescription;
/**
* process definition locations
*/
private String processDefinitionLocations;
/**
* process definition connects
*/
private String processDefinitionConnects;
/**
* warning type
*/
private String scheduleWarningType;
/**
* warning group id
*/
private Integer scheduleWarningGroupId;
/**
* warning group name
*/
private String scheduleWarningGroupName;
/**
* start time
*/
private String scheduleStartTime;
/**
* end time
*/
private String scheduleEndTime;
/**
* crontab
*/
private String scheduleCrontab;
/**
* failure strategy
*/
private String scheduleFailureStrategy;
/**
* release state
*/
private String scheduleReleaseState;
/**
* process instance priority
*/
private String scheduleProcessInstancePriority;
/**
* worker group id
*/
private Integer scheduleWorkerGroupId;
/**
* worker group name
*/
private String scheduleWorkerGroupName;
public ProcessMeta() {
}
public String getProjectName() {
return projectName;
}
public void setProjectName(String projectName) {
this.projectName = projectName;
}
public String getProcessDefinitionName() {
return processDefinitionName;
}
public void setProcessDefinitionName(String processDefinitionName) {
this.processDefinitionName = processDefinitionName;
}
public String getProcessDefinitionJson() {
return processDefinitionJson;
}
public void setProcessDefinitionJson(String processDefinitionJson) {
this.processDefinitionJson = processDefinitionJson;
}
public String getProcessDefinitionDescription() {
return processDefinitionDescription;
}
public void setProcessDefinitionDescription(String processDefinitionDescription) {
this.processDefinitionDescription = processDefinitionDescription;
}
public String getProcessDefinitionLocations() {
return processDefinitionLocations;
}
public void setProcessDefinitionLocations(String processDefinitionLocations) {
this.processDefinitionLocations = processDefinitionLocations;
}
public String getProcessDefinitionConnects() {
return processDefinitionConnects;
}
public void setProcessDefinitionConnects(String processDefinitionConnects) {
this.processDefinitionConnects = processDefinitionConnects;
}
public String getScheduleWarningType() {
return scheduleWarningType;
}
public void setScheduleWarningType(String scheduleWarningType) {
this.scheduleWarningType = scheduleWarningType;
}
public Integer getScheduleWarningGroupId() {
return scheduleWarningGroupId;
}
public void setScheduleWarningGroupId(int scheduleWarningGroupId) {
this.scheduleWarningGroupId = scheduleWarningGroupId;
}
public String getScheduleWarningGroupName() {
return scheduleWarningGroupName;
}
public void setScheduleWarningGroupName(String scheduleWarningGroupName) {
this.scheduleWarningGroupName = scheduleWarningGroupName;
}
public String getScheduleStartTime() {
return scheduleStartTime;
}
public void setScheduleStartTime(String scheduleStartTime) {
this.scheduleStartTime = scheduleStartTime;
}
public String getScheduleEndTime() {
return scheduleEndTime;
}
public void setScheduleEndTime(String scheduleEndTime) {
this.scheduleEndTime = scheduleEndTime;
}
public String getScheduleCrontab() {
return scheduleCrontab;
}
public void setScheduleCrontab(String scheduleCrontab) {
this.scheduleCrontab = scheduleCrontab;
}
public String getScheduleFailureStrategy() {
return scheduleFailureStrategy;
}
public void setScheduleFailureStrategy(String scheduleFailureStrategy) {
this.scheduleFailureStrategy = scheduleFailureStrategy;
}
public String getScheduleReleaseState() {
return scheduleReleaseState;
}
public void setScheduleReleaseState(String scheduleReleaseState) {
this.scheduleReleaseState = scheduleReleaseState;
}
public String getScheduleProcessInstancePriority() {
return scheduleProcessInstancePriority;
}
public void setScheduleProcessInstancePriority(String scheduleProcessInstancePriority) {
this.scheduleProcessInstancePriority = scheduleProcessInstancePriority;
}
public Integer getScheduleWorkerGroupId() {
return scheduleWorkerGroupId;
}
public void setScheduleWorkerGroupId(int scheduleWorkerGroupId) {
this.scheduleWorkerGroupId = scheduleWorkerGroupId;
}
public String getScheduleWorkerGroupName() {
return scheduleWorkerGroupName;
}
public void setScheduleWorkerGroupName(String scheduleWorkerGroupName) {
this.scheduleWorkerGroupName = scheduleWorkerGroupName;
}
}

603
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionService.java

@ -22,11 +22,15 @@ import com.alibaba.fastjson.JSONObject;
import com.baomidou.mybatisplus.core.metadata.IPage;
import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
import com.fasterxml.jackson.core.JsonProcessingException;
import org.apache.dolphinscheduler.api.dto.ProcessMeta;
import org.apache.dolphinscheduler.api.dto.treeview.Instance;
import org.apache.dolphinscheduler.api.dto.treeview.TreeViewDto;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.utils.CheckUtils;
import org.apache.dolphinscheduler.api.utils.FileUtils;
import org.apache.dolphinscheduler.api.utils.PageInfo;
import org.apache.dolphinscheduler.api.utils.exportprocess.ProcessAddTaskParam;
import org.apache.dolphinscheduler.api.utils.exportprocess.TaskNodeParamFactory;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.*;
import org.apache.dolphinscheduler.common.graph.DAG;
@ -53,9 +57,7 @@ import org.springframework.web.multipart.MultipartFile;
import javax.servlet.ServletOutputStream;
import javax.servlet.http.HttpServletResponse;
import java.io.BufferedOutputStream;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.nio.charset.StandardCharsets;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
@ -94,9 +96,6 @@ public class ProcessDefinitionService extends BaseDAGService {
@Autowired
private ProcessDao processDao;
@Autowired
private DataSourceMapper dataSourceMapper;
@Autowired
private WorkerGroupMapper workerGroupMapper;
@ -496,43 +495,7 @@ public class ProcessDefinitionService extends BaseDAGService {
ProcessDefinition processDefinition = processDefineMapper.queryByDefineId(processDefinitionId);
if (null != processDefinition) {
//correct task param which has data source or dependent param
String correctProcessDefinitionJson = addTaskNodeSpecialParam(processDefinition.getProcessDefinitionJson());
processDefinition.setProcessDefinitionJson(correctProcessDefinitionJson);
Map<String, Object> row = new LinkedHashMap<>();
row.put("projectName", processDefinition.getProjectName());
row.put("processDefinitionName", processDefinition.getName());
row.put("processDefinitionJson", processDefinition.getProcessDefinitionJson());
row.put("processDefinitionDescription", processDefinition.getDescription());
row.put("processDefinitionLocations", processDefinition.getLocations());
row.put("processDefinitionConnects", processDefinition.getConnects());
//schedule info
List<Schedule> schedules = scheduleMapper.queryByProcessDefinitionId(processDefinitionId);
if (!schedules.isEmpty()) {
Schedule schedule = schedules.get(0);
row.put("scheduleWarningType", schedule.getWarningType());
row.put("scheduleWarningGroupId", schedule.getWarningGroupId());
row.put("scheduleStartTime", DateUtils.dateToString(schedule.getStartTime()));
row.put("scheduleEndTime", DateUtils.dateToString(schedule.getEndTime()));
row.put("scheduleCrontab", schedule.getCrontab());
row.put("scheduleFailureStrategy", schedule.getFailureStrategy());
row.put("scheduleReleaseState", ReleaseState.OFFLINE);
row.put("scheduleProcessInstancePriority", schedule.getProcessInstancePriority());
if(schedule.getId() == -1){
row.put("scheduleWorkerGroupId", -1);
}else{
WorkerGroup workerGroup = workerGroupMapper.selectById(schedule.getWorkerGroupId());
if(workerGroup != null){
row.put("scheduleWorkerGroupName", workerGroup.getName());
}
}
}
//create workflow json file
String rowsJson = JSONUtils.toJsonString(row);
String exportProcessJson = exportProcessMetaDataStr(processDefinitionId, processDefinition);
response.setContentType(MediaType.APPLICATION_JSON_UTF8_VALUE);
response.setHeader("Content-Disposition", "attachment;filename="+processDefinition.getName()+".json");
BufferedOutputStream buff = null;
@ -540,7 +503,7 @@ public class ProcessDefinitionService extends BaseDAGService {
try {
out = response.getOutputStream();
buff = new BufferedOutputStream(out);
buff.write(rowsJson.getBytes(StandardCharsets.UTF_8));
buff.write(exportProcessJson.getBytes(StandardCharsets.UTF_8));
buff.flush();
buff.close();
} catch (IOException e) {
@ -560,18 +523,66 @@ public class ProcessDefinitionService extends BaseDAGService {
logger.warn("export process output stream not close", e);
}
}
}
}
}
}
/**
* get export process metadata string
* @param processDefinitionId process definition id
* @param processDefinition process definition
* @return export process metadata string
*/
public String exportProcessMetaDataStr(Integer processDefinitionId, ProcessDefinition processDefinition) {
//correct task param which has data source or dependent param
String correctProcessDefinitionJson = addExportTaskNodeSpecialParam(processDefinition.getProcessDefinitionJson());
processDefinition.setProcessDefinitionJson(correctProcessDefinitionJson);
//export process metadata
ProcessMeta exportProcessMeta = new ProcessMeta();
exportProcessMeta.setProjectName(processDefinition.getProjectName());
exportProcessMeta.setProcessDefinitionName(processDefinition.getName());
exportProcessMeta.setProcessDefinitionJson(processDefinition.getProcessDefinitionJson());
exportProcessMeta.setProcessDefinitionLocations(processDefinition.getLocations());
exportProcessMeta.setProcessDefinitionConnects(processDefinition.getConnects());
//schedule info
List<Schedule> schedules = scheduleMapper.queryByProcessDefinitionId(processDefinitionId);
if (!schedules.isEmpty()) {
Schedule schedule = schedules.get(0);
WorkerGroup workerGroup = workerGroupMapper.selectById(schedule.getWorkerGroupId());
if (null == workerGroup && schedule.getWorkerGroupId() == -1) {
workerGroup = new WorkerGroup();
workerGroup.setId(-1);
workerGroup.setName("");
}
exportProcessMeta.setScheduleWarningType(schedule.getWarningType().toString());
exportProcessMeta.setScheduleWarningGroupId(schedule.getWarningGroupId());
exportProcessMeta.setScheduleStartTime(DateUtils.dateToString(schedule.getStartTime()));
exportProcessMeta.setScheduleEndTime(DateUtils.dateToString(schedule.getEndTime()));
exportProcessMeta.setScheduleCrontab(schedule.getCrontab());
exportProcessMeta.setScheduleFailureStrategy(String.valueOf(schedule.getFailureStrategy()));
exportProcessMeta.setScheduleReleaseState(String.valueOf(ReleaseState.OFFLINE));
exportProcessMeta.setScheduleProcessInstancePriority(String.valueOf(schedule.getProcessInstancePriority()));
if (null != workerGroup) {
exportProcessMeta.setScheduleWorkerGroupId(workerGroup.getId());
exportProcessMeta.setScheduleWorkerGroupName(workerGroup.getName());
}
}
//create workflow json file
return JSONUtils.toJsonString(exportProcessMeta);
}
/**
* correct task param which has datasource or dependent
* @param processDefinitionJson processDefinitionJson
* @return correct processDefinitionJson
*/
public String addTaskNodeSpecialParam(String processDefinitionJson) {
public String addExportTaskNodeSpecialParam(String processDefinitionJson) {
JSONObject jsonObject = JSONUtils.parseObject(processDefinitionJson);
JSONArray jsonArray = (JSONArray) jsonObject.get("tasks");
@ -580,35 +591,9 @@ public class ProcessDefinitionService extends BaseDAGService {
if (StringUtils.isNotEmpty(taskNode.getString("type"))) {
String taskType = taskNode.getString("type");
if(checkTaskHasDataSource(taskType)){
// add sqlParameters
JSONObject sqlParameters = JSONUtils.parseObject(taskNode.getString("params"));
DataSource dataSource = dataSourceMapper.selectById((Integer) sqlParameters.get("datasource"));
if (null != dataSource) {
sqlParameters.put("datasourceName", dataSource.getName());
}
taskNode.put("params", sqlParameters);
}else if(checkTaskHasDependent(taskType)){
// add dependent param
JSONObject dependentParameters = JSONUtils.parseObject(taskNode.getString("dependence"));
if(null != dependentParameters){
JSONArray dependTaskList = (JSONArray) dependentParameters.get("dependTaskList");
for (int j = 0; j < dependTaskList.size(); j++) {
JSONObject dependentTaskModel = dependTaskList.getJSONObject(j);
JSONArray dependItemList = (JSONArray) dependentTaskModel.get("dependItemList");
for (int k = 0; k < dependItemList.size(); k++) {
JSONObject dependentItem = dependItemList.getJSONObject(k);
int definitionId = dependentItem.getInteger("definitionId");
ProcessDefinition definition = processDefineMapper.queryByDefineId(definitionId);
if(null != definition){
dependentItem.put("projectName",definition.getProjectName());
dependentItem.put("definitionName",definition.getName());
}
}
}
taskNode.put("dependence", dependentParameters);
}
ProcessAddTaskParam addTaskParam = TaskNodeParamFactory.getByTaskType(taskType);
if (null != addTaskParam) {
addTaskParam.addExportSpecialParam(taskNode);
}
}
}
@ -616,24 +601,6 @@ public class ProcessDefinitionService extends BaseDAGService {
return jsonObject.toString();
}
/**
* check task if has dependent
* @param taskType task type
* @return if task has dependent return true else false
*/
private boolean checkTaskHasDependent(String taskType) {
return taskType.equals(TaskType.DEPENDENT.name());
}
/**
* check task if has data source info
* @param taskType task type
* @return if task has data source return true else false
*/
private boolean checkTaskHasDataSource(String taskType) {
return taskType.equals(TaskType.SQL.name()) || taskType.equals(TaskType.PROCEDURE.name());
}
/**
* check task if has sub process
* @param taskType task type
@ -648,211 +615,173 @@ public class ProcessDefinitionService extends BaseDAGService {
* @param loginUser login user
* @param file process metadata json file
* @param currentProjectName current project name
* @return
* @return import process
*/
@Transactional(rollbackFor = Exception.class)
public Map<String, Object> importProcessDefinition(User loginUser, MultipartFile file, String currentProjectName) {
Map<String, Object> result = new HashMap<>(5);
String processMetaJson = FileUtils.file2String(file);
ProcessMeta processMeta = JSONUtils.parseObject(processMetaJson, ProcessMeta.class);
JSONObject json;
//check file content
if (null == processMeta) {
putMsg(result, Status.DATA_IS_NULL, "fileContent");
return result;
}
if (StringUtils.isEmpty(processMeta.getProjectName())) {
putMsg(result, Status.DATA_IS_NULL, "projectName");
return result;
}
if (StringUtils.isEmpty(processMeta.getProcessDefinitionName())) {
putMsg(result, Status.DATA_IS_NULL, "processDefinitionName");
return result;
}
if (StringUtils.isEmpty(processMeta.getProcessDefinitionJson())) {
putMsg(result, Status.DATA_IS_NULL, "processDefinitionJson");
return result;
}
//read workflow json
try(InputStreamReader inputStreamReader = new InputStreamReader( file.getInputStream(), StandardCharsets.UTF_8)) {
BufferedReader streamReader = new BufferedReader(inputStreamReader);
StringBuilder respomseStrBuilder = new StringBuilder();
String inputStr;
//deal with process name
String processDefinitionName = processMeta.getProcessDefinitionName();
//use currentProjectName to query
Project targetProject = projectMapper.queryByName(currentProjectName);
if(null != targetProject){
processDefinitionName = recursionProcessDefinitionName(targetProject.getId(),
processDefinitionName, 1);
}
while ((inputStr = streamReader.readLine())!= null){
respomseStrBuilder.append( inputStr );
}
//add special task param
String importProcessParam = addImportTaskNodeParam(loginUser, processMeta.getProcessDefinitionJson(), targetProject);
json = JSONObject.parseObject( respomseStrBuilder.toString() );
if(null != json){
String originProjectName = null;
String processDefinitionName = null;
String processDefinitionJson = null;
String processDefinitionDesc = null;
String processDefinitionLocations = null;
String processDefinitionConnects = null;
String scheduleWarningType = null;
String scheduleWarningGroupId = null;
String scheduleStartTime = null;
String scheduleEndTime = null;
String scheduleCrontab = null;
String scheduleFailureStrategy = null;
String scheduleReleaseState = null;
String scheduleProcessInstancePriority = null;
String scheduleWorkerGroupId = null;
String scheduleWorkerGroupName = null;
if (Objects.nonNull(json.get("projectName"))) {
originProjectName = json.get("projectName").toString();
} else {
putMsg(result, Status.DATA_IS_NULL, "processDefinitionName");
return result;
}
if (Objects.nonNull(json.get("processDefinitionName"))) {
processDefinitionName = json.get("processDefinitionName").toString();
} else {
putMsg(result, Status.DATA_IS_NULL, "processDefinitionName");
return result;
}
if (Objects.nonNull(json.get("processDefinitionJson"))) {
processDefinitionJson = json.get("processDefinitionJson").toString();
} else {
putMsg(result, Status.DATA_IS_NULL, "processDefinitionJson");
return result;
}
if (Objects.nonNull(json.get("processDefinitionDescription"))) {
processDefinitionDesc = json.get("processDefinitionDescription").toString();
}
if (Objects.nonNull(json.get("processDefinitionLocations"))) {
processDefinitionLocations = json.get("processDefinitionLocations").toString();
}
if (Objects.nonNull(json.get("processDefinitionConnects"))) {
processDefinitionConnects = json.get("processDefinitionConnects").toString();
}
//check user access for org project
Project originProject = projectMapper.queryByName(originProjectName);
Map<String, Object> checkResult = projectService.checkProjectAndAuth(loginUser, originProject, originProjectName);
Status resultStatus = (Status) checkResult.get(Constants.STATUS);
Map<String, Object> createProcessResult;
try {
createProcessResult = createProcessDefinition(loginUser
,currentProjectName,
processDefinitionName,
importProcessParam,
processMeta.getProcessDefinitionDescription(),
processMeta.getProcessDefinitionLocations(),
processMeta.getProcessDefinitionConnects());
} catch (JsonProcessingException e) {
logger.error("import process meta json data: {}", e.getMessage(), e);
putMsg(result, Status.IMPORT_PROCESS_DEFINE_ERROR);
return result;
}
if (resultStatus == Status.SUCCESS) {
//use currentProjectName to query
Project targetProject = projectMapper.queryByName(currentProjectName);
if(null != targetProject){
processDefinitionName = recursionProcessDefinitionName(targetProject.getId(), processDefinitionName, 1);
}
putMsg(result, Status.SUCCESS);
//create process definition
Integer processDefinitionId = null;
if (null != createProcessResult && Objects.nonNull(createProcessResult.get("processDefinitionId"))) {
processDefinitionId = Integer.parseInt(createProcessResult.get("processDefinitionId").toString());
}
//scheduler param
if (null != processMeta.getScheduleCrontab() && null != processDefinitionId) {
int scheduleInsert = importProcessSchedule(loginUser,
currentProjectName,
processMeta,
processDefinitionName,
processDefinitionId);
if (0 == scheduleInsert) {
putMsg(result, Status.IMPORT_PROCESS_DEFINE_ERROR);
return result;
}
}
JSONObject jsonObject = JSONUtils.parseObject(processDefinitionJson);
JSONArray jsonArray = (JSONArray) jsonObject.get("tasks");
for (int j = 0; j < jsonArray.size(); j++) {
JSONObject taskNode = jsonArray.getJSONObject(j);
String taskType = taskNode.getString("type");
if(checkTaskHasDataSource(taskType)) {
JSONObject sqlParameters = JSONUtils.parseObject(taskNode.getString("params"));
List<DataSource> dataSources = dataSourceMapper.queryDataSourceByName(sqlParameters.getString("datasourceName"));
if (!dataSources.isEmpty()) {
DataSource dataSource = dataSources.get(0);
sqlParameters.put("datasource", dataSource.getId());
}
taskNode.put("params", sqlParameters);
}else if(checkTaskHasDependent(taskType)){
JSONObject dependentParameters = JSONUtils.parseObject(taskNode.getString("dependence"));
if(dependentParameters != null){
JSONArray dependTaskList = (JSONArray) dependentParameters.get("dependTaskList");
for (int h = 0; h < dependTaskList.size(); h++) {
JSONObject dependentTaskModel = dependTaskList.getJSONObject(h);
JSONArray dependItemList = (JSONArray) dependentTaskModel.get("dependItemList");
for (int k = 0; k < dependItemList.size(); k++) {
JSONObject dependentItem = dependItemList.getJSONObject(k);
Project dependentItemProject = projectMapper.queryByName(dependentItem.getString("projectName"));
if(dependentItemProject != null){
ProcessDefinition definition = processDefineMapper.queryByDefineName(dependentItemProject.getId(),dependentItem.getString("definitionName"));
if(definition != null){
dependentItem.put("projectId",dependentItemProject.getId());
dependentItem.put("definitionId",definition.getId());
}
}
}
}
taskNode.put("dependence", dependentParameters);
}
}
}
return result;
}
//recursive sub-process parameter correction map key for old process id value for new process id
Map<Integer, Integer> subProcessIdMap = new HashMap<>(20);
/**
* import process add special task param
* @param loginUser login user
* @param processDefinitionJson process definition json
* @param targetProject target project
* @return import process param
*/
private String addImportTaskNodeParam(User loginUser, String processDefinitionJson, Project targetProject) {
JSONObject jsonObject = JSONUtils.parseObject(processDefinitionJson);
JSONArray jsonArray = (JSONArray) jsonObject.get("tasks");
//add sql and dependent param
for (int i = 0; i < jsonArray.size(); i++) {
JSONObject taskNode = jsonArray.getJSONObject(i);
String taskType = taskNode.getString("type");
ProcessAddTaskParam addTaskParam = TaskNodeParamFactory.getByTaskType(taskType);
if (null != addTaskParam) {
addTaskParam.addImportSpecialParam(taskNode);
}
}
List<Object> subProcessList = jsonArray.stream()
.filter(elem -> checkTaskHasSubProcess(JSONUtils.parseObject(elem.toString()).getString("type")))
.collect(Collectors.toList());
//recursive sub-process parameter correction map key for old process id value for new process id
Map<Integer, Integer> subProcessIdMap = new HashMap<>(20);
if (!subProcessList.isEmpty()) {
importSubProcess(loginUser, targetProject, jsonArray, subProcessIdMap);
}
List<Object> subProcessList = jsonArray.stream()
.filter(elem -> checkTaskHasSubProcess(JSONUtils.parseObject(elem.toString()).getString("type")))
.collect(Collectors.toList());
jsonObject.put("tasks", jsonArray);
if (CollectionUtils.isNotEmpty(subProcessList)) {
importSubProcess(loginUser, targetProject, jsonArray, subProcessIdMap);
}
Map<String, Object> createProcessDefinitionResult = createProcessDefinition(loginUser,currentProjectName,processDefinitionName,jsonObject.toString(),processDefinitionDesc,processDefinitionLocations,processDefinitionConnects);
Integer processDefinitionId = null;
if (Objects.nonNull(createProcessDefinitionResult.get("processDefinitionId"))) {
processDefinitionId = Integer.parseInt(createProcessDefinitionResult.get("processDefinitionId").toString());
}
if (Objects.nonNull(json.get("scheduleCrontab")) && processDefinitionId != null) {
Date now = new Date();
Schedule scheduleObj = new Schedule();
scheduleObj.setProjectName(currentProjectName);
scheduleObj.setProcessDefinitionId(processDefinitionId);
scheduleObj.setProcessDefinitionName(processDefinitionName);
scheduleObj.setCreateTime(now);
scheduleObj.setUpdateTime(now);
scheduleObj.setUserId(loginUser.getId());
scheduleObj.setUserName(loginUser.getUserName());
scheduleCrontab = json.get("scheduleCrontab").toString();
scheduleObj.setCrontab(scheduleCrontab);
if (Objects.nonNull(json.get("scheduleStartTime"))) {
scheduleStartTime = json.get("scheduleStartTime").toString();
scheduleObj.setStartTime(DateUtils.stringToDate(scheduleStartTime));
}
if (Objects.nonNull(json.get("scheduleEndTime"))) {
scheduleEndTime = json.get("scheduleEndTime").toString();
scheduleObj.setEndTime(DateUtils.stringToDate(scheduleEndTime));
}
if (Objects.nonNull(json.get("scheduleWarningType"))) {
scheduleWarningType = json.get("scheduleWarningType").toString();
scheduleObj.setWarningType(WarningType.valueOf(scheduleWarningType));
}
if (Objects.nonNull(json.get("scheduleWarningGroupId"))) {
scheduleWarningGroupId = json.get("scheduleWarningGroupId").toString();
scheduleObj.setWarningGroupId(Integer.parseInt(scheduleWarningGroupId));
}
if (Objects.nonNull(json.get("scheduleFailureStrategy"))) {
scheduleFailureStrategy = json.get("scheduleFailureStrategy").toString();
scheduleObj.setFailureStrategy(FailureStrategy.valueOf(scheduleFailureStrategy));
}
if (Objects.nonNull(json.get("scheduleReleaseState"))) {
scheduleReleaseState = json.get("scheduleReleaseState").toString();
scheduleObj.setReleaseState(ReleaseState.valueOf(scheduleReleaseState));
}
if (Objects.nonNull(json.get("scheduleProcessInstancePriority"))) {
scheduleProcessInstancePriority = json.get("scheduleProcessInstancePriority").toString();
scheduleObj.setProcessInstancePriority(Priority.valueOf(scheduleProcessInstancePriority));
}
if (Objects.nonNull(json.get("scheduleWorkerGroupId"))) {
scheduleWorkerGroupId = json.get("scheduleWorkerGroupId").toString();
if(scheduleWorkerGroupId != null){
scheduleObj.setWorkerGroupId(Integer.parseInt(scheduleWorkerGroupId));
}else{
if (Objects.nonNull(json.get("scheduleWorkerGroupName"))) {
scheduleWorkerGroupName = json.get("scheduleWorkerGroupName").toString();
List<WorkerGroup> workerGroups = workerGroupMapper.queryWorkerGroupByName(scheduleWorkerGroupName);
if(!workerGroups.isEmpty()){
scheduleObj.setWorkerGroupId(workerGroups.get(0).getId());
}
}
}
}
scheduleMapper.insert(scheduleObj);
}
jsonObject.put("tasks", jsonArray);
return jsonObject.toString();
}
putMsg(result, Status.SUCCESS);
return result;
/**
* import process schedule
* @param loginUser login user
* @param currentProjectName current project name
* @param processMeta process meta data
* @param processDefinitionName process definition name
* @param processDefinitionId process definition id
* @return insert schedule flag
*/
public int importProcessSchedule(User loginUser, String currentProjectName, ProcessMeta processMeta,
String processDefinitionName, Integer processDefinitionId) {
Date now = new Date();
Schedule scheduleObj = new Schedule();
scheduleObj.setProjectName(currentProjectName);
scheduleObj.setProcessDefinitionId(processDefinitionId);
scheduleObj.setProcessDefinitionName(processDefinitionName);
scheduleObj.setCreateTime(now);
scheduleObj.setUpdateTime(now);
scheduleObj.setUserId(loginUser.getId());
scheduleObj.setUserName(loginUser.getUserName());
scheduleObj.setCrontab(processMeta.getScheduleCrontab());
if (null != processMeta.getScheduleStartTime()) {
scheduleObj.setStartTime(DateUtils.stringToDate(processMeta.getScheduleStartTime()));
}
if (null != processMeta.getScheduleEndTime()) {
scheduleObj.setEndTime(DateUtils.stringToDate(processMeta.getScheduleEndTime()));
}
if (null != processMeta.getScheduleWarningType()) {
scheduleObj.setWarningType(WarningType.valueOf(processMeta.getScheduleWarningType()));
}
if (null != processMeta.getScheduleWarningGroupId()) {
scheduleObj.setWarningGroupId(processMeta.getScheduleWarningGroupId());
}
if (null != processMeta.getScheduleFailureStrategy()) {
scheduleObj.setFailureStrategy(FailureStrategy.valueOf(processMeta.getScheduleFailureStrategy()));
}
if (null != processMeta.getScheduleReleaseState()) {
scheduleObj.setReleaseState(ReleaseState.valueOf(processMeta.getScheduleReleaseState()));
}
if (null != processMeta.getScheduleProcessInstancePriority()) {
scheduleObj.setProcessInstancePriority(Priority.valueOf(processMeta.getScheduleProcessInstancePriority()));
}
if (null != processMeta.getScheduleWorkerGroupId()) {
scheduleObj.setWorkerGroupId(processMeta.getScheduleWorkerGroupId());
} else {
if (null != processMeta.getScheduleWorkerGroupName()) {
List<WorkerGroup> workerGroups = workerGroupMapper.queryWorkerGroupByName(processMeta.getScheduleWorkerGroupName());
if(CollectionUtils.isNotEmpty(workerGroups)){
scheduleObj.setWorkerGroupId(workerGroups.get(0).getId());
}
}else{
putMsg(result, Status.IMPORT_PROCESS_DEFINE_ERROR);
return result;
}
} catch (IOException e) {
throw new RuntimeException(e.getMessage(), e);
}
return result;
return scheduleMapper.insert(scheduleObj);
}
/**
@ -860,6 +789,8 @@ public class ProcessDefinitionService extends BaseDAGService {
* recursion create sub process
* @param loginUser login user
* @param targetProject target project
* @param jsonArray process task array
* @param subProcessIdMap correct sub process id map
*/
public void importSubProcess(User loginUser, Project targetProject, JSONArray jsonArray, Map<Integer, Integer> subProcessIdMap) {
for (int i = 0; i < jsonArray.size(); i++) {
@ -871,66 +802,68 @@ public class ProcessDefinitionService extends BaseDAGService {
JSONObject subParams = JSONUtils.parseObject(taskNode.getString("params"));
Integer subProcessId = subParams.getInteger("processDefinitionId");
ProcessDefinition subProcess = processDefineMapper.queryByDefineId(subProcessId);
String subProcessJson = subProcess.getProcessDefinitionJson();
//check current project has sub process
ProcessDefinition currentProjectSubProcess = processDefineMapper.queryByDefineName(targetProject.getId(), subProcess.getName());
if (null == currentProjectSubProcess) {
JSONArray subJsonArray = (JSONArray) JSONUtils.parseObject(subProcess.getProcessDefinitionJson()).get("tasks");
List<Object> subProcessList = subJsonArray.stream()
.filter(item -> checkTaskHasSubProcess(JSONUtils.parseObject(item.toString()).getString("type")))
.collect(Collectors.toList());
if (!subProcessList.isEmpty()) {
importSubProcess(loginUser, targetProject, subJsonArray, subProcessIdMap);
//sub process processId correct
if (!subProcessIdMap.isEmpty()) {
for (Map.Entry<Integer, Integer> entry : subProcessIdMap.entrySet()) {
String oldSubProcessId = "\"processDefinitionId\":" + entry.getKey();
String newSubProcessId = "\"processDefinitionId\":" + entry.getValue();
subProcessJson = subProcessJson.replaceAll(oldSubProcessId, newSubProcessId);
}
//check is sub process exist in db
if (null != subProcess) {
String subProcessJson = subProcess.getProcessDefinitionJson();
//check current project has sub process
ProcessDefinition currentProjectSubProcess = processDefineMapper.queryByDefineName(targetProject.getId(), subProcess.getName());
if (null == currentProjectSubProcess) {
JSONArray subJsonArray = (JSONArray) JSONUtils.parseObject(subProcess.getProcessDefinitionJson()).get("tasks");
List<Object> subProcessList = subJsonArray.stream()
.filter(item -> checkTaskHasSubProcess(JSONUtils.parseObject(item.toString()).getString("type")))
.collect(Collectors.toList());
if (CollectionUtils.isNotEmpty(subProcessList)) {
importSubProcess(loginUser, targetProject, subJsonArray, subProcessIdMap);
//sub process processId correct
if (!subProcessIdMap.isEmpty()) {
for (Map.Entry<Integer, Integer> entry : subProcessIdMap.entrySet()) {
String oldSubProcessId = "\"processDefinitionId\":" + entry.getKey();
String newSubProcessId = "\"processDefinitionId\":" + entry.getValue();
subProcessJson = subProcessJson.replaceAll(oldSubProcessId, newSubProcessId);
}
subProcessIdMap.clear();
subProcessIdMap.clear();
}
}
}
//if sub-process recursion
Date now = new Date();
//create sub process in target project
ProcessDefinition processDefine = new ProcessDefinition();
processDefine.setName(subProcess.getName());
processDefine.setVersion(subProcess.getVersion());
processDefine.setReleaseState(subProcess.getReleaseState());
processDefine.setProjectId(targetProject.getId());
processDefine.setUserId(loginUser.getId());
processDefine.setProcessDefinitionJson(subProcessJson);
processDefine.setDescription(subProcess.getDescription());
processDefine.setLocations(subProcess.getLocations());
processDefine.setConnects(subProcess.getConnects());
processDefine.setTimeout(subProcess.getTimeout());
processDefine.setTenantId(subProcess.getTenantId());
processDefine.setGlobalParams(subProcess.getGlobalParams());
processDefine.setCreateTime(now);
processDefine.setUpdateTime(now);
processDefine.setFlag(subProcess.getFlag());
processDefine.setReceivers(subProcess.getReceivers());
processDefine.setReceiversCc(subProcess.getReceiversCc());
processDefineMapper.insert(processDefine);
logger.info("create sub process, project: {}, process name: {}", targetProject.getName(), processDefine.getName());
//modify task node
ProcessDefinition newSubProcessDefine = processDefineMapper.queryByDefineName(processDefine.getProjectId(),processDefine.getName());
if (null != newSubProcessDefine) {
subProcessIdMap.put(subProcessId, newSubProcessDefine.getId());
subParams.put("processDefinitionId", newSubProcessDefine.getId());
taskNode.put("params", subParams);
//if sub-process recursion
Date now = new Date();
//create sub process in target project
ProcessDefinition processDefine = new ProcessDefinition();
processDefine.setName(subProcess.getName());
processDefine.setVersion(subProcess.getVersion());
processDefine.setReleaseState(subProcess.getReleaseState());
processDefine.setProjectId(targetProject.getId());
processDefine.setUserId(loginUser.getId());
processDefine.setProcessDefinitionJson(subProcessJson);
processDefine.setDescription(subProcess.getDescription());
processDefine.setLocations(subProcess.getLocations());
processDefine.setConnects(subProcess.getConnects());
processDefine.setTimeout(subProcess.getTimeout());
processDefine.setTenantId(subProcess.getTenantId());
processDefine.setGlobalParams(subProcess.getGlobalParams());
processDefine.setCreateTime(now);
processDefine.setUpdateTime(now);
processDefine.setFlag(subProcess.getFlag());
processDefine.setReceivers(subProcess.getReceivers());
processDefine.setReceiversCc(subProcess.getReceiversCc());
processDefineMapper.insert(processDefine);
logger.info("create sub process, project: {}, process name: {}", targetProject.getName(), processDefine.getName());
//modify task node
ProcessDefinition newSubProcessDefine = processDefineMapper.queryByDefineName(processDefine.getProjectId(),processDefine.getName());
if (null != newSubProcessDefine) {
subProcessIdMap.put(subProcessId, newSubProcessDefine.getId());
subParams.put("processDefinitionId", newSubProcessDefine.getId());
taskNode.put("params", subParams);
}
}
}
}
}
@ -1316,7 +1249,7 @@ public class ProcessDefinitionService extends BaseDAGService {
private String recursionProcessDefinitionName(Integer projectId,String processDefinitionName,int num){
ProcessDefinition processDefinition = processDefineMapper.queryByDefineName(projectId, processDefinitionName);
if (processDefinition != null) {
if(num>1){
if(num > 1){
String str = processDefinitionName.substring(0,processDefinitionName.length() - 3);
processDefinitionName = str + "("+num+")";
}else{

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

@ -19,7 +19,6 @@ package org.apache.dolphinscheduler.api.service;
import com.baomidou.mybatisplus.core.metadata.IPage;
import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
import org.apache.commons.collections.BeanMap;
import org.apache.commons.lang.StringUtils;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.utils.PageInfo;
import org.apache.dolphinscheduler.api.utils.Result;
@ -28,6 +27,7 @@ import org.apache.dolphinscheduler.common.enums.ResourceType;
import org.apache.dolphinscheduler.common.utils.FileUtils;
import org.apache.dolphinscheduler.common.utils.HadoopUtils;
import org.apache.dolphinscheduler.common.utils.PropertyUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.Tenant;
import org.apache.dolphinscheduler.dao.entity.UdfFunc;
@ -104,7 +104,7 @@ public class ResourcesService extends BaseService {
String nameSuffix = FileUtils.suffix(name);
// determine file suffix
if (!StringUtils.equals(fileSuffix, nameSuffix)) {
if (!(StringUtils.isNotEmpty(fileSuffix) && fileSuffix.equalsIgnoreCase(nameSuffix))) {
/**
* rename file suffix and original suffix must be consistent
*/
@ -347,7 +347,7 @@ public class ResourcesService extends BaseService {
String nameSuffix = FileUtils.suffix(name);
// determine file suffix
if (!StringUtils.equals(fileSuffix, nameSuffix)) {
if (!(StringUtils.isNotEmpty(fileSuffix) && fileSuffix.equalsIgnoreCase(nameSuffix))) {
return false;
}
// query tenant
@ -545,7 +545,7 @@ public class ResourcesService extends BaseService {
putMsg(result, Status.SUCCESS);
Map<String, Object> map = new HashMap<>();
map.put(ALIAS, resource.getAlias());
map.put(CONTENT, StringUtils.join(content.toArray(), "\n"));
map.put(CONTENT, StringUtils.join(content, "\n"));
result.setData(map);
}else{
logger.error("read file {} not exist in hdfs", hdfsFileName);
@ -608,7 +608,7 @@ public class ResourcesService extends BaseService {
putMsg(result, Status.SUCCESS);
Map<Object, Object> dataMap = new BeanMap(resource);
Map<String, Object> resultMap = new HashMap<>(5);
Map<String, Object> resultMap = new HashMap<>();
for (Map.Entry<Object, Object> entry: dataMap.entrySet()) {
if (!Constants.CLASS.equalsIgnoreCase(entry.getKey().toString())) {
resultMap.put(entry.getKey().toString(), entry.getValue());

2
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/UdfFuncService.java

@ -22,6 +22,7 @@ import org.apache.dolphinscheduler.api.utils.Result;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.UdfType;
import org.apache.dolphinscheduler.common.utils.PropertyUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.UdfFunc;
import org.apache.dolphinscheduler.dao.entity.User;
@ -30,7 +31,6 @@ import org.apache.dolphinscheduler.dao.mapper.UDFUserMapper;
import org.apache.dolphinscheduler.dao.mapper.UdfFuncMapper;
import com.baomidou.mybatisplus.core.metadata.IPage;
import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
import org.apache.commons.lang.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;

30
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/FileUtils.java

@ -22,9 +22,12 @@ import org.springframework.core.io.Resource;
import org.springframework.core.io.UrlResource;
import org.springframework.web.multipart.MultipartFile;
import java.io.BufferedReader;
import java.io.File;
import java.io.IOException;
import java.io.InputStreamReader;
import java.net.MalformedURLException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
@ -38,7 +41,7 @@ public class FileUtils {
/**
* copy source file to target file
*
* @param file file
* @param file file
* @param destFilename destination file name
*/
@ -54,7 +57,7 @@ public class FileUtils {
Files.copy(file.getInputStream(), Paths.get(destFilename));
} catch (IOException e) {
logger.error(String.format("failed to copy file , {} is empty file", file.getOriginalFilename()), e);
logger.error("failed to copy file , {} is empty file", file.getOriginalFilename(), e);
}
}
@ -77,4 +80,27 @@ public class FileUtils {
}
return null;
}
/**
* file convert String
* @param file MultipartFile file
* @return file content string
*/
public static String file2String(MultipartFile file) {
StringBuilder strBuilder = new StringBuilder();
try (InputStreamReader inputStreamReader = new InputStreamReader(file.getInputStream(), StandardCharsets.UTF_8)) {
BufferedReader streamReader = new BufferedReader(inputStreamReader);
String inputStr;
while ((inputStr = streamReader.readLine()) != null) {
strBuilder.append(inputStr);
}
} catch (IOException e) {
logger.error("file convert to string failed: {}", file.getName());
}
return strBuilder.toString();
}
}

58
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/ZooKeeperState.java

@ -55,41 +55,41 @@ public class ZooKeeperState {
public void getZookeeperInfo() {
String content = cmd("srvr");
if (StringUtils.isNotBlank(content)) {
Scanner scannerForStat = new Scanner(content);
while (scannerForStat.hasNext()) {
String line = scannerForStat.nextLine();
if (line.startsWith("Latency min/avg/max:")) {
String[] latencys = getStringValueFromLine(line).split("/");
minLatency = Integer.parseInt(latencys[0]);
avgLatency = Integer.parseInt(latencys[1]);
maxLatency = Integer.parseInt(latencys[2]);
} else if (line.startsWith("Received:")) {
received = Long.parseLong(getStringValueFromLine(line));
} else if (line.startsWith("Sent:")) {
sent = Long.parseLong(getStringValueFromLine(line));
} else if (line.startsWith("Outstanding:")) {
outStanding = Integer.parseInt(getStringValueFromLine(line));
} else if (line.startsWith("Zxid:")) {
zxid = Long.parseLong(getStringValueFromLine(line).substring(2), 16);
} else if (line.startsWith("Mode:")) {
mode = getStringValueFromLine(line);
} else if (line.startsWith("Node count:")) {
nodeCount = Integer.parseInt(getStringValueFromLine(line));
try (Scanner scannerForStat = new Scanner(content)) {
while (scannerForStat.hasNext()) {
String line = scannerForStat.nextLine();
if (line.startsWith("Latency min/avg/max:")) {
String[] latencys = getStringValueFromLine(line).split("/");
minLatency = Integer.parseInt(latencys[0]);
avgLatency = Integer.parseInt(latencys[1]);
maxLatency = Integer.parseInt(latencys[2]);
} else if (line.startsWith("Received:")) {
received = Long.parseLong(getStringValueFromLine(line));
} else if (line.startsWith("Sent:")) {
sent = Long.parseLong(getStringValueFromLine(line));
} else if (line.startsWith("Outstanding:")) {
outStanding = Integer.parseInt(getStringValueFromLine(line));
} else if (line.startsWith("Zxid:")) {
zxid = Long.parseLong(getStringValueFromLine(line).substring(2), 16);
} else if (line.startsWith("Mode:")) {
mode = getStringValueFromLine(line);
} else if (line.startsWith("Node count:")) {
nodeCount = Integer.parseInt(getStringValueFromLine(line));
}
}
}
scannerForStat.close();
}
}
String wchsText = cmd("wchs");
if (StringUtils.isNotBlank(wchsText)) {
Scanner scannerForWchs = new Scanner(wchsText);
while (scannerForWchs.hasNext()) {
String line = scannerForWchs.nextLine();
if (line.startsWith("Total watches:")) {
watches = Integer.parseInt(getStringValueFromLine(line));
try (Scanner scannerForWchs = new Scanner(wchsText)) {
while (scannerForWchs.hasNext()) {
String line = scannerForWchs.nextLine();
if (line.startsWith("Total watches:")) {
watches = Integer.parseInt(getStringValueFromLine(line));
}
}
}
scannerForWchs.close();
}
}
String consText = cmd("cons");

83
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/exportprocess/DataSourceParam.java

@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.api.utils.exportprocess;
import com.alibaba.fastjson.JSONObject;
import org.apache.dolphinscheduler.common.enums.TaskType;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.DataSource;
import org.apache.dolphinscheduler.dao.mapper.DataSourceMapper;
import org.springframework.beans.factory.InitializingBean;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import java.util.List;
/**
* task node add datasource param strategy
*/
@Service
public class DataSourceParam implements ProcessAddTaskParam, InitializingBean {
@Autowired
private DataSourceMapper dataSourceMapper;
/**
* add datasource params
* @param taskNode task node json object
* @return task node json object
*/
@Override
public JSONObject addExportSpecialParam(JSONObject taskNode) {
// add sqlParameters
JSONObject sqlParameters = JSONUtils.parseObject(taskNode.getString("params"));
DataSource dataSource = dataSourceMapper.selectById((Integer) sqlParameters.get("datasource"));
if (null != dataSource) {
sqlParameters.put("datasourceName", dataSource.getName());
}
taskNode.put("params", sqlParameters);
return taskNode;
}
/**
* import process add datasource params
* @param taskNode task node json object
* @return task node json object
*/
@Override
public JSONObject addImportSpecialParam(JSONObject taskNode) {
JSONObject sqlParameters = JSONUtils.parseObject(taskNode.getString("params"));
List<DataSource> dataSources = dataSourceMapper.queryDataSourceByName(sqlParameters.getString("datasourceName"));
if (!dataSources.isEmpty()) {
DataSource dataSource = dataSources.get(0);
sqlParameters.put("datasource", dataSource.getId());
}
taskNode.put("params", sqlParameters);
return taskNode;
}
/**
* put datasource strategy
*/
@Override
public void afterPropertiesSet() {
TaskNodeParamFactory.register(TaskType.SQL.name(), this);
TaskNodeParamFactory.register(TaskType.PROCEDURE.name(), this);
}
}

112
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/exportprocess/DependentParam.java

@ -0,0 +1,112 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.api.utils.exportprocess;
import com.alibaba.fastjson.JSONArray;
import com.alibaba.fastjson.JSONObject;
import org.apache.dolphinscheduler.common.enums.TaskType;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.springframework.beans.factory.InitializingBean;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
/**
* task node add dependent param strategy
*/
@Service
public class DependentParam implements ProcessAddTaskParam, InitializingBean {
@Autowired
ProcessDefinitionMapper processDefineMapper;
@Autowired
ProjectMapper projectMapper;
/**
* add dependent param
* @param taskNode task node json object
* @return task node json object
*/
@Override
public JSONObject addExportSpecialParam(JSONObject taskNode) {
// add dependent param
JSONObject dependentParameters = JSONUtils.parseObject(taskNode.getString("dependence"));
if (null != dependentParameters) {
JSONArray dependTaskList = (JSONArray) dependentParameters.get("dependTaskList");
for (int j = 0; j < dependTaskList.size(); j++) {
JSONObject dependentTaskModel = dependTaskList.getJSONObject(j);
JSONArray dependItemList = (JSONArray) dependentTaskModel.get("dependItemList");
for (int k = 0; k < dependItemList.size(); k++) {
JSONObject dependentItem = dependItemList.getJSONObject(k);
int definitionId = dependentItem.getInteger("definitionId");
ProcessDefinition definition = processDefineMapper.queryByDefineId(definitionId);
if (null != definition) {
dependentItem.put("projectName", definition.getProjectName());
dependentItem.put("definitionName", definition.getName());
}
}
}
taskNode.put("dependence", dependentParameters);
}
return taskNode;
}
/**
* import process add dependent param
* @param taskNode task node json object
* @return
*/
@Override
public JSONObject addImportSpecialParam(JSONObject taskNode) {
JSONObject dependentParameters = JSONUtils.parseObject(taskNode.getString("dependence"));
if(dependentParameters != null){
JSONArray dependTaskList = (JSONArray) dependentParameters.get("dependTaskList");
for (int h = 0; h < dependTaskList.size(); h++) {
JSONObject dependentTaskModel = dependTaskList.getJSONObject(h);
JSONArray dependItemList = (JSONArray) dependentTaskModel.get("dependItemList");
for (int k = 0; k < dependItemList.size(); k++) {
JSONObject dependentItem = dependItemList.getJSONObject(k);
Project dependentItemProject = projectMapper.queryByName(dependentItem.getString("projectName"));
if(dependentItemProject != null){
ProcessDefinition definition = processDefineMapper.queryByDefineName(dependentItemProject.getId(),dependentItem.getString("definitionName"));
if(definition != null){
dependentItem.put("projectId",dependentItemProject.getId());
dependentItem.put("definitionId",definition.getId());
}
}
}
}
taskNode.put("dependence", dependentParameters);
}
return taskNode;
}
/**
* put dependent strategy
*/
@Override
public void afterPropertiesSet() {
TaskNodeParamFactory.register(TaskType.DEPENDENT.name(), this);
}
}

39
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/exportprocess/ProcessAddTaskParam.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.exportprocess;
import com.alibaba.fastjson.JSONObject;
/**
* ProcessAddTaskParam
*/
public interface ProcessAddTaskParam {
/**
* add export task special param: sql task dependent task
* @param taskNode task node json object
* @return task node json object
*/
JSONObject addExportSpecialParam(JSONObject taskNode);
/**
* add task special param: sql task dependent task
* @param taskNode task node json object
* @return task node json object
*/
JSONObject addImportSpecialParam(JSONObject taskNode);
}

38
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/exportprocess/TaskNodeParamFactory.java

@ -0,0 +1,38 @@
/*
* 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.exportprocess;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
/**
* task node param factory
*/
public class TaskNodeParamFactory {
private static Map<String, ProcessAddTaskParam> taskServices = new ConcurrentHashMap<>();
public static ProcessAddTaskParam getByTaskType(String taskType){
return taskServices.get(taskType);
}
static void register(String taskType, ProcessAddTaskParam addSpecialTaskParam){
if (null != taskType) {
taskServices.put(taskType, addSpecialTaskParam);
}
}
}

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

@ -20,18 +20,15 @@ import com.alibaba.fastjson.JSON;
import com.alibaba.fastjson.JSONArray;
import com.alibaba.fastjson.JSONObject;
import org.apache.dolphinscheduler.api.ApiApplicationServer;
import org.apache.dolphinscheduler.api.dto.ProcessMeta;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.common.enums.*;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.FileUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.DataSource;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.DataSourceMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.apache.dolphinscheduler.dao.entity.*;
import org.apache.dolphinscheduler.dao.mapper.*;
import org.apache.http.entity.ContentType;
import org.json.JSONException;
import org.junit.Assert;
@ -52,8 +49,7 @@ import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.text.MessageFormat;
import java.util.HashMap;
import java.util.Map;
import java.util.*;
@RunWith(MockitoJUnitRunner.Silent.class)
@SpringBootTest(classes = ApiApplicationServer.class)
@ -75,6 +71,29 @@ public class ProcessDefinitionServiceTest {
@Mock
private ProjectService projectService;
@Mock
private ScheduleMapper scheduleMapper;
@Mock
private WorkerGroupMapper workerGroupMapper;
private String sqlDependentJson = "{\"globalParams\":[]," +
"\"tasks\":[{\"type\":\"SQL\",\"id\":\"tasks-27297\",\"name\":\"sql\"," +
"\"params\":{\"type\":\"MYSQL\",\"datasource\":1,\"sql\":\"select * from test\"," +
"\"udfs\":\"\",\"sqlType\":\"1\",\"title\":\"\",\"receivers\":\"\",\"receiversCc\":\"\",\"showType\":\"TABLE\"" +
",\"localParams\":[],\"connParams\":\"\"," +
"\"preStatements\":[],\"postStatements\":[]}," +
"\"description\":\"\",\"runFlag\":\"NORMAL\",\"dependence\":{},\"maxRetryTimes\":\"0\"," +
"\"retryInterval\":\"1\",\"timeout\":{\"strategy\":\"\"," +
"\"enable\":false},\"taskInstancePriority\":\"MEDIUM\",\"workerGroupId\":-1," +
"\"preTasks\":[\"dependent\"]},{\"type\":\"DEPENDENT\",\"id\":\"tasks-33787\"," +
"\"name\":\"dependent\",\"params\":{},\"description\":\"\",\"runFlag\":\"NORMAL\"," +
"\"dependence\":{\"relation\":\"AND\",\"dependTaskList\":[{\"relation\":\"AND\"," +
"\"dependItemList\":[{\"projectId\":2,\"definitionId\":46,\"depTasks\":\"ALL\"," +
"\"cycle\":\"day\",\"dateValue\":\"today\"}]}]},\"maxRetryTimes\":\"0\",\"retryInterval\":\"1\"," +
"\"timeout\":{\"strategy\":\"\",\"enable\":false},\"taskInstancePriority\":\"MEDIUM\"," +
"\"workerGroupId\":-1,\"preTasks\":[]}],\"tenantId\":1,\"timeout\":0}";
@Test
public void queryProccessDefinitionList() throws Exception {
String projectName = "project_test1";
@ -147,28 +166,77 @@ public class ProcessDefinitionServiceTest {
Mockito.when(dataSourceMapper.selectById(1)).thenReturn(getDataSource());
Mockito.when(processDefineMapper.queryByDefineId(2)).thenReturn(getProcessDefinition());
String corSqlDependentJson = processDefinitionService.addExportTaskNodeSpecialParam(sqlDependentJson);
String sqlDependentJson = "{\"globalParams\":[]," +
"\"tasks\":[{\"type\":\"SQL\",\"id\":\"tasks-27297\",\"name\":\"sql\"," +
"\"params\":{\"type\":\"MYSQL\",\"datasource\":1,\"sql\":\"select * from test\"," +
"\"udfs\":\"\",\"sqlType\":\"1\",\"title\":\"\",\"receivers\":\"\",\"receiversCc\":\"\",\"showType\":\"TABLE\"" +
",\"localParams\":[],\"connParams\":\"\"," +
"\"preStatements\":[],\"postStatements\":[]}," +
"\"description\":\"\",\"runFlag\":\"NORMAL\",\"dependence\":{},\"maxRetryTimes\":\"0\"," +
"\"retryInterval\":\"1\",\"timeout\":{\"strategy\":\"\"," +
"\"enable\":false},\"taskInstancePriority\":\"MEDIUM\",\"workerGroupId\":-1," +
"\"preTasks\":[\"dependent\"]},{\"type\":\"DEPENDENT\",\"id\":\"tasks-33787\"," +
"\"name\":\"dependent\",\"params\":{},\"description\":\"\",\"runFlag\":\"NORMAL\"," +
"\"dependence\":{\"relation\":\"AND\",\"dependTaskList\":[{\"relation\":\"AND\"," +
"\"dependItemList\":[{\"projectId\":2,\"definitionId\":46,\"depTasks\":\"ALL\"," +
"\"cycle\":\"day\",\"dateValue\":\"today\"}]}]},\"maxRetryTimes\":\"0\",\"retryInterval\":\"1\"," +
"\"timeout\":{\"strategy\":\"\",\"enable\":false},\"taskInstancePriority\":\"MEDIUM\"," +
JSONAssert.assertEquals(sqlDependentJson,corSqlDependentJson,false);
}
@Test
public void testExportProcessMetaDataStr() {
Mockito.when(scheduleMapper.queryByProcessDefinitionId(46)).thenReturn(getSchedulerList());
Mockito.when(workerGroupMapper.selectById(-1)).thenReturn(null);
ProcessDefinition processDefinition = getProcessDefinition();
processDefinition.setProcessDefinitionJson(sqlDependentJson);
String exportProcessMetaDataStr = processDefinitionService.exportProcessMetaDataStr(46, processDefinition);
Assert.assertNotEquals(sqlDependentJson,exportProcessMetaDataStr);
}
@Test
public void testAddExportTaskNodeSpecialParam() throws JSONException {
String shellJson = "{\"globalParams\":[],\"tasks\":[{\"id\":\"tasks-9527\",\"name\":\"shell-1\"," +
"\"params\":{\"resourceList\":[],\"localParams\":[],\"rawScript\":\"#!/bin/bash\\necho \\\"shell-1\\\"\"}," +
"\"description\":\"\",\"runFlag\":\"NORMAL\",\"dependence\":{},\"maxRetryTimes\":\"0\",\"retryInterval\":\"1\"," +
"\"timeout\":{\"strategy\":\"\",\"interval\":1,\"enable\":false},\"taskInstancePriority\":\"MEDIUM\"," +
"\"workerGroupId\":-1,\"preTasks\":[]}],\"tenantId\":1,\"timeout\":0}";
String corSqlDependentJson = processDefinitionService.addTaskNodeSpecialParam(sqlDependentJson);
String resultStr = processDefinitionService.addExportTaskNodeSpecialParam(shellJson);
JSONAssert.assertEquals(shellJson, resultStr, false);
}
@Test
public void testImportProcessSchedule() {
User loginUser = new User();
loginUser.setId(1);
loginUser.setUserType(UserType.GENERAL_USER);
String currentProjectName = "test";
String processDefinitionName = "test_process";
Integer processDefinitionId = 1;
Schedule schedule = getSchedule();
ProcessMeta processMeta = getProcessMeta();
int insertFlag = processDefinitionService.importProcessSchedule(loginUser, currentProjectName, processMeta,
processDefinitionName, processDefinitionId);
Assert.assertEquals(0, insertFlag);
ProcessMeta processMetaCron = new ProcessMeta();
processMetaCron.setScheduleCrontab(schedule.getCrontab());
int insertFlagCron = processDefinitionService.importProcessSchedule(loginUser, currentProjectName, processMetaCron,
processDefinitionName, processDefinitionId);
Assert.assertEquals(0, insertFlagCron);
WorkerGroup workerGroup = new WorkerGroup();
workerGroup.setName("ds-test-workergroup");
workerGroup.setId(2);
List<WorkerGroup> workerGroups = new ArrayList<>();
workerGroups.add(workerGroup);
Mockito.when(workerGroupMapper.queryWorkerGroupByName("ds-test")).thenReturn(workerGroups);
processMetaCron.setScheduleWorkerGroupName("ds-test");
int insertFlagWorker = processDefinitionService.importProcessSchedule(loginUser, currentProjectName, processMetaCron,
processDefinitionName, processDefinitionId);
Assert.assertEquals(0, insertFlagWorker);
Mockito.when(workerGroupMapper.queryWorkerGroupByName("ds-test")).thenReturn(null);
int workerNullFlag = processDefinitionService.importProcessSchedule(loginUser, currentProjectName, processMetaCron,
processDefinitionName, processDefinitionId);
Assert.assertEquals(0, workerNullFlag);
JSONAssert.assertEquals(sqlDependentJson,corSqlDependentJson,false);
}
@ -311,9 +379,50 @@ public class ProcessDefinitionServiceTest {
Assert.assertTrue(delete);
String processMetaJson = "";
improssProcessCheckData(file, loginUser, currentProjectName, processMetaJson);
processMetaJson = "{\"scheduleWorkerGroupId\":-1}";
improssProcessCheckData(file, loginUser, currentProjectName, processMetaJson);
processMetaJson = "{\"scheduleWorkerGroupId\":-1,\"projectName\":\"test\"}";
improssProcessCheckData(file, loginUser, currentProjectName, processMetaJson);
processMetaJson = "{\"scheduleWorkerGroupId\":-1,\"projectName\":\"test\",\"processDefinitionName\":\"test_definition\"}";
improssProcessCheckData(file, loginUser, currentProjectName, processMetaJson);
}
/**
* check import process metadata
* @param file file
* @param loginUser login user
* @param currentProjectName current project name
* @param processMetaJson process meta json
* @throws IOException IO exception
*/
private void improssProcessCheckData(File file, User loginUser, String currentProjectName, String processMetaJson) throws IOException {
//check null
FileUtils.writeStringToFile(new File("/tmp/task.json"),processMetaJson);
File fileEmpty = new File("/tmp/task.json");
FileInputStream fileEmptyInputStream = new FileInputStream("/tmp/task.json");
MultipartFile multiFileEmpty = new MockMultipartFile(fileEmpty.getName(), fileEmpty.getName(),
ContentType.APPLICATION_OCTET_STREAM.toString(), fileEmptyInputStream);
Map<String, Object> resEmptyProcess = processDefinitionService.importProcessDefinition(loginUser, multiFileEmpty, currentProjectName);
Assert.assertEquals(Status.DATA_IS_NULL, resEmptyProcess.get(Constants.STATUS));
boolean deleteFlag = file.delete();
Assert.assertTrue(deleteFlag);
}
/**
* get mock datasource
* @return DataSource
@ -350,6 +459,54 @@ public class ProcessDefinitionServiceTest {
return project;
}
/**
* get mock schedule
* @return schedule
*/
private Schedule getSchedule() {
Date date = new Date();
Schedule schedule = new Schedule();
schedule.setId(46);
schedule.setProcessDefinitionId(1);
schedule.setStartTime(date);
schedule.setEndTime(date);
schedule.setCrontab("0 0 5 * * ? *");
schedule.setFailureStrategy(FailureStrategy.END);
schedule.setUserId(1);
schedule.setReleaseState(ReleaseState.OFFLINE);
schedule.setProcessInstancePriority(Priority.MEDIUM);
schedule.setWarningType(WarningType.NONE);
schedule.setWarningGroupId(1);
schedule.setWorkerGroupId(-1);
return schedule;
}
/**
* get mock processMeta
* @return processMeta
*/
private ProcessMeta getProcessMeta() {
ProcessMeta processMeta = new ProcessMeta();
Schedule schedule = getSchedule();
processMeta.setScheduleCrontab(schedule.getCrontab());
processMeta.setScheduleStartTime(DateUtils.dateToString(schedule.getStartTime()));
processMeta.setScheduleEndTime(DateUtils.dateToString(schedule.getEndTime()));
processMeta.setScheduleWarningType(String.valueOf(schedule.getWarningType()));
processMeta.setScheduleWarningGroupId(schedule.getWarningGroupId());
processMeta.setScheduleFailureStrategy(String.valueOf(schedule.getFailureStrategy()));
processMeta.setScheduleReleaseState(String.valueOf(schedule.getReleaseState()));
processMeta.setScheduleProcessInstancePriority(String.valueOf(schedule.getProcessInstancePriority()));
processMeta.setScheduleWorkerGroupId(schedule.getWorkerGroupId());
processMeta.setScheduleWorkerGroupName("workgroup1");
return processMeta;
}
private List<Schedule> getSchedulerList() {
List<Schedule> scheduleList = new ArrayList<>();
scheduleList.add(getSchedule());
return scheduleList;
}
private void putMsg(Map<String, Object> result, Status status, Object... statusParams) {
result.put(Constants.STATUS, status);
if (statusParams != null && statusParams.length > 0) {

3
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ResourcesServiceTest.java

@ -41,6 +41,7 @@ import org.mockito.InjectMocks;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.powermock.api.mockito.PowerMockito;
import org.powermock.core.classloader.annotations.PowerMockIgnore;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
import org.slf4j.Logger;
@ -53,6 +54,7 @@ import java.util.List;
import java.util.Map;
@RunWith(PowerMockRunner.class)
@PowerMockIgnore({"sun.security.*", "javax.net.*"})
@PrepareForTest({HadoopUtils.class,PropertyUtils.class, FileUtils.class,org.apache.dolphinscheduler.api.utils.FileUtils.class})
public class ResourcesServiceTest {
private static final Logger logger = LoggerFactory.getLogger(ResourcesServiceTest.class);
@ -115,6 +117,7 @@ public class ResourcesServiceTest {
//UDF_RESOURCE_SUFFIX_NOT_JAR
mockMultipartFile = new MockMultipartFile("ResourcesServiceTest.pdf","ResourcesServiceTest.pdf","pdf",new String("test").getBytes());
PowerMockito.when(FileUtils.suffix("ResourcesServiceTest.pdf")).thenReturn("pdf");
result = resourcesService.createResource(user,"ResourcesServiceTest.pdf","ResourcesServiceTest",ResourceType.UDF,mockMultipartFile);
logger.info(result.toString());
Assert.assertEquals(Status.UDF_RESOURCE_SUFFIX_NOT_JAR.getMsg(),result.getMsg());

31
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/FileUtilsTest.java

@ -17,21 +17,17 @@
package org.apache.dolphinscheduler.api.utils;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.apache.http.entity.ContentType;
import org.junit.*;
import org.junit.rules.TemporaryFolder;
import org.mockito.Mockito;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.core.io.Resource;
import org.springframework.mock.web.MockMultipartFile;
import org.springframework.web.multipart.MultipartFile;
import java.io.ByteArrayInputStream;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.*;
import static org.junit.Assert.*;
@ -106,4 +102,23 @@ public class FileUtilsTest {
assertNull(resource1);
}
@Test
public void testFile2String() throws IOException {
String content = "123";
org.apache.dolphinscheduler.common.utils.FileUtils.writeStringToFile(new File("/tmp/task.json"),content);
File file = new File("/tmp/task.json");
FileInputStream fileInputStream = new FileInputStream("/tmp/task.json");
MultipartFile multipartFile = new MockMultipartFile(file.getName(), file.getName(),
ContentType.APPLICATION_OCTET_STREAM.toString(), fileInputStream);
String resultStr = FileUtils.file2String(multipartFile);
Assert.assertEquals(content, resultStr);
boolean delete = file.delete();
Assert.assertTrue(delete);
}
}

86
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/exportprocess/DataSourceParamTest.java

@ -0,0 +1,86 @@
/*
* 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.exportprocess;
import com.alibaba.fastjson.JSONObject;
import org.apache.dolphinscheduler.api.ApiApplicationServer;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.json.JSONException;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.skyscreamer.jsonassert.JSONAssert;
import org.springframework.boot.test.context.SpringBootTest;
import org.springframework.test.context.junit4.SpringRunner;
/**
* DataSourceParamTest
*/
@RunWith(SpringRunner.class)
@SpringBootTest(classes = ApiApplicationServer.class)
public class DataSourceParamTest {
@Test
public void testAddExportDependentSpecialParam() throws JSONException {
String sqlJson = "{\"type\":\"SQL\",\"id\":\"tasks-27297\",\"name\":\"sql\"," +
"\"params\":{\"type\":\"MYSQL\",\"datasource\":1,\"sql\":\"select * from test\"," +
"\"udfs\":\"\",\"sqlType\":\"1\",\"title\":\"\",\"receivers\":\"\",\"receiversCc\":\"\",\"showType\":\"TABLE\"" +
",\"localParams\":[],\"connParams\":\"\"," +
"\"preStatements\":[],\"postStatements\":[]}," +
"\"description\":\"\",\"runFlag\":\"NORMAL\",\"dependence\":{},\"maxRetryTimes\":\"0\"," +
"\"retryInterval\":\"1\",\"timeout\":{\"strategy\":\"\"," +
"\"enable\":false},\"taskInstancePriority\":\"MEDIUM\",\"workerGroupId\":-1," +
"\"preTasks\":[\"dependent\"]}";
JSONObject taskNode = JSONUtils.parseObject(sqlJson);
if (StringUtils.isNotEmpty(taskNode.getString("type"))) {
String taskType = taskNode.getString("type");
ProcessAddTaskParam addTaskParam = TaskNodeParamFactory.getByTaskType(taskType);
JSONObject sql = addTaskParam.addExportSpecialParam(taskNode);
JSONAssert.assertEquals(taskNode.toString(), sql.toString(), false);
}
}
@Test
public void testAddImportDependentSpecialParam() throws JSONException {
String sqlJson = "{\"workerGroupId\":-1,\"description\":\"\",\"runFlag\":\"NORMAL\"," +
"\"type\":\"SQL\",\"params\":{\"postStatements\":[]," +
"\"connParams\":\"\",\"receiversCc\":\"\",\"udfs\":\"\"," +
"\"type\":\"MYSQL\",\"title\":\"\",\"sql\":\"show tables\",\"" +
"preStatements\":[],\"sqlType\":\"1\",\"receivers\":\"\",\"datasource\":1," +
"\"showType\":\"TABLE\",\"localParams\":[],\"datasourceName\":\"dsmetadata\"},\"timeout\"" +
":{\"enable\":false,\"strategy\":\"\"},\"maxRetryTimes\":\"0\"," +
"\"taskInstancePriority\":\"MEDIUM\",\"name\":\"mysql\",\"dependence\":{}," +
"\"retryInterval\":\"1\",\"preTasks\":[\"dependent\"],\"id\":\"tasks-8745\"}";
JSONObject taskNode = JSONUtils.parseObject(sqlJson);
if (StringUtils.isNotEmpty(taskNode.getString("type"))) {
String taskType = taskNode.getString("type");
ProcessAddTaskParam addTaskParam = TaskNodeParamFactory.getByTaskType(taskType);
JSONObject sql = addTaskParam.addImportSpecialParam(taskNode);
JSONAssert.assertEquals(taskNode.toString(), sql.toString(), false);
}
}
}

112
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/exportprocess/DependentParamTest.java

@ -0,0 +1,112 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.api.utils.exportprocess;
import com.alibaba.fastjson.JSONObject;
import org.apache.dolphinscheduler.api.ApiApplicationServer;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.json.JSONException;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.skyscreamer.jsonassert.JSONAssert;
import org.springframework.boot.test.context.SpringBootTest;
import org.springframework.test.context.junit4.SpringRunner;
/**
* DependentParamTest
*/
@RunWith(SpringRunner.class)
@SpringBootTest(classes = ApiApplicationServer.class)
public class DependentParamTest {
@Test
public void testAddExportDependentSpecialParam() throws JSONException {
String dependentJson = "{\"type\":\"DEPENDENT\",\"id\":\"tasks-33787\"," +
"\"name\":\"dependent\",\"params\":{},\"description\":\"\",\"runFlag\":\"NORMAL\"," +
"\"dependence\":{\"relation\":\"AND\",\"dependTaskList\":[{\"relation\":\"AND\"," +
"\"dependItemList\":[{\"projectId\":2,\"definitionId\":46,\"depTasks\":\"ALL\"," +
"\"cycle\":\"day\",\"dateValue\":\"today\"}]}]}}";
JSONObject taskNode = JSONUtils.parseObject(dependentJson);
if (StringUtils.isNotEmpty(taskNode.getString("type"))) {
String taskType = taskNode.getString("type");
ProcessAddTaskParam addTaskParam = TaskNodeParamFactory.getByTaskType(taskType);
JSONObject dependent = addTaskParam.addExportSpecialParam(taskNode);
JSONAssert.assertEquals(taskNode.toString(), dependent.toString(), false);
}
String dependentEmpty = "{\"type\":\"DEPENDENT\",\"id\":\"tasks-33787\"," +
"\"name\":\"dependent\",\"params\":{},\"description\":\"\",\"runFlag\":\"NORMAL\"}";
JSONObject taskEmpty = JSONUtils.parseObject(dependentEmpty);
if (StringUtils.isNotEmpty(taskEmpty.getString("type"))) {
String taskType = taskEmpty.getString("type");
ProcessAddTaskParam addTaskParam = TaskNodeParamFactory.getByTaskType(taskType);
JSONObject dependent = addTaskParam.addImportSpecialParam(taskEmpty);
JSONAssert.assertEquals(taskEmpty.toString(), dependent.toString(), false);
}
}
@Test
public void testAddImportDependentSpecialParam() throws JSONException {
String dependentJson = "{\"workerGroupId\":-1,\"description\":\"\",\"runFlag\":\"NORMAL\"" +
",\"type\":\"DEPENDENT\",\"params\":{},\"timeout\":{\"enable\":false," +
"\"strategy\":\"\"},\"maxRetryTimes\":\"0\",\"taskInstancePriority\":\"MEDIUM\"" +
",\"name\":\"dependent\"," +
"\"dependence\":{\"dependTaskList\":[{\"dependItemList\":[{\"dateValue\":\"today\"," +
"\"definitionName\":\"shell-1\",\"depTasks\":\"shell-1\",\"projectName\":\"test\"," +
"\"projectId\":1,\"cycle\":\"day\",\"definitionId\":7}],\"relation\":\"AND\"}]," +
"\"relation\":\"AND\"},\"retryInterval\":\"1\",\"preTasks\":[],\"id\":\"tasks-55485\"}";
JSONObject taskNode = JSONUtils.parseObject(dependentJson);
if (StringUtils.isNotEmpty(taskNode.getString("type"))) {
String taskType = taskNode.getString("type");
ProcessAddTaskParam addTaskParam = TaskNodeParamFactory.getByTaskType(taskType);
JSONObject dependent = addTaskParam.addImportSpecialParam(taskNode);
JSONAssert.assertEquals(taskNode.toString(), dependent.toString(), false);
}
String dependentEmpty = "{\"workerGroupId\":-1,\"description\":\"\",\"runFlag\":\"NORMAL\"" +
",\"type\":\"DEPENDENT\",\"params\":{},\"timeout\":{\"enable\":false," +
"\"strategy\":\"\"},\"maxRetryTimes\":\"0\",\"taskInstancePriority\":\"MEDIUM\"" +
",\"name\":\"dependent\",\"retryInterval\":\"1\",\"preTasks\":[],\"id\":\"tasks-55485\"}";
JSONObject taskNodeEmpty = JSONUtils.parseObject(dependentEmpty);
if (StringUtils.isNotEmpty(taskNodeEmpty.getString("type"))) {
String taskType = taskNodeEmpty.getString("type");
ProcessAddTaskParam addTaskParam = TaskNodeParamFactory.getByTaskType(taskType);
JSONObject dependent = addTaskParam.addImportSpecialParam(taskNode);
JSONAssert.assertEquals(taskNodeEmpty.toString(), dependent.toString(), false);
}
}
}

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

@ -471,6 +471,8 @@ public final class Constants {
public static final String PASSWORD = "password";
public static final String XXXXXX = "******";
public static final String NULL = "NULL";
public static final String THREAD_NAME_MASTER_SERVER = "Master-Server";
public static final String THREAD_NAME_WORKER_SERVER = "Worker-Server";
public static String TASK_RECORD_TABLE_HIVE_LOG = "eamp_hive_log_hd";

7
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/ITaskQueue.java

@ -30,6 +30,13 @@ public interface ITaskQueue {
*/
List<String> getAllTasks(String key);
/**
* check if has a task
* @param key queue name
* @return true if has; false if not
*/
boolean hasTask(String key);
/**
* check task exists in the task queue or not
*

15
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/TaskQueueZkImpl.java

@ -75,6 +75,21 @@ public class TaskQueueZkImpl implements ITaskQueue {
return new ArrayList<>();
}
/**
* check if has a task
* @param key queue name
* @return true if has; false if not
*/
@Override
public boolean hasTask(String key) {
try {
return zookeeperOperator.hasChildren(getTasksPath(key));
} catch (Exception e) {
logger.error("check has task in tasks queue exception",e);
}
return false;
}
/**
* check task exists in the task queue or not
*

2
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/thread/Stopper.java

@ -25,7 +25,7 @@ public class Stopper {
private static volatile AtomicBoolean signal = new AtomicBoolean(false);
public static final boolean isStoped(){
public static final boolean isStopped(){
return signal.get();
}

84
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/AbstractZKClient.java

@ -16,18 +16,6 @@
*/
package org.apache.dolphinscheduler.common.zk;
import static org.apache.dolphinscheduler.common.Constants.ADD_ZK_OP;
import static org.apache.dolphinscheduler.common.Constants.DELETE_ZK_OP;
import static org.apache.dolphinscheduler.common.Constants.MASTER_PREFIX;
import static org.apache.dolphinscheduler.common.Constants.SINGLE_SLASH;
import static org.apache.dolphinscheduler.common.Constants.UNDERLINE;
import static org.apache.dolphinscheduler.common.Constants.WORKER_PREFIX;
import java.util.ArrayList;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.imps.CuratorFrameworkState;
import org.apache.curator.framework.recipes.locks.InterProcessMutex;
@ -42,6 +30,8 @@ import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.*;
import static org.apache.dolphinscheduler.common.Constants.*;
/**
* abstract zookeeper client
@ -84,7 +74,7 @@ public abstract class AbstractZKClient extends ZookeeperCachedOperator{
zkClient.setData().forPath(znode,str.getBytes());
} catch (Exception e) {
logger.error("heartbeat for zk failed : " + e.getMessage(), e);
logger.error("heartbeat for zk failed", e);
stoppable.stop("heartbeat for zk exception, release resources and stop myself");
}
}
@ -118,28 +108,26 @@ public abstract class AbstractZKClient extends ZookeeperCachedOperator{
List<String> deadServers = super.getChildrenKeys(getDeadZNodeParentPath());
for(String serverPath : deadServers){
if(serverPath.startsWith(serverType+UNDERLINE+host)){
String server = getDeadZNodeParentPath() + SINGLE_SLASH + serverPath;
super.remove(server);
logger.info("{} server {} deleted from zk dead server path success" , serverType , host);
String server = getDeadZNodeParentPath() + SINGLE_SLASH + serverPath;
super.remove(server);
logger.info("{} server {} deleted from zk dead server path success" , serverType , host);
}
}
}
/**
* create zookeeper path according the zk node type.
* @param zkNodeType zookeeper node type
* @return register zookeeper path
* @throws Exception
*/
private String createZNodePath(ZKNodeType zkNodeType) throws Exception {
private String createZNodePath(ZKNodeType zkNodeType, String host) throws Exception {
// specify the format of stored data in ZK nodes
String heartbeatZKInfo = ResInfo.getHeartBeatInfo(new Date());
// create temporary sequence nodes for master znode
String parentPath = getZNodeParentPath(zkNodeType);
String serverPathPrefix = parentPath + "/" + OSUtils.getHost();
String registerPath = serverPathPrefix + UNDERLINE;
super.persistEphemeral(registerPath, heartbeatZKInfo);
String registerPath= getZNodeParentPath(zkNodeType) + SINGLE_SLASH + host;
super.persistEphemeral(registerPath, heartbeatZKInfo);
logger.info("register {} node {} success" , zkNodeType.toString(), registerPath);
return registerPath;
}
@ -158,7 +146,7 @@ public abstract class AbstractZKClient extends ZookeeperCachedOperator{
zkNodeType.toString(), host);
return registerPath;
}
registerPath = createZNodePath(zkNodeType);
registerPath = createZNodePath(zkNodeType, host);
// handle dead server
handleDeadServer(registerPath, zkNodeType, Constants.DELETE_ZK_OP);
@ -176,25 +164,19 @@ public abstract class AbstractZKClient extends ZookeeperCachedOperator{
* @throws Exception errors
*/
public void handleDeadServer(String zNode, ZKNodeType zkNodeType, String opType) throws Exception {
//ip_sequenceno
String[] zNodesPath = zNode.split("\\/");
String ipSeqNo = zNodesPath[zNodesPath.length - 1];
String host = getHostByEventDataPath(zNode);
String type = (zkNodeType == ZKNodeType.MASTER) ? MASTER_PREFIX : WORKER_PREFIX;
//check server restart, if restart , dead server path in zk should be delete
if(opType.equals(DELETE_ZK_OP)){
String[] ipAndSeqNo = ipSeqNo.split(UNDERLINE);
String ip = ipAndSeqNo[0];
removeDeadServerByHost(ip, type);
removeDeadServerByHost(host, type);
}else if(opType.equals(ADD_ZK_OP)){
String deadServerPath = getDeadZNodeParentPath() + SINGLE_SLASH + type + UNDERLINE + ipSeqNo;
String deadServerPath = getDeadZNodeParentPath() + SINGLE_SLASH + type + UNDERLINE + host;
if(!super.isExisted(deadServerPath)){
//add dead server info to zk dead server path : /dead-servers/
super.persist(deadServerPath,(type + UNDERLINE + ipSeqNo));
super.persist(deadServerPath,(type + UNDERLINE + host));
logger.info("{} server dead , and {} added to zk dead server path success" ,
zkNodeType.toString(), zNode);
@ -274,7 +256,7 @@ public abstract class AbstractZKClient extends ZookeeperCachedOperator{
masterMap.putIfAbsent(server, super.get(path + "/" + server));
}
} catch (Exception e) {
logger.error("get server list failed : " + e.getMessage(), e);
logger.error("get server list failed", e);
}
return masterMap;
@ -295,21 +277,13 @@ public abstract class AbstractZKClient extends ZookeeperCachedOperator{
}
Map<String, String> serverMaps = getServerMaps(zkNodeType);
for(String hostKey : serverMaps.keySet()){
if(hostKey.startsWith(host + UNDERLINE)){
if(hostKey.startsWith(host)){
return true;
}
}
return false;
}
/**
* get zkclient
* @return zookeeper client
*/
public CuratorFramework getZkClient() {
return zkClient;
}
/**
*
* @return get worker node parent path
@ -406,7 +380,7 @@ public abstract class AbstractZKClient extends ZookeeperCachedOperator{
if(e.getMessage().equals("instance must be started before calling this method")){
logger.warn("lock release");
}else{
logger.error("lock release failed : " + e.getMessage(),e);
logger.error("lock release failed",e);
}
}
@ -422,8 +396,9 @@ public abstract class AbstractZKClient extends ZookeeperCachedOperator{
persist(getWorkerZNodeParentPath(), "");
persist(getDeadZNodeParentPath(), "");
logger.info("initialize server nodes success.");
} catch (Exception e) {
logger.error("init system znode failed : " + e.getMessage(),e);
logger.error("init system znode failed",e);
}
}
@ -445,19 +420,22 @@ public abstract class AbstractZKClient extends ZookeeperCachedOperator{
}
/**
* get host ip, string format: masterParentPath/ip_000001/value
* get host ip, string format: masterParentPath/ip
* @param path path
* @return host ip, string format: masterParentPath/ip_000001/value
* @return host ip, string format: masterParentPath/ip
*/
protected String getHostByEventDataPath(String path) {
int startIndex = path.lastIndexOf("/")+1;
int endIndex = path.lastIndexOf("_");
if(startIndex >= endIndex){
logger.error("parse ip error");
if(StringUtils.isEmpty(path)){
logger.error("empty path!");
return "";
}
String[] pathArray = path.split(SINGLE_SLASH);
if(pathArray.length < 1){
logger.error("parse ip error: {}", path);
return "";
}
return path.substring(startIndex, endIndex);
return pathArray[pathArray.length - 1];
}
/**
* acquire zk lock

1
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperCachedOperator.java

@ -77,6 +77,7 @@ public class ZookeeperCachedOperator extends ZookeeperOperator {
return treeCache;
}
@Override
public void close() {
treeCache.close();
try {

11
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperOperator.java

@ -27,6 +27,7 @@ import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.InitializingBean;
@ -139,6 +140,16 @@ public class ZookeeperOperator implements InitializingBean {
}
}
public boolean hasChildren(final String key){
Stat stat ;
try {
stat = zkClient.checkExists().forPath(key);
return stat.getNumChildren() >= 1;
} catch (Exception ex) {
throw new IllegalStateException(ex);
}
}
public boolean isExisted(final String key) {
try {
return zkClient.checkExists().forPath(key) != null;

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

@ -63,7 +63,7 @@ login.user.keytab.username=hdfs-mycluster@ESZ.COM
login.user.keytab.path=/opt/hdfs.headless.keytab
# system env path. self configuration, please make sure the directory and file exists and have read write execute permissions
dolphinscheduler.env.path=/opt/.dolphinscheduler_env.sh
dolphinscheduler.env.path=/opt/dolphinscheduler_env.sh
#resource.view.suffixs
resource.view.suffixs=txt,log,sh,conf,cfg,py,java,sql,hql,xml,properties

6
dolphinscheduler-common/src/main/resources/quartz.properties

@ -20,10 +20,12 @@
#============================================================================
#org.quartz.jobStore.driverDelegateClass = org.quartz.impl.jdbcjobstore.StdJDBCDelegate
org.quartz.jobStore.driverDelegateClass = org.quartz.impl.jdbcjobstore.PostgreSQLDelegate
#org.quartz.dataSource.myDs.driver = com.mysql.jdbc.Driver
# postgre
org.quartz.dataSource.myDs.driver = org.postgresql.Driver
#org.quartz.dataSource.myDs.URL = jdbc:mysql://192.168.xx.xx:3306/dolphinscheduler?characterEncoding=utf8
org.quartz.dataSource.myDs.URL = jdbc:postgresql://localhost:5432/dolphinscheduler?characterEncoding=utf8
# mysql
#org.quartz.dataSource.myDs.driver = com.mysql.jdbc.Driver
#org.quartz.dataSource.myDs.URL = jdbc:mysql://localhost:3306/dolphinscheduler?characterEncoding=utf8
org.quartz.dataSource.myDs.user = test
org.quartz.dataSource.myDs.password = test

11
dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/queue/TaskQueueZKImplTest.java

@ -64,7 +64,16 @@ public class TaskQueueZKImplTest extends BaseTaskQueueTest {
allTasks = tasksQueue.getAllTasks(Constants.DOLPHINSCHEDULER_TASKS_QUEUE);
assertEquals(allTasks.size(),0);
}
@Test
public void hasTask(){
init();
boolean hasTask = tasksQueue.hasTask(Constants.DOLPHINSCHEDULER_TASKS_QUEUE);
assertTrue(hasTask);
//delete all
tasksQueue.delete();
hasTask = tasksQueue.hasTask(Constants.DOLPHINSCHEDULER_TASKS_QUEUE);
assertFalse(hasTask);
}
/**
* test check task exists in the task queue or not
*/

2
dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/threadutils/ThreadUtilsTest.java

@ -98,7 +98,7 @@ public class ThreadUtilsTest {
public void testStopper() {
assertTrue(Stopper.isRunning());
Stopper.stop();
assertTrue(Stopper.isStoped());
assertTrue(Stopper.isStopped());
}
/**

6
dolphinscheduler-dao/pom.xml

@ -49,11 +49,7 @@
<groupId>org.postgresql</groupId>
<artifactId>postgresql</artifactId>
</dependency>
<dependency>
<groupId>org.projectlombok</groupId>
<artifactId>lombok</artifactId>
<version>${lombok.version}</version>
</dependency>
<dependency>
<groupId>org.springframework.boot</groupId>
<artifactId>spring-boot-starter-test</artifactId>

5
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/ProcessDao.java

@ -457,6 +457,11 @@ public class ProcessDao {
if(tenantId >= 0){
tenant = tenantMapper.queryById(tenantId);
}
if (userId == 0){
return null;
}
if(null == tenant){
User user = userMapper.selectById(userId);

82
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/AccessToken.java

@ -25,28 +25,38 @@ import java.util.Date;
@TableName("t_ds_access_token")
public class AccessToken {
@TableId(value="id", type=IdType.AUTO)
/**
* primary key
*/
@TableId(value = "id", type = IdType.AUTO)
private int id;
@TableField("user_id")
/**
* user_id
*/
@TableField(value = "user_id")
private int userId;
@TableField("token")
/**
* token
*/
@TableField(value = "token")
private String token;
@TableField(exist = false)
private String userName;
@TableField("expire_time")
/**
* expire_time
*/
@TableField(value = "expire_time")
private Date expireTime;
@TableField("create_time")
/**
* create_time
*/
@TableField(value = "create_time")
private Date createTime;
@TableField("update_time")
/**
* update_time
*/
@TableField(value = "update_time")
private Date updateTime;
@TableField(exist = false)
private String userName;
public int getId() {
return id;
@ -106,26 +116,44 @@ public class AccessToken {
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
AccessToken that = (AccessToken) o;
if (id != that.id) return false;
if (userId != that.userId) return false;
if (token != null ? !token.equals(that.token) : that.token != null) return false;
if (userName != null ? !userName.equals(that.userName) : that.userName != null) return false;
if (expireTime != null ? !expireTime.equals(that.expireTime) : that.expireTime != null) return false;
if (createTime != null ? !createTime.equals(that.createTime) : that.createTime != null) return false;
return updateTime != null ? updateTime.equals(that.updateTime) : that.updateTime == null;
if (id != that.id) {
return false;
}
if (userId != that.userId) {
return false;
}
if (userName != null && !userName.equals(that.userName)) {
return false;
}
if (token != null && !token.equals(that.token)) {
return false;
}
if (expireTime != null && !expireTime.equals(that.expireTime)) {
return false;
}
if (createTime != null && !createTime.equals(that.createTime)) {
return false;
}
if (updateTime != null && !updateTime.equals(that.updateTime)) {
return false;
}
return true;
}
@Override
public int hashCode() {
int result = id;
result = 31 * result + userId;
result = 31 * result + (token != null ? token.hashCode() : 0);
result = 31 * result + (userName != null ? userName.hashCode() : 0);
result = 31 * result + (token != null ? token.hashCode() : 0);
result = 31 * result + (expireTime != null ? expireTime.hashCode() : 0);
result = 31 * result + (createTime != null ? createTime.hashCode() : 0);
result = 31 * result + (updateTime != null ? updateTime.hashCode() : 0);

79
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Alert.java

@ -28,96 +28,74 @@ import java.util.Date;
import java.util.HashMap;
import java.util.Map;
/**
* alert
*/
@TableName("t_ds_alert")
public class Alert {
/**
* id
* primary key
*/
@TableId(value="id", type=IdType.AUTO)
@TableId(value = "id", type = IdType.AUTO)
private int id;
/**
* title
*/
@TableField(value = "title")
private String title;
/**
* show type
* show_type
*/
@TableField(value = "show_type")
private ShowType showType;
/**
* content
*/
@TableField(value = "content")
private String content;
/**
* alert type
* alert_type
*/
@TableField(value = "alert_type")
private AlertType alertType;
/**
* alert status
* alert_status
*/
@TableField(value = "alert_status")
private AlertStatus alertStatus;
/**
* log
*/
@TableField(value = "log")
private String log;
/**
* alert group id
* alertgroup_id
*/
@TableField("alertgroup_id")
private int alertGroupId;
/**
* receivers
*/
@TableField("receivers")
private String receivers;
/**
* receivers cc
* receivers_cc
*/
@TableField("receivers_cc")
private String receiversCc;
/**
* create time
* create_time
*/
@TableField("create_time")
private Date createTime;
/**
* update time
* update_time
*/
@TableField("update_time")
private Date updateTime;
@TableField(exist = false)
private Map<String,Object> info = new HashMap<>();
public Map<String, Object> getInfo() {
return info;
}
public void setInfo(Map<String, Object> info) {
this.info = info;
}
private Map<String, Object> info = new HashMap<>();
public Alert() {
}
public Alert(int id, String title) {
this.id = id;
this.title = title;
}
public int getId() {
return id;
}
@ -158,7 +136,6 @@ public class Alert {
this.alertType = alertType;
}
public AlertStatus getAlertStatus() {
return alertStatus;
}
@ -183,10 +160,6 @@ public class Alert {
this.alertGroupId = alertGroupId;
}
public Date getCreateTime() {
return createTime;
}
public String getReceivers() {
return receivers;
}
@ -203,6 +176,10 @@ public class Alert {
this.receiversCc = receiversCc;
}
public Date getCreateTime() {
return createTime;
}
public void setCreateTime(Date createTime) {
this.createTime = createTime;
}
@ -215,6 +192,14 @@ public class Alert {
this.updateTime = updateTime;
}
public Map<String, Object> getInfo() {
return info;
}
public void setInfo(Map<String, Object> info) {
this.info = info;
}
@Override
public boolean equals(Object o) {
if (this == o) {

25
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/AlertGroup.java

@ -16,6 +16,7 @@
*/
package org.apache.dolphinscheduler.dao.entity;
import com.baomidou.mybatisplus.annotation.TableField;
import org.apache.dolphinscheduler.common.enums.AlertType;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableId;
@ -25,37 +26,35 @@ import java.util.Date;
@TableName("t_ds_alertgroup")
public class AlertGroup {
/**
* primary key
*/
@TableId(value="id", type=IdType.AUTO)
@TableId(value = "id", type = IdType.AUTO)
private int id;
/**
* alert group name
* group_name
*/
@TableField(value = "group_name")
private String groupName;
/**
* alert group type
* group_type
*/
@TableField(value = "group_type")
private AlertType groupType;
/**
* alert group description
* description
*/
@TableField(value = "description")
private String description;
/**
* create time
* create_time
*/
@TableField(value = "create_time")
private Date createTime;
/**
* update time
* update_time
*/
@TableField(value = "update_time")
private Date updateTime;
public int getId() {

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

@ -24,6 +24,7 @@ import com.baomidou.mybatisplus.core.toolkit.StringUtils;
import org.apache.dolphinscheduler.common.enums.*;
import java.util.Date;
import java.util.Objects;
/**
* process instance
@ -625,90 +626,11 @@ public class ProcessInstance {
ProcessInstance that = (ProcessInstance) o;
if (id != that.id) return false;
if (processDefinitionId != that.processDefinitionId) return false;
if (runTimes != that.runTimes) return false;
if (maxTryTimes != that.maxTryTimes) return false;
if (executorId != that.executorId) return false;
if (workerGroupId != that.workerGroupId) return false;
if (timeout != that.timeout) return false;
if (tenantId != that.tenantId) return false;
if (state != that.state) return false;
if (recovery != that.recovery) return false;
if (startTime != null ? !startTime.equals(that.startTime) : that.startTime != null) return false;
if (endTime != null ? !endTime.equals(that.endTime) : that.endTime != null) return false;
if (name != null ? !name.equals(that.name) : that.name != null) return false;
if (host != null ? !host.equals(that.host) : that.host != null) return false;
if (processDefinition != null ? !processDefinition.equals(that.processDefinition) : that.processDefinition != null)
return false;
if (commandType != that.commandType) return false;
if (commandParam != null ? !commandParam.equals(that.commandParam) : that.commandParam != null) return false;
if (taskDependType != that.taskDependType) return false;
if (failureStrategy != that.failureStrategy) return false;
if (warningType != that.warningType) return false;
if (warningGroupId != null ? !warningGroupId.equals(that.warningGroupId) : that.warningGroupId != null)
return false;
if (scheduleTime != null ? !scheduleTime.equals(that.scheduleTime) : that.scheduleTime != null) return false;
if (commandStartTime != null ? !commandStartTime.equals(that.commandStartTime) : that.commandStartTime != null)
return false;
if (globalParams != null ? !globalParams.equals(that.globalParams) : that.globalParams != null) return false;
if (processInstanceJson != null ? !processInstanceJson.equals(that.processInstanceJson) : that.processInstanceJson != null)
return false;
if (tenantCode != null ? !tenantCode.equals(that.tenantCode) : that.tenantCode != null) return false;
if (queue != null ? !queue.equals(that.queue) : that.queue != null) return false;
if (isSubProcess != that.isSubProcess) return false;
if (locations != null ? !locations.equals(that.locations) : that.locations != null) return false;
if (connects != null ? !connects.equals(that.connects) : that.connects != null) return false;
if (historyCmd != null ? !historyCmd.equals(that.historyCmd) : that.historyCmd != null) return false;
if (dependenceScheduleTimes != null ? !dependenceScheduleTimes.equals(that.dependenceScheduleTimes) : that.dependenceScheduleTimes != null)
return false;
if (duration != null ? !duration.equals(that.duration) : that.duration != null) return false;
if (processInstancePriority != that.processInstancePriority) return false;
if (workerGroupName != null ? !workerGroupName.equals(that.workerGroupName) : that.workerGroupName != null)
return false;
if (receivers != null ? !receivers.equals(that.receivers) : that.receivers != null) return false;
return receiversCc != null ? receiversCc.equals(that.receiversCc) : that.receiversCc == null;
return id == that.id;
}
@Override
public int hashCode() {
int result = id;
result = 31 * result + processDefinitionId;
result = 31 * result + (state != null ? state.hashCode() : 0);
result = 31 * result + (recovery != null ? recovery.hashCode() : 0);
result = 31 * result + (startTime != null ? startTime.hashCode() : 0);
result = 31 * result + (endTime != null ? endTime.hashCode() : 0);
result = 31 * result + runTimes;
result = 31 * result + (name != null ? name.hashCode() : 0);
result = 31 * result + (host != null ? host.hashCode() : 0);
result = 31 * result + (processDefinition != null ? processDefinition.hashCode() : 0);
result = 31 * result + (commandType != null ? commandType.hashCode() : 0);
result = 31 * result + (commandParam != null ? commandParam.hashCode() : 0);
result = 31 * result + (taskDependType != null ? taskDependType.hashCode() : 0);
result = 31 * result + maxTryTimes;
result = 31 * result + (failureStrategy != null ? failureStrategy.hashCode() : 0);
result = 31 * result + (warningType != null ? warningType.hashCode() : 0);
result = 31 * result + (warningGroupId != null ? warningGroupId.hashCode() : 0);
result = 31 * result + (scheduleTime != null ? scheduleTime.hashCode() : 0);
result = 31 * result + (commandStartTime != null ? commandStartTime.hashCode() : 0);
result = 31 * result + (globalParams != null ? globalParams.hashCode() : 0);
result = 31 * result + (processInstanceJson != null ? processInstanceJson.hashCode() : 0);
result = 31 * result + executorId;
result = 31 * result + (tenantCode != null ? tenantCode.hashCode() : 0);
result = 31 * result + (queue != null ? queue.hashCode() : 0);
result = 31 * result + (isSubProcess != null ? isSubProcess.hashCode() : 0);
result = 31 * result + (locations != null ? locations.hashCode() : 0);
result = 31 * result + (connects != null ? connects.hashCode() : 0);
result = 31 * result + (historyCmd != null ? historyCmd.hashCode() : 0);
result = 31 * result + (dependenceScheduleTimes != null ? dependenceScheduleTimes.hashCode() : 0);
result = 31 * result + (duration != null ? duration.hashCode() : 0);
result = 31 * result + (processInstancePriority != null ? processInstancePriority.hashCode() : 0);
result = 31 * result + workerGroupId;
result = 31 * result + timeout;
result = 31 * result + tenantId;
result = 31 * result + (workerGroupName != null ? workerGroupName.hashCode() : 0);
result = 31 * result + (receivers != null ? receivers.hashCode() : 0);
result = 31 * result + (receiversCc != null ? receiversCc.hashCode() : 0);
return result;
return Objects.hash(id);
}
}

2
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProjectUser.java

@ -20,11 +20,9 @@ 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 lombok.Data;
import java.util.Date;
@Data
@TableName("t_ds_relation_project_user")
public class ProjectUser {
/**

22
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Queue.java

@ -19,14 +19,12 @@ package org.apache.dolphinscheduler.dao.entity;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
import lombok.Data;
import java.util.Date;
/**
* queue
*/
@Data
@TableName("t_ds_queue")
public class Queue {
@ -103,4 +101,24 @@ public class Queue {
", updateTime=" + updateTime +
'}';
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Queue queue1 = (Queue) o;
if (id != queue1.id) return false;
if (!queueName.equals(queue1.queueName)) return false;
return queue.equals(queue1.queue);
}
@Override
public int hashCode() {
int result = id;
result = 31 * result + queueName.hashCode();
result = 31 * result + queue.hashCode();
return result;
}
}

2
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Resource.java

@ -21,11 +21,9 @@ import org.apache.dolphinscheduler.common.enums.ResourceType;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
import lombok.Data;
import java.util.Date;
@Data
@TableName("t_ds_resources")
public class Resource {
/**

2
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ResourcesUser.java

@ -19,14 +19,12 @@ package org.apache.dolphinscheduler.dao.entity;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
import lombok.Data;
import java.util.Date;
/**
* resource user relation
*/
@Data
@TableName("t_ds_relation_resources_user")
public class ResourcesUser {

24
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Session.java

@ -19,14 +19,12 @@ package org.apache.dolphinscheduler.dao.entity;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
import lombok.Data;
import java.util.Date;
/**
* session
*/
@Data
@TableName("t_ds_session")
public class Session {
@ -92,4 +90,26 @@ public class Session {
", lastLoginTime=" + lastLoginTime +
'}';
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Session session = (Session) o;
if (userId != session.userId) return false;
if (!id.equals(session.id)) return false;
if (!lastLoginTime.equals(session.lastLoginTime)) return false;
return ip.equals(session.ip);
}
@Override
public int hashCode() {
int result = id.hashCode();
result = 31 * result + userId;
result = 31 * result + lastLoginTime.hashCode();
result = 31 * result + ip.hashCode();
return result;
}
}

18
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Tenant.java

@ -20,14 +20,13 @@ 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 lombok.Data;
import java.util.Date;
import java.util.Objects;
/**
* tenant
*/
@Data
@TableName("t_ds_tenant")
public class Tenant {
@ -164,4 +163,19 @@ public class Tenant {
public void setDescription(String description) {
this.description = description;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Tenant tenant = (Tenant) o;
return id == tenant.id;
}
@Override
public int hashCode() {
return Objects.hash(id);
}
}

2
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/UDFUser.java

@ -19,14 +19,12 @@ package org.apache.dolphinscheduler.dao.entity;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
import lombok.Data;
import java.util.Date;
/**
* udf user relation
*/
@Data
@TableName("t_ds_relation_udfs_user")
public class UDFUser {

2
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/UdfFunc.java

@ -21,14 +21,12 @@ import org.apache.dolphinscheduler.common.enums.UdfType;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
import lombok.Data;
import java.util.Date;
/**
* udf function
*/
@Data
@TableName("t_ds_udfs")
public class UdfFunc {
/**

39
dolphinscheduler-dao/src/main/resources/application.properties

@ -95,6 +95,45 @@ mybatis-plus.configuration.cache-enabled=false
mybatis-plus.configuration.call-setters-on-nulls=true
mybatis-plus.configuration.jdbc-type-for-null=null
# master settings
# master execute thread num
master.exec.threads=100
# master execute task number in parallel
master.exec.task.num=20
# master heartbeat interval
master.heartbeat.interval=10
# master commit task retry times
master.task.commit.retryTimes=5
# master commit task interval
master.task.commit.interval=1000
# only less than cpu avg load, master server can work. default value : the number of cpu cores * 2
master.max.cpuload.avg=100
# only larger than reserved memory, master server can work. default value : physical memory * 1/10, unit is G.
master.reserved.memory=0.1
# worker settings
# worker execute thread num
worker.exec.threads=100
# worker heartbeat interval
worker.heartbeat.interval=10
# submit the number of tasks at a time
worker.fetch.task.num = 3
# only less than cpu avg load, worker server can work. default value : the number of cpu cores * 2
worker.max.cpuload.avg=100
# only larger than reserved memory, worker server can work. default value : physical memory * 1/6, unit is G.
worker.reserved.memory=0.1
# data quality analysis is not currently in use. please ignore the following configuration
# task record
task.record.flag=false

39
dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/AccessTokenMapperTest.java

@ -37,9 +37,7 @@ import java.util.*;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.greaterThan;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.*;
/**
* AccessToken mapper test
@ -77,14 +75,34 @@ public class AccessTokenMapperTest {
@Test
public void testSelectById() throws Exception{
Integer userId = 1;
AccessToken accessToken = createAccessToken(userId);
AccessToken resultAccessToken
= accessTokenMapper.selectById(accessToken.getId());
AccessToken resultAccessToken = accessTokenMapper.selectById(accessToken.getId());
assertEquals(accessToken, resultAccessToken);
}
/**
* test hashCode method
* @throws Exception
*/
@Test
public void testHashCodeMethod() throws Exception {
Integer userId = 1;
AccessToken accessToken = createAccessToken(userId);
AccessToken resultAccessToken = accessTokenMapper.selectById(accessToken.getId());
boolean flag = accessToken.equals(resultAccessToken);
assertTrue(flag);
}
/**
* test equals method
* @throws Exception
*/
@Test
public void testEqualsMethod() throws Exception {
Integer userId = 1;
AccessToken accessToken = createAccessToken(userId);
int result = accessToken.hashCode();
assertNotNull(result);
}
/**
@ -98,7 +116,6 @@ public class AccessTokenMapperTest {
Integer offset = 2;
Integer size = 2;
Map<Integer, AccessToken> accessTokenMap = createAccessTokens(count, userName);
Page page = new Page(offset, size);
@ -106,7 +123,6 @@ public class AccessTokenMapperTest {
assertEquals(Integer.valueOf(accessTokenPage.getRecords().size()),size);
for (AccessToken accessToken : accessTokenPage.getRecords()){
AccessToken resultAccessToken = accessTokenMap.get(accessToken.getId());
assertEquals(accessToken,resultAccessToken);
@ -120,15 +136,12 @@ public class AccessTokenMapperTest {
@Test
public void testUpdate() throws Exception{
Integer userId = 1;
AccessToken accessToken = createAccessToken(userId);
//update
accessToken.setToken("56789");
accessToken.setExpireTime(DateUtils.getCurrentDate());
accessToken.setUpdateTime(DateUtils.getCurrentDate());
accessTokenMapper.updateById(accessToken);
AccessToken resultAccessToken = accessTokenMapper.selectById(accessToken.getId());
assertEquals(accessToken, resultAccessToken);
}

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

@ -97,7 +97,9 @@ public class MasterServer implements IStoppable {
* @param args arguments
*/
public static void main(String[] args) {
Thread.currentThread().setName(Constants.THREAD_NAME_MASTER_SERVER);
new SpringApplicationBuilder(MasterServer.class).web(WebApplicationType.NONE).run(args);
}
/**
@ -133,6 +135,7 @@ public class MasterServer implements IStoppable {
// start QuartzExecutors
// what system should do if exception
try {
logger.info("start Quartz server...");
ProcessScheduleJob.init(processDao);
QuartzExecutors.getInstance().start();
} catch (Exception e) {
@ -170,7 +173,7 @@ public class MasterServer implements IStoppable {
try {
//execute only once
if(Stopper.isStoped()){
if(Stopper.isStopped()){
return;
}
@ -239,6 +242,7 @@ public class MasterServer implements IStoppable {
* @return
*/
private Runnable heartBeatThread(){
logger.info("start master heart beat thread...");
Runnable heartBeatThread = new Runnable() {
@Override
public void run() {

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

@ -22,25 +22,25 @@ import org.springframework.stereotype.Component;
@Component
public class MasterConfig {
@Value("${master.exec.threads:100}")
@Value("${master.exec.threads}")
private int masterExecThreads;
@Value("${master.exec.task.num:20}")
@Value("${master.exec.task.num}")
private int masterExecTaskNum;
@Value("${master.heartbeat.interval:10}")
@Value("${master.heartbeat.interval}")
private int masterHeartbeatInterval;
@Value("${master.task.commit.retryTimes:5}")
@Value("${master.task.commit.retryTimes}")
private int masterTaskCommitRetryTimes;
@Value("${master.task.commit.interval:100}")
@Value("${master.task.commit.interval}")
private int masterTaskCommitInterval;
@Value("${master.max.cpuload.avg:100}")
@Value("${master.max.cpuload.avg}")
private double masterMaxCpuloadAvg;
@Value("${master.reserved.memory:0.1}")
@Value("${master.reserved.memory}")
private double masterReservedMemory;
public int getMasterExecThreads() {

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/log/MasterLogFilter.java

@ -37,7 +37,7 @@ public class MasterLogFilter extends Filter<ILoggingEvent> {
*/
@Override
public FilterReply decide(ILoggingEvent event) {
if (event.getThreadName().startsWith("Master-")){
if (event.getThreadName().startsWith("Master-") ){
return FilterReply.ACCEPT;
}
return FilterReply.DENY;

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java

@ -90,6 +90,7 @@ public class MasterSchedulerThread implements Runnable {
*/
@Override
public void run() {
logger.info("master scheduler start successfully...");
while (Stopper.isRunning()){
// process instance
@ -139,6 +140,7 @@ public class MasterSchedulerThread implements Runnable {
AbstractZKClient.releaseMutex(mutex);
}
}
logger.info("master server stopped...");
}

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/monitor/RunConfig.java

@ -24,7 +24,7 @@ import org.springframework.stereotype.Component;
* zookeeper conf
*/
@Component
@PropertySource("classpath:config/run_config.conf")
@PropertySource("classpath:config/install_config.conf")
public class RunConfig {
//zk connect config

11
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java

@ -129,6 +129,7 @@ public class WorkerServer implements IStoppable {
* @param args arguments
*/
public static void main(String[] args) {
Thread.currentThread().setName(Constants.THREAD_NAME_WORKER_SERVER);
new SpringApplicationBuilder(WorkerServer.class).web(WebApplicationType.NONE).run(args);
}
@ -138,6 +139,8 @@ public class WorkerServer implements IStoppable {
*/
@PostConstruct
public void run(){
logger.info("start worker server...");
zkWorkerClient.init();
this.taskQueue = TaskQueueFactory.getTaskQueueInstance();
@ -198,7 +201,7 @@ public class WorkerServer implements IStoppable {
try {
//execute only once
if(Stopper.isStoped()){
if(Stopper.isStopped()){
return;
}
@ -211,7 +214,7 @@ public class WorkerServer implements IStoppable {
//thread sleep 3 seconds for thread quitely stop
Thread.sleep(3000L);
}catch (Exception e){
logger.warn("thread sleep exception:" + e.getMessage(), e);
logger.warn("thread sleep exception", e);
}
try {
@ -252,7 +255,7 @@ public class WorkerServer implements IStoppable {
logger.info("zookeeper service stopped");
} catch (Exception e) {
logger.error("worker server stop exception : " + e.getMessage(), e);
logger.error("worker server stop exception ", e);
System.exit(-1);
}
}
@ -264,6 +267,7 @@ public class WorkerServer implements IStoppable {
* @return
*/
private Runnable heartBeatThread(){
logger.info("start worker heart beat thread...");
Runnable heartBeatThread = new Runnable() {
@Override
public void run() {
@ -288,6 +292,7 @@ public class WorkerServer implements IStoppable {
Runnable killProcessThread = new Runnable() {
@Override
public void run() {
logger.info("start listening kill process thread...");
while (Stopper.isRunning()){
Set<String> taskInfoSet = taskQueue.smembers(Constants.DOLPHINSCHEDULER_TASKS_KILL);
if (CollectionUtils.isNotEmpty(taskInfoSet)){

10
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/config/WorkerConfig.java

@ -22,19 +22,19 @@ import org.springframework.stereotype.Component;
@Component
public class WorkerConfig {
@Value("${worker.exec.threads:100}")
@Value("${worker.exec.threads}")
private int workerExecThreads;
@Value("${worker.heartbeat.interval:10}")
@Value("${worker.heartbeat.interval}")
private int workerHeartbeatInterval;
@Value("${worker.fetch.task.num:3}")
@Value("${worker.fetch.task.num}")
private int workerFetchTaskNum;
@Value("${worker.max.cpuload.avg:10}")
@Value("${worker.max.cpuload.avg}")
private int workerMaxCpuloadAvg;
@Value("${master.reserved.memory:1}")
@Value("${master.reserved.memory}")
private double workerReservedMemory;
public int getWorkerExecThreads() {

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/log/SensitiveDataConverter.java

@ -19,7 +19,6 @@ package org.apache.dolphinscheduler.server.worker.log;
import ch.qos.logback.classic.pattern.MessageConverter;
import ch.qos.logback.classic.spi.ILoggingEvent;
import lombok.extern.slf4j.Slf4j;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.server.utils.SensitiveLogUtil;
@ -30,7 +29,6 @@ import java.util.regex.Pattern;
/**
* sensitive data log converter
*/
@Slf4j
public class SensitiveDataConverter extends MessageConverter {
/**

47
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/FetchTaskThread.java

@ -18,6 +18,7 @@ package org.apache.dolphinscheduler.server.worker.runner;
import org.apache.curator.framework.recipes.locks.InterProcessMutex;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.queue.ITaskQueue;
import org.apache.dolphinscheduler.common.thread.Stopper;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
@ -136,8 +137,10 @@ public class FetchTaskThread implements Runnable{
@Override
public void run() {
logger.info("worker start fetch tasks...");
while (Stopper.isRunning()){
InterProcessMutex mutex = null;
String currentTaskQueueStr = null;
try {
ThreadPoolExecutor poolExecutor = (ThreadPoolExecutor) workerExecService;
//check memory and cpu usage and threads
@ -149,8 +152,9 @@ public class FetchTaskThread implements Runnable{
}
//whether have tasks, if no tasks , no need lock //get all tasks
List<String> tasksQueueList = taskQueue.getAllTasks(Constants.DOLPHINSCHEDULER_TASKS_QUEUE);
if (CollectionUtils.isEmpty(tasksQueueList)){
boolean hasTask = taskQueue.hasTask(Constants.DOLPHINSCHEDULER_TASKS_QUEUE);
if (!hasTask){
Thread.sleep(Constants.SLEEP_TIME_MILLIS);
continue;
}
@ -163,6 +167,9 @@ public class FetchTaskThread implements Runnable{
List<String> taskQueueStrArr = taskQueue.poll(Constants.DOLPHINSCHEDULER_TASKS_QUEUE, taskNum);
for(String taskQueueStr : taskQueueStrArr){
currentTaskQueueStr = taskQueueStr;
if (StringUtils.isEmpty(taskQueueStr)) {
continue;
}
@ -182,7 +189,7 @@ public class FetchTaskThread implements Runnable{
// verify task instance is null
if (verifyTaskInstanceIsNull(taskInstance)) {
logger.warn("remove task queue : {} due to taskInstance is null", taskQueueStr);
removeNodeFromTaskQueue(taskQueueStr);
processErrorTask(taskQueueStr);
continue;
}
@ -190,13 +197,17 @@ public class FetchTaskThread implements Runnable{
continue;
}
Tenant tenant = processDao.getTenantForProcess(taskInstance.getProcessInstance().getTenantId(),
taskInstance.getProcessDefine().getUserId());
// if process definition is null ,process definition already deleted
int userId = taskInstance.getProcessDefine() == null ? 0 : taskInstance.getProcessDefine().getUserId();
Tenant tenant = processDao.getTenantForProcess(
taskInstance.getProcessInstance().getTenantId(),
userId);
// verify tenant is null
if (verifyTenantIsNull(tenant)) {
logger.warn("remove task queue : {} due to tenant is null", taskQueueStr);
removeNodeFromTaskQueue(taskQueueStr);
processErrorTask(taskQueueStr);
continue;
}
@ -230,6 +241,7 @@ public class FetchTaskThread implements Runnable{
}
}catch (Exception e){
processErrorTask(currentTaskQueueStr);
logger.error("fetch task thread failure" ,e);
}finally {
AbstractZKClient.releaseMutex(mutex);
@ -237,6 +249,26 @@ public class FetchTaskThread implements Runnable{
}
}
/**
* process error task
*
* @param taskQueueStr task queue str
*/
private void processErrorTask(String taskQueueStr){
// remove from zk
removeNodeFromTaskQueue(taskQueueStr);
if (taskInstance != null){
processDao.changeTaskState(ExecutionStatus.FAILURE,
taskInstance.getStartTime(),
taskInstance.getHost(),
null,
null,
taskInstId);
}
}
/**
* remove node from task queue
*
@ -267,8 +299,7 @@ public class FetchTaskThread implements Runnable{
*/
private boolean verifyTenantIsNull(Tenant tenant) {
if(tenant == null){
logger.error("tenant not exists,process define id : {},process instance id : {},task instance id : {}",
taskInstance.getProcessDefine().getId(),
logger.error("tenant not exists,process instance id : {},task instance id : {}",
taskInstance.getProcessInstance().getId(),
taskInstance.getId());
return true;

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

@ -292,7 +292,7 @@ public abstract class AbstractCommandExecutor {
Runtime.getRuntime().exec(cmd);
} catch (IOException e) {
logger.info("kill attempt failed." + e.getMessage(), e);
logger.info("kill attempt failed", e);
}
}
@ -312,7 +312,7 @@ public abstract class AbstractCommandExecutor {
Runtime.getRuntime().exec(cmd);
} catch (IOException e) {
logger.error("kill attempt failed." + e.getMessage(), e);
logger.error("kill attempt failed ", e);
}
}
}
@ -407,7 +407,7 @@ public abstract class AbstractCommandExecutor {
}
}
} catch (Exception e) {
logger.error(String.format("yarn applications: %s status failed : " + e.getMessage(), appIds.toString()),e);
logger.error(String.format("yarn applications: %s status failed ", appIds.toString()),e);
result = false;
}
return result;

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

@ -84,6 +84,9 @@ public class ZKMasterClient extends AbstractZKClient {
* init
*/
public void init(){
logger.info("initialize master client...");
// init dao
this.initDao();
@ -107,7 +110,7 @@ public class ZKMasterClient extends AbstractZKClient {
}
}catch (Exception e){
logger.error("master start up exception : " + e.getMessage(),e);
logger.error("master start up exception",e);
}finally {
releaseMutex(mutex);
}
@ -143,7 +146,7 @@ public class ZKMasterClient extends AbstractZKClient {
}
masterZNode = serverPath;
} catch (Exception e) {
logger.error("register master failure : " + e.getMessage(),e);
logger.error("register master failure ",e);
System.exit(-1);
}
}
@ -192,7 +195,7 @@ public class ZKMasterClient extends AbstractZKClient {
}
}catch (Exception e){
logger.error("{} server failover failed.", zkNodeType.toString());
logger.error("failover exception : " + e.getMessage(),e);
logger.error("failover exception ",e);
}
finally {
releaseMutex(mutex);

3
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKWorkerClient.java

@ -51,6 +51,7 @@ public class ZKWorkerClient extends AbstractZKClient {
*/
public void init(){
logger.info("initialize worker client...");
// init system znode
this.initSystemZNode();
@ -69,7 +70,7 @@ public class ZKWorkerClient extends AbstractZKClient {
}
workerZNode = serverPath;
} catch (Exception e) {
logger.error("register worker failure : " + e.getMessage(),e);
logger.error("register worker failure",e);
System.exit(-1);
}
}

4
dolphinscheduler-server/src/main/resources/config/install_config.conf

@ -19,3 +19,7 @@ installPath=/data1_1T/dolphinscheduler
deployUser=dolphinscheduler
ips=ark0,ark1,ark2,ark3,ark4
sshPort=22
masters=ark0,ark1
workers=ark2,ark3,ark4
alertServer=ark3
apiServers=ark1

22
dolphinscheduler-server/src/main/resources/config/run_config.conf

@ -1,22 +0,0 @@
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
masters=ark0,ark1
workers=ark2,ark3,ark4
alertServer=ark3
apiServers=ark1
sshPort=22

4
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/EnvFileTest.java

@ -33,7 +33,7 @@ public class EnvFileTest {
@Test
public void test() {
String path = System.getProperty("user.dir")+"/script/env/.dolphinscheduler_env.sh";
String path = System.getProperty("user.dir")+"/script/env/dolphinscheduler_env.sh";
String pythonHome = getPythonHome(path);
logger.info(pythonHome);
}
@ -65,7 +65,7 @@ public class EnvFileTest {
}
}catch (IOException e){
logger.error("read file failed : " + e.getMessage(),e);
logger.error("read file failed",e);
}finally {
try {
if (br != null){

18
dolphinscheduler-ui/.editorconfig

@ -1,5 +1,19 @@
# editorconfig.org
# author: axin
# 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.
root = true
[*]

15
dolphinscheduler-ui/.env

@ -1,3 +1,18 @@
# 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.
# back end interface address
API_BASE = http://192.168.xx.xx:12345

8
dolphinscheduler-ui/_test_/.babelrc

@ -0,0 +1,8 @@
{
"presets": [["env", { "modules": false }]],
"env": {
"test": {
"presets": [["env", { "targets": { "node": "current" } }]]
}
}
}

55
dolphinscheduler-ui/_test_/Counter.spec.js

@ -0,0 +1,55 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import { mount } from '@vue/test-utils'
import Counter from '../src/components/Counter.vue'
describe("Counter.vue", () => {
it("渲染Counter组件", () => {
const wrapper = mount(Counter)
expect(wrapper.element).toMatchSnapshot()
})
it("初始化之为0", () => {
const wrapper = mount(Counter)
expect(wrapper.vm.count).toEqual(0)
})
it("加1", () => {
const wrapper = mount(Counter)
wrapper.vm.inc()
expect(wrapper.vm.count).toEqual(1)
})
it("减1", () => {
const wrapper = mount(Counter)
wrapper.vm.dec()
expect(wrapper.vm.count).toEqual(-1)
})
it("重置", () => {
const wrapper = mount(Counter)
wrapper.vm.reset()
expect(wrapper.vm.count).toEqual(0)
})
it("因数为10加1操作", () => {
const wrapper = mount(Counter, { propsData: { factor: 10 } })
wrapper.vm.inc()
expect(wrapper.vm.computedCount).toEqual(10)
})
})

43
dolphinscheduler-ui/_test_/package.json

@ -0,0 +1,43 @@
{
"name": "testjest",
"description": "jest",
"version": "1.0.0",
"author": "xiangcaibiao",
"private": true,
"scripts": {
"test": "jest --coverage"
},
"dependencies": {
"vue": "^2.4.4"
},
"jest": {
"moduleFileExtensions": [
"js",
"vue"
],
"moduleNameMapper": {
"^@/(.*)$": "<rootDir>/src/$1"
},
"transform": {
"^.+\\.js$": "<rootDir>/_test_/node_modules/babel-jest",
".*\\.(vue)$": "<rootDir>/_test_/node_modules/vue-jest"
},
"snapshotSerializers": [
"<rootDir>/_test_/node_modules/jest-serializer-vue"
],
"collectCoverage": true,
"collectCoverageFrom": [
"src/**/*.{js,vue}",
"!**/node_modules/**"
],
"rootDir": "../"
},
"devDependencies": {
"@vue/test-utils": "^1.0.0-beta.30",
"babel-jest": "^24.9.0",
"babel-preset-env": "^1.7.0",
"jest": "^24.9.0",
"jest-serializer-vue": "^2.0.2",
"vue-jest": "^3.0.5"
}
}

34
dolphinscheduler-ui/_test_/test.spec.js

@ -0,0 +1,34 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import { shallowMount } from '@vue/test-utils'
import Message from '../src/components/Message.vue'
describe('Message', () => {
it('renders props.msg when passed', () => {
const msg = 'new message'
const wrapper = shallowMount(Message, {
propsData: { msg }
})
expect(wrapper.text()).toBe(msg)
})
it('renders default message if not passed a prop', () => {
const defaultMessage = 'default message'
const wrapper = shallowMount(Message)
expect(wrapper.text()).toBe(defaultMessage)
})
})

100
dolphinscheduler-ui/build/webpack.config.test.js

@ -1,100 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
const webpack = require('webpack')
const merge = require('webpack-merge')
const { baseConfig } = require('./config')
const ExtractTextPlugin = require('extract-text-webpack-plugin')
const config = merge.smart(baseConfig, {
devtool: 'inline-source-map',
output: {
filename: 'js/[name].js'
},
module: {
rules: [
{
test: /\.vue$/,
loader: 'vue-loader',
options: {
hotReload: true // Open hot overload
}
},
{
test: /\.css$/,
loader: ExtractTextPlugin.extract({
use: [
'css-loader',
{
loader: 'postcss-loader',
options: {
plugins: (loader) => [
require('autoprefixer')({
overrideBrowserslist: [
"Android 4.1",
"iOS 7.1",
"Chrome > 31",
"ff > 31",
"ie >= 8"
]
}),
require('cssnano')
]
}
}
],
fallback: ['vue-style-loader']
})
},
{
test: /\.scss$/,
loader: ExtractTextPlugin.extract({
use: [
'css-loader',
'sass-loader',
{
loader: 'postcss-loader',
options: {
plugins: (loader) => [
require('autoprefixer')({
overrideBrowserslist: [
"Android 4.1",
"iOS 7.1",
"Chrome > 31",
"ff > 31",
"ie >= 8"
]
}),
require('cssnano')
]
}
}
],
fallback: ['vue-style-loader']
})
}
]
},
externals: '',
plugins: [
new webpack.HotModuleReplacementPlugin(),
new ExtractTextPlugin({ filename: 'css/[name].css', allChunks: true }),
new webpack.optimize.OccurrenceOrderPlugin()
]
})
module.exports = config

53
dolphinscheduler-ui/src/components/Counter.vue

@ -0,0 +1,53 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
<template>
<div>
<div>{{ computedCount }}</div>
<button @click="inc"></button>
<button @click="dec"></button>
<button @click="reset">重置</button>
</div>
</template>
<script>
export default {
props: {
factor: { type: Number, default: 1 }
},
data() {
return {
count: 0
};
},
methods: {
inc() {
this.count++;
},
dec() {
this.count--;
},
reset() {
this.count = 0;
}
},
computed: {
computedCount: function() {
return this.count * this.factor;
}
}
};
</script>

28
dolphinscheduler-ui/src/components/Message.vue

@ -0,0 +1,28 @@
/*
* 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>
<h1> {{ msg || 'default message' }}</h1>
</template>
<script>
export default {
name: 'message',
props: [
'msg'
]
}
</script>

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

@ -157,6 +157,9 @@
background: #F2F3F7;
position: relative;
border-radius: 0 3px 0 0;
.ans-btn-text {
color: #337ab7;
}
.assist-btn {
position: absolute;
left: 10px;

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

@ -143,6 +143,7 @@
<m-sql
v-if="taskType === 'SQL'"
@on-params="_onParams"
@on-cache-params="_onCacheParams"
ref="SQL"
:create-node-id="id"
:backfill-item="backfillItem">
@ -151,12 +152,14 @@
<m-spark
v-if="taskType === 'SPARK'"
@on-params="_onParams"
@on-cache-params="_onCacheParams"
ref="SPARK"
:backfill-item="backfillItem">
</m-spark>
<m-flink
v-if="taskType === 'FLINK'"
@on-params="_onParams"
@on-cache-params="_onCacheParams"
ref="FLINK"
:backfill-item="backfillItem">
</m-flink>
@ -346,7 +349,6 @@
type: this.taskType,
id: this.id,
name: this.name,
params: this.params,
description: this.description,
runFlag: this.runFlag,
dependence: this.dependence,
@ -511,6 +513,9 @@
this.workerGroupId = o.workerGroupId
}
this.params = o.params || {}
this.dependence = o.dependence || {}
}
this.isContentBox = true
},

17
dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/datasource.vue

@ -112,7 +112,20 @@
})
}
},
watch: {},
computed: {
cacheParams () {
return {
type: this.type,
datasource: this.datasource
}
}
},
// Watch the cacheParams
watch: {
cacheParams (val) {
this.$emit('on-dsData', val);
}
},
created () {
let supportType = this.supportType || []
this.typeList = _.cloneDeep(this.store.state.dag.dsTypeListS)
@ -146,4 +159,4 @@
},
components: { }
}
</script>
</script>

50
dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/dependItemList.vue

@ -22,7 +22,7 @@
</x-option>
</x-select>
<x-select filterable :style="{width:isInstance ? '450px' : '450px'}" :disabled="isDetails" v-model="el.definitionId" @on-change="_onChangeDefinitionId">
<x-option v-for="item in definitionList" :key="item.value" :value="item.value" :label="item.label">
<x-option v-for="item in el.definitionList" :key="item.value" :value="item.value" :label="item.label">
</x-option>
</x-select>
<x-select filterable :style="{width:isInstance ? '450px' : '450px'}" :disabled="isDetails" v-model="el.depTasks">
@ -64,7 +64,6 @@
data () {
return {
list: [],
definitionList: [],
projectList: [],
cycleList: cycleList,
isInstance: false,
@ -88,16 +87,19 @@
_add () {
// btn loading
this.isLoading = true
// dependItemList index
let is = (value) => _.some(this.dependItemList, { definitionId: value })
let noArr = _.filter(this.definitionList, v => !is(v.value))
let value = noArr[0] && noArr[0].value || null
let val = value || this.definitionList[0].value
// add task list
let projectId = this.projectList[0].value
this._getDependItemList(val).then(depTasksList => {
this.$nextTick(() => {
this.$emit('dependItemListEvent', _.concat(this.dependItemList, this._rtNewParams(val, depTasksList,projectId)))
this._getProcessByProjectId(projectId).then(definitionList => {
// dependItemList index
let is = (value) => _.some(this.dependItemList, { definitionId: value })
let noArr = _.filter(definitionList, v => !is(v.value))
let value = noArr[0] && noArr[0].value || null
let val = value || definitionList[0].value
this._getDependItemList(val).then(depTasksList => {
this.$nextTick(() => {
this.$emit('dependItemListEvent', _.concat(this.dependItemList, this._rtNewParams(val, definitionList, depTasksList, projectId)))
})
})
})
// remove tooltip
@ -131,25 +133,25 @@
*/
_getProcessList () {
return new Promise((resolve, reject) => {
this.definitionList = _.map(_.cloneDeep(this.store.state.dag.processListS), v => {
let definitionList = _.map(_.cloneDeep(this.store.state.dag.processListS), v => {
return {
value: v.id,
label: v.name
}
})
resolve()
resolve(definitionList)
})
},
_getProcessByProjectId (id) {
return new Promise((resolve, reject) => {
this.store.dispatch('dag/getProcessByProjectId', { projectId: id }).then(res => {
this.definitionList = _.map(_.cloneDeep(res), v => {
let definitionList = _.map(_.cloneDeep(res), v => {
return {
value: v.id,
label: v.name
}
})
resolve(res)
resolve(definitionList)
})
})
},
@ -175,7 +177,7 @@
_onChangeProjectId ({ value }) {
this._getProcessByProjectId(value).then(definitionList => {
/*this.$set(this.dependItemList, this.itemIndex, this._dlOldParams(value, definitionList, item))*/
let definitionId = definitionList[0].id
let definitionId = definitionList[0].value
this._getDependItemList(definitionId).then(depTasksList => {
let item = this.dependItemList[this.itemIndex]
// init set depTasks All
@ -192,7 +194,7 @@
// init set depTasks All
item.depTasks = 'ALL'
// set dependItemList item data
this.$set(this.dependItemList, this.itemIndex, this._rtOldParams(value, depTasksList, item))
this.$set(this.dependItemList, this.itemIndex, this._rtOldParams(value, item.definitionList, depTasksList, item))
})
},
_onChangeCycle ({ value }) {
@ -200,10 +202,12 @@
this.$set(this.dependItemList[this.itemIndex], 'dateValue', list[0].value)
this.$set(this.dependItemList[this.itemIndex], 'dateValueList', list)
},
_rtNewParams (value, depTasksList,projectId) {
_rtNewParams (value, definitionList, depTasksList, projectId) {
return {
projectId: projectId,
definitionId: value,
// dependItem need private definitionList
definitionList: definitionList,
depTasks: 'ALL',
depTasksList: depTasksList,
cycle: 'day',
@ -212,10 +216,12 @@
state: ''
}
},
_rtOldParams (value,depTasksList, item) {
_rtOldParams (value, definitionList, depTasksList, item) {
return {
projectId: item.projectId,
definitionId: value,
// dependItem need private definitionList
definitionList: definitionList,
depTasks: item.depTasks || 'ALL',
depTasksList: depTasksList,
cycle: item.cycle,
@ -254,12 +260,12 @@
this.isInstance = this.router.history.current.name === 'projects-instance-details'
// get processlist
this._getProjectList().then(() => {
let projectId = this.projectList[0].value
if (!this.dependItemList.length) {
let projectId = this.projectList[0].value
this._getProcessByProjectId(projectId).then(definitionList => {
let value = this.definitionList[0].value
let value = definitionList[0].value
this._getDependItemList(value).then(depTasksList => {
this.$emit('dependItemListEvent', _.concat(this.dependItemList, this._rtNewParams(value, depTasksList,projectId)))
this.$emit('dependItemListEvent', _.concat(this.dependItemList, this._rtNewParams(value, definitionList, depTasksList, projectId)))
})
})
} else {
@ -269,7 +275,7 @@
this._getDependItemList(ids, false).then(res => {
_.map(this.dependItemList, (v, i) => {
this._getProcessByProjectId(v.projectId).then(definitionList => {
this.$set(this.dependItemList, i, this._rtOldParams(v.definitionId, ['ALL'].concat(_.map(res[v.definitionId] || [], v => v.name)), v))
this.$set(this.dependItemList, i, this._rtOldParams(v.definitionId, definitionList, ['ALL'].concat(_.map(res[v.definitionId] || [], v => v.name)), v))
})
})
})

7
dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/resources.vue

@ -66,6 +66,13 @@
// Listening data source
resourceList (a) {
this.value = _.map(_.cloneDeep(a), v => v.res)
},
value (val) {
this.$emit('on-cache-resourcesData', _.map(val, v => {
return {
res: v
}
}))
}
},
created () {

3
dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/sqlType.vue

@ -20,7 +20,7 @@
v-model="sqlTypeId"
:disabled="isDetails"
@on-change="_handleSqlTypeChanged"
style="width: 90px;">
style="width: 120px;">
<x-option
v-for="city in sqlTypeList"
:key="city.id"
@ -60,7 +60,6 @@
},
created () {
this.$nextTick(() => {
console.log(this.sqlType)
if (this.sqlType != 0) {
this.sqlTypeId = this.sqlType
} else {

11
dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/udfs.vue

@ -24,7 +24,7 @@
v-for="city in udfsList"
:key="city.id"
:value="city.id"
:label="city.code">
:label="city.code">
</x-option>
</x-select>
</div>
@ -53,7 +53,7 @@
* verification
*/
_verifUdfs () {
this.$emit('on-udfsData', _.map(this.udfsStr, v => v.id).join(','))
this.$emit('on-udfsData', this.udfsStr.join(','))
return true
},
/**
@ -68,18 +68,18 @@
code: v.funcName
}
})
let udfs = _.cloneDeep(this.udfs.split(','))
if (udfs.length) {
let arr = []
_.map(udfs, v => {
_.map(this.udfsList, v1 => {
if (parseInt(v) === v1.id) {
arr.push(v1)
arr.push(parseInt(v))
}
})
})
this.$nextTick(() => {
_.map(_.cloneDeep(this.udfsList), v => v.res)
this.udfsStr = arr
})
}
@ -87,6 +87,9 @@
}
},
watch: {
udfsStr (val) {
this._verifUdfs()
},
type (a) {
// The props parameter needs to be changed due to the scene.
this.udfs = ''

35
dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/flink.vue

@ -154,6 +154,7 @@
<m-resources
ref="refResources"
@on-resourcesData="_onResourcesData"
@on-cache-resourcesData="_onCacheResourcesData"
:resource-list="resourceList">
</m-resources>
</div>
@ -193,6 +194,8 @@
deployMode: 'cluster',
// Resource(list)
resourceList: [],
// Cache ResourceList
cacheResourceList: [],
// Custom function
localParams: [],
// Driver Number of cores
@ -232,6 +235,12 @@
_onResourcesData (a) {
this.resourceList = a
},
/**
* cache resourceList
*/
_onCacheResourcesData (a) {
this.cacheResourceList = a
},
/**
* verification
*/
@ -336,6 +345,31 @@
if (type === 'PYTHON') {
this.mainClass = ''
}
},
//Watch the cacheParams
cacheParams (val) {
this.$emit('on-cache-params', val);
}
},
computed: {
cacheParams () {
return {
mainClass: this.mainClass,
mainJar: {
res: this.mainJar
},
deployMode: this.deployMode,
resourceList: this.cacheResourceList,
localParams: this.localParams,
slot: this.slot,
taskManager: this.taskManager,
jobManagerMemory: this.jobManagerMemory,
taskManagerMemory: this.taskManagerMemory,
executorCores: this.executorCores,
mainArgs: this.mainArgs,
others: this.others,
programType: this.programType
}
}
},
created () {
@ -360,6 +394,7 @@
let resourceList = o.params.resourceList || []
if (resourceList.length) {
this.resourceList = resourceList
this.cacheResourceList = resourceList
}
// backfill localParams

36
dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/spark.vue

@ -182,6 +182,7 @@
<m-resources
ref="refResources"
@on-resourcesData="_onResourcesData"
@on-cache-resourcesData="_onCacheResourcesData"
:resource-list="resourceList">
</m-resources>
</div>
@ -221,6 +222,8 @@
deployMode: 'cluster',
// Resource(list)
resourceList: [],
// Cache ResourceList
cacheResourceList: [],
// Custom function
localParams: [],
// Driver Number of cores
@ -264,6 +267,12 @@
_onResourcesData (a) {
this.resourceList = a
},
/**
* cache resourceList
*/
_onCacheResourcesData (a) {
this.cacheResourceList = a
},
/**
* verification
*/
@ -368,6 +377,32 @@
if (type === 'PYTHON') {
this.mainClass = ''
}
},
//Watch the cacheParams
cacheParams (val) {
this.$emit('on-cache-params', val)
}
},
computed: {
cacheParams () {
return {
mainClass: this.mainClass,
mainJar: {
res: this.mainJar
},
deployMode: this.deployMode,
resourceList: this.cacheResourceList,
localParams: this.localParams,
driverCores: this.driverCores,
driverMemory: this.driverMemory,
numExecutors: this.numExecutors,
executorMemory: this.executorMemory,
executorCores: this.executorCores,
mainArgs: this.mainArgs,
others: this.others,
programType: this.programType,
sparkVersion: this.sparkVersion
}
}
},
created () {
@ -393,6 +428,7 @@
let resourceList = o.params.resourceList || []
if (resourceList.length) {
this.resourceList = resourceList
this.cacheResourceList = resourceList
}
// backfill localParams

35
dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sql.vue

@ -371,6 +371,10 @@
this.connParams = ''
}
},
//Watch the cacheParams
cacheParams (val) {
this.$emit('on-cache-params', val);
}
},
created () {
let o = this.backfillItem
@ -396,7 +400,8 @@
this.receivers = o.params.receivers && o.params.receivers.split(',') || []
this.receiversCc = o.params.receiversCc && o.params.receiversCc.split(',') || []
}
if (!_.some(this.store.state.dag.tasks, { id: this.createNodeId }) &&
// read tasks from cache
if (!_.some(this.store.state.dag.cacheTasks, { id: this.createNodeId }) &&
this.router.history.current.name !== 'definition-create') {
this._getReceiver()
}
@ -415,7 +420,33 @@
editor.off($('.code-sql-mirror'), 'keypress', this.keypress)
}
},
computed: {},
computed: {
cacheParams () {
return {
type: this.type,
datasource: this.rtDatasource,
sql: editor ? editor.getValue() : '',
udfs: this.udfs,
sqlType: this.sqlType,
title: this.title,
receivers: this.receivers.join(','),
receiversCc: this.receiversCc.join(','),
showType: (() => {
let showType = this.showType
if (showType.length === 2 && showType[0] === 'ATTACHMENT') {
return [showType[1], showType[0]].join(',')
} else {
return showType.join(',')
}
})(),
localParams: this.localParams,
connParams: this.connParams,
preStatements: this.preStatements,
postStatements: this.postStatements
}
}
},
components: { mListBox, mDatasource, mLocalParams, mUdfs, mSqlType, mStatementList, mEmail }
}
</script>

4
dolphinscheduler-ui/src/js/conf/home/pages/datasource/pages/list/_source/createDataSource.vue

@ -42,6 +42,7 @@
<x-input
type="input"
v-model="name"
maxlength="60"
:placeholder="$t('Please enter datasource name')"
autocomplete="off">
</x-input>
@ -64,6 +65,7 @@
<x-input
type="input"
v-model="host"
maxlength="60"
:placeholder="$t('Please enter IP')"
autocomplete="off">
</x-input>
@ -97,6 +99,7 @@
<x-input
type="input"
v-model="userName"
maxlength="60"
:placeholder="$t('Please enter user name')"
autocomplete="off">
</x-input>
@ -119,6 +122,7 @@
<x-input
type="input"
v-model="database"
maxlength="60"
:placeholder="$t('Please enter database name')"
autocomplete="off">
</x-input>

2
dolphinscheduler-ui/src/js/conf/home/pages/datasource/pages/list/_source/list.vue

@ -148,7 +148,7 @@
this.deleteDatasource({
id: item.id
}).then(res => {
this.list.splice(i, 1)
this.$emit('on-update')
this.$message.success(res.msg)
}).catch(e => {
this.$message.error(e.msg || '')

23
dolphinscheduler-ui/src/js/conf/home/pages/datasource/pages/list/index.vue

@ -24,13 +24,13 @@
</m-conditions>
</template>
<template slot="content">
<template v-if="datasourcesList.length">
<m-list :datasources-list="datasourcesList" :page-no="searchParams.pageNo" :page-size="searchParams.pageSize"></m-list>
<template v-if="datasourcesList.length || total>0">
<m-list @on-update="_onUpdate" :datasources-list="datasourcesList" :page-no="searchParams.pageNo" :page-size="searchParams.pageSize"></m-list>
<div class="page-box">
<x-page :current="parseInt(searchParams.pageNo)" :total="total" :page-size="searchParams.pageSize" show-elevator @on-change="_page" show-sizer :page-size-options="[10,30,50]" @on-size-change="_pageSize"></x-page>
</div>
</template>
<template v-if="!datasourcesList.length">
<template v-if="!datasourcesList.length && total<=0">
<m-no-data></m-no-data>
</template>
<m-spin :is-spin="isLoading" :is-left="false">
@ -124,14 +124,21 @@
_getList (flag) {
this.isLoading = !flag
this.getDatasourcesListP(this.searchParams).then(res => {
this.datasourcesList = []
this.datasourcesList = res.totalList
this.total = res.total
this.isLoading = false
if(this.searchParams.pageNo>1 && res.totalList.length == 0) {
this.searchParams.pageNo = this.searchParams.pageNo -1
} else {
this.datasourcesList = []
this.datasourcesList = res.totalList
this.total = res.total
this.isLoading = false
}
}).catch(e => {
this.isLoading = false
})
}
},
_onUpdate () {
this._debounceGET('false')
},
},
watch: {
// router

4
dolphinscheduler-ui/src/js/conf/home/pages/monitor/pages/servers/servers.scss

@ -60,7 +60,7 @@
height: 276px;
line-height: 276px;
text-align: center;
>b {
>strong {
font-size: 100px;
color: #333;
}
@ -85,7 +85,7 @@
font-size: 72px;
}
>.state {
>i {
>em {
font-size: 50px;
}
.success {

4
dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/_source/instanceConditions/index.vue

@ -52,10 +52,10 @@
</x-select>
</div>
<div class="list">
<x-input v-model="searchParams.host" @on-enterkey="_ckQuery" style="width: 140px;" size="small" :placeholder="$t('host')"></x-input>
<x-input v-model.trim="searchParams.host" @on-enterkey="_ckQuery" style="width: 140px;" size="small" :placeholder="$t('host')"></x-input>
</div>
<div class="list">
<x-input v-model="searchParams.searchVal" @on-enterkey="_ckQuery" style="width: 200px;" size="small" :placeholder="$t('name')"></x-input>
<x-input v-model.trim="searchParams.searchVal" @on-enterkey="_ckQuery" style="width: 200px;" size="small" :placeholder="$t('name')"></x-input>
</div>
</template>
</m-conditions>

4
dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/_source/taskRecordList/index.vue

@ -20,14 +20,14 @@
<m-conditions @on-query="_onQuery"></m-conditions>
</template>
<template slot="content">
<template v-if="taskRecordList.length">
<template v-if="taskRecordList.length || total>0">
<m-list :task-record-list="taskRecordList" @on-update="_onUpdate" :page-no="searchParams.pageNo" :page-size="searchParams.pageSize">
</m-list>
<div class="page-box">
<x-page :current="parseInt(searchParams.pageNo)" :total="total" show-elevator @on-change="_page"></x-page>
</div>
</template>
<template v-if="!taskRecordList.length">
<template v-if="!taskRecordList.length && total<=0">
<m-no-data></m-no-data>
</template>
<m-spin :is-spin="isLoading"></m-spin>

16
dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/index.vue

@ -26,13 +26,13 @@
</m-conditions>
</template>
<template slot="content">
<template v-if="processListP.length">
<template v-if="processListP.length || total>0">
<m-list :process-list="processListP" @on-update="_onUpdate" :page-no="searchParams.pageNo" :page-size="searchParams.pageSize"></m-list>
<div class="page-box">
<x-page :current="parseInt(searchParams.pageNo)" :total="total" show-elevator @on-change="_page" show-sizer :page-size-options="[10,30,50]" @on-size-change="_pageSize"></x-page>
</div>
</template>
<template v-if="!processListP.length">
<template v-if="!processListP.length && total<=0">
<m-no-data></m-no-data>
</template>
<m-spin :is-spin="isLoading"></m-spin>
@ -100,10 +100,14 @@
_getList (flag) {
this.isLoading = !flag
this.getProcessListP(this.searchParams).then(res => {
this.processListP = []
this.processListP = res.totalList
this.total = res.total
this.isLoading = false
if(this.searchParams.pageNo>1 && res.totalList.length == 0) {
this.searchParams.pageNo = this.searchParams.pageNo -1
} else {
this.processListP = []
this.processListP = res.totalList
this.total = res.total
this.isLoading = false
}
}).catch(e => {
this.isLoading = false
})

16
dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/index.vue

@ -20,14 +20,14 @@
<m-instance-conditions @on-query="_onQuery"></m-instance-conditions>
</template>
<template slot="content">
<template v-if="processInstanceList.length">
<template v-if="processInstanceList.length || total>0">
<m-list :process-instance-list="processInstanceList" @on-update="_onUpdate" :page-no="searchParams.pageNo" :page-size="searchParams.pageSize">
</m-list>
<div class="page-box">
<x-page :current="parseInt(searchParams.pageNo)" :total="total" show-elevator @on-change="_page" show-sizer :page-size-options="[10,30,50]" @on-size-change="_pageSize"></x-page>
</div>
</template>
<template v-if="!processInstanceList.length">
<template v-if="!processInstanceList.length && total<=0">
<m-no-data></m-no-data>
</template>
<m-spin :is-spin="isLoading"></m-spin>
@ -105,10 +105,14 @@
_getProcessInstanceListP (flag) {
this.isLoading = !flag
this.getProcessInstance(this.searchParams).then(res => {
this.processInstanceList = []
this.processInstanceList = res.totalList
this.total = res.total
this.isLoading = false
if(this.searchParams.pageNo>1 && res.totalList.length == 0) {
this.searchParams.pageNo = this.searchParams.pageNo -1
} else {
this.processInstanceList = []
this.processInstanceList = res.totalList
this.total = res.total
this.isLoading = false
}
}).catch(e => {
this.isLoading = false
})

1
dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/list/_source/createProject.vue

@ -24,6 +24,7 @@
<x-input
type="input"
v-model="projectName"
maxlength="60"
:placeholder="$t('Please enter name')"
autocomplete="off">
</x-input>

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

@ -152,7 +152,7 @@
projectId: item.id
}).then(res => {
this.$refs[`poptip-${i}`][0].doClose()
this.list.splice(i, 1)
this.$emit('on-update')
this.$message.success(res.msg)
}).catch(e => {
this.$refs[`poptip-${i}`][0].doClose()
@ -165,7 +165,8 @@
*/
_edit (item) {
findComponentDownward(this.$root, 'projects-list')._create(item)
}
},
},
watch: {
projectsList (a) {

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

Loading…
Cancel
Save