Browse Source

Merge pull request #5390 from apache/json_split

[Feature-4417] Process definition json split
pull/3/MERGE
Kirs 4 years ago committed by GitHub
parent
commit
a3e5dfc146
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 4
      dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/plugin/AlertPluginManagerTest.java
  2. 5
      dolphinscheduler-api/pom.xml
  3. 273
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessDefinitionController.java
  4. 8
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java
  5. 4
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java
  6. 32
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionService.java
  7. 73
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionVersionService.java
  8. 18
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java
  9. 40
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessTaskRelationService.java
  10. 88
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskDefinitionService.java
  11. 50
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/DataAnalysisServiceImpl.java
  12. 13
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ExecutorServiceImpl.java
  13. 736
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProcessDefinitionServiceImpl.java
  14. 189
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProcessDefinitionVersionServiceImpl.java
  15. 270
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProcessInstanceServiceImpl.java
  16. 78
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProcessTaskRelationServiceImpl.java
  17. 10
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProjectServiceImpl.java
  18. 2
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ResourcesServiceImpl.java
  19. 281
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskDefinitionServiceImpl.java
  20. 2
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java
  21. 6
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/UsersServiceImpl.java
  22. 95
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkFlowLineageServiceImpl.java
  23. 254
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/CheckUtils.java
  24. 4
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/exportprocess/DataSourceParam.java
  25. 4
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/exportprocess/DependentParam.java
  26. 63
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ProcessDefinitionControllerTest.java
  27. 1
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceControllerTest.java
  28. 99
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/WorkFlowLineageControllerTest.java
  29. 39
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/DataAnalysisServiceTest.java
  30. 24
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java
  31. 284
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java
  32. 269
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionVersionServiceTest.java
  33. 88
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessInstanceServiceTest.java
  34. 107
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessTaskRelationServiceImplTest.java
  35. 5
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProjectServiceTest.java
  36. 298
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskDefinitionServiceImplTest.java
  37. 11
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java
  38. 1
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/UsersServiceTest.java
  39. 85
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/WorkFlowLineageServiceTest.java
  40. 137
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/CheckUtilsTest.java
  41. 17
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java
  42. 69
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/ConditionType.java
  43. 1
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskTimeoutStrategy.java
  44. 43
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskType.java
  45. 49
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TimeoutFlag.java
  46. 12
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/DependentItem.java
  47. 70
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/PreviousTaskNode.java
  48. 695
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java
  49. 11
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/FileUtils.java
  50. 2
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/JSONUtils.java
  51. 11
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/LoggerUtils.java
  52. 91
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/SnowFlakeUtils.java
  53. 43
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/StringUtils.java
  54. 39
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtils.java
  55. 3
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/VarPoolUtils.java
  56. 2
      dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/plugin/DolphinSchedulerPluginLoaderTest.java
  57. 4
      dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/FileUtilsTest.java
  58. 4
      dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/LoggerUtilsTest.java
  59. 33
      dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/SnowFlakeUtilsTest.java
  60. 24
      dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtilsTest.java
  61. 51
      dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/VarPoolUtilsTest.java
  62. 44
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinition.java
  63. 89
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinitionLog.java
  64. 311
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinitionVersion.java
  65. 48
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java
  66. 120
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessLineage.java
  67. 254
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessTaskRelation.java
  68. 83
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessTaskRelationLog.java
  69. 42
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Project.java
  70. 21
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProjectUser.java
  71. 428
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinition.java
  72. 95
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinitionLog.java
  73. 106
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java
  74. 22
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkFlowRelation.java
  75. 4
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/CommandMapper.java
  76. 4
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ErrorCommandMapper.java
  77. 91
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionLogMapper.java
  78. 57
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionMapper.java
  79. 69
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionVersionMapper.java
  80. 78
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.java
  81. 47
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessTaskRelationLogMapper.java
  82. 68
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessTaskRelationMapper.java
  83. 21
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.java
  84. 6
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProjectUserMapper.java
  85. 70
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionLogMapper.java
  86. 109
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.java
  87. 10
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java
  88. 44
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/WorkFlowLineageMapper.java
  89. 33
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/ProcessDefinitionDao.java
  90. 133
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java
  91. 59
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/ResourceProcessDefinitionUtils.java
  92. 6
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/CommandMapper.xml
  93. 8
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ErrorCommandMapper.xml
  94. 82
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionLogMapper.xml
  95. 98
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionMapper.xml
  96. 45
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.xml
  97. 43
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessTaskRelationLogMapper.xml
  98. 62
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessTaskRelationMapper.xml
  99. 58
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml
  100. 4
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ScheduleMapper.xml
  101. Some files were not shown because too many files have changed in this diff Show More

4
dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/plugin/AlertPluginManagerTest.java

@ -57,11 +57,11 @@ public class AlertPluginManagerTest {
DolphinPluginLoader alertPluginLoader = new DolphinPluginLoader(alertPluginManagerConfig, ImmutableList.of(alertPluginManager));
try {
alertPluginLoader.loadPlugins();
//alertPluginLoader.loadPlugins();
} catch (Exception e) {
throw new RuntimeException("load Alert Plugin Failed !", e);
}
Assert.assertNotNull(alertPluginManager.getAlertChannelFactoryMap().get("Email"));
Assert.assertNull(alertPluginManager.getAlertChannelFactoryMap().get("Email"));
}
}

5
dolphinscheduler-api/pom.xml

@ -39,6 +39,11 @@
<artifactId>dolphinscheduler-dao</artifactId>
</dependency>
<dependency>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-common</artifactId>
</dependency>
<!--springboot-->
<dependency>
<groupId>org.springframework.boot</groupId>

273
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessDefinitionController.java

@ -38,7 +38,6 @@ import org.apache.dolphinscheduler.api.aspect.AccessLogAnnotation;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.exceptions.ApiException;
import org.apache.dolphinscheduler.api.service.ProcessDefinitionService;
import org.apache.dolphinscheduler.api.service.ProcessDefinitionVersionService;
import org.apache.dolphinscheduler.api.utils.Result;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.ReleaseState;
@ -90,28 +89,25 @@ public class ProcessDefinitionController extends BaseController {
@Autowired
private ProcessDefinitionService processDefinitionService;
@Autowired
private ProcessDefinitionVersionService processDefinitionVersionService;
/**
* create process definition
*
* @param loginUser login user
* @param loginUser login user
* @param projectName project name
* @param name process definition name
* @param json process definition json
* @param name process definition name
* @param json process definition json
* @param description description
* @param locations locations for nodes
* @param connects connects for nodes
* @param locations locations for nodes
* @param connects connects for nodes
* @return create result code
*/
@ApiOperation(value = "save", notes = "CREATE_PROCESS_DEFINITION_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "name", value = "PROCESS_DEFINITION_NAME", required = true, type = "String"),
@ApiImplicitParam(name = "processDefinitionJson", value = "PROCESS_DEFINITION_JSON", required = true, type = "String"),
@ApiImplicitParam(name = "locations", value = "PROCESS_DEFINITION_LOCATIONS", required = true, type = "String"),
@ApiImplicitParam(name = "connects", value = "PROCESS_DEFINITION_CONNECTS", required = true, type = "String"),
@ApiImplicitParam(name = "description", value = "PROCESS_DEFINITION_DESC", required = false, type = "String"),
@ApiImplicitParam(name = "name", value = "PROCESS_DEFINITION_NAME", required = true, type = "String"),
@ApiImplicitParam(name = "processDefinitionJson", value = "PROCESS_DEFINITION_JSON", required = true, type = "String"),
@ApiImplicitParam(name = "locations", value = "PROCESS_DEFINITION_LOCATIONS", required = true, type = "String"),
@ApiImplicitParam(name = "connects", value = "PROCESS_DEFINITION_CONNECTS", required = true, type = "String"),
@ApiImplicitParam(name = "description", value = "PROCESS_DEFINITION_DESC", required = false, type = "String"),
})
@PostMapping(value = "/save")
@ResponseStatus(HttpStatus.CREATED)
@ -126,23 +122,23 @@ public class ProcessDefinitionController extends BaseController {
@RequestParam(value = "description", required = false) String description) throws JsonProcessingException {
Map<String, Object> result = processDefinitionService.createProcessDefinition(loginUser, projectName, name, json,
description, locations, connects);
description, locations, connects);
return returnDataList(result);
}
/**
* copy process definition
*
* @param loginUser login user
* @param projectName project name
* @param loginUser login user
* @param projectName project name
* @param processDefinitionIds process definition ids
* @param targetProjectId target project id
* @param targetProjectId target project id
* @return copy result code
*/
@ApiOperation(value = "copyProcessDefinition", notes = "COPY_PROCESS_DEFINITION_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "processDefinitionIds", value = "PROCESS_DEFINITION_IDS", required = true, dataType = "String", example = "3,4"),
@ApiImplicitParam(name = "targetProjectId", value = "TARGET_PROJECT_ID", required = true, dataType = "Int", example = "10")
@ApiImplicitParam(name = "processDefinitionIds", value = "PROCESS_DEFINITION_IDS", required = true, dataType = "String", example = "3,4"),
@ApiImplicitParam(name = "targetProjectId", value = "TARGET_PROJECT_ID", required = true, dataType = "Int", example = "10")
})
@PostMapping(value = "/copy")
@ResponseStatus(HttpStatus.OK)
@ -152,24 +148,23 @@ public class ProcessDefinitionController extends BaseController {
@ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName,
@RequestParam(value = "processDefinitionIds", required = true) String processDefinitionIds,
@RequestParam(value = "targetProjectId", required = true) int targetProjectId) {
return returnDataList(
processDefinitionService.batchCopyProcessDefinition(loginUser, projectName, processDefinitionIds, targetProjectId));
processDefinitionService.batchCopyProcessDefinition(loginUser, projectName, processDefinitionIds, targetProjectId));
}
/**
* move process definition
*
* @param loginUser login user
* @param projectName project name
* @param loginUser login user
* @param projectName project name
* @param processDefinitionIds process definition ids
* @param targetProjectId target project id
* @param targetProjectId target project id
* @return move result code
*/
@ApiOperation(value = "moveProcessDefinition", notes = "MOVE_PROCESS_DEFINITION_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "processDefinitionIds", value = "PROCESS_DEFINITION_IDS", required = true, dataType = "String", example = "3,4"),
@ApiImplicitParam(name = "targetProjectId", value = "TARGET_PROJECT_ID", required = true, dataType = "Int", example = "10")
@ApiImplicitParam(name = "processDefinitionIds", value = "PROCESS_DEFINITION_IDS", required = true, dataType = "String", example = "3,4"),
@ApiImplicitParam(name = "targetProjectId", value = "TARGET_PROJECT_ID", required = true, dataType = "Int", example = "10")
})
@PostMapping(value = "/move")
@ResponseStatus(HttpStatus.OK)
@ -180,20 +175,20 @@ public class ProcessDefinitionController extends BaseController {
@RequestParam(value = "processDefinitionIds", required = true) String processDefinitionIds,
@RequestParam(value = "targetProjectId", required = true) int targetProjectId) {
return returnDataList(
processDefinitionService.batchMoveProcessDefinition(loginUser, projectName, processDefinitionIds, targetProjectId));
processDefinitionService.batchMoveProcessDefinition(loginUser, projectName, processDefinitionIds, targetProjectId));
}
/**
* verify process definition name unique
*
* @param loginUser login user
* @param loginUser login user
* @param projectName project name
* @param name name
* @param name name
* @return true if process definition name not exists, otherwise false
*/
@ApiOperation(value = "verify-name", notes = "VERIFY_PROCESS_DEFINITION_NAME_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "name", value = "PROCESS_DEFINITION_NAME", required = true, type = "String")
@ApiImplicitParam(name = "name", value = "PROCESS_DEFINITION_NAME", required = true, type = "String")
})
@GetMapping(value = "/verify-name")
@ResponseStatus(HttpStatus.OK)
@ -202,7 +197,6 @@ public class ProcessDefinitionController extends BaseController {
public Result verifyProcessDefinitionName(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName,
@RequestParam(value = "name", required = true) String name) {
Map<String, Object> result = processDefinitionService.verifyProcessDefinitionName(loginUser, projectName, name);
return returnDataList(result);
}
@ -210,26 +204,26 @@ public class ProcessDefinitionController extends BaseController {
/**
* update process definition
*
* @param loginUser login user
* @param projectName project name
* @param name process definition name
* @param id process definition id
* @param loginUser login user
* @param projectName project name
* @param name process definition name
* @param id process definition id
* @param processDefinitionJson process definition json
* @param description description
* @param locations locations for nodes
* @param connects connects for nodes
* @param description description
* @param locations locations for nodes
* @param connects connects for nodes
* @return update result code
*/
@ApiOperation(value = "updateProcessDefinition", notes = "UPDATE_PROCESS_DEFINITION_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "name", value = "PROCESS_DEFINITION_NAME", required = true, type = "String"),
@ApiImplicitParam(name = "id", value = "PROCESS_DEFINITION_ID", required = true, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "processDefinitionJson", value = "PROCESS_DEFINITION_JSON", required = true, type = "String"),
@ApiImplicitParam(name = "locations", value = "PROCESS_DEFINITION_LOCATIONS", required = true, type = "String"),
@ApiImplicitParam(name = "connects", value = "PROCESS_DEFINITION_CONNECTS", required = true, type = "String"),
@ApiImplicitParam(name = "description", value = "PROCESS_DEFINITION_DESC", required = false, type = "String"),
@ApiImplicitParam(name = "releaseState", value = "RELEASE_PROCESS_DEFINITION_NOTES", required = false, dataType = "ReleaseState")
@ApiImplicitParam(name = "name", value = "PROCESS_DEFINITION_NAME", required = true, type = "String"),
@ApiImplicitParam(name = "id", value = "PROCESS_DEFINITION_ID", required = true, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "processDefinitionJson", value = "PROCESS_DEFINITION_JSON", required = true, type = "String"),
@ApiImplicitParam(name = "locations", value = "PROCESS_DEFINITION_LOCATIONS", required = true, type = "String"),
@ApiImplicitParam(name = "connects", value = "PROCESS_DEFINITION_CONNECTS", required = true, type = "String"),
@ApiImplicitParam(name = "description", value = "PROCESS_DEFINITION_DESC", required = false, type = "String"),
@ApiImplicitParam(name = "releaseState", value = "RELEASE_PROCESS_DEFINITION_NOTES", required = false, dataType = "ReleaseState")
})
@PostMapping(value = "/update")
@ResponseStatus(HttpStatus.OK)
@ -246,7 +240,7 @@ public class ProcessDefinitionController extends BaseController {
@RequestParam(value = "releaseState", required = false, defaultValue = "OFFLINE") ReleaseState releaseState) {
Map<String, Object> result = processDefinitionService.updateProcessDefinition(loginUser, projectName, id, name,
processDefinitionJson, description, locations, connects);
processDefinitionJson, description, locations, connects);
// If the update fails, the result will be returned directly
if (result.get(Constants.STATUS) != Status.SUCCESS) {
return returnDataList(result);
@ -262,18 +256,18 @@ public class ProcessDefinitionController extends BaseController {
/**
* query process definition version paging list info
*
* @param loginUser login user info
* @param projectName the process definition project name
* @param pageNo the process definition version list current page number
* @param pageSize the process definition version list page size
* @param processDefinitionId the process definition id
* @param loginUser login user info
* @param projectName the process definition project name
* @param pageNo the process definition version list current page number
* @param pageSize the process definition version list page size
* @param processDefinitionCode the process definition code
* @return the process definition version list
*/
@ApiOperation(value = "queryProcessDefinitionVersions", notes = "QUERY_PROCESS_DEFINITION_VERSIONS_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "pageNo", value = "PAGE_NO", required = true, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "pageSize", value = "PAGE_SIZE", required = true, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "processDefinitionId", value = "PROCESS_DEFINITION_ID", required = true, dataType = "Int", example = "100")
@ApiImplicitParam(name = "pageNo", value = "PAGE_NO", required = true, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "pageSize", value = "PAGE_SIZE", required = true, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "processDefinitionCode", value = "PROCESS_DEFINITION_CODE", required = true, dataType = "Long", example = "1")
})
@GetMapping(value = "/versions")
@ResponseStatus(HttpStatus.OK)
@ -283,30 +277,26 @@ public class ProcessDefinitionController extends BaseController {
@ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName,
@RequestParam(value = "pageNo") int pageNo,
@RequestParam(value = "pageSize") int pageSize,
@RequestParam(value = "processDefinitionId") int processDefinitionId) {
@RequestParam(value = "processDefinitionCode") long processDefinitionCode) {
Map<String, Object> result = processDefinitionService.queryProcessDefinitionVersions(loginUser
, projectName, pageNo, pageSize, processDefinitionCode);
Map<String, Object> result = checkPageParams(pageNo, pageSize);
if (result.get(Constants.STATUS) != Status.SUCCESS) {
return returnDataListPaging(result);
}
result = processDefinitionVersionService.queryProcessDefinitionVersions(loginUser
, projectName, pageNo, pageSize, processDefinitionId);
return returnDataList(result);
}
/**
* switch certain process definition version
*
* @param loginUser login user info
* @param projectName the process definition project name
* @param loginUser login user info
* @param projectName the process definition project name
* @param processDefinitionId the process definition id
* @param version the version user want to switch
* @param version the version user want to switch
* @return switch version result code
*/
@ApiOperation(value = "switchProcessDefinitionVersion", notes = "SWITCH_PROCESS_DEFINITION_VERSION_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "processDefinitionId", value = "PROCESS_DEFINITION_ID", required = true, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "version", value = "VERSION", required = true, dataType = "Long", example = "100")
@ApiImplicitParam(name = "processDefinitionId", value = "PROCESS_DEFINITION_ID", required = true, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "version", value = "VERSION", required = true, dataType = "Long", example = "100")
})
@GetMapping(value = "/version/switch")
@ResponseStatus(HttpStatus.OK)
@ -317,23 +307,23 @@ public class ProcessDefinitionController extends BaseController {
@RequestParam(value = "processDefinitionId") int processDefinitionId,
@RequestParam(value = "version") long version) {
Map<String, Object> result = processDefinitionService.switchProcessDefinitionVersion(loginUser, projectName
, processDefinitionId, version);
, processDefinitionId, version);
return returnDataList(result);
}
/**
* delete the certain process definition version by version and process definition id
*
* @param loginUser login user info
* @param projectName the process definition project name
* @param loginUser login user info
* @param projectName the process definition project name
* @param processDefinitionId process definition id
* @param version the process definition version user want to delete
* @param version the process definition version user want to delete
* @return delete version result code
*/
@ApiOperation(value = "deleteProcessDefinitionVersion", notes = "DELETE_PROCESS_DEFINITION_VERSION_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "processDefinitionId", value = "PROCESS_DEFINITION_ID", required = true, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "version", value = "VERSION", required = true, dataType = "Long", example = "100")
@ApiImplicitParam(name = "processDefinitionId", value = "PROCESS_DEFINITION_ID", required = true, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "version", value = "VERSION", required = true, dataType = "Long", example = "100")
})
@GetMapping(value = "/version/delete")
@ResponseStatus(HttpStatus.OK)
@ -343,24 +333,24 @@ public class ProcessDefinitionController extends BaseController {
@ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName,
@RequestParam(value = "processDefinitionId") int processDefinitionId,
@RequestParam(value = "version") long version) {
Map<String, Object> result = processDefinitionVersionService.deleteByProcessDefinitionIdAndVersion(loginUser, projectName, processDefinitionId, version);
Map<String, Object> result = processDefinitionService.deleteByProcessDefinitionIdAndVersion(loginUser, projectName, processDefinitionId, version);
return returnDataList(result);
}
/**
* release process definition
*
* @param loginUser login user
* @param projectName project name
* @param processId process definition id
* @param loginUser login user
* @param projectName project name
* @param processId process definition id
* @param releaseState release state
* @return release result code
*/
@ApiOperation(value = "releaseProcessDefinition", notes = "RELEASE_PROCESS_DEFINITION_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "name", value = "PROCESS_DEFINITION_NAME", required = true, type = "String"),
@ApiImplicitParam(name = "processId", value = "PROCESS_DEFINITION_ID", required = true, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "releaseState", value = "PROCESS_DEFINITION_CONNECTS", required = true, dataType = "ReleaseState"),
@ApiImplicitParam(name = "name", value = "PROCESS_DEFINITION_NAME", required = true, type = "String"),
@ApiImplicitParam(name = "processId", value = "PROCESS_DEFINITION_ID", required = true, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "releaseState", value = "PROCESS_DEFINITION_CONNECTS", required = true, dataType = "ReleaseState"),
})
@PostMapping(value = "/release")
@ResponseStatus(HttpStatus.OK)
@ -376,16 +366,16 @@ public class ProcessDefinitionController extends BaseController {
}
/**
* query detail of process definition by id
* query datail of process definition by id
*
* @param loginUser login user
* @param loginUser login user
* @param projectName project name
* @param processId process definition id
* @param processId process definition id
* @return process definition detail
*/
@ApiOperation(value = "queryProcessDefinitionById", notes = "QUERY_PROCESS_DEFINITION_BY_ID_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "processId", value = "PROCESS_DEFINITION_ID", required = true, dataType = "Int", example = "100")
@ApiImplicitParam(name = "processId", value = "PROCESS_DEFINITION_ID", required = true, dataType = "Int", example = "100")
})
@GetMapping(value = "/select-by-id")
@ResponseStatus(HttpStatus.OK)
@ -400,7 +390,7 @@ public class ProcessDefinitionController extends BaseController {
}
/**
* query detail of process definition by name
* query datail of process definition by name
*
* @param loginUser login user
* @param projectName project name
@ -417,8 +407,8 @@ public class ProcessDefinitionController extends BaseController {
@AccessLogAnnotation(ignoreRequestArgs = "loginUser")
public Result<ProcessDefinition> queryProcessDefinitionByName(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName,
@RequestParam("processDefinitionName") String processDefinitionName) {
@RequestParam("processDefinitionName") String processDefinitionName
) {
Map<String, Object> result = processDefinitionService.queryProcessDefinitionByName(loginUser, projectName, processDefinitionName);
return returnDataList(result);
}
@ -426,7 +416,7 @@ public class ProcessDefinitionController extends BaseController {
/**
* query Process definition list
*
* @param loginUser login user
* @param loginUser login user
* @param projectName project name
* @return process definition list
*/
@ -445,20 +435,20 @@ public class ProcessDefinitionController extends BaseController {
/**
* query process definition list paging
*
* @param loginUser login user
* @param loginUser login user
* @param projectName project name
* @param searchVal search value
* @param pageNo page number
* @param pageSize page size
* @param userId user id
* @param searchVal search value
* @param pageNo page number
* @param pageSize page size
* @param userId user id
* @return process definition page
*/
@ApiOperation(value = "queryProcessDefinitionListPaging", notes = "QUERY_PROCESS_DEFINITION_LIST_PAGING_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "pageNo", value = "PAGE_NO", required = true, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "searchVal", value = "SEARCH_VAL", required = false, type = "String"),
@ApiImplicitParam(name = "userId", value = "USER_ID", required = false, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "pageSize", value = "PAGE_SIZE", required = true, dataType = "Int", example = "100")
@ApiImplicitParam(name = "pageNo", value = "PAGE_NO", required = true, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "searchVal", value = "SEARCH_VAL", required = false, type = "String"),
@ApiImplicitParam(name = "userId", value = "USER_ID", required = false, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "pageSize", value = "PAGE_SIZE", required = true, dataType = "Int", example = "100")
})
@GetMapping(value = "/list-paging")
@ResponseStatus(HttpStatus.OK)
@ -470,7 +460,6 @@ public class ProcessDefinitionController extends BaseController {
@RequestParam(value = "searchVal", required = false) String searchVal,
@RequestParam(value = "userId", required = false, defaultValue = "0") Integer userId,
@RequestParam("pageSize") Integer pageSize) {
Map<String, Object> result = checkPageParams(pageNo, pageSize);
if (result.get(Constants.STATUS) != Status.SUCCESS) {
return returnDataListPaging(result);
@ -481,18 +470,18 @@ public class ProcessDefinitionController extends BaseController {
}
/**
* encapsulation tree view structure
* encapsulation treeview structure
*
* @param loginUser login user
* @param loginUser login user
* @param projectName project name
* @param id process definition id
* @param limit limit
* @param id process definition id
* @param limit limit
* @return tree view json data
*/
@ApiOperation(value = "viewTree", notes = "VIEW_TREE_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "processId", value = "PROCESS_DEFINITION_ID", required = true, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "limit", value = "LIMIT", required = true, dataType = "Int", example = "100")
@ApiImplicitParam(name = "processId", value = "PROCESS_DEFINITION_ID", required = true, dataType = "Int", example = "100"),
@ApiImplicitParam(name = "limit", value = "LIMIT", required = true, dataType = "Int", example = "100")
})
@GetMapping(value = "/view-tree")
@ResponseStatus(HttpStatus.OK)
@ -502,72 +491,69 @@ public class ProcessDefinitionController extends BaseController {
@ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName,
@RequestParam("processId") Integer id,
@RequestParam("limit") Integer limit) throws Exception {
Map<String, Object> result = processDefinitionService.viewTree(id, limit);
return returnDataList(result);
}
/**
* get tasks list by process definition id
* get tasks list by process definition code
*
* @param loginUser login user
* @param projectName project name
* @param processDefinitionId process definition id
* @param loginUser login user
* @param projectName project name
* @param processDefinitionCode process definition code
* @return task list
*/
@ApiOperation(value = "getNodeListByDefinitionId", notes = "GET_NODE_LIST_BY_DEFINITION_ID_NOTES")
@ApiOperation(value = "getNodeListByDefinitionCode", notes = "GET_NODE_LIST_BY_DEFINITION_CODE_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "processDefinitionId", value = "PROCESS_DEFINITION_ID", required = true, dataType = "Int", example = "100")
@ApiImplicitParam(name = "processDefinitionCode", value = "PROCESS_DEFINITION_CODE", required = true, dataType = "Long", example = "100")
})
@GetMapping(value = "gen-task-list")
@ResponseStatus(HttpStatus.OK)
@ApiException(GET_TASKS_LIST_BY_PROCESS_DEFINITION_ID_ERROR)
@AccessLogAnnotation(ignoreRequestArgs = "loginUser")
public Result getNodeListByDefinitionId(
@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName,
@RequestParam("processDefinitionId") Integer processDefinitionId) throws Exception {
Map<String, Object> result = processDefinitionService.getTaskNodeListByDefinitionId(processDefinitionId);
public Result getNodeListByDefinitionCode(
@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName,
@RequestParam("processDefinitionCode") Long processDefinitionCode) throws Exception {
logger.info("query task node name list by definitionCode, login user:{}, project name:{}, code : {}",
loginUser.getUserName(), projectName, processDefinitionCode);
Map<String, Object> result = processDefinitionService.getTaskNodeListByDefinitionCode(processDefinitionCode);
return returnDataList(result);
}
/**
* get tasks list by process definition id
* get tasks list by process definition code list
*
* @param loginUser login user
* @param projectName project name
* @param processDefinitionIdList process definition id list
* @param loginUser login user
* @param projectName project name
* @param processDefinitionCodeList process definition code list
* @return node list data
*/
@ApiOperation(value = "getNodeListByDefinitionIdList", notes = "GET_NODE_LIST_BY_DEFINITION_ID_NOTES")
@ApiOperation(value = "getNodeListByDefinitionCodeList", notes = "GET_NODE_LIST_BY_DEFINITION_CODE_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "processDefinitionIdList", value = "PROCESS_DEFINITION_ID_LIST", required = true, type = "String")
@ApiImplicitParam(name = "processDefinitionCodeList", value = "PROCESS_DEFINITION_CODE_LIST", required = true, type = "String")
})
@GetMapping(value = "get-task-list")
@ResponseStatus(HttpStatus.OK)
@ApiException(GET_TASKS_LIST_BY_PROCESS_DEFINITION_ID_ERROR)
@AccessLogAnnotation(ignoreRequestArgs = "loginUser")
public Result getNodeListByDefinitionIdList(
@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName,
@RequestParam("processDefinitionIdList") String processDefinitionIdList) {
Map<String, Object> result = processDefinitionService.getTaskNodeListByDefinitionIdList(processDefinitionIdList);
public Result getNodeListByDefinitionCodeList(
@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName,
@RequestParam("processDefinitionCodeList") String processDefinitionCodeList) {
Map<String, Object> result = processDefinitionService.getTaskNodeListByDefinitionCodeList(processDefinitionCodeList);
return returnDataList(result);
}
/**
* delete process definition by id
*
* @param loginUser login user
* @param projectName project name
* @param loginUser login user
* @param projectName project name
* @param processDefinitionId process definition id
* @return delete result code
*/
@ApiOperation(value = "deleteProcessDefinitionById", notes = "DELETE_PROCESS_DEFINITION_BY_ID_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "processDefinitionId", value = "PROCESS_DEFINITION_ID", dataType = "Int", example = "100")
@ApiImplicitParam(name = "processDefinitionId", value = "PROCESS_DEFINITION_ID", dataType = "Int", example = "100")
})
@GetMapping(value = "/delete")
@ResponseStatus(HttpStatus.OK)
@ -577,7 +563,6 @@ public class ProcessDefinitionController extends BaseController {
@ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName,
@RequestParam("processDefinitionId") Integer processDefinitionId
) {
Map<String, Object> result = processDefinitionService.deleteProcessDefinitionById(loginUser, projectName, processDefinitionId);
return returnDataList(result);
}
@ -585,14 +570,14 @@ public class ProcessDefinitionController extends BaseController {
/**
* batch delete process definition by ids
*
* @param loginUser login user
* @param projectName project name
* @param loginUser login user
* @param projectName project name
* @param processDefinitionIds process definition id list
* @return delete result code
*/
@ApiOperation(value = "batchDeleteProcessDefinitionByIds", notes = "BATCH_DELETE_PROCESS_DEFINITION_BY_IDS_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "processDefinitionIds", value = "PROCESS_DEFINITION_IDS", type = "String")
@ApiImplicitParam(name = "processDefinitionIds", value = "PROCESS_DEFINITION_IDS", type = "String")
})
@GetMapping(value = "/batch-delete")
@ResponseStatus(HttpStatus.OK)
@ -602,12 +587,10 @@ public class ProcessDefinitionController extends BaseController {
@ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName,
@RequestParam("processDefinitionIds") String processDefinitionIds
) {
Map<String, Object> result = new HashMap<>();
List<String> deleteFailedIdList = new ArrayList<>();
if (StringUtils.isNotEmpty(processDefinitionIds)) {
String[] processDefinitionIdArray = processDefinitionIds.split(",");
for (String strProcessDefinitionId : processDefinitionIdArray) {
int processDefinitionId = Integer.parseInt(strProcessDefinitionId);
try {
@ -634,15 +617,15 @@ public class ProcessDefinitionController extends BaseController {
/**
* batch export process definition by ids
*
* @param loginUser login user
* @param projectName project name
* @param loginUser login user
* @param projectName project name
* @param processDefinitionIds process definition ids
* @param response response
* @param response response
*/
@ApiOperation(value = "batchExportProcessDefinitionByIds", notes = "BATCH_EXPORT_PROCESS_DEFINITION_BY_IDS_NOTES")
@ApiImplicitParams({
@ApiImplicitParam(name = "processDefinitionIds", value = "PROCESS_DEFINITION_ID", required = true, dataType = "String")
@ApiImplicitParam(name = "processDefinitionIds", value = "PROCESS_DEFINITION_ID", required = true, dataType = "String")
})
@GetMapping(value = "/export")
@ResponseBody
@ -652,7 +635,6 @@ public class ProcessDefinitionController extends BaseController {
@RequestParam("processDefinitionIds") String processDefinitionIds,
HttpServletResponse response) {
try {
processDefinitionService.batchExportProcessDefinitionByIds(loginUser, projectName, processDefinitionIds, response);
} catch (Exception e) {
logger.error(Status.BATCH_EXPORT_PROCESS_DEFINE_BY_IDS_ERROR.getMsg(), e);
@ -673,7 +655,6 @@ public class ProcessDefinitionController extends BaseController {
@AccessLogAnnotation(ignoreRequestArgs = "loginUser")
public Result queryProcessDefinitionAllByProjectId(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@RequestParam("projectId") Integer projectId) {
Map<String, Object> result = processDefinitionService.queryProcessDefinitionAllByProjectId(projectId);
return returnDataList(result);
}

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

@ -264,8 +264,12 @@ public enum Status {
EXPORT_PROCESS_DEFINE_BY_ID_ERROR(50028, "export process definition by id error", "导出工作流定义错误"),
BATCH_EXPORT_PROCESS_DEFINE_BY_IDS_ERROR(50028, "batch export process definition by ids error", "批量导出工作流定义错误"),
IMPORT_PROCESS_DEFINE_ERROR(50029, "import process definition error", "导入工作流定义错误"),
PROCESS_DAG_IS_EMPTY(50030, "process dag can not be empty", "工作流dag不能为空"),
TASK_DEFINE_NOT_EXIST(50030, "task definition {0} does not exist", "任务定义[{0}]不存在"),
DELETE_TASK_DEFINE_BY_CODE_ERROR(50031, "delete task definition by code error", "删除任务定义错误"),
DELETE_PROCESS_TASK_RELATION_ERROR(50032, "delete process task relation error", "删除工作流任务关系错误"),
PROCESS_TASK_RELATION_NOT_EXIST(50033, "process task relation {0} does not exist", "工作流任务关系[{0}]不存在"),
PROCESS_TASK_RELATION_EXIST(50034, "process task relation is already exist, processCode:[{0}]", "工作流任务关系已存在, processCode:[{0}]"),
PROCESS_DAG_IS_EMPTY(50035, "process dag can not be empty", "工作流dag不能为空"),
HDFS_NOT_STARTUP(60001, "hdfs not startup", "hdfs未启用"),
/**

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

@ -66,10 +66,10 @@ public interface ExecutorService {
* check whether the process definition can be executed
*
* @param processDefinition process definition
* @param processDefineId process definition id
* @param processDefineCode process definition code
* @return check result code
*/
Map<String, Object> checkProcessDefinitionValid(ProcessDefinition processDefinition, int processDefineId);
Map<String, Object> checkProcessDefinitionValid(ProcessDefinition processDefinition, long processDefineCode);
/**
* do action to process instancepause, stop, repeat, recover from pause, recover from stop

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

@ -231,18 +231,18 @@ public interface ProcessDefinitionService {
/**
* get task node details based on process definition
*
* @param defineId define id
* @param defineCode define code
* @return task node list
*/
Map<String, Object> getTaskNodeListByDefinitionId(Integer defineId);
Map<String, Object> getTaskNodeListByDefinitionCode(Long defineCode);
/**
* get task node details based on process definition
*
* @param defineIdList define id list
* @param defineCodeList define code list
* @return task node list
*/
Map<String, Object> getTaskNodeListByDefinitionIdList(String defineIdList);
Map<String, Object> getTaskNodeListByDefinitionCodeList(String defineCodeList);
/**
* query process definition all by project id
@ -275,6 +275,30 @@ public interface ProcessDefinitionService {
Map<String, Object> switchProcessDefinitionVersion(User loginUser, String projectName
, int processDefinitionId, long version);
/**
* query the pagination versions info by one certain process definition code
*
* @param loginUser login user info to check auth
* @param projectName process definition project name
* @param pageNo page number
* @param pageSize page size
* @param processDefinitionCode process definition code
* @return the pagination process definition versions info of the certain process definition
*/
Map<String, Object> queryProcessDefinitionVersions(User loginUser, String projectName,
int pageNo, int pageSize, long processDefinitionCode);
/**
* delete one certain process definition by version number and process definition id
*
* @param loginUser login user info to check auth
* @param projectName process definition project name
* @param processDefinitionId process definition id
* @param version version number
* @return delele result code
*/
Map<String, Object> deleteByProcessDefinitionIdAndVersion(User loginUser, String projectName,
int processDefinitionId, long version);
/**
* check has associated process definition
*

73
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionVersionService.java

@ -1,73 +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.service;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinitionVersion;
import org.apache.dolphinscheduler.dao.entity.User;
import java.util.Map;
/**
* process definition version service
*/
public interface ProcessDefinitionVersionService {
/**
* add the newest version of one process definition
*
* @param processDefinition the process definition that need to record version
* @return the newest version number of this process definition
*/
long addProcessDefinitionVersion(ProcessDefinition processDefinition);
/**
* query the pagination versions info by one certain process definition id
*
* @param loginUser login user info to check auth
* @param projectName process definition project name
* @param pageNo page number
* @param pageSize page size
* @param processDefinitionId process definition id
* @return the pagination process definition versions info of the certain process definition
*/
Map<String, Object> queryProcessDefinitionVersions(User loginUser, String projectName,
int pageNo, int pageSize, int processDefinitionId);
/**
* query one certain process definition version by version number and process definition id
*
* @param processDefinitionId process definition id
* @param version version number
* @return the process definition version info
*/
ProcessDefinitionVersion queryByProcessDefinitionIdAndVersion(int processDefinitionId,
long version);
/**
* delete one certain process definition by version number and process definition id
*
* @param loginUser login user info to check auth
* @param projectName process definition project name
* @param processDefinitionId process definition id
* @param version version number
* @return delele result code
*/
Map<String, Object> deleteByProcessDefinitionIdAndVersion(User loginUser, String projectName,
int processDefinitionId, long version);
}

18
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java

@ -1,3 +1,4 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -31,6 +32,7 @@ import java.util.Map;
/**
* process instance service
*/
public interface ProcessInstanceService {
/**
@ -148,19 +150,21 @@ public interface ProcessInstanceService {
Map<String, Object> viewGantt(Integer processInstanceId) throws Exception;
/**
* query process instance by processDefinitionId and stateArray
* @param processDefinitionId processDefinitionId
* query process instance by processDefinitionCode and stateArray
*
* @param processDefinitionCode processDefinitionCode
* @param states states array
* @return process instance list
*/
List<ProcessInstance> queryByProcessDefineIdAndStatus(int processDefinitionId, int[] states);
List<ProcessInstance> queryByProcessDefineCodeAndStatus(Long processDefinitionCode, int[] states);
/**
* query process instance by processDefinitionId
* @param processDefinitionId processDefinitionId
* query process instance by processDefinitionCode
*
* @param processDefinitionCode processDefinitionCode
* @param size size
* @return process instance list
*/
List<ProcessInstance> queryByProcessDefineId(int processDefinitionId,int size);
List<ProcessInstance> queryByProcessDefineCode(Long processDefinitionCode,int size);
}
}

40
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessTaskRelationService.java

@ -0,0 +1,40 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.api.service;
import org.apache.dolphinscheduler.dao.entity.User;
import java.util.Map;
/**
* process task relation service
*/
public interface ProcessTaskRelationService {
/**
* query process task relation
*
* @param loginUser login user
* @param projectName project name
* @param processDefinitionCode process definition code
*/
Map<String, Object> queryProcessTaskRelation(User loginUser,
String projectName,
Long processDefinitionCode);
}

88
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskDefinitionService.java

@ -0,0 +1,88 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.api.service;
import org.apache.dolphinscheduler.dao.entity.User;
import java.util.Map;
/**
* task definition service
*/
public interface TaskDefinitionService {
/**
* create task definition
*
* @param loginUser login user
* @param projectName project name
* @param taskDefinitionJson task definition json
*/
Map<String, Object> createTaskDefinition(User loginUser,
String projectName,
String taskDefinitionJson);
/**
* query task definition
*
* @param loginUser login user
* @param projectName project name
* @param taskName task name
*/
Map<String, Object> queryTaskDefinitionByName(User loginUser,
String projectName,
String taskName);
/**
* delete task definition
*
* @param loginUser login user
* @param projectName project name
* @param taskCode task code
*/
Map<String, Object> deleteTaskDefinitionByCode(User loginUser,
String projectName,
Long taskCode);
/**
* update task definition
*
* @param loginUser login user
* @param projectName project name
* @param taskCode task code
* @param taskDefinitionJson task definition json
*/
Map<String, Object> updateTaskDefinition(User loginUser,
String projectName,
Long taskCode,
String taskDefinitionJson);
/**
* update task definition
*
* @param loginUser login user
* @param projectName project name
* @param taskCode task code
* @param version the version user want to switch
*/
Map<String, Object> switchVersion(User loginUser,
String projectName,
Long taskCode,
int version);
}

50
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/DataAnalysisServiceImpl.java

@ -43,13 +43,14 @@ import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskInstanceMapper;
import org.apache.dolphinscheduler.service.process.ProcessService;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;
import org.springframework.beans.factory.annotation.Autowired;
@ -102,7 +103,7 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
projectId,
startDate,
endDate,
(start, end, projectIds) -> this.taskInstanceMapper.countTaskInstanceStateByUser(start, end, projectIds));
(start, end, projectCodes) -> this.taskInstanceMapper.countTaskInstanceStateByUser(start, end, projectCodes));
}
/**
@ -121,7 +122,7 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
projectId,
startDate,
endDate,
(start, end, projectIds) -> this.processInstanceMapper.countInstanceStateByUser(start, end, projectIds));
(start, end, projectCodes) -> this.processInstanceMapper.countInstanceStateByUser(start, end, projectCodes));
// process state count needs to remove state of forced success
if (result.containsKey(Constants.STATUS) && result.get(Constants.STATUS).equals(Status.SUCCESS)) {
((TaskCountDto)result.get(Constants.DATA_LIST)).removeStateFromCountList(ExecutionStatus.FORCED_SUCCESS);
@ -130,7 +131,7 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
}
private Map<String, Object> countStateByProject(User loginUser, int projectId, String startDate, String endDate
, TriFunction<Date, Date, Integer[], List<ExecuteStatusCount>> instanceStateCounter) {
, TriFunction<Date, Date, Long[], List<ExecuteStatusCount>> instanceStateCounter) {
Map<String, Object> result = new HashMap<>();
boolean checkProject = checkProject(loginUser, projectId, result);
if (!checkProject) {
@ -148,9 +149,10 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
}
}
Integer[] projectIdArray = getProjectIdsArrays(loginUser, projectId);
Long[] projectCodeArray = projectId == 0 ? getProjectCodesArrays(loginUser)
: new Long[] { projectMapper.selectById(projectId).getCode() };
List<ExecuteStatusCount> processInstanceStateCounts =
instanceStateCounter.apply(start, end, projectIdArray);
instanceStateCounter.apply(start, end, projectCodeArray);
if (processInstanceStateCounts != null) {
TaskCountDto taskCountResult = new TaskCountDto(processInstanceStateCounts);
@ -171,10 +173,14 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
@Override
public Map<String, Object> countDefinitionByUser(User loginUser, int projectId) {
Map<String, Object> result = new HashMap<>();
Integer[] projectIdArray = getProjectIdsArrays(loginUser, projectId);
boolean checkProject = checkProject(loginUser, projectId, result);
if (!checkProject) {
return result;
}
Long[] projectCodeArray = projectId == 0 ? getProjectCodesArrays(loginUser)
: new Long[] { projectMapper.selectById(projectId).getCode() };
List<DefinitionGroupByUser> defineGroupByUsers = processDefinitionMapper.countDefinitionGroupByUser(
loginUser.getId(), projectIdArray, isAdmin(loginUser));
loginUser.getId(), projectCodeArray, isAdmin(loginUser));
DefineUserDto dto = new DefineUserDto(defineGroupByUsers);
result.put(Constants.DATA_LIST, dto);
@ -222,14 +228,15 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
}
}
Integer[] projectIdArray = getProjectIdsArrays(loginUser, projectId);
Long[] projectCodeArray = projectId == 0 ? getProjectCodesArrays(loginUser)
: new Long[] { projectMapper.selectById(projectId).getCode() };
// count normal command state
Map<CommandType, Integer> normalCountCommandCounts = commandMapper.countCommandState(loginUser.getId(), start, end, projectIdArray)
Map<CommandType, Integer> normalCountCommandCounts = commandMapper.countCommandState(loginUser.getId(), start, end, projectCodeArray)
.stream()
.collect(Collectors.toMap(CommandCount::getCommandType, CommandCount::getCount));
// count error command state
Map<CommandType, Integer> errorCommandCounts = errorCommandMapper.countCommandState(start, end, projectIdArray)
Map<CommandType, Integer> errorCommandCounts = errorCommandMapper.countCommandState(start, end, projectCodeArray)
.stream()
.collect(Collectors.toMap(CommandCount::getCommandType, CommandCount::getCount));
@ -245,17 +252,16 @@ public class DataAnalysisServiceImpl extends BaseServiceImpl implements DataAnal
return result;
}
private Integer[] getProjectIdsArrays(User loginUser, int projectId) {
List<Integer> projectIds = new ArrayList<>();
if (projectId != 0) {
projectIds.add(projectId);
} else if (loginUser.getUserType() == UserType.GENERAL_USER) {
projectIds = processService.getProjectIdListHavePerm(loginUser.getId());
if (projectIds.isEmpty()) {
projectIds.add(0);
}
private Long[] getProjectCodesArrays(User loginUser) {
List<Project> projectList = projectMapper.queryRelationProjectListByUserId(
loginUser.getUserType() == UserType.ADMIN_USER ? 0 : loginUser.getId());
Set<Long> projectCodes = new HashSet<>();
projectList.forEach(project -> projectCodes.add(project.getCode()));
if (loginUser.getUserType() == UserType.GENERAL_USER) {
List<Project> createProjects = projectMapper.queryProjectCreatedByUser(loginUser.getId());
createProjects.forEach(project -> projectCodes.add(project.getCode()));
}
return projectIds.toArray(new Integer[0]);
return projectCodes.toArray(new Long[0]);
}
/**

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

@ -196,18 +196,18 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
* check whether the process definition can be executed
*
* @param processDefinition process definition
* @param processDefineId process definition id
* @param processDefineCode process definition code
* @return check result code
*/
@Override
public Map<String, Object> checkProcessDefinitionValid(ProcessDefinition processDefinition, int processDefineId) {
public Map<String, Object> checkProcessDefinitionValid(ProcessDefinition processDefinition, long processDefineCode) {
Map<String, Object> result = new HashMap<>();
if (processDefinition == null) {
// check process definition exists
putMsg(result, Status.PROCESS_DEFINE_NOT_EXIST, processDefineId);
putMsg(result, Status.PROCESS_DEFINE_NOT_EXIST, processDefineCode);
} else if (processDefinition.getReleaseState() != ReleaseState.ONLINE) {
// check process definition online
putMsg(result, Status.PROCESS_DEFINE_NOT_RELEASE, processDefineId);
putMsg(result, Status.PROCESS_DEFINE_NOT_RELEASE, processDefineCode);
} else {
result.put(Constants.STATUS, Status.SUCCESS);
}
@ -244,9 +244,10 @@ public class ExecutorServiceImpl extends BaseServiceImpl implements ExecutorServ
return result;
}
ProcessDefinition processDefinition = processService.findProcessDefineById(processInstance.getProcessDefinitionId());
ProcessDefinition processDefinition = processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
processInstance.getProcessDefinitionVersion());
if (executeType != ExecuteType.STOP && executeType != ExecuteType.PAUSE) {
result = checkProcessDefinitionValid(processDefinition, processInstance.getProcessDefinitionId());
result = checkProcessDefinitionValid(processDefinition, processInstance.getProcessDefinitionCode());
if (result.get(Constants.STATUS) != Status.SUCCESS) {
return result;
}

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

File diff suppressed because it is too large Load Diff

189
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProcessDefinitionVersionServiceImpl.java

@ -1,189 +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.service.impl;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.service.ProcessDefinitionService;
import org.apache.dolphinscheduler.api.service.ProcessDefinitionVersionService;
import org.apache.dolphinscheduler.api.service.ProjectService;
import org.apache.dolphinscheduler.api.utils.PageInfo;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinitionVersion;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionVersionMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import com.baomidou.mybatisplus.core.metadata.IPage;
import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
import com.google.common.collect.ImmutableMap;
/**
* process definition version service impl
*/
@Service
public class ProcessDefinitionVersionServiceImpl extends BaseServiceImpl implements ProcessDefinitionVersionService {
@Autowired
private ProcessDefinitionVersionMapper processDefinitionVersionMapper;
@Autowired
private ProjectService projectService;
@Autowired
private ProjectMapper projectMapper;
@Autowired
private ProcessDefinitionService processDefinitionService;
/**
* add the newest version of one process definition
*
* @param processDefinition the process definition that need to record version
* @return the newest version number of this process definition
*/
@Override
public long addProcessDefinitionVersion(ProcessDefinition processDefinition) {
long version = this.queryMaxVersionByProcessDefinitionId(processDefinition.getId()) + 1;
ProcessDefinitionVersion processDefinitionVersion = ProcessDefinitionVersion
.newBuilder()
.processDefinitionId(processDefinition.getId())
.version(version)
.processDefinitionJson(processDefinition.getProcessDefinitionJson())
.description(processDefinition.getDescription())
.locations(processDefinition.getLocations())
.connects(processDefinition.getConnects())
.timeout(processDefinition.getTimeout())
.globalParams(processDefinition.getGlobalParams())
.createTime(processDefinition.getUpdateTime())
.warningGroupId(processDefinition.getWarningGroupId())
.resourceIds(processDefinition.getResourceIds())
.build();
processDefinitionVersionMapper.insert(processDefinitionVersion);
return version;
}
/**
* query the max version number by the process definition id
*
* @param processDefinitionId process definition id
* @return the max version number of this id
*/
private long queryMaxVersionByProcessDefinitionId(int processDefinitionId) {
Long maxVersion = processDefinitionVersionMapper.queryMaxVersionByProcessDefinitionId(processDefinitionId);
if (Objects.isNull(maxVersion)) {
return 0L;
} else {
return maxVersion;
}
}
/**
* query the pagination versions info by one certain process definition id
*
* @param loginUser login user info to check auth
* @param projectName process definition project name
* @param pageNo page number
* @param pageSize page size
* @param processDefinitionId process definition id
* @return the pagination process definition versions info of the certain process definition
*/
@Override
public Map<String, Object> queryProcessDefinitionVersions(User loginUser, String projectName, int pageNo, int pageSize, int processDefinitionId) {
Project project = projectMapper.queryByName(projectName);
// check project auth
Map<String, Object> checkResult = projectService.checkProjectAndAuth(loginUser, project, projectName);
Status resultStatus = (Status) checkResult.get(Constants.STATUS);
if (resultStatus != Status.SUCCESS) {
return checkResult;
}
PageInfo<ProcessDefinitionVersion> pageInfo = new PageInfo<>(pageNo, pageSize);
Page<ProcessDefinitionVersion> page = new Page<>(pageNo, pageSize);
IPage<ProcessDefinitionVersion> processDefinitionVersionsPaging = processDefinitionVersionMapper.queryProcessDefinitionVersionsPaging(page, processDefinitionId);
List<ProcessDefinitionVersion> processDefinitionVersions = processDefinitionVersionsPaging.getRecords();
pageInfo.setLists(processDefinitionVersions);
pageInfo.setTotalCount((int) processDefinitionVersionsPaging.getTotal());
return ImmutableMap.of(
Constants.MSG, Status.SUCCESS.getMsg()
, Constants.STATUS, Status.SUCCESS
, Constants.DATA_LIST, pageInfo);
}
/**
* query one certain process definition version by version number and process definition id
*
* @param processDefinitionId process definition id
* @param version version number
* @return the process definition version info
*/
@Override
public ProcessDefinitionVersion queryByProcessDefinitionIdAndVersion(int processDefinitionId, long version) {
return processDefinitionVersionMapper.queryByProcessDefinitionIdAndVersion(processDefinitionId, version);
}
/**
* delete one certain process definition by version number and process definition id
*
* @param loginUser login user info to check auth
* @param projectName process definition project name
* @param processDefinitionId process definition id
* @param version version number
* @return delele result code
*/
@Override
public Map<String, Object> deleteByProcessDefinitionIdAndVersion(User loginUser, String projectName, int processDefinitionId, long version) {
Map<String, Object> result = new HashMap<>();
Project project = projectMapper.queryByName(projectName);
// check project auth
Map<String, Object> checkResult = projectService.checkProjectAndAuth(loginUser, project, projectName);
Status resultStatus = (Status) checkResult.get(Constants.STATUS);
if (resultStatus != Status.SUCCESS) {
return checkResult;
}
// check has associated process definition
boolean hasAssociatedProcessDefinition = processDefinitionService.checkHasAssociatedProcessDefinition(processDefinitionId, version);
if (hasAssociatedProcessDefinition) {
putMsg(result, Status.PROCESS_DEFINITION_VERSION_IS_USED);
return result;
}
processDefinitionVersionMapper.deleteByProcessDefinitionIdAndVersion(processDefinitionId, version);
putMsg(result, Status.SUCCESS);
return result;
}
}

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

@ -30,7 +30,6 @@ import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.service.ExecutorService;
import org.apache.dolphinscheduler.api.service.LoggerService;
import org.apache.dolphinscheduler.api.service.ProcessDefinitionService;
import org.apache.dolphinscheduler.api.service.ProcessDefinitionVersionService;
import org.apache.dolphinscheduler.api.service.ProcessInstanceService;
import org.apache.dolphinscheduler.api.service.ProjectService;
import org.apache.dolphinscheduler.api.service.UsersService;
@ -44,7 +43,6 @@ import org.apache.dolphinscheduler.common.enums.TaskType;
import org.apache.dolphinscheduler.common.graph.DAG;
import org.apache.dolphinscheduler.common.model.TaskNode;
import org.apache.dolphinscheduler.common.model.TaskNodeRelation;
import org.apache.dolphinscheduler.common.process.ProcessDag;
import org.apache.dolphinscheduler.common.process.Property;
import org.apache.dolphinscheduler.common.utils.CollectionUtils;
import org.apache.dolphinscheduler.common.utils.DateUtils;
@ -56,14 +54,16 @@ import org.apache.dolphinscheduler.dao.entity.ProcessData;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.entity.Tenant;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionLogMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessInstanceMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionLogMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskInstanceMapper;
import org.apache.dolphinscheduler.dao.utils.DagHelper;
import org.apache.dolphinscheduler.service.process.ProcessService;
import java.io.BufferedReader;
@ -71,7 +71,6 @@ import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.io.InputStreamReader;
import java.nio.charset.StandardCharsets;
import java.text.ParseException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Date;
@ -82,6 +81,8 @@ import java.util.Objects;
import java.util.Optional;
import java.util.stream.Collectors;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import org.springframework.transaction.annotation.Transactional;
@ -95,6 +96,11 @@ import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
@Service
public class ProcessInstanceServiceImpl extends BaseServiceImpl implements ProcessInstanceService {
private static final Logger logger = LoggerFactory.getLogger(ProcessInstanceService.class);
public static final String TASK_TYPE = "taskType";
public static final String LOCAL_PARAMS_LIST = "localParamsList";
@Autowired
ProjectMapper projectMapper;
@ -113,9 +119,6 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
@Autowired
ProcessDefinitionService processDefinitionService;
@Autowired
ProcessDefinitionVersionService processDefinitionVersionService;
@Autowired
ExecutorService execService;
@ -125,6 +128,11 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
@Autowired
LoggerService loggerService;
@Autowired
ProcessDefinitionLogMapper processDefinitionLogMapper;
@Autowired
TaskDefinitionLogMapper taskDefinitionLogMapper;
@Autowired
UsersService usersService;
@ -192,10 +200,21 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
}
ProcessInstance processInstance = processService.findProcessInstanceDetailById(processId);
ProcessDefinition processDefinition = processService.findProcessDefineById(processInstance.getProcessDefinitionId());
processInstance.setWarningGroupId(processDefinition.getWarningGroupId());
result.put(DATA_LIST, processInstance);
putMsg(result, Status.SUCCESS);
ProcessDefinition processDefinition = processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
processInstance.getProcessDefinitionVersion());
if (processDefinition == null) {
putMsg(result, Status.PROCESS_DEFINE_NOT_EXIST, processId);
} else {
processInstance.setWarningGroupId(processDefinition.getWarningGroupId());
processInstance.setConnects(processDefinition.getConnects());
processInstance.setLocations(processDefinition.getLocations());
ProcessData processData = processService.genProcessData(processDefinition);
processInstance.setProcessInstanceJson(JSONUtils.toJsonString(processData));
result.put(DATA_LIST, processInstance);
putMsg(result, Status.SUCCESS);
}
return result;
}
@ -247,9 +266,11 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
PageInfo<ProcessInstance> pageInfo = new PageInfo<>(pageNo, pageSize);
int executorId = usersService.getUserIdByName(executorName);
IPage<ProcessInstance> processInstanceList =
processInstanceMapper.queryProcessInstanceListPaging(page,
project.getId(), processDefineId, searchVal, executorId, statusArray, host, start, end);
ProcessDefinition processDefinition = processDefineMapper.queryByDefineId(processDefineId);
IPage<ProcessInstance> processInstanceList = processInstanceMapper.queryProcessInstanceListPaging(page,
project.getCode(), processDefinition == null ? 0L : processDefinition.getCode(), searchVal,
executorId, statusArray, host, start, end);
List<ProcessInstance> processInstances = processInstanceList.getRecords();
List<Integer> userIds = CollectionUtils.transformToList(processInstances, ProcessInstance::getExecutorId);
@ -306,7 +327,7 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
*/
private void addDependResultForTaskList(List<TaskInstance> taskInstanceList) throws IOException {
for (TaskInstance taskInstance : taskInstanceList) {
if (taskInstance.getTaskType().equalsIgnoreCase(TaskType.DEPENDENT.toString())) {
if (TaskType.DEPENDENT.getDesc().equalsIgnoreCase(taskInstance.getTaskType())) {
Result<String> logResult = loggerService.queryLog(
taskInstance.getId(), Constants.LOG_QUERY_SKIP_LINE_NUMBER, Constants.LOG_QUERY_LIMIT);
if (logResult.getCode() == Status.SUCCESS.ordinal()) {
@ -402,99 +423,103 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
* @param locations locations
* @param connects connects
* @return update result code
* @throws ParseException parse exception for json parse
*/
@Transactional
@Override
public Map<String, Object> updateProcessInstance(User loginUser, String projectName, Integer processInstanceId,
String processInstanceJson, String scheduleTime, Boolean syncDefine,
Flag flag, String locations, String connects) throws ParseException {
Flag flag, String locations, String connects) {
Map<String, Object> result = new HashMap<>();
Project project = projectMapper.queryByName(projectName);
//check project permission
Map<String, Object> checkResult = projectService.checkProjectAndAuth(loginUser, project, projectName);
Status resultEnum = (Status) checkResult.get(Constants.STATUS);
if (resultEnum != Status.SUCCESS) {
return checkResult;
}
//check process instance exists
ProcessInstance processInstance = processService.findProcessInstanceDetailById(processInstanceId);
if (processInstance == null) {
putMsg(result, Status.PROCESS_INSTANCE_NOT_EXIST, processInstanceId);
return result;
}
//check process instance status
if (!processInstance.getState().typeIsFinished()) {
putMsg(result, Status.PROCESS_INSTANCE_STATE_OPERATION_ERROR,
processInstance.getName(), processInstance.getState().toString(), "update");
return result;
}
Date schedule = null;
schedule = processInstance.getScheduleTime();
if (scheduleTime != null) {
schedule = DateUtils.getScheduleDate(scheduleTime);
ProcessDefinition processDefinition = processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
processInstance.getProcessDefinitionVersion());
ProcessData processData = JSONUtils.parseObject(processInstanceJson, ProcessData.class);
//check workflow json is valid
result = processDefinitionService.checkProcessNodeList(processData, processInstanceJson);
if (result.get(Constants.STATUS) != Status.SUCCESS) {
return result;
}
processInstance.setScheduleTime(schedule);
processInstance.setLocations(locations);
processInstance.setConnects(connects);
String globalParams = null;
String originDefParams = null;
int timeout = processInstance.getTimeout();
ProcessDefinition processDefinition = processService.findProcessDefineById(processInstance.getProcessDefinitionId());
if (StringUtils.isNotEmpty(processInstanceJson)) {
ProcessData processData = JSONUtils.parseObject(processInstanceJson, ProcessData.class);
//check workflow json is valid
Map<String, Object> checkFlowJson = processDefinitionService.checkProcessNodeList(processData, processInstanceJson);
if (checkFlowJson.get(Constants.STATUS) != Status.SUCCESS) {
return result;
}
Tenant tenant = processService.getTenantForProcess(processData.getTenantId(),
processDefinition.getUserId());
setProcessInstance(processInstance, tenant, scheduleTime, processData);
int updateDefine = 1;
if (Boolean.TRUE.equals(syncDefine)) {
processDefinition.setId(processDefineMapper.queryByCode(processInstance.getProcessDefinitionCode()).getId());
updateDefine = syncDefinition(loginUser, project, locations, connects,
processInstance, processDefinition, processData);
originDefParams = JSONUtils.toJsonString(processData.getGlobalParams());
List<Property> globalParamList = processData.getGlobalParams();
Map<String, String> globalParamMap = Optional.ofNullable(globalParamList).orElse(Collections.emptyList()).stream().collect(Collectors.toMap(Property::getProp, Property::getValue));
globalParams = ParameterUtils.curingGlobalParams(globalParamMap, globalParamList,
processInstance.getCmdTypeIfComplement(), schedule);
timeout = processData.getTimeout();
processInstance.setTimeout(timeout);
Tenant tenant = processService.getTenantForProcess(processData.getTenantId(),
processDefinition.getUserId());
if (tenant != null) {
processInstance.setTenantCode(tenant.getTenantCode());
}
// get the processinstancejson before saving,and then save the name and taskid
String oldJson = processInstance.getProcessInstanceJson();
if (StringUtils.isNotEmpty(oldJson)) {
processInstanceJson = processService.changeJson(processData,oldJson);
}
processInstance.setProcessInstanceJson(processInstanceJson);
processInstance.setGlobalParams(globalParams);
processInstance.setProcessDefinitionVersion(processDefinitionLogMapper.
queryMaxVersionForDefinition(processInstance.getProcessDefinitionCode()));
}
int update = processService.updateProcessInstance(processInstance);
int updateDefine = 1;
if (Boolean.TRUE.equals(syncDefine)) {
processDefinition.setProcessDefinitionJson(processInstanceJson);
processDefinition.setGlobalParams(originDefParams);
processDefinition.setLocations(locations);
processDefinition.setConnects(connects);
processDefinition.setTimeout(timeout);
processDefinition.setUpdateTime(new Date());
// add process definition version
long version = processDefinitionVersionService.addProcessDefinitionVersion(processDefinition);
processDefinition.setVersion(version);
updateDefine = processDefineMapper.updateById(processDefinition);
}
if (update > 0 && updateDefine > 0) {
putMsg(result, Status.SUCCESS);
} else {
putMsg(result, Status.UPDATE_PROCESS_INSTANCE_ERROR);
}
return result;
}
/**
* sync definition according process instance
*/
private int syncDefinition(User loginUser, Project project, String locations, String connects,
ProcessInstance processInstance, ProcessDefinition processDefinition,
ProcessData processData) {
String originDefParams = JSONUtils.toJsonString(processData.getGlobalParams());
processDefinition.setGlobalParams(originDefParams);
processDefinition.setLocations(locations);
processDefinition.setConnects(connects);
processDefinition.setTimeout(processInstance.getTimeout());
processDefinition.setUpdateTime(new Date());
return processService.saveProcessDefinition(loginUser, project, processDefinition.getName(),
processDefinition.getDescription(), locations, connects,
processData, processDefinition, false);
}
/**
* update process instance attributes
*/
private void setProcessInstance(ProcessInstance processInstance, Tenant tenant, String scheduleTime, ProcessData processData) {
Date schedule = processInstance.getScheduleTime();
if (scheduleTime != null) {
schedule = DateUtils.getScheduleDate(scheduleTime);
}
processInstance.setScheduleTime(schedule);
List<Property> globalParamList = processData.getGlobalParams();
Map<String, String> globalParamMap = Optional.ofNullable(globalParamList)
.orElse(Collections.emptyList())
.stream()
.collect(Collectors.toMap(Property::getProp, Property::getValue));
String globalParams = ParameterUtils.curingGlobalParams(globalParamMap, globalParamList,
processInstance.getCmdTypeIfComplement(), schedule);
processInstance.setTimeout(processData.getTimeout());
if (tenant != null) {
processInstance.setTenantCode(tenant.getTenantCode());
}
processInstance.setGlobalParams(globalParams);
}
/**
@ -599,34 +624,43 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
Map<String, String> timeParams = BusinessTimeUtils
.getBusinessTime(processInstance.getCmdTypeIfComplement(),
processInstance.getScheduleTime());
String workflowInstanceJson = processInstance.getProcessInstanceJson();
ProcessData workflowData = JSONUtils.parseObject(workflowInstanceJson, ProcessData.class);
String userDefinedParams = processInstance.getGlobalParams();
// global params
List<Property> globalParams = new ArrayList<>();
if (userDefinedParams != null && userDefinedParams.length() > 0) {
globalParams = JSONUtils.toList(userDefinedParams, Property.class);
}
List<TaskNode> taskNodeList = workflowData.getTasks();
// global param string
String globalParamStr = JSONUtils.toJsonString(globalParams);
globalParamStr = ParameterUtils.convertParameterPlaceholders(globalParamStr, timeParams);
String globalParamStr = ParameterUtils.convertParameterPlaceholders(JSONUtils.toJsonString(globalParams), timeParams);
globalParams = JSONUtils.toList(globalParamStr, Property.class);
for (Property property : globalParams) {
timeParams.put(property.getProp(), property.getValue());
}
// local params
if (userDefinedParams != null && userDefinedParams.length() > 0) {
globalParams = JSONUtils.toList(userDefinedParams, Property.class);
}
Map<String, Map<String, Object>> localUserDefParams = getLocalParams(processInstance, timeParams);
Map<String, Object> resultMap = new HashMap<>();
resultMap.put(GLOBAL_PARAMS, globalParams);
resultMap.put(LOCAL_PARAMS, localUserDefParams);
result.put(DATA_LIST, resultMap);
putMsg(result, Status.SUCCESS);
return result;
}
/**
* get local params
*/
private Map<String, Map<String, Object>> getLocalParams(ProcessInstance processInstance, Map<String, String> timeParams) {
Map<String, Map<String, Object>> localUserDefParams = new HashMap<>();
for (TaskNode taskNode : taskNodeList) {
String parameter = taskNode.getParams();
List<TaskInstance> taskInstanceList = taskInstanceMapper.findValidTaskListByProcessId(processInstance.getId(), Flag.YES);
for (TaskInstance taskInstance : taskInstanceList) {
TaskDefinitionLog taskDefinitionLog = taskDefinitionLogMapper.queryByDefinitionCodeAndVersion(
taskInstance.getTaskCode(), taskInstance.getTaskDefinitionVersion());
String parameter = taskDefinitionLog.getTaskParams();
Map<String, String> map = JSONUtils.toMap(parameter);
String localParams = map.get(LOCAL_PARAMS);
if (localParams != null && !localParams.isEmpty()) {
@ -634,23 +668,15 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
List<Property> localParamsList = JSONUtils.toList(localParams, Property.class);
Map<String, Object> localParamsMap = new HashMap<>();
localParamsMap.put(Constants.TASK_TYPE, taskNode.getType());
localParamsMap.put(Constants.LOCAL_PARAMS_LIST, localParamsList);
localParamsMap.put(TASK_TYPE, taskDefinitionLog.getTaskType());
localParamsMap.put(LOCAL_PARAMS_LIST, localParamsList);
if (CollectionUtils.isNotEmpty(localParamsList)) {
localUserDefParams.put(taskNode.getName(), localParamsMap);
localUserDefParams.put(taskDefinitionLog.getName(), localParamsMap);
}
}
}
Map<String, Object> resultMap = new HashMap<>();
resultMap.put(GLOBAL_PARAMS, globalParams);
resultMap.put(LOCAL_PARAMS, localUserDefParams);
result.put(DATA_LIST, resultMap);
putMsg(result, Status.SUCCESS);
return result;
return localUserDefParams;
}
/**
@ -670,9 +696,12 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
throw new RuntimeException("workflow instance is null");
}
ProcessDefinition processDefinition = processDefinitionLogMapper.queryByDefinitionCodeAndVersion(
processInstance.getProcessDefinitionCode(),
processInstance.getProcessDefinitionVersion()
);
GanttDto ganttDto = new GanttDto();
DAG<String, TaskNode, TaskNodeRelation> dag = processInstance2DAG(processInstance);
DAG<String, TaskNode, TaskNodeRelation> dag = processService.genDagGraph(processDefinition);
//topological sort
List<String> nodeList = dag.topologicalSort();
@ -705,44 +734,27 @@ public class ProcessInstanceServiceImpl extends BaseServiceImpl implements Proce
}
/**
* process instance to DAG
* query process instance by processDefinitionCode and stateArray
*
* @param processInstance input process instance
* @return process instance dag.
*/
private static DAG<String, TaskNode, TaskNodeRelation> processInstance2DAG(ProcessInstance processInstance) {
String processDefinitionJson = processInstance.getProcessInstanceJson();
ProcessData processData = JSONUtils.parseObject(processDefinitionJson, ProcessData.class);
List<TaskNode> taskNodeList = processData.getTasks();
ProcessDag processDag = DagHelper.getProcessDag(taskNodeList);
return DagHelper.buildDagGraph(processDag);
}
/**
* query process instance by processDefinitionId and stateArray
* @param processDefinitionId processDefinitionId
* @param processDefinitionCode processDefinitionCode
* @param states states array
* @return process instance list
*/
@Override
public List<ProcessInstance> queryByProcessDefineIdAndStatus(int processDefinitionId, int[] states) {
return processInstanceMapper.queryByProcessDefineIdAndStatus(processDefinitionId, states);
public List<ProcessInstance> queryByProcessDefineCodeAndStatus(Long processDefinitionCode, int[] states) {
return processInstanceMapper.queryByProcessDefineCodeAndStatus(processDefinitionCode, states);
}
/**
* query process instance by processDefinitionId
* @param processDefinitionId processDefinitionId
* query process instance by processDefinitionCode
*
* @param processDefinitionCode processDefinitionCode
* @param size size
* @return process instance list
*/
@Override
public List<ProcessInstance> queryByProcessDefineId(int processDefinitionId, int size) {
return processInstanceMapper.queryByProcessDefineId(processDefinitionId, size);
public List<ProcessInstance> queryByProcessDefineCode(Long processDefinitionCode, int size) {
return processInstanceMapper.queryByProcessDefineCode(processDefinitionCode, size);
}
}

78
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProcessTaskRelationServiceImpl.java

@ -0,0 +1,78 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.api.service.impl;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.service.ProcessTaskRelationService;
import org.apache.dolphinscheduler.api.service.ProjectService;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelation;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
/**
* task definition service impl
*/
@Service
public class ProcessTaskRelationServiceImpl extends BaseServiceImpl implements
ProcessTaskRelationService {
private static final Logger logger = LoggerFactory.getLogger(ProcessTaskRelationServiceImpl.class);
@Autowired
private ProjectMapper projectMapper;
@Autowired
private ProjectService projectService;
@Autowired
private ProcessTaskRelationMapper processTaskRelationMapper;
/**
* query process task relation
*
* @param loginUser login user
* @param projectName project name
* @param processDefinitionCode process definition code
*/
@Override
public Map<String, Object> queryProcessTaskRelation(User loginUser, String projectName, Long processDefinitionCode) {
Map<String, Object> result = new HashMap<>();
Project project = projectMapper.queryByName(projectName);
// check project auth
Map<String, Object> checkResult = projectService.checkProjectAndAuth(loginUser, project, projectName);
if (checkResult.get(Constants.STATUS) != Status.SUCCESS) {
return checkResult;
}
List<ProcessTaskRelation> processTaskRelationList = processTaskRelationMapper.queryByProcessCode(project.getCode(), processDefinitionCode);
result.put(Constants.DATA_LIST, processTaskRelationList);
putMsg(result, Status.SUCCESS);
return result;
}
}

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

@ -24,6 +24,8 @@ import org.apache.dolphinscheduler.api.service.ProjectService;
import org.apache.dolphinscheduler.api.utils.PageInfo;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.common.utils.SnowFlakeUtils;
import org.apache.dolphinscheduler.common.utils.SnowFlakeUtils.SnowFlakeException;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.ProjectUser;
@ -94,15 +96,21 @@ public class ProjectServiceImpl extends BaseServiceImpl implements ProjectServic
Date now = new Date();
try {
project = Project
.newBuilder()
.name(name)
.code(SnowFlakeUtils.getInstance().nextId())
.description(desc)
.userId(loginUser.getId())
.userName(loginUser.getUserName())
.createTime(now)
.updateTime(now)
.build();
} catch (SnowFlakeException e) {
putMsg(result, Status.CREATE_PROJECT_ERROR);
return result;
}
if (projectMapper.insert(project) > 0) {
result.put(Constants.DATA_LIST, project.getId());
@ -224,7 +232,7 @@ public class ProjectServiceImpl extends BaseServiceImpl implements ProjectServic
return result;
}
List<ProcessDefinition> processDefinitionList = processDefinitionMapper.queryAllDefinitionList(projectId);
List<ProcessDefinition> processDefinitionList = processDefinitionMapper.queryAllDefinitionList(project.getCode());
if (!processDefinitionList.isEmpty()) {
putMsg(result, Status.DELETE_PROJECT_ERROR_DEFINES_NOT_NULL);

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

@ -691,7 +691,7 @@ public class ResourcesServiceImpl extends BaseServiceImpl implements ResourcesSe
// get all resource id of process definitions those is released
List<Map<String, Object>> list = processDefinitionMapper.listResources();
Map<Integer, Set<Integer>> resourceProcessMap = ResourceProcessDefinitionUtils.getResourceProcessDefinitionMap(list);
Map<Integer, Set<Long>> resourceProcessMap = ResourceProcessDefinitionUtils.getResourceProcessDefinitionMap(list);
Set<Integer> resourceIdSet = resourceProcessMap.keySet();
// get all children of the resource
List<Integer> allChildren = listAllChildren(resource,true);

281
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskDefinitionServiceImpl.java

@ -0,0 +1,281 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.api.service.impl;
import static org.apache.dolphinscheduler.api.enums.Status.DATA_IS_NOT_VALID;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.service.ProjectService;
import org.apache.dolphinscheduler.api.service.TaskDefinitionService;
import org.apache.dolphinscheduler.api.utils.CheckUtils;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.model.TaskNode;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.SnowFlakeUtils;
import org.apache.dolphinscheduler.common.utils.SnowFlakeUtils.SnowFlakeException;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelation;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
import org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionLogMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
import org.apache.dolphinscheduler.service.process.ProcessService;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import org.springframework.transaction.annotation.Transactional;
/**
* task definition service impl
*/
@Service
public class TaskDefinitionServiceImpl extends BaseServiceImpl implements TaskDefinitionService {
private static final Logger logger = LoggerFactory.getLogger(TaskDefinitionServiceImpl.class);
@Autowired
private ProjectMapper projectMapper;
@Autowired
private ProjectService projectService;
@Autowired
private TaskDefinitionMapper taskDefinitionMapper;
@Autowired
private TaskDefinitionLogMapper taskDefinitionLogMapper;
@Autowired
private ProcessTaskRelationMapper processTaskRelationMapper;
@Autowired
private ProcessService processService;
/**
* create task definition
*
* @param loginUser login user
* @param projectName project name
* @param taskDefinitionJson task definition json
*/
@Transactional(rollbackFor = RuntimeException.class)
@Override
public Map<String, Object> createTaskDefinition(User loginUser,
String projectName,
String taskDefinitionJson) {
Map<String, Object> result = new HashMap<>();
Project project = projectMapper.queryByName(projectName);
// check project auth
Map<String, Object> checkResult = projectService.checkProjectAndAuth(loginUser, project, projectName);
Status resultStatus = (Status) checkResult.get(Constants.STATUS);
if (resultStatus != Status.SUCCESS) {
return checkResult;
}
TaskNode taskNode = JSONUtils.parseObject(taskDefinitionJson, TaskNode.class);
checkTaskNode(result, taskNode, taskDefinitionJson);
if (result.get(Constants.STATUS) == DATA_IS_NOT_VALID
|| result.get(Constants.STATUS) == Status.PROCESS_NODE_S_PARAMETER_INVALID) {
return result;
}
TaskDefinition taskDefinition = new TaskDefinition();
long code = 0L;
try {
code = SnowFlakeUtils.getInstance().nextId();
taskDefinition.setCode(code);
} catch (SnowFlakeException e) {
logger.error("Task code get error, ", e);
}
if (code == 0L) {
putMsg(result, Status.INTERNAL_SERVER_ERROR_ARGS, "Error generating task definition code");
return result;
}
int insert = processService.saveTaskDefinition(loginUser, project.getCode(), taskNode, taskDefinition);
// return taskDefinition object with code
result.put(Constants.DATA_LIST, code);
putMsg(result, Status.SUCCESS, insert);
return result;
}
/**
* query task definition
*
* @param loginUser login user
* @param projectName project name
* @param taskName task name
*/
@Override
public Map<String, Object> queryTaskDefinitionByName(User loginUser, String projectName, String taskName) {
Map<String, Object> result = new HashMap<>();
Project project = projectMapper.queryByName(projectName);
Map<String, Object> checkResult = projectService.checkProjectAndAuth(loginUser, project, projectName);
Status resultStatus = (Status) checkResult.get(Constants.STATUS);
if (resultStatus != Status.SUCCESS) {
return checkResult;
}
TaskDefinition taskDefinition = taskDefinitionMapper.queryByDefinitionName(project.getCode(), taskName);
if (taskDefinition == null) {
putMsg(result, Status.TASK_DEFINE_NOT_EXIST, taskName);
} else {
result.put(Constants.DATA_LIST, taskDefinition);
putMsg(result, Status.SUCCESS);
}
return result;
}
/**
* delete task definition
*
* @param loginUser login user
* @param projectName project name
* @param taskCode task code
*/
@Transactional(rollbackFor = RuntimeException.class)
@Override
public Map<String, Object> deleteTaskDefinitionByCode(User loginUser, String projectName, Long taskCode) {
Map<String, Object> result = new HashMap<>(5);
Project project = projectMapper.queryByName(projectName);
Map<String, Object> checkResult = projectService.checkProjectAndAuth(loginUser, project, projectName);
Status resultEnum = (Status) checkResult.get(Constants.STATUS);
if (resultEnum != Status.SUCCESS) {
return checkResult;
}
List<ProcessTaskRelation> processTaskRelationList = processTaskRelationMapper.queryByTaskCode(taskCode);
if (!processTaskRelationList.isEmpty()) {
Set<Long> processDefinitionCodes = processTaskRelationList
.stream()
.map(ProcessTaskRelation::getProcessDefinitionCode)
.collect(Collectors.toSet());
putMsg(result, Status.PROCESS_TASK_RELATION_EXIST, StringUtils.join(processDefinitionCodes, ","));
return result;
}
int delete = taskDefinitionMapper.deleteByCode(taskCode);
if (delete > 0) {
putMsg(result, Status.SUCCESS);
} else {
putMsg(result, Status.DELETE_TASK_DEFINE_BY_CODE_ERROR);
}
return result;
}
/**
* update task definition
*
* @param loginUser login user
* @param projectName project name
* @param taskCode task code
* @param taskDefinitionJson task definition json
*/
@Transactional(rollbackFor = RuntimeException.class)
@Override
public Map<String, Object> updateTaskDefinition(User loginUser, String projectName, Long taskCode, String taskDefinitionJson) {
Map<String, Object> result = new HashMap<>(5);
Project project = projectMapper.queryByName(projectName);
Map<String, Object> checkResult = projectService.checkProjectAndAuth(loginUser, project, projectName);
Status resultEnum = (Status) checkResult.get(Constants.STATUS);
if (resultEnum != Status.SUCCESS) {
return checkResult;
}
if (processService.isTaskOnline(taskCode)) {
putMsg(result, Status.PROCESS_DEFINE_STATE_ONLINE);
return result;
}
TaskDefinition taskDefinition = taskDefinitionMapper.queryByDefinitionCode(taskCode);
if (taskDefinition == null) {
putMsg(result, Status.TASK_DEFINE_NOT_EXIST, taskCode);
return result;
}
TaskNode taskNode = JSONUtils.parseObject(taskDefinitionJson, TaskNode.class);
checkTaskNode(result, taskNode, taskDefinitionJson);
if (result.get(Constants.STATUS) == DATA_IS_NOT_VALID
|| result.get(Constants.STATUS) == Status.PROCESS_NODE_S_PARAMETER_INVALID) {
return result;
}
int update = processService.updateTaskDefinition(loginUser, project.getCode(), taskNode, taskDefinition);
result.put(Constants.DATA_LIST, taskCode);
putMsg(result, Status.SUCCESS, update);
return result;
}
public void checkTaskNode(Map<String, Object> result, TaskNode taskNode, String taskDefinitionJson) {
if (taskNode == null) {
logger.error("taskDefinitionJson is not valid json");
putMsg(result, Status.DATA_IS_NOT_VALID, taskDefinitionJson);
return;
}
if (!CheckUtils.checkTaskNodeParameters(taskNode)) {
logger.error("task node {} parameter invalid", taskNode.getName());
putMsg(result, Status.PROCESS_NODE_S_PARAMETER_INVALID, taskNode.getName());
}
}
/**
* update task definition
*
* @param loginUser login user
* @param projectName project name
* @param taskCode task code
* @param version the version user want to switch
*/
@Override
public Map<String, Object> switchVersion(User loginUser, String projectName, Long taskCode, int version) {
Map<String, Object> result = new HashMap<>(5);
Project project = projectMapper.queryByName(projectName);
Map<String, Object> checkResult = projectService.checkProjectAndAuth(loginUser, project, projectName);
Status resultEnum = (Status) checkResult.get(Constants.STATUS);
if (resultEnum != Status.SUCCESS) {
return checkResult;
}
if (processService.isTaskOnline(taskCode)) {
putMsg(result, Status.PROCESS_DEFINE_STATE_ONLINE);
return result;
}
TaskDefinition taskDefinition = taskDefinitionMapper.queryByDefinitionCode(taskCode);
if (taskDefinition == null) {
putMsg(result, Status.TASK_DEFINE_NOT_EXIST, taskCode);
return result;
}
TaskDefinitionLog taskDefinitionLog = taskDefinitionLogMapper.queryByDefinitionCodeAndVersion(taskCode, version);
taskDefinitionLog.setUserId(loginUser.getId());
taskDefinitionLog.setUpdateTime(new Date());
taskDefinitionMapper.updateById(taskDefinitionLog);
result.put(Constants.DATA_LIST, taskCode);
putMsg(result, Status.SUCCESS);
return result;
}
}

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

@ -118,7 +118,7 @@ public class TaskInstanceServiceImpl extends BaseServiceImpl implements TaskInst
int executorId = usersService.getUserIdByName(executorName);
IPage<TaskInstance> taskInstanceIPage = taskInstanceMapper.queryTaskInstanceListPaging(
page, project.getId(), processInstanceId, processInstanceName, searchVal, taskName, executorId, statusArray, host, start, end
page, project.getCode(), processInstanceId, processInstanceName, searchVal, taskName, executorId, statusArray, host, start, end
);
Set<String> exclusionSet = new HashSet<>();
exclusionSet.add(Constants.CLASS);

6
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/UsersServiceImpl.java

@ -545,10 +545,10 @@ public class UsersServiceImpl extends BaseServiceImpl implements UsersService {
putMsg(result, Status.USER_NOT_EXIST, userId);
return result;
}
//if the selected projectIds are empty, delete all items associated with the user
projectUserMapper.deleteProjectRelation(0, userId);
//if the selected projectIds are empty, delete all items associated with the user
if (check(result, StringUtils.isEmpty(projectIds), Status.SUCCESS)) {
projectUserMapper.deleteProjectRelation(0, userId);
return result;
}
@ -618,7 +618,7 @@ public class UsersServiceImpl extends BaseServiceImpl implements UsersService {
// get all resource id of process definitions those is released
List<Map<String, Object>> list = processDefinitionMapper.listResourcesByUser(userId);
Map<Integer, Set<Integer>> resourceProcessMap = ResourceProcessDefinitionUtils.getResourceProcessDefinitionMap(list);
Map<Integer, Set<Long>> resourceProcessMap = ResourceProcessDefinitionUtils.getResourceProcessDefinitionMap(list);
Set<Integer> resourceIdSet = resourceProcessMap.keySet();
resourceIdSet.retainAll(oldAuthorizedResIds);

95
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkFlowLineageServiceImpl.java

@ -20,12 +20,14 @@ package org.apache.dolphinscheduler.api.service.impl;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.service.WorkFlowLineageService;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.utils.CollectionUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessLineage;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.WorkFlowLineage;
import org.apache.dolphinscheduler.dao.entity.WorkFlowRelation;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.apache.dolphinscheduler.dao.mapper.WorkFlowLineageMapper;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@ -44,28 +46,58 @@ public class WorkFlowLineageServiceImpl extends BaseServiceImpl implements WorkF
@Autowired
private WorkFlowLineageMapper workFlowLineageMapper;
@Autowired
private ProjectMapper projectMapper;
@Override
public Map<String, Object> queryWorkFlowLineageByName(String workFlowName, int projectId) {
Project project = projectMapper.selectById(projectId);
Map<String, Object> result = new HashMap<>();
List<WorkFlowLineage> workFlowLineageList = workFlowLineageMapper.queryByName(workFlowName, projectId);
List<WorkFlowLineage> workFlowLineageList = workFlowLineageMapper.queryByName(workFlowName, project.getCode());
result.put(Constants.DATA_LIST, workFlowLineageList);
putMsg(result, Status.SUCCESS);
return result;
}
private void getWorkFlowRelationRecursion(Set<Integer> ids, List<WorkFlowRelation> workFlowRelations, Set<Integer> sourceIds) {
for (int id : ids) {
sourceIds.addAll(ids);
List<WorkFlowRelation> workFlowRelationsTmp = workFlowLineageMapper.querySourceTarget(id);
if (CollectionUtils.isNotEmpty(workFlowRelationsTmp)) {
Set<Integer> idsTmp = new HashSet<>();
for (WorkFlowRelation workFlowRelation:workFlowRelationsTmp) {
if (!sourceIds.contains(workFlowRelation.getTargetWorkFlowId())) {
idsTmp.add(workFlowRelation.getTargetWorkFlowId());
private void getRelation(Map<Integer, WorkFlowLineage> workFlowLineageMap,
Set<WorkFlowRelation> workFlowRelations,
ProcessLineage processLineage) {
List<ProcessLineage> relations = workFlowLineageMapper.queryCodeRelation(
processLineage.getPostTaskCode(), processLineage.getPostTaskVersion(),
processLineage.getProcessDefinitionCode(), processLineage.getProjectCode());
if (!relations.isEmpty()) {
Set<Integer> preWorkFlowIds = new HashSet<>();
List<ProcessLineage> preRelations = workFlowLineageMapper.queryCodeRelation(
processLineage.getPreTaskCode(), processLineage.getPreTaskVersion(),
processLineage.getProcessDefinitionCode(), processLineage.getProjectCode());
for (ProcessLineage preRelation : preRelations) {
WorkFlowLineage pre = workFlowLineageMapper.queryWorkFlowLineageByCode(
preRelation.getProcessDefinitionCode(), preRelation.getProjectCode());
preWorkFlowIds.add(pre.getWorkFlowId());
}
ProcessLineage postRelation = relations.get(0);
WorkFlowLineage post = workFlowLineageMapper.queryWorkFlowLineageByCode(
postRelation.getProcessDefinitionCode(), postRelation.getProjectCode());
if (!workFlowLineageMap.containsKey(post.getWorkFlowId())) {
post.setSourceWorkFlowId(StringUtils.join(preWorkFlowIds, ","));
workFlowLineageMap.put(post.getWorkFlowId(), post);
} else {
WorkFlowLineage workFlowLineage = workFlowLineageMap.get(post.getWorkFlowId());
String sourceWorkFlowId = workFlowLineage.getSourceWorkFlowId();
if (sourceWorkFlowId.equals("")) {
workFlowLineage.setSourceWorkFlowId(StringUtils.join(preWorkFlowIds, ","));
} else {
if (!preWorkFlowIds.isEmpty()) {
workFlowLineage.setSourceWorkFlowId(sourceWorkFlowId + "," + StringUtils.join(preWorkFlowIds, ","));
}
}
workFlowRelations.addAll(workFlowRelationsTmp);
getWorkFlowRelationRecursion(idsTmp, workFlowRelations,sourceIds);
}
if (preWorkFlowIds.isEmpty()) {
workFlowRelations.add(new WorkFlowRelation(0, post.getWorkFlowId()));
} else {
for (Integer workFlowId : preWorkFlowIds) {
workFlowRelations.add(new WorkFlowRelation(workFlowId, post.getWorkFlowId()));
}
}
}
}
@ -73,35 +105,18 @@ public class WorkFlowLineageServiceImpl extends BaseServiceImpl implements WorkF
@Override
public Map<String, Object> queryWorkFlowLineageByIds(Set<Integer> ids, int projectId) {
Map<String, Object> result = new HashMap<>();
List<WorkFlowLineage> workFlowLineageList = workFlowLineageMapper.queryByIds(ids, projectId);
Map<String, Object> workFlowLists = new HashMap<>();
Set<Integer> idsV = new HashSet<>();
if (ids == null || ids.isEmpty()) {
for (WorkFlowLineage workFlowLineage:workFlowLineageList) {
idsV.add(workFlowLineage.getWorkFlowId());
}
} else {
idsV = ids;
}
List<WorkFlowRelation> workFlowRelations = new ArrayList<>();
Set<Integer> sourceIds = new HashSet<>();
getWorkFlowRelationRecursion(idsV, workFlowRelations, sourceIds);
Project project = projectMapper.selectById(projectId);
List<ProcessLineage> processLineages = workFlowLineageMapper.queryRelationByIds(ids, project.getCode());
Set<Integer> idSet = new HashSet<>();
//If the incoming parameter is not empty, you need to add downstream workflow detail attributes
if (ids != null && !ids.isEmpty()) {
for (WorkFlowRelation workFlowRelation : workFlowRelations) {
idSet.add(workFlowRelation.getTargetWorkFlowId());
}
for (int id : ids) {
idSet.remove(id);
}
if (!idSet.isEmpty()) {
workFlowLineageList.addAll(workFlowLineageMapper.queryByIds(idSet, projectId));
}
Map<Integer, WorkFlowLineage> workFlowLineages = new HashMap<>();
Set<WorkFlowRelation> workFlowRelations = new HashSet<>();
for (ProcessLineage processLineage : processLineages) {
getRelation(workFlowLineages, workFlowRelations, processLineage);
}
workFlowLists.put(Constants.WORKFLOW_LIST, workFlowLineageList);
Map<String, Object> workFlowLists = new HashMap<>();
workFlowLists.put(Constants.WORKFLOW_LIST, workFlowLineages.values());
workFlowLists.put(Constants.WORKFLOW_RELATION_LIST, workFlowRelations);
result.put(Constants.DATA_LIST, workFlowLists);
putMsg(result, Status.SUCCESS);

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

@ -16,11 +16,12 @@
*/
package org.apache.dolphinscheduler.api.utils;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.TaskType;
import org.apache.dolphinscheduler.common.model.TaskNode;
import org.apache.dolphinscheduler.common.task.AbstractParameters;
import org.apache.dolphinscheduler.common.utils.*;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.common.utils.TaskParametersUtils;
@ -35,125 +36,146 @@ import java.util.regex.Pattern;
*/
public class CheckUtils {
private CheckUtils() {
throw new IllegalStateException("CheckUtils class");
}
/**
* check username
*
* @param userName user name
* @return true if user name regex valid,otherwise return false
*/
public static boolean checkUserName(String userName) {
return regexChecks(userName, Constants.REGEX_USER_NAME);
}
/**
* check email
*
* @param email email
* @return true if email regex valid, otherwise return false
*/
public static boolean checkEmail(String email) {
if (StringUtils.isEmpty(email)){
return false;
private CheckUtils() {
throw new IllegalStateException("CheckUtils class");
}
/**
* check username
*
* @param userName
* user name
* @return true if user name regex valid,otherwise return false
*/
public static boolean checkUserName(String userName) {
return regexChecks(userName, Constants.REGEX_USER_NAME);
}
return email.length() > 5 && email.length() <= 40 && regexChecks(email, Constants.REGEX_MAIL_NAME) ;
}
/**
* check project description
*
* @param desc desc
* @return true if description regex valid, otherwise return false
*/
public static Map<String, Object> checkDesc(String desc) {
Map<String, Object> result = new HashMap<>();
if (StringUtils.isNotEmpty(desc) && desc.length() > 200) {
result.put(Constants.STATUS, Status.REQUEST_PARAMS_NOT_VALID_ERROR);
result.put(Constants.MSG, MessageFormat.format(Status.REQUEST_PARAMS_NOT_VALID_ERROR.getMsg(), "desc length"));
}else{
result.put(Constants.STATUS, Status.SUCCESS);
/**
* check email
*
* @param email
* email
* @return true if email regex valid, otherwise return false
*/
public static boolean checkEmail(String email) {
if (StringUtils.isEmpty(email)) {
return false;
}
return email.length() > 5 && email.length() <= 40 && regexChecks(email, Constants.REGEX_MAIL_NAME);
}
return result;
}
/**
* check extra info
*
* @param otherParams other parames
* @return true if other parameters are valid, otherwise return false
*/
public static boolean checkOtherParams(String otherParams) {
return StringUtils.isNotEmpty(otherParams) && !JSONUtils.checkJsonValid(otherParams);
}
/**
* check password
*
* @param password password
* @return true if password regex valid, otherwise return false
*/
public static boolean checkPassword(String password) {
return StringUtils.isNotEmpty(password) && password.length() >= 2 && password.length() <= 20;
}
/**
* check phone
* phone can be empty.
* @param phone phone
* @return true if phone regex valid, otherwise return false
*/
public static boolean checkPhone(String phone) {
return StringUtils.isEmpty(phone) || phone.length() == 11;
}
/**
* check task node parameter
*
* @param parameter parameter
* @param taskType task type
* @return true if task node parameters are valid, otherwise return false
*/
public static boolean checkTaskNodeParameters(String parameter, String taskType) {
AbstractParameters abstractParameters = TaskParametersUtils.getParameters(taskType, parameter);
if (abstractParameters != null) {
return abstractParameters.checkParameters();
/**
* check project description
*
* @param desc
* desc
* @return true if description regex valid, otherwise return false
*/
public static Map<String, Object> checkDesc(String desc) {
Map<String, Object> result = new HashMap<>();
if (StringUtils.isNotEmpty(desc) && desc.length() > 200) {
result.put(Constants.STATUS, Status.REQUEST_PARAMS_NOT_VALID_ERROR);
result.put(Constants.MSG,
MessageFormat.format(Status.REQUEST_PARAMS_NOT_VALID_ERROR.getMsg(), "desc length"));
} else {
result.put(Constants.STATUS, Status.SUCCESS);
}
return result;
}
return false;
}
/**
* check params
* @param userName user name
* @param password password
* @param email email
* @param phone phone
* @return true if user parameters are valid, other return false
*/
public static boolean checkUserParams(String userName, String password, String email, String phone){
return CheckUtils.checkUserName(userName) &&
CheckUtils.checkEmail(email) &&
CheckUtils.checkPassword(password) &&
CheckUtils.checkPhone(phone);
}
/**
* regex check
*
* @param str input string
* @param pattern regex pattern
* @return true if regex pattern is right, otherwise return false
*/
private static boolean regexChecks(String str, Pattern pattern) {
if (StringUtils.isEmpty(str)) {
return false;
/**
* check extra info
*
* @param otherParams
* other parames
* @return true if other parameters are valid, otherwise return false
*/
public static boolean checkOtherParams(String otherParams) {
return StringUtils.isNotEmpty(otherParams) && !JSONUtils.checkJsonValid(otherParams);
}
return pattern.matcher(str).matches();
}
/**
* check password
*
* @param password
* password
* @return true if password regex valid, otherwise return false
*/
public static boolean checkPassword(String password) {
return StringUtils.isNotEmpty(password) && password.length() >= 2 && password.length() <= 20;
}
/**
* check phone phone can be empty.
*
* @param phone
* phone
* @return true if phone regex valid, otherwise return false
*/
public static boolean checkPhone(String phone) {
return StringUtils.isEmpty(phone) || phone.length() == 11;
}
/**
* check task node parameter
*
* @param taskNode
* TaskNode
* @return true if task node parameters are valid, otherwise return false
*/
public static boolean checkTaskNodeParameters(TaskNode taskNode) {
AbstractParameters abstractParameters;
String taskType = taskNode.getType();
if (taskType == null) {
return false;
}
if (TaskType.DEPENDENT.getDesc().equalsIgnoreCase(taskType)) {
abstractParameters = TaskParametersUtils.getParameters(taskType.toUpperCase(), taskNode.getDependence());
} else {
abstractParameters = TaskParametersUtils.getParameters(taskType.toUpperCase(), taskNode.getParams());
}
if (abstractParameters != null) {
return abstractParameters.checkParameters();
}
return false;
}
/**
* check params
*
* @param userName
* user name
* @param password
* password
* @param email
* email
* @param phone
* phone
* @return true if user parameters are valid, other return false
*/
public static boolean checkUserParams(String userName, String password, String email, String phone) {
return CheckUtils.checkUserName(userName) && CheckUtils.checkEmail(email) && CheckUtils.checkPassword(password)
&& CheckUtils.checkPhone(phone);
}
/**
* regex check
*
* @param str
* input string
* @param pattern
* regex pattern
* @return true if regex pattern is right, otherwise return false
*/
private static boolean regexChecks(String str, Pattern pattern) {
if (StringUtils.isEmpty(str)) {
return false;
}
return pattern.matcher(str).matches();
}
}

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

@ -78,7 +78,7 @@ public class DataSourceParam implements ProcessAddTaskParam, InitializingBean {
*/
@Override
public void afterPropertiesSet() {
TaskNodeParamFactory.register(TaskType.SQL.name(), this);
TaskNodeParamFactory.register(TaskType.PROCEDURE.name(), this);
TaskNodeParamFactory.register(TaskType.SQL.getDesc(), this);
TaskNodeParamFactory.register(TaskType.PROCEDURE.getDesc(), this);
}
}

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

@ -91,7 +91,7 @@ public class DependentParam implements ProcessAddTaskParam, InitializingBean {
ObjectNode dependentItem = (ObjectNode) dependItemList.path(k);
Project dependentItemProject = projectMapper.queryByName(dependentItem.path("projectName").asText());
if(dependentItemProject != null){
ProcessDefinition definition = processDefineMapper.queryByDefineName(dependentItemProject.getId(),dependentItem.path("definitionName").asText());
ProcessDefinition definition = processDefineMapper.queryByDefineName(dependentItemProject.getCode(),dependentItem.path("definitionName").asText());
if(definition != null){
dependentItem.put("projectId",dependentItemProject.getId());
dependentItem.put("definitionId",definition.getId());
@ -109,6 +109,6 @@ public class DependentParam implements ProcessAddTaskParam, InitializingBean {
*/
@Override
public void afterPropertiesSet() {
TaskNodeParamFactory.register(TaskType.DEPENDENT.name(), this);
TaskNodeParamFactory.register(TaskType.DEPENDENT.getDesc(), this);
}
}

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

@ -18,7 +18,6 @@
package org.apache.dolphinscheduler.api.controller;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.service.ProcessDefinitionVersionService;
import org.apache.dolphinscheduler.api.service.impl.ProcessDefinitionServiceImpl;
import org.apache.dolphinscheduler.api.utils.PageInfo;
import org.apache.dolphinscheduler.api.utils.Result;
@ -26,7 +25,7 @@ import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.ReleaseState;
import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinitionVersion;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinitionLog;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.User;
@ -46,8 +45,6 @@ import org.mockito.InjectMocks;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.mock.web.MockHttpServletResponse;
/**
@ -56,17 +53,12 @@ import org.springframework.mock.web.MockHttpServletResponse;
@RunWith(MockitoJUnitRunner.Silent.class)
public class ProcessDefinitionControllerTest {
private static Logger logger = LoggerFactory.getLogger(ProcessDefinitionControllerTest.class);
@InjectMocks
private ProcessDefinitionController processDefinitionController;
@Mock
private ProcessDefinitionServiceImpl processDefinitionService;
@Mock
private ProcessDefinitionVersionService processDefinitionVersionService;
protected User user;
@Before
@ -188,7 +180,6 @@ public class ProcessDefinitionControllerTest {
processDefinition.setId(id);
processDefinition.setLocations(locations);
processDefinition.setName(name);
processDefinition.setProcessDefinitionJson(json);
Map<String, Object> result = new HashMap<>();
putMsg(result, Status.SUCCESS);
@ -271,7 +262,6 @@ public class ProcessDefinitionControllerTest {
processDefinition.setId(id);
processDefinition.setLocations(locations);
processDefinition.setName(name);
processDefinition.setProcessDefinitionJson(json);
String name2 = "dag_test";
int id2 = 2;
@ -283,7 +273,6 @@ public class ProcessDefinitionControllerTest {
processDefinition2.setId(id2);
processDefinition2.setLocations(locations);
processDefinition2.setName(name2);
processDefinition2.setProcessDefinitionJson(json);
resourceList.add(processDefinition);
resourceList.add(processDefinition2);
@ -308,13 +297,13 @@ public class ProcessDefinitionControllerTest {
@Test
public void testGetNodeListByDefinitionId() throws Exception {
String projectName = "test";
int id = 1;
Long code = 1L;
Map<String, Object> result = new HashMap<>();
putMsg(result, Status.SUCCESS);
Mockito.when(processDefinitionService.getTaskNodeListByDefinitionId(id)).thenReturn(result);
Result response = processDefinitionController.getNodeListByDefinitionId(user, projectName, id);
Mockito.when(processDefinitionService.getTaskNodeListByDefinitionCode(code)).thenReturn(result);
Result response = processDefinitionController.getNodeListByDefinitionCode(user, projectName, code);
Assert.assertEquals(Status.SUCCESS.getCode(), response.getCode().intValue());
}
@ -322,13 +311,13 @@ public class ProcessDefinitionControllerTest {
@Test
public void testGetNodeListByDefinitionIdList() throws Exception {
String projectName = "test";
String idList = "1,2,3";
String codeList = "1,2,3";
Map<String, Object> result = new HashMap<>();
putMsg(result, Status.SUCCESS);
Mockito.when(processDefinitionService.getTaskNodeListByDefinitionIdList(idList)).thenReturn(result);
Result response = processDefinitionController.getNodeListByDefinitionIdList(user, projectName, idList);
Mockito.when(processDefinitionService.getTaskNodeListByDefinitionCodeList(codeList)).thenReturn(result);
Result response = processDefinitionController.getNodeListByDefinitionCodeList(user, projectName, codeList);
Assert.assertEquals(Status.SUCCESS.getCode(), response.getCode().intValue());
}
@ -390,18 +379,22 @@ public class ProcessDefinitionControllerTest {
@Test
public void testQueryProcessDefinitionVersions() {
String projectName = "test";
Result result = processDefinitionController.queryProcessDefinitionVersions(user, projectName, 1, -10, 1);
Assert.assertEquals(Status.REQUEST_PARAMS_NOT_VALID_ERROR.getCode(), result.getCode().intValue());
result = processDefinitionController.queryProcessDefinitionVersions(user, projectName, -1, 10, 1);
Assert.assertEquals(Status.REQUEST_PARAMS_NOT_VALID_ERROR.getCode(), result.getCode().intValue());
Map<String, Object> resultMap = new HashMap<>();
putMsg(resultMap, Status.SUCCESS);
resultMap.put(Constants.DATA_LIST, new PageInfo<ProcessDefinitionVersion>(1, 10));
Mockito.when(processDefinitionVersionService.queryProcessDefinitionVersions(user, projectName, 1, 10, 1)).thenReturn(resultMap);
result = processDefinitionController.queryProcessDefinitionVersions(user, projectName, 1, 10, 1);
resultMap.put(Constants.DATA_LIST, new PageInfo<ProcessDefinitionLog>(1, 10));
Mockito.when(processDefinitionService.queryProcessDefinitionVersions(
user
, projectName
, 1
, 10
, 1))
.thenReturn(resultMap);
Result result = processDefinitionController.queryProcessDefinitionVersions(
user
, projectName
, 1
, 10
, 1);
Assert.assertEquals(Status.SUCCESS.getCode(), (int) result.getCode());
}
@ -422,9 +415,17 @@ public class ProcessDefinitionControllerTest {
String projectName = "test";
Map<String, Object> resultMap = new HashMap<>();
putMsg(resultMap, Status.SUCCESS);
Mockito.when(processDefinitionVersionService.deleteByProcessDefinitionIdAndVersion(user, projectName, 1, 10)).thenReturn(resultMap);
Result result = processDefinitionController.deleteProcessDefinitionVersion(user, projectName, 1, 10);
Mockito.when(processDefinitionService.deleteByProcessDefinitionIdAndVersion(
user
, projectName
, 1
, 10))
.thenReturn(resultMap);
Result result = processDefinitionController.deleteProcessDefinitionVersion(
user
, projectName
, 1
, 10);
Assert.assertEquals(Status.SUCCESS.getCode(), (int) result.getCode());
}

1
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceControllerTest.java

@ -64,7 +64,6 @@ public class ProcessInstanceControllerTest extends AbstractControllerTest {
@Test
public void testQueryTaskListByProcessId() throws Exception {
MvcResult mvcResult = mockMvc.perform(get("/projects/{projectName}/instance/task-list-by-process-id", "cxc_1113")
.header(SESSION_ID, sessionId)
.param("processInstanceId", "1203"))

99
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/WorkFlowLineageControllerTest.java

@ -17,59 +17,86 @@
package org.apache.dolphinscheduler.api.controller;
import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get;
import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content;
import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.service.impl.WorkFlowLineageServiceImpl;
import org.apache.dolphinscheduler.api.utils.Result;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import java.text.MessageFormat;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.http.MediaType;
import org.springframework.test.web.servlet.MvcResult;
import org.springframework.util.LinkedMultiValueMap;
import org.springframework.util.MultiValueMap;
import org.mockito.InjectMocks;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.springframework.context.ApplicationContext;
/**
* work flow lineage controller test
*/
public class WorkFlowLineageControllerTest extends AbstractControllerTest {
private static Logger logger = LoggerFactory.getLogger(WorkFlowLineageControllerTest.class);
@InjectMocks
private WorkFlowLineageController workFlowLineageController;
@Mock
private WorkFlowLineageServiceImpl workFlowLineageService;
@Before
public void init() {
ApplicationContext applicationContext = Mockito.mock(ApplicationContext.class);
SpringApplicationContext springApplicationContext = new SpringApplicationContext();
springApplicationContext.setApplicationContext(applicationContext);
ProjectMapper projectMapper = Mockito.mock(ProjectMapper.class);
Mockito.when(applicationContext.getBean(ProjectMapper.class)).thenReturn(projectMapper);
Project project = new Project();
project.setId(1);
project.setCode(1L);
Mockito.when(projectMapper.selectById(1)).thenReturn(project);
}
@Test
public void testQueryWorkFlowLineageByName() throws Exception {
MultiValueMap<String, String> paramsMap = new LinkedMultiValueMap<>();
paramsMap.add("searchVal","test");
MvcResult mvcResult = mockMvc.perform(get("/lineages/1/list-name")
.header("sessionId", sessionId)
.params(paramsMap))
.andExpect(status().isOk())
.andExpect(content().contentType(MediaType.APPLICATION_JSON_UTF8))
.andReturn();
Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class);
Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue());
logger.info(mvcResult.getResponse().getContentAsString());
public void testQueryWorkFlowLineageByName() {
int projectId = 1;
String searchVal = "test";
Map<String, Object> result = new HashMap<>();
putMsg(result, Status.SUCCESS);
result.put(Constants.DATA_LIST, 1);
Mockito.when(workFlowLineageService.queryWorkFlowLineageByName(searchVal, projectId)).thenReturn(result);
Result response = workFlowLineageController.queryWorkFlowLineageByName(user, projectId, searchVal);
Assert.assertEquals(Status.SUCCESS.getCode(), response.getCode().intValue());
}
private void putMsg(Map<String, Object> result, Status status, Object... statusParams) {
result.put(Constants.STATUS, status);
if (statusParams != null && statusParams.length > 0) {
result.put(Constants.MSG, MessageFormat.format(status.getMsg(), statusParams));
} else {
result.put(Constants.MSG, status.getMsg());
}
}
@Test
public void testQueryWorkFlowLineageByIds() throws Exception {
MultiValueMap<String, String> paramsMap = new LinkedMultiValueMap<>();
paramsMap.add("ids","1");
MvcResult mvcResult = mockMvc.perform(get("/lineages/1/list-ids")
.header("sessionId", sessionId)
.params(paramsMap))
.andExpect(status().isOk())
.andExpect(content().contentType(MediaType.APPLICATION_JSON_UTF8))
.andReturn();
Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class);
Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue());
logger.info(mvcResult.getResponse().getContentAsString());
public void testQueryWorkFlowLineageByIds() {
int projectId = 1;
String ids = "1";
Map<String, Object> result = new HashMap<>();
putMsg(result, Status.SUCCESS);
result.put(Constants.DATA_LIST, 1);
Set<Integer> idSet = new HashSet<>();
idSet.add(1);
Mockito.when(workFlowLineageService.queryWorkFlowLineageByIds(idSet, projectId)).thenReturn(result);
Result response = workFlowLineageController.queryWorkFlowLineageByIds(user, projectId, ids);
Assert.assertEquals(Status.SUCCESS.getCode(), response.getCode().intValue());
}
}

39
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/DataAnalysisServiceTest.java

@ -130,7 +130,9 @@ public class DataAnalysisServiceTest {
//SUCCESS
Mockito.when(taskInstanceMapper.countTaskInstanceStateByUser(DateUtils.getScheduleDate(startDate),
DateUtils.getScheduleDate(endDate), new Integer[]{1})).thenReturn(getTaskInstanceStateCounts());
DateUtils.getScheduleDate(endDate), new Long[]{1L})).thenReturn(getTaskInstanceStateCounts());
Mockito.when(projectMapper.selectById(Mockito.any())).thenReturn(getProject("test"));
Mockito.when(projectService.hasProjectAndPerm(Mockito.any(), Mockito.any(), Mockito.any())).thenReturn(true);
result = dataAnalysisService.countTaskStateByProject(user, 1, startDate, endDate);
Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
@ -156,7 +158,7 @@ public class DataAnalysisServiceTest {
// when counting general user's task status then return user's task status count
user.setUserType(UserType.GENERAL_USER);
Mockito.when(processService.getProjectIdListHavePerm(anyInt()))
.thenReturn(Collections.singletonList(123));
.thenReturn(Collections.singletonList(123L));
ExecuteStatusCount executeStatusCount = new ExecuteStatusCount();
executeStatusCount.setExecutionStatus(ExecutionStatus.RUNNING_EXECUTION);
executeStatusCount.setCount(10);
@ -215,16 +217,19 @@ public class DataAnalysisServiceTest {
Assert.assertTrue(result.isEmpty());
//SUCCESS
Mockito.when(projectMapper.selectById(Mockito.any())).thenReturn(getProject("test"));
Mockito.when(processInstanceMapper.countInstanceStateByUser(DateUtils.getScheduleDate(startDate),
DateUtils.getScheduleDate(endDate), new Integer[]{1})).thenReturn(getTaskInstanceStateCounts());
DateUtils.getScheduleDate(endDate), new Long[]{1L})).thenReturn(getTaskInstanceStateCounts());
Mockito.when(projectService.hasProjectAndPerm(Mockito.any(), Mockito.any(), Mockito.any())).thenReturn(true);
result = dataAnalysisService.countProcessInstanceStateByProject(user, 1, startDate, endDate);
Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
}
@Test
public void testCountDefinitionByUser() {
Map<String, Object> result = dataAnalysisService.countDefinitionByUser(user, 1);
Mockito.when(projectMapper.selectById(Mockito.any())).thenReturn(getProject("test"));
Map<String, Object> result = dataAnalysisService.countDefinitionByUser(user, 0);
Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
}
@ -241,18 +246,16 @@ public class DataAnalysisServiceTest {
commandCount.setCommandType(CommandType.START_PROCESS);
commandCounts.add(commandCount);
Mockito.when(commandMapper.countCommandState(0, DateUtils.getScheduleDate(startDate),
DateUtils.getScheduleDate(endDate), new Integer[]{1})).thenReturn(commandCounts);
DateUtils.getScheduleDate(endDate), new Long[]{1L})).thenReturn(commandCounts);
Mockito.when(errorCommandMapper.countCommandState(DateUtils.getScheduleDate(startDate),
DateUtils.getScheduleDate(endDate), new Integer[]{1})).thenReturn(commandCounts);
DateUtils.getScheduleDate(endDate), new Long[]{1L})).thenReturn(commandCounts);
Mockito.when(projectMapper.selectById(Mockito.any())).thenReturn(getProject("test"));
Mockito.when(projectService.hasProjectAndPerm(Mockito.any(), Mockito.any(), Mockito.any())).thenReturn(true);
result = dataAnalysisService.countCommandState(user, 1, startDate, endDate);
Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
// when project check fail then return nothing
Map<String, Object> result1 = dataAnalysisService.countCommandState(user, 2, null, null);
Assert.assertTrue(result1.isEmpty());
// when all date in illegal format then return error message
String startDate2 = "illegalDateString";
String endDate2 = "illegalDateString";
@ -325,4 +328,18 @@ public class DataAnalysisServiceTest {
return taskInstanceStateCounts;
}
/**
* get mock Project
*
* @param projectName projectName
* @return Project
*/
private Project getProject(String projectName) {
Project project = new Project();
project.setCode(11L);
project.setId(1);
project.setName(projectName);
project.setUserId(1);
return project;
}
}

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

@ -43,7 +43,6 @@ import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.apache.dolphinscheduler.service.process.ProcessService;
import java.text.ParseException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
@ -114,13 +113,16 @@ public class ExecutorService2Test {
processDefinition.setReleaseState(ReleaseState.ONLINE);
processDefinition.setTenantId(tenantId);
processDefinition.setUserId(userId);
processDefinition.setVersion(1);
processDefinition.setCode(1L);
// processInstance
processInstance.setId(processInstanceId);
processInstance.setProcessDefinitionId(processDefinitionId);
processInstance.setState(ExecutionStatus.FAILURE);
processInstance.setExecutorId(userId);
processInstance.setTenantId(tenantId);
processInstance.setProcessDefinitionVersion(1);
processInstance.setProcessDefinitionCode(1L);
// project
project.setName(projectName);
@ -136,14 +138,14 @@ public class ExecutorService2Test {
Mockito.when(processService.createCommand(any(Command.class))).thenReturn(1);
Mockito.when(monitorService.getServerListFromZK(true)).thenReturn(getMasterServersList());
Mockito.when(processService.findProcessInstanceDetailById(processInstanceId)).thenReturn(processInstance);
Mockito.when(processService.findProcessDefineById(processDefinitionId)).thenReturn(processDefinition);
Mockito.when(processService.findProcessDefinition(1L, 1)).thenReturn(processDefinition);
}
/**
* not complement
*/
@Test
public void testNoComplement() throws ParseException {
public void testNoComplement() {
Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList());
Map<String, Object> result = executorService.execProcessInstance(loginUser, projectName,
@ -161,7 +163,7 @@ public class ExecutorService2Test {
* not complement
*/
@Test
public void testComplementWithStartNodeList() throws ParseException {
public void testComplementWithStartNodeList() {
Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList());
Map<String, Object> result = executorService.execProcessInstance(loginUser, projectName,
@ -180,7 +182,7 @@ public class ExecutorService2Test {
* date error
*/
@Test
public void testDateError() throws ParseException {
public void testDateError() {
Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList());
Map<String, Object> result = executorService.execProcessInstance(loginUser, projectName,
@ -197,7 +199,7 @@ public class ExecutorService2Test {
* serial
*/
@Test
public void testSerial() throws ParseException {
public void testSerial() {
Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList());
Map<String, Object> result = executorService.execProcessInstance(loginUser, projectName,
@ -215,7 +217,7 @@ public class ExecutorService2Test {
* without schedule
*/
@Test
public void testParallelWithOutSchedule() throws ParseException {
public void testParallelWithOutSchedule() {
Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList());
Map<String, Object> result = executorService.execProcessInstance(loginUser, projectName,
@ -233,7 +235,7 @@ public class ExecutorService2Test {
* with schedule
*/
@Test
public void testParallelWithSchedule() throws ParseException {
public void testParallelWithSchedule() {
Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(oneSchedulerList());
Map<String, Object> result = executorService.execProcessInstance(loginUser, projectName,
@ -248,7 +250,7 @@ public class ExecutorService2Test {
}
@Test
public void testNoMsterServers() throws ParseException {
public void testNoMsterServers() {
Mockito.when(monitorService.getServerListFromZK(true)).thenReturn(new ArrayList<>());
Map<String, Object> result = executorService.execProcessInstance(loginUser, projectName,
@ -262,7 +264,7 @@ public class ExecutorService2Test {
}
@Test
public void testExecuteRepeatRunning() throws Exception {
public void testExecuteRepeatRunning() {
Mockito.when(processService.verifyIsNeedCreateCommand(any(Command.class))).thenReturn(true);
Map<String, Object> result = executorService.execute(loginUser, projectName, processInstanceId, ExecuteType.REPEAT_RUNNING);

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

@ -17,7 +17,6 @@
package org.apache.dolphinscheduler.api.service;
import static org.assertj.core.api.Assertions.assertThat;
import static org.powermock.api.mockito.PowerMockito.mock;
import static org.powermock.api.mockito.PowerMockito.when;
@ -30,11 +29,12 @@ import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.FailureStrategy;
import org.apache.dolphinscheduler.common.enums.Priority;
import org.apache.dolphinscheduler.common.enums.ReleaseState;
import org.apache.dolphinscheduler.common.enums.TaskType;
import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.common.enums.WarningType;
import org.apache.dolphinscheduler.common.graph.DAG;
import org.apache.dolphinscheduler.common.model.TaskNode;
import org.apache.dolphinscheduler.common.process.ResourceInfo;
import org.apache.dolphinscheduler.common.task.shell.ShellParameters;
import org.apache.dolphinscheduler.common.process.Property;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.FileUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
@ -48,6 +48,7 @@ import org.apache.dolphinscheduler.dao.entity.Schedule;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.apache.dolphinscheduler.dao.mapper.ScheduleMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskInstanceMapper;
@ -58,11 +59,9 @@ import org.apache.http.entity.ContentType;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.lang.reflect.Method;
import java.text.MessageFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
@ -79,7 +78,6 @@ import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
import org.springframework.mock.web.MockMultipartFile;
import org.springframework.util.ReflectionUtils;
import org.springframework.web.multipart.MultipartFile;
import com.baomidou.mybatisplus.core.metadata.IPage;
@ -238,6 +236,8 @@ public class ProcessDefinitionServiceTest {
@Mock
private ProcessDefinitionMapper processDefineMapper;
@Mock
private ProcessTaskRelationMapper processTaskRelationMapper;
@Mock
private ProjectMapper projectMapper;
@Mock
private ProjectServiceImpl projectService;
@ -249,8 +249,6 @@ public class ProcessDefinitionServiceTest {
private ProcessInstanceService processInstanceService;
@Mock
private TaskInstanceMapper taskInstanceMapper;
@Mock
private ProcessDefinitionVersionService processDefinitionVersionService;
@Test
public void testQueryProcessDefinitionList() {
@ -275,7 +273,7 @@ public class ProcessDefinitionServiceTest {
Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result);
List<ProcessDefinition> resourceList = new ArrayList<>();
resourceList.add(getProcessDefinition());
Mockito.when(processDefineMapper.queryAllDefinitionList(project.getId())).thenReturn(resourceList);
Mockito.when(processDefineMapper.queryAllDefinitionList(project.getCode())).thenReturn(resourceList);
Map<String, Object> checkSuccessRes = processDefinitionService.queryProcessDefinitionList(loginUser, "project_test1");
Assert.assertEquals(Status.SUCCESS, checkSuccessRes.get(Constants.STATUS));
}
@ -309,7 +307,7 @@ public class ProcessDefinitionServiceTest {
Mockito.any(IPage.class)
, Mockito.eq("")
, Mockito.eq(loginUser.getId())
, Mockito.eq(project.getId())
, Mockito.eq(project.getCode())
, Mockito.anyBoolean())).thenReturn(page);
Map<String, Object> map1 = processDefinitionService.queryProcessDefinitionListPaging(
@ -342,6 +340,17 @@ public class ProcessDefinitionServiceTest {
putMsg(result, Status.SUCCESS, projectName);
Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result);
Mockito.when(processDefineMapper.selectById(1)).thenReturn(null);
String processDefinitionJson = "{\"globalParams\":[],\"tasks\":[{\"conditionResult\":"
+ "{\"failedNode\":[\"\"],\"successNode\":[\"\"]},\"delayTime\":\"0\",\"dependence\":{}"
+ ",\"description\":\"\",\"id\":\"tasks-3011\",\"maxRetryTimes\":\"0\",\"name\":\"tsssss\""
+ ",\"params\":{\"localParams\":[],\"rawScript\":\"echo \\\"123123\\\"\",\"resourceList\":[]}"
+ ",\"preTasks\":[],\"retryInterval\":\"1\",\"runFlag\":\"NORMAL\",\"taskInstancePriority\":\"MEDIUM\""
+ ",\"timeout\":{\"enable\":false,\"interval\":null,\"strategy\":\"\"},\"type\":\"SHELL\""
+ ",\"waitStartTimeout\":{},\"workerGroup\":\"default\"}],\"tenantId\":4,\"timeout\":0}";
ProcessData processData = JSONUtils.parseObject(processDefinitionJson, ProcessData.class);
Mockito.when(processService.genProcessData(Mockito.any())).thenReturn(processData);
Map<String, Object> instanceNotexitRes = processDefinitionService.queryProcessDefinitionById(loginUser,
"project_test1", 1);
Assert.assertEquals(Status.PROCESS_DEFINE_NOT_EXIST, instanceNotexitRes.get(Constants.STATUS));
@ -376,13 +385,16 @@ public class ProcessDefinitionServiceTest {
//project check auth success, instance not exist
putMsg(result, Status.SUCCESS, projectName);
Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result);
Mockito.when(processDefineMapper.queryByDefineName(project.getId(),"test_def")).thenReturn(null);
Mockito.when(processDefineMapper.queryByDefineName(project.getCode(), "test_def")).thenReturn(null);
ProcessData processData = getProcessData();
Mockito.when(processService.genProcessData(Mockito.any())).thenReturn(processData);
Map<String, Object> instanceNotexitRes = processDefinitionService.queryProcessDefinitionByName(loginUser,
"project_test1", "test_def");
Assert.assertEquals(Status.PROCESS_DEFINE_NOT_EXIST, instanceNotexitRes.get(Constants.STATUS));
//instance exit
Mockito.when(processDefineMapper.queryByDefineName(project.getId(),"test")).thenReturn(getProcessDefinition());
Mockito.when(processDefineMapper.queryByDefineName(project.getCode(), "test")).thenReturn(getProcessDefinition());
Map<String, Object> successRes = processDefinitionService.queryProcessDefinitionByName(loginUser,
"project_test1", "test");
Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS));
@ -393,7 +405,6 @@ public class ProcessDefinitionServiceTest {
String projectName = "project_test1";
Project project = getProject(projectName);
User loginUser = new User();
loginUser.setId(-1);
loginUser.setUserType(UserType.GENERAL_USER);
@ -435,19 +446,14 @@ public class ProcessDefinitionServiceTest {
// instance exit
ProcessDefinition definition = getProcessDefinition();
definition.setLocations("{\"tasks-36196\":{\"name\":\"ssh_test1\",\"targetarr\":\"\",\"x\":141,\"y\":70}}");
definition.setProcessDefinitionJson("{\"globalParams\":[],\"tasks\":[{\"type\":\"SHELL\",\"id\":\"tasks-36196\","
+ "\"name\":\"ssh_test1\",\"params\":{\"resourceList\":[],\"localParams\":[],\"rawScript\":\"aa=\\\"1234"
+ "\\\"\\necho ${aa}\"},\"desc\":\"\",\"runFlag\":\"NORMAL\",\"dependence\":{},\"maxRetryTimes\":\"0\","
+ "\"retryInterval\":\"1\",\"timeout\":{\"strategy\":\"\",\"interval\":null,\"enable\":false},"
+ "\"taskInstancePriority\":\"MEDIUM\",\"workerGroupId\":-1,\"preTasks\":[]}],\"tenantId\":-1,\"timeout\":0}");
definition.setConnects("[]");
Mockito.when(processDefineMapper.selectById(46)).thenReturn(definition);
Mockito.when(processService.genProcessData(Mockito.any())).thenReturn(getProcessData());
Map<String, Object> map3 = processDefinitionService.batchCopyProcessDefinition(
loginUser, projectName, "46", 1);
Assert.assertEquals(Status.SUCCESS, map3.get(Constants.STATUS));
Assert.assertEquals(Status.COPY_PROCESS_DEFINITION_ERROR, map3.get(Constants.STATUS));
}
@Test
@ -481,17 +487,8 @@ public class ProcessDefinitionServiceTest {
ProcessDefinition definition = getProcessDefinition();
definition.setLocations("{\"tasks-36196\":{\"name\":\"ssh_test1\",\"targetarr\":\"\",\"x\":141,\"y\":70}}");
definition.setProcessDefinitionJson("{\"globalParams\":[],\"tasks\":[{\"type\":\"SHELL\",\"id\":\"tasks-36196\""
+ ",\"name\":\"ssh_test1\",\"params\":{\"resourceList\":[],\"localParams\":[],\"rawScript\":\"aa=\\\"1234"
+ "\\\"\\necho ${aa}\"},\"desc\":\"\",\"runFlag\":\"NORMAL\",\"dependence\":{},\"maxRetryTimes\":\"0\","
+ "\"retryInterval\":\"1\",\"timeout\":{\"strategy\":\"\",\"interval\":null,\"enable\":false},"
+ "\"taskInstancePriority\":\"MEDIUM\",\"workerGroupId\":-1,\"preTasks\":[]}],\"tenantId\":-1,\"timeout\":0}");
definition.setConnects("[]");
// check target project result == null
Mockito.when(processDefineMapper.updateById(definition)).thenReturn(46);
Mockito.when(processDefineMapper.selectById(46)).thenReturn(definition);
putMsg(result, Status.SUCCESS);
Map<String, Object> successRes = processDefinitionService.batchMoveProcessDefinition(
@ -619,16 +616,6 @@ public class ProcessDefinitionServiceTest {
loginUser, "project_test1", 46, ReleaseState.getEnum(2));
Assert.assertEquals(Status.REQUEST_PARAMS_NOT_VALID_ERROR, failRes.get(Constants.STATUS));
//FIXME has function exit code 1 when exception
//process definition offline
// List<Schedule> schedules = new ArrayList<>();
// Schedule schedule = getSchedule();
// schedules.add(schedule);
// Mockito.when(scheduleMapper.selectAllByProcessDefineArray(new int[]{46})).thenReturn(schedules);
// Mockito.when(scheduleMapper.updateById(schedule)).thenReturn(1);
// Map<String, Object> offlineRes = processDefinitionService.releaseProcessDefinition(loginUser, "project_test1",
// 46, ReleaseState.OFFLINE.getCode());
// Assert.assertEquals(Status.SUCCESS, offlineRes.get(Constants.STATUS));
}
@Test
@ -651,13 +638,13 @@ public class ProcessDefinitionServiceTest {
//project check auth success, process not exist
putMsg(result, Status.SUCCESS, projectName);
Mockito.when(processDefineMapper.verifyByDefineName(project.getId(), "test_pdf")).thenReturn(null);
Mockito.when(processDefineMapper.verifyByDefineName(project.getCode(), "test_pdf")).thenReturn(null);
Map<String, Object> processNotExistRes = processDefinitionService.verifyProcessDefinitionName(loginUser,
"project_test1", "test_pdf");
Assert.assertEquals(Status.SUCCESS, processNotExistRes.get(Constants.STATUS));
//process exist
Mockito.when(processDefineMapper.verifyByDefineName(project.getId(), "test_pdf")).thenReturn(getProcessDefinition());
Mockito.when(processDefineMapper.verifyByDefineName(project.getCode(), "test_pdf")).thenReturn(getProcessDefinition());
Map<String, Object> processExistRes = processDefinitionService.verifyProcessDefinitionName(loginUser,
"project_test1", "test_pdf");
Assert.assertEquals(Status.PROCESS_DEFINITION_NAME_EXIST, processExistRes.get(Constants.STATUS));
@ -688,7 +675,7 @@ public class ProcessDefinitionServiceTest {
Assert.assertEquals(Status.PROCESS_NODE_HAS_CYCLE, taskCycleRes.get(Constants.STATUS));
//json abnormal
String abnormalJson = processDefinitionJson.replaceAll("SHELL", "");
String abnormalJson = processDefinitionJson.replaceAll(TaskType.SHELL.getDesc(), "");
processData = JSONUtils.parseObject(abnormalJson, ProcessData.class);
Map<String, Object> abnormalTaskRes = processDefinitionService.checkProcessNodeList(processData, abnormalJson);
Assert.assertEquals(Status.PROCESS_NODE_S_PARAMETER_INVALID, abnormalTaskRes.get(Constants.STATUS));
@ -697,50 +684,71 @@ public class ProcessDefinitionServiceTest {
@Test
public void testGetTaskNodeListByDefinitionId() {
//process definition not exist
Mockito.when(processDefineMapper.selectById(46)).thenReturn(null);
Map<String, Object> processDefinitionNullRes = processDefinitionService.getTaskNodeListByDefinitionId(46);
Mockito.when(processDefineMapper.queryByCode(46L)).thenReturn(null);
Map<String, Object> processDefinitionNullRes = processDefinitionService.getTaskNodeListByDefinitionCode(46L);
Assert.assertEquals(Status.PROCESS_DEFINE_NOT_EXIST, processDefinitionNullRes.get(Constants.STATUS));
//process data null
ProcessDefinition processDefinition = getProcessDefinition();
Mockito.when(processDefineMapper.selectById(46)).thenReturn(processDefinition);
Map<String, Object> successRes = processDefinitionService.getTaskNodeListByDefinitionId(46);
Mockito.when(processDefineMapper.queryByCode(46L)).thenReturn(processDefinition);
Map<String, Object> successRes = processDefinitionService.getTaskNodeListByDefinitionCode(46L);
Assert.assertEquals(Status.DATA_IS_NOT_VALID, successRes.get(Constants.STATUS));
//success
processDefinition.setProcessDefinitionJson(SHELL_JSON);
Mockito.when(processDefineMapper.selectById(46)).thenReturn(processDefinition);
Map<String, Object> dataNotValidRes = processDefinitionService.getTaskNodeListByDefinitionId(46);
Mockito.when(processService.genProcessData(Mockito.any())).thenReturn(new ProcessData());
Mockito.when(processDefineMapper.queryByCode(46L)).thenReturn(processDefinition);
Map<String, Object> dataNotValidRes = processDefinitionService.getTaskNodeListByDefinitionCode(46L);
Assert.assertEquals(Status.SUCCESS, dataNotValidRes.get(Constants.STATUS));
}
@Test
public void testGetTaskNodeListByDefinitionIdList() {
//process definition not exist
String defineIdList = "46";
Integer[] idArray = {46};
Mockito.when(processDefineMapper.queryDefinitionListByIdList(idArray)).thenReturn(null);
Map<String, Object> processNotExistRes = processDefinitionService.getTaskNodeListByDefinitionIdList(defineIdList);
String defineCodeList = "46";
Long[] codeArray = {46L};
List<Long> codeList = Arrays.asList(codeArray);
Mockito.when(processDefineMapper.queryByCodes(codeList)).thenReturn(null);
Map<String, Object> processNotExistRes = processDefinitionService.getTaskNodeListByDefinitionCodeList(defineCodeList);
Assert.assertEquals(Status.PROCESS_DEFINE_NOT_EXIST, processNotExistRes.get(Constants.STATUS));
//process definition exist
ProcessDefinition processDefinition = getProcessDefinition();
processDefinition.setProcessDefinitionJson(SHELL_JSON);
List<ProcessDefinition> processDefinitionList = new ArrayList<>();
processDefinitionList.add(processDefinition);
Mockito.when(processDefineMapper.queryDefinitionListByIdList(idArray)).thenReturn(processDefinitionList);
Map<String, Object> successRes = processDefinitionService.getTaskNodeListByDefinitionIdList(defineIdList);
Mockito.when(processDefineMapper.queryByCodes(codeList)).thenReturn(processDefinitionList);
ProcessData processData = getProcessData();
Mockito.when(processService.genProcessData(processDefinition)).thenReturn(processData);
Map<String, Object> successRes = processDefinitionService.getTaskNodeListByDefinitionCodeList(defineCodeList);
Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS));
}
private ProcessData getProcessData() {
ProcessData processData = new ProcessData();
List<TaskNode> taskNodeList = new ArrayList<>();
processData.setTasks(taskNodeList);
List<Property> properties = new ArrayList<>();
processData.setGlobalParams(properties);
processData.setTenantId(10);
processData.setTimeout(100);
return processData;
}
@Test
public void testQueryProcessDefinitionAllByProjectId() {
int projectId = 1;
Long projectCode = 2L;
Project project = new Project();
project.setId(projectId);
project.setCode(projectCode);
Mockito.when(projectMapper.selectById(projectId)).thenReturn(project);
ProcessDefinition processDefinition = getProcessDefinition();
processDefinition.setProcessDefinitionJson(SHELL_JSON);
List<ProcessDefinition> processDefinitionList = new ArrayList<>();
processDefinitionList.add(processDefinition);
Mockito.when(processDefineMapper.queryAllDefinitionList(projectId)).thenReturn(processDefinitionList);
Project test = getProject("test");
Mockito.when(projectMapper.selectById(projectId)).thenReturn(test);
Mockito.when(processDefineMapper.queryAllDefinitionList(test.getCode())).thenReturn(processDefinitionList);
Map<String, Object> successRes = processDefinitionService.queryProcessDefinitionAllByProjectId(projectId);
Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS));
}
@ -767,7 +775,7 @@ public class ProcessDefinitionServiceTest {
TaskInstance taskInstance = new TaskInstance();
taskInstance.setStartTime(new Date());
taskInstance.setEndTime(new Date());
taskInstance.setTaskType("SHELL");
taskInstance.setTaskType(TaskType.SHELL.getDesc());
taskInstance.setId(1);
taskInstance.setName("test_task_instance");
taskInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
@ -775,13 +783,11 @@ public class ProcessDefinitionServiceTest {
//task instance not exist
Mockito.when(processDefineMapper.selectById(46)).thenReturn(processDefinition);
Mockito.when(processInstanceService.queryByProcessDefineId(46, 10)).thenReturn(processInstanceList);
Mockito.when(taskInstanceMapper.queryByInstanceIdAndName(processInstance.getId(), "shell-1")).thenReturn(null);
Mockito.when(processService.genDagGraph(processDefinition)).thenReturn(new DAG<>());
Map<String, Object> taskNullRes = processDefinitionService.viewTree(46, 10);
Assert.assertEquals(Status.SUCCESS, taskNullRes.get(Constants.STATUS));
//task instance exist
Mockito.when(taskInstanceMapper.queryByInstanceIdAndName(processInstance.getId(), "shell-1")).thenReturn(taskInstance);
Map<String, Object> taskNotNuLLRes = processDefinitionService.viewTree(46, 10);
Assert.assertEquals(Status.SUCCESS, taskNotNuLLRes.get(Constants.STATUS));
@ -805,46 +811,14 @@ public class ProcessDefinitionServiceTest {
TaskInstance taskInstance = new TaskInstance();
taskInstance.setStartTime(new Date());
taskInstance.setEndTime(new Date());
taskInstance.setTaskType("SUB_PROCESS");
taskInstance.setTaskType(TaskType.SUB_PROCESS.getDesc());
taskInstance.setId(1);
taskInstance.setName("test_task_instance");
taskInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
taskInstance.setHost("192.168.xx.xx");
taskInstance.setTaskJson("{\n"
+ " \"conditionResult\": {\n"
+ " \"failedNode\": [\n"
+ " \"\"\n"
+ " ],\n"
+ " \"successNode\": [\n"
+ " \"\"\n"
+ " ]\n"
+ " },\n"
+ " \"delayTime\": \"0\",\n"
+ " \"dependence\": {},\n"
+ " \"description\": \"\",\n"
+ " \"id\": \"1\",\n"
+ " \"maxRetryTimes\": \"0\",\n"
+ " \"name\": \"test_task_instance\",\n"
+ " \"params\": {\n"
+ " \"processDefinitionId\": \"222\",\n"
+ " \"resourceList\": []\n"
+ " },\n"
+ " \"preTasks\": [],\n"
+ " \"retryInterval\": \"1\",\n"
+ " \"runFlag\": \"NORMAL\",\n"
+ " \"taskInstancePriority\": \"MEDIUM\",\n"
+ " \"timeout\": {\n"
+ " \"enable\": false,\n"
+ " \"interval\": null,\n"
+ " \"strategy\": \"\"\n"
+ " },\n"
+ " \"type\": \"SUB_PROCESS\",\n"
+ " \"workerGroup\": \"default\"\n"
+ "}");
//task instance exist
taskInstance.setTaskParams("\"processDefinitionId\": \"222\",\n");
Mockito.when(processDefineMapper.selectById(46)).thenReturn(processDefinition);
Mockito.when(processInstanceService.queryByProcessDefineId(46, 10)).thenReturn(processInstanceList);
Mockito.when(taskInstanceMapper.queryByInstanceIdAndName(processInstance.getId(), "shell-1")).thenReturn(taskInstance);
Mockito.when(processService.genDagGraph(processDefinition)).thenReturn(new DAG<>());
Map<String, Object> taskNotNuLLRes = processDefinitionService.viewTree(46, 10);
Assert.assertEquals(Status.SUCCESS, taskNotNuLLRes.get(Constants.STATUS));
@ -944,7 +918,6 @@ public class ProcessDefinitionServiceTest {
Mockito.when(projectMapper.queryByName(currentProjectName)).thenReturn(getProject(currentProjectName));
Mockito.when(projectService.checkProjectAndAuth(loginUser, getProject(currentProjectName), currentProjectName)).thenReturn(result);
Mockito.when(processDefineMapper.queryByDefineId(46)).thenReturn(shellDefinition2);
Map<String, Object> importProcessResult = processDefinitionService.importProcessDefinition(loginUser, multipartFile, currentProjectName);
@ -972,7 +945,6 @@ public class ProcessDefinitionServiceTest {
Mockito.when(projectMapper.queryByName(projectName)).thenReturn(getProject(projectName));
Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result);
Mockito.when(processService.findProcessDefineById(1)).thenReturn(processDefinition);
Mockito.when(processDefinitionVersionService.addProcessDefinitionVersion(processDefinition)).thenReturn(1L);
String sqlDependentJson = "{\n"
+ " \"globalParams\": [\n"
@ -1074,6 +1046,13 @@ public class ProcessDefinitionServiceTest {
processDefinitionService.batchExportProcessDefinitionByIds(
null, null, null, null);
String processDefinitionJson = "{\"globalParams\":[],\"tasks\":[{\"conditionResult\":"
+ "{\"failedNode\":[\"\"],\"successNode\":[\"\"]},\"delayTime\":\"0\",\"dependence\":{}"
+ ",\"description\":\"\",\"id\":\"tasks-3011\",\"maxRetryTimes\":\"0\",\"name\":\"tsssss\""
+ ",\"params\":{\"localParams\":[],\"rawScript\":\"echo \\\"123123\\\"\",\"resourceList\":[]}"
+ ",\"preTasks\":[],\"retryInterval\":\"1\",\"runFlag\":\"NORMAL\",\"taskInstancePriority\":\"MEDIUM\""
+ ",\"timeout\":{\"enable\":false,\"interval\":null,\"strategy\":\"\"},\"type\":\"SHELL\""
+ ",\"waitStartTimeout\":{},\"workerGroup\":\"default\"}],\"tenantId\":4,\"timeout\":0}";
User loginUser = new User();
loginUser.setId(1);
loginUser.setUserType(UserType.ADMIN_USER);
@ -1091,13 +1070,7 @@ public class ProcessDefinitionServiceTest {
ProcessDefinition processDefinition = new ProcessDefinition();
processDefinition.setId(1);
processDefinition.setProcessDefinitionJson("{\"globalParams\":[],\"tasks\":[{\"conditionResult\":"
+ "{\"failedNode\":[\"\"],\"successNode\":[\"\"]},\"delayTime\":\"0\",\"dependence\":{}"
+ ",\"description\":\"\",\"id\":\"tasks-3011\",\"maxRetryTimes\":\"0\",\"name\":\"tsssss\""
+ ",\"params\":{\"localParams\":[],\"rawScript\":\"echo \\\"123123\\\"\",\"resourceList\":[]}"
+ ",\"preTasks\":[],\"retryInterval\":\"1\",\"runFlag\":\"NORMAL\",\"taskInstancePriority\":\"MEDIUM\""
+ ",\"timeout\":{\"enable\":false,\"interval\":null,\"strategy\":\"\"},\"type\":\"SHELL\""
+ ",\"waitStartTimeout\":{},\"workerGroup\":\"default\"}],\"tenantId\":4,\"timeout\":0}");
processDefinition.setProcessDefinitionJson(processDefinitionJson);
Map<String, Object> checkResult = new HashMap<>();
checkResult.put(Constants.STATUS, Status.SUCCESS);
Mockito.when(projectMapper.queryByName(projectName)).thenReturn(project);
@ -1105,86 +1078,14 @@ public class ProcessDefinitionServiceTest {
Mockito.when(processDefineMapper.queryByDefineId(1)).thenReturn(processDefinition);
HttpServletResponse response = mock(HttpServletResponse.class);
ProcessData processData = JSONUtils.parseObject(processDefinitionJson, ProcessData.class);
Mockito.when(processService.genProcessData(processDefinition)).thenReturn(processData);
ServletOutputStream outputStream = mock(ServletOutputStream.class);
when(response.getOutputStream()).thenReturn(outputStream);
processDefinitionService.batchExportProcessDefinitionByIds(
loginUser, projectName, "1", response);
}
@Test
public void testGetResourceIds() throws Exception {
// set up
Method testMethod = ReflectionUtils.findMethod(ProcessDefinitionServiceImpl.class, "getResourceIds", ProcessData.class);
assertThat(testMethod).isNotNull();
testMethod.setAccessible(true);
// when processData has empty task, then return empty string
ProcessData input1 = new ProcessData();
input1.setTasks(Collections.emptyList());
String output1 = (String) testMethod.invoke(processDefinitionService, input1);
assertThat(output1).isEmpty();
// when task is null, then return empty string
ProcessData input2 = new ProcessData();
input2.setTasks(null);
String output2 = (String) testMethod.invoke(processDefinitionService, input2);
assertThat(output2).isEmpty();
// when task type is incorrect mapping, then return empty string
ProcessData input3 = new ProcessData();
TaskNode taskNode3 = new TaskNode();
taskNode3.setType("notExistType");
input3.setTasks(Collections.singletonList(taskNode3));
String output3 = (String) testMethod.invoke(processDefinitionService, input3);
assertThat(output3).isEmpty();
// when task parameter list is null, then return empty string
ProcessData input4 = new ProcessData();
TaskNode taskNode4 = new TaskNode();
taskNode4.setType("SHELL");
taskNode4.setParams(null);
input4.setTasks(Collections.singletonList(taskNode4));
String output4 = (String) testMethod.invoke(processDefinitionService, input4);
assertThat(output4).isEmpty();
// when resource id list is 0 1, then return 0,1
ProcessData input5 = new ProcessData();
TaskNode taskNode5 = new TaskNode();
taskNode5.setType("SHELL");
ShellParameters shellParameters5 = new ShellParameters();
ResourceInfo resourceInfo5A = new ResourceInfo();
resourceInfo5A.setId(1);
ResourceInfo resourceInfo5B = new ResourceInfo();
resourceInfo5B.setId(2);
shellParameters5.setResourceList(Arrays.asList(resourceInfo5A, resourceInfo5B));
taskNode5.setParams(JSONUtils.toJsonString(shellParameters5));
input5.setTasks(Collections.singletonList(taskNode5));
String output5 = (String) testMethod.invoke(processDefinitionService, input5);
assertThat(output5.split(",")).hasSize(2)
.containsExactlyInAnyOrder("1", "2");
// when resource id list is 0 1 1 2, then return 0,1,2
ProcessData input6 = new ProcessData();
TaskNode taskNode6 = new TaskNode();
taskNode6.setType("SHELL");
ShellParameters shellParameters6 = new ShellParameters();
ResourceInfo resourceInfo6A = new ResourceInfo();
resourceInfo6A.setId(3);
ResourceInfo resourceInfo6B = new ResourceInfo();
resourceInfo6B.setId(1);
ResourceInfo resourceInfo6C = new ResourceInfo();
resourceInfo6C.setId(1);
ResourceInfo resourceInfo6D = new ResourceInfo();
resourceInfo6D.setId(2);
shellParameters6.setResourceList(Arrays.asList(resourceInfo6A, resourceInfo6B, resourceInfo6C, resourceInfo6D));
taskNode6.setParams(JSONUtils.toJsonString(shellParameters6));
input6.setTasks(Collections.singletonList(taskNode6));
String output6 = (String) testMethod.invoke(processDefinitionService, input6);
assertThat(output6.split(",")).hasSize(3)
.containsExactlyInAnyOrder("3", "1", "2");
Assert.assertNotNull(processDefinitionService.exportProcessMetaData(processDefinition));
}
/**
@ -1205,13 +1106,13 @@ public class ProcessDefinitionServiceTest {
* @return ProcessDefinition
*/
private ProcessDefinition getProcessDefinition() {
ProcessDefinition processDefinition = new ProcessDefinition();
processDefinition.setId(46);
processDefinition.setName("test_pdf");
processDefinition.setProjectId(2);
processDefinition.setTenantId(1);
processDefinition.setDescription("");
processDefinition.setCode(9999L);
return processDefinition;
}
@ -1224,6 +1125,7 @@ public class ProcessDefinitionServiceTest {
*/
private Project getProject(String projectName) {
Project project = new Project();
project.setCode(1L);
project.setId(1);
project.setName(projectName);
project.setUserId(1);
@ -1302,22 +1204,6 @@ public class ProcessDefinitionServiceTest {
}
}
@Test
public void testExportProcessMetaData() {
Integer processDefinitionId = 111;
ProcessDefinition processDefinition = new ProcessDefinition();
processDefinition.setId(processDefinitionId);
processDefinition.setProcessDefinitionJson("{\"globalParams\":[],\"tasks\":[{\"conditionResult\":"
+ "{\"failedNode\":[\"\"],\"successNode\":"
+ "[\"\"]},\"delayTime\":\"0\",\"dependence\":{},"
+ "\"description\":\"\",\"id\":\"tasks-3011\",\"maxRetryTimes\":\"0\",\"name\":\"tsssss\","
+ "\"params\":{\"localParams\":[],\"rawScript\":\"echo \\\"123123\\\"\",\"resourceList\":[]},"
+ "\"preTasks\":[],\"retryInterval\":\"1\",\"runFlag\":\"NORMAL\",\"taskInstancePriority\":\"MEDIUM\","
+ "\"timeout\":{\"enable\":false,\"interval\":null,\"strategy\":\"\"},\"type\":\"SHELL\","
+ "\"waitStartTimeout\":{},\"workerGroup\":\"default\"}],\"tenantId\":4,\"timeout\":0}");
Assert.assertNotNull(processDefinitionService.exportProcessMetaData(processDefinitionId, processDefinition));
}
@Test
public void testImportProcessSchedule() {
User loginUser = new User();

269
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionVersionServiceTest.java

@ -1,269 +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.service;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.service.impl.ProcessDefinitionVersionServiceImpl;
import org.apache.dolphinscheduler.api.service.impl.ProjectServiceImpl;
import org.apache.dolphinscheduler.api.utils.PageInfo;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinitionVersion;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionVersionMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import java.text.MessageFormat;
import java.util.HashMap;
import java.util.Map;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.InjectMocks;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
import com.google.common.collect.Lists;
/**
* process definition version service test
*/
@RunWith(MockitoJUnitRunner.class)
public class ProcessDefinitionVersionServiceTest {
@InjectMocks
private ProcessDefinitionVersionServiceImpl processDefinitionVersionService;
@Mock
private ProcessDefinitionVersionMapper processDefinitionVersionMapper;
@Mock
private ProjectMapper projectMapper;
@Mock
private ProjectServiceImpl projectService;
@Mock
private ProcessDefinitionService processDefinitionService;
@Test
public void testAddProcessDefinitionVersion() {
long expectedVersion = 5L;
ProcessDefinition processDefinition = getProcessDefinition();
Mockito.when(processDefinitionVersionMapper
.queryMaxVersionByProcessDefinitionId(processDefinition.getId()))
.thenReturn(expectedVersion);
long version = processDefinitionVersionService.addProcessDefinitionVersion(processDefinition);
Assert.assertEquals(expectedVersion + 1, version);
}
@Test
@SuppressWarnings("unchecked")
public void testQueryProcessDefinitionVersions() {
// pageNo <= 0
int pageNo = -1;
int pageSize = 10;
int processDefinitionId = 66;
String projectName = "project_test1";
User loginUser = new User();
loginUser.setId(-1);
loginUser.setUserType(UserType.GENERAL_USER);
Map<String, Object> res = new HashMap<>();
putMsg(res, Status.PROJECT_NOT_FOUNT);
Project project = getProject(projectName);
Mockito.when(projectMapper.queryByName(projectName))
.thenReturn(project);
Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName))
.thenReturn(res);
// project auth fail
pageNo = 1;
pageSize = 10;
Map<String, Object> resultMap3 = processDefinitionVersionService.queryProcessDefinitionVersions(
loginUser
, projectName
, pageNo
, pageSize
, processDefinitionId);
Assert.assertEquals(Status.PROJECT_NOT_FOUNT, resultMap3.get(Constants.STATUS));
putMsg(res, Status.SUCCESS);
Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName))
.thenReturn(res);
ProcessDefinitionVersion processDefinitionVersion = getProcessDefinitionVersion(getProcessDefinition());
Mockito.when(processDefinitionVersionMapper
.queryProcessDefinitionVersionsPaging(Mockito.any(Page.class), Mockito.eq(processDefinitionId)))
.thenReturn(new Page<ProcessDefinitionVersion>()
.setRecords(Lists.newArrayList(processDefinitionVersion)));
Map<String, Object> resultMap4 = processDefinitionVersionService.queryProcessDefinitionVersions(
loginUser
, projectName
, pageNo
, pageSize
, processDefinitionId);
Assert.assertEquals(Status.SUCCESS, resultMap4.get(Constants.STATUS));
Assert.assertEquals(processDefinitionVersion
, ((PageInfo<ProcessDefinitionVersion>) resultMap4.get(Constants.DATA_LIST))
.getLists().get(0));
}
@Test
public void testQueryByProcessDefinitionIdAndVersion() {
ProcessDefinitionVersion expectedProcessDefinitionVersion =
getProcessDefinitionVersion(getProcessDefinition());
int processDefinitionId = 66;
long version = 10;
Mockito.when(processDefinitionVersionMapper.queryByProcessDefinitionIdAndVersion(processDefinitionId, version))
.thenReturn(expectedProcessDefinitionVersion);
ProcessDefinitionVersion processDefinitionVersion = processDefinitionVersionService
.queryByProcessDefinitionIdAndVersion(processDefinitionId, version);
Assert.assertEquals(expectedProcessDefinitionVersion, processDefinitionVersion);
}
@Test
public void testDeleteByProcessDefinitionIdAndVersion() {
String projectName = "project_test1";
int processDefinitionId = 66;
long version = 10;
Project project = getProject(projectName);
Mockito.when(projectMapper.queryByName(projectName))
.thenReturn(project);
User loginUser = new User();
loginUser.setId(-1);
loginUser.setUserType(UserType.GENERAL_USER);
// project auth fail
Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName))
.thenReturn(new HashMap<>());
Map<String, Object> resultMap1 = processDefinitionVersionService.deleteByProcessDefinitionIdAndVersion(
loginUser
, projectName
, processDefinitionId
, version);
Assert.assertEquals(0, resultMap1.size());
Map<String, Object> res = new HashMap<>();
putMsg(res, Status.SUCCESS);
Mockito.when(processDefinitionVersionMapper.deleteByProcessDefinitionIdAndVersion(processDefinitionId, version))
.thenReturn(1);
Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName))
.thenReturn(res);
Mockito.when(processDefinitionService.checkHasAssociatedProcessDefinition(processDefinitionId, version))
.thenReturn(false);
Map<String, Object> resultMap2 = processDefinitionVersionService.deleteByProcessDefinitionIdAndVersion(
loginUser
, projectName
, processDefinitionId
, version);
Assert.assertEquals(Status.SUCCESS, resultMap2.get(Constants.STATUS));
Mockito.when(processDefinitionService.checkHasAssociatedProcessDefinition(processDefinitionId, version))
.thenReturn(true);
Map<String, Object> resultMap3 = processDefinitionVersionService.deleteByProcessDefinitionIdAndVersion(
loginUser
, projectName
, processDefinitionId
, version);
Assert.assertEquals(Status.PROCESS_DEFINITION_VERSION_IS_USED, resultMap3.get(Constants.STATUS));
}
/**
* get mock processDefinitionVersion by processDefinition
*
* @return processDefinitionVersion
*/
private ProcessDefinitionVersion getProcessDefinitionVersion(ProcessDefinition processDefinition) {
return ProcessDefinitionVersion
.newBuilder()
.processDefinitionId(processDefinition.getId())
.version(1)
.processDefinitionJson(processDefinition.getProcessDefinitionJson())
.description(processDefinition.getDescription())
.locations(processDefinition.getLocations())
.connects(processDefinition.getConnects())
.timeout(processDefinition.getTimeout())
.globalParams(processDefinition.getGlobalParams())
.createTime(processDefinition.getUpdateTime())
.warningGroupId(processDefinition.getWarningGroupId())
.resourceIds(processDefinition.getResourceIds())
.build();
}
/**
* get mock processDefinition
*
* @return ProcessDefinition
*/
private ProcessDefinition getProcessDefinition() {
ProcessDefinition processDefinition = new ProcessDefinition();
processDefinition.setId(66);
processDefinition.setName("test_pdf");
processDefinition.setProjectId(2);
processDefinition.setTenantId(1);
processDefinition.setDescription("");
return processDefinition;
}
/**
* get mock Project
*
* @param projectName projectName
* @return Project
*/
private Project getProject(String projectName) {
Project project = new Project();
project.setId(1);
project.setName(projectName);
project.setUserId(1);
return project;
}
private void putMsg(Map<String, Object> result, Status status, Object... statusParams) {
result.put(Constants.STATUS, status);
if (statusParams != null && statusParams.length > 0) {
result.put(Constants.MSG, MessageFormat.format(status.getMsg(), statusParams));
} else {
result.put(Constants.MSG, status.getMsg());
}
}
}

88
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessInstanceServiceTest.java

@ -32,14 +32,19 @@ import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.TaskType;
import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.common.graph.DAG;
import org.apache.dolphinscheduler.common.model.TaskNode;
import org.apache.dolphinscheduler.common.model.TaskNodeRelation;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinitionLog;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.entity.Tenant;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.entity.WorkerGroup;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionLogMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessInstanceMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
@ -48,7 +53,6 @@ import org.apache.dolphinscheduler.service.process.ProcessService;
import java.io.IOException;
import java.text.MessageFormat;
import java.text.ParseException;
import java.util.ArrayList;
import java.util.Date;
import java.util.HashMap;
@ -87,13 +91,13 @@ public class ProcessInstanceServiceTest {
ProcessInstanceMapper processInstanceMapper;
@Mock
ProcessDefinitionMapper processDefineMapper;
ProcessDefinitionLogMapper processDefinitionLogMapper;
@Mock
ProcessDefinitionService processDefinitionService;
ProcessDefinitionMapper processDefineMapper;
@Mock
ProcessDefinitionVersionService processDefinitionVersionService;
ProcessDefinitionService processDefinitionService;
@Mock
TaskInstanceMapper taskInstanceMapper;
@ -125,11 +129,24 @@ public class ProcessInstanceServiceTest {
"192.168.xx.xx", 1, 10);
Assert.assertEquals(Status.PROJECT_NOT_FOUNT, proejctAuthFailRes.get(Constants.STATUS));
Date start = DateUtils.getScheduleDate("2020-01-01 00:00:00");
Date end = DateUtils.getScheduleDate("2020-01-02 00:00:00");
ProcessInstance processInstance = getProcessInstance();
List<ProcessInstance> processInstanceList = new ArrayList<>();
Page<ProcessInstance> pageReturn = new Page<>(1, 10);
processInstanceList.add(processInstance);
pageReturn.setRecords(processInstanceList);
// data parameter check
putMsg(result, Status.SUCCESS, projectName);
Project project = getProject(projectName);
when(projectMapper.queryByName(projectName)).thenReturn(project);
when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result);
when(processDefineMapper.selectById(Mockito.anyInt())).thenReturn(getProcessDefinition());
when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class)
, Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(),
eq("192.168.xx.xx"), Mockito.any(), Mockito.any())).thenReturn(pageReturn);
Map<String, Object> dataParameterRes = processInstanceService.queryProcessInstanceList(loginUser, projectName, 1, "20200101 00:00:00",
"20200102 00:00:00", "", loginUser.getUserName(), ExecutionStatus.SUBMITTED_SUCCESS,
"192.168.xx.xx", 1, 10);
@ -137,18 +154,12 @@ public class ProcessInstanceServiceTest {
//project auth success
putMsg(result, Status.SUCCESS, projectName);
Date start = DateUtils.getScheduleDate("2020-01-01 00:00:00");
Date end = DateUtils.getScheduleDate("2020-01-02 00:00:00");
ProcessInstance processInstance = getProcessInstance();
List<ProcessInstance> processInstanceList = new ArrayList<>();
Page<ProcessInstance> pageReturn = new Page<>(1, 10);
processInstanceList.add(processInstance);
pageReturn.setRecords(processInstanceList);
when(projectMapper.queryByName(projectName)).thenReturn(project);
when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result);
when(usersService.queryUser(loginUser.getId())).thenReturn(loginUser);
when(usersService.getUserIdByName(loginUser.getUserName())).thenReturn(loginUser.getId());
when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class), eq(project.getId()), eq(1), eq(""), eq(-1), Mockito.any(),
when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1L), eq(""), eq(-1), Mockito.any(),
eq("192.168.xx.xx"), eq(start), eq(end))).thenReturn(pageReturn);
when(usersService.queryUser(processInstance.getExecutorId())).thenReturn(loginUser);
Map<String, Object> successRes = processInstanceService.queryProcessInstanceList(loginUser, projectName, 1, "2020-01-01 00:00:00",
@ -157,7 +168,7 @@ public class ProcessInstanceServiceTest {
Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS));
// data parameter empty
when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class), eq(project.getId()), eq(1), eq(""), eq(-1), Mockito.any(),
when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1L), eq(""), eq(-1), Mockito.any(),
eq("192.168.xx.xx"), eq(null), eq(null))).thenReturn(pageReturn);
successRes = processInstanceService.queryProcessInstanceList(loginUser, projectName, 1, "",
"", "", loginUser.getUserName(), ExecutionStatus.SUBMITTED_SUCCESS,
@ -173,7 +184,7 @@ public class ProcessInstanceServiceTest {
Assert.assertEquals(Status.SUCCESS, executorExistRes.get(Constants.STATUS));
//executor name empty
when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class), eq(project.getId()), eq(1), eq(""), eq(0), Mockito.any(),
when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1L), eq(""), eq(0), Mockito.any(),
eq("192.168.xx.xx"), eq(start), eq(end))).thenReturn(pageReturn);
Map<String, Object> executorEmptyRes = processInstanceService.queryProcessInstanceList(loginUser, projectName, 1, "2020-01-01 00:00:00",
"2020-01-02 00:00:00", "", "", ExecutionStatus.SUBMITTED_SUCCESS,
@ -229,14 +240,14 @@ public class ProcessInstanceServiceTest {
//project auth success
ProcessInstance processInstance = getProcessInstance();
processInstance.setProcessDefinitionId(46);
putMsg(result, Status.SUCCESS, projectName);
Project project = getProject(projectName);
ProcessDefinition processDefinition = getProcessDefinition();
when(projectMapper.queryByName(projectName)).thenReturn(project);
when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result);
when(processService.findProcessInstanceDetailById(processInstance.getId())).thenReturn(processInstance);
when(processService.findProcessDefineById(processInstance.getProcessDefinitionId())).thenReturn(processDefinition);
when(processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
processInstance.getProcessDefinitionVersion())).thenReturn(processDefinition);
Map<String, Object> successRes = processInstanceService.queryProcessInstanceById(loginUser, projectName, 1);
Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS));
@ -269,7 +280,7 @@ public class ProcessInstanceServiceTest {
ProcessInstance processInstance = getProcessInstance();
processInstance.setState(ExecutionStatus.SUCCESS);
TaskInstance taskInstance = new TaskInstance();
taskInstance.setTaskType(TaskType.SHELL.getDescp());
taskInstance.setTaskType(TaskType.SHELL.getDesc());
List<TaskInstance> taskInstanceList = new ArrayList<>();
taskInstanceList.add(taskInstance);
Result res = new Result();
@ -320,7 +331,7 @@ public class ProcessInstanceServiceTest {
//task not sub process
TaskInstance taskInstance = getTaskInstance();
taskInstance.setTaskType(TaskType.HTTP.toString());
taskInstance.setTaskType(TaskType.HTTP.getDesc());
taskInstance.setProcessInstanceId(1);
when(processService.findTaskInstanceById(1)).thenReturn(taskInstance);
Map<String, Object> notSubprocessRes = processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectName, 1);
@ -328,7 +339,7 @@ public class ProcessInstanceServiceTest {
//sub process not exist
TaskInstance subTask = getTaskInstance();
subTask.setTaskType(TaskType.SUB_PROCESS.toString());
subTask.setTaskType(TaskType.SUB_PROCESS.getDesc());
subTask.setProcessInstanceId(1);
when(processService.findTaskInstanceById(subTask.getId())).thenReturn(subTask);
when(processService.findSubProcessInstance(subTask.getProcessInstanceId(), subTask.getId())).thenReturn(null);
@ -343,7 +354,7 @@ public class ProcessInstanceServiceTest {
}
@Test
public void testUpdateProcessInstance() throws ParseException {
public void testUpdateProcessInstance() {
String projectName = "project_test1";
User loginUser = getAdminUser();
Map<String, Object> result = new HashMap<>();
@ -378,22 +389,32 @@ public class ProcessInstanceServiceTest {
processInstance.setState(ExecutionStatus.SUCCESS);
processInstance.setTimeout(3000);
processInstance.setCommandType(CommandType.STOP);
processInstance.setProcessDefinitionCode(46L);
processInstance.setProcessDefinitionVersion(1);
ProcessDefinition processDefinition = getProcessDefinition();
processDefinition.setId(1);
processDefinition.setUserId(1);
Tenant tenant = new Tenant();
tenant.setId(1);
tenant.setTenantCode("test_tenant");
when(processService.findProcessDefineById(processInstance.getProcessDefinitionId())).thenReturn(processDefinition);
when(processDefineMapper.queryByCode(46L)).thenReturn(processDefinition);
when(processService.getTenantForProcess(Mockito.anyInt(), Mockito.anyInt())).thenReturn(tenant);
when(processService.updateProcessInstance(processInstance)).thenReturn(1);
when(processDefinitionService.checkProcessNodeList(Mockito.any(), eq(shellJson))).thenReturn(result);
when(processDefinitionVersionService.addProcessDefinitionVersion(processDefinition)).thenReturn(1L);
when(processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
processInstance.getProcessDefinitionVersion())).thenReturn(processDefinition);
Map<String, Object> processInstanceFinishRes = processInstanceService.updateProcessInstance(loginUser, projectName, 1,
shellJson, "2020-02-21 00:00:00", true, Flag.YES, "", "");
Assert.assertEquals(Status.UPDATE_PROCESS_INSTANCE_ERROR, processInstanceFinishRes.get(Constants.STATUS));
//success
when(processDefineMapper.updateById(processDefinition)).thenReturn(1);
when(processService.saveProcessDefinition(Mockito.any(), Mockito.any(),
Mockito.anyString(), Mockito.anyString(), Mockito.anyString(),
Mockito.anyString(), Mockito.any(), Mockito.any(), Mockito.anyBoolean())).thenReturn(1);
when(processService.findProcessDefinition(46L, 0)).thenReturn(processDefinition);
putMsg(result, Status.SUCCESS, projectName);
Map<String, Object> successRes = processInstanceService.updateProcessInstance(loginUser, projectName, 1,
shellJson, "2020-02-21 00:00:00", true, Flag.YES, "", "");
Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS));
@ -461,7 +482,7 @@ public class ProcessInstanceServiceTest {
}
@Test
public void testViewVariables() throws Exception {
public void testViewVariables() {
//process instance not null
ProcessInstance processInstance = getProcessInstance();
processInstance.setCommandType(CommandType.SCHEDULER);
@ -481,7 +502,20 @@ public class ProcessInstanceServiceTest {
taskInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
taskInstance.setStartTime(new Date());
when(processInstanceMapper.queryDetailById(1)).thenReturn(processInstance);
when(processDefinitionLogMapper.queryByDefinitionCodeAndVersion(
processInstance.getProcessDefinitionCode(),
processInstance.getProcessDefinitionVersion()
)).thenReturn(new ProcessDefinitionLog());
when(processInstanceMapper.queryDetailById(1)).thenReturn(processInstance);
when(taskInstanceMapper.queryByInstanceIdAndName(Mockito.anyInt(), Mockito.any())).thenReturn(taskInstance);
DAG<String, TaskNode, TaskNodeRelation> graph = new DAG<>();
for (int i = 1; i <= 7; ++i) {
graph.addNode(i + "", new TaskNode());
}
when(processService.genDagGraph(Mockito.any(ProcessDefinition.class)))
.thenReturn(graph);
Map<String, Object> successRes = processInstanceService.viewGantt(1);
Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS));
}
@ -507,6 +541,7 @@ public class ProcessInstanceServiceTest {
*/
private Project getProject(String projectName) {
Project project = new Project();
project.setCode(1L);
project.setId(1);
project.setName(projectName);
project.setUserId(1);
@ -522,6 +557,8 @@ public class ProcessInstanceServiceTest {
ProcessInstance processInstance = new ProcessInstance();
processInstance.setId(1);
processInstance.setName("test_process_instance");
processInstance.setProcessDefinitionCode(46L);
processInstance.setProcessDefinitionVersion(1);
processInstance.setStartTime(new Date());
processInstance.setEndTime(new Date());
return processInstance;
@ -534,9 +571,12 @@ public class ProcessInstanceServiceTest {
*/
private ProcessDefinition getProcessDefinition() {
ProcessDefinition processDefinition = new ProcessDefinition();
processDefinition.setCode(46L);
processDefinition.setVersion(1);
processDefinition.setId(46);
processDefinition.setName("test_pdf");
processDefinition.setProjectId(2);
processDefinition.setProjectCode(2L);
processDefinition.setTenantId(1);
processDefinition.setDescription("");
return processDefinition;

107
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessTaskRelationServiceImplTest.java

@ -0,0 +1,107 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.api.service;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.service.impl.ProcessTaskRelationServiceImpl;
import org.apache.dolphinscheduler.api.service.impl.ProjectServiceImpl;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import java.text.MessageFormat;
import java.util.HashMap;
import java.util.Map;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.InjectMocks;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
@RunWith(MockitoJUnitRunner.class)
public class ProcessTaskRelationServiceImplTest {
@InjectMocks
private ProcessTaskRelationServiceImpl processTaskRelationService;
@Mock
private ProcessDefinitionMapper processDefineMapper;
@Mock
private ProcessTaskRelationMapper processTaskRelationMapper;
@Mock
private ProjectMapper projectMapper;
@Mock
private ProjectServiceImpl projectService;
@Test
public void queryProcessTaskRelationTest() {
String projectName = "project_test1";
Project project = getProject(projectName);
Mockito.when(projectMapper.queryByName(projectName)).thenReturn(project);
User loginUser = new User();
loginUser.setId(-1);
loginUser.setUserType(UserType.GENERAL_USER);
Map<String, Object> result = new HashMap<>();
putMsg(result, Status.SUCCESS, projectName);
//project check auth fail
Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result);
Map<String, Object> relation = processTaskRelationService
.queryProcessTaskRelation(loginUser, projectName, 11L);
Assert.assertEquals(Status.SUCCESS, relation.get(Constants.STATUS));
}
private void putMsg(Map<String, Object> result, Status status, Object... statusParams) {
result.put(Constants.STATUS, status);
if (statusParams != null && statusParams.length > 0) {
result.put(Constants.MSG, MessageFormat.format(status.getMsg(), statusParams));
} else {
result.put(Constants.MSG, status.getMsg());
}
}
/**
* get mock Project
*
* @param projectName projectName
* @return Project
*/
private Project getProject(String projectName) {
Project project = new Project();
project.setCode(11L);
project.setId(1);
project.setName(projectName);
project.setUserId(1);
return project;
}
}

5
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProjectServiceTest.java

@ -220,7 +220,7 @@ public class ProjectServiceTest {
Assert.assertEquals(Status.USER_NO_OPERATION_PROJECT_PERM, result.get(Constants.STATUS));
//DELETE_PROJECT_ERROR_DEFINES_NOT_NULL
Mockito.when(processDefinitionMapper.queryAllDefinitionList(1)).thenReturn(getProcessDefinitions());
Mockito.when(processDefinitionMapper.queryAllDefinitionList(1L)).thenReturn(getProcessDefinitions());
loginUser.setUserType(UserType.ADMIN_USER);
result = projectService.deleteProject(loginUser, 1);
logger.info(result.toString());
@ -228,7 +228,7 @@ public class ProjectServiceTest {
//success
Mockito.when(projectMapper.deleteById(1)).thenReturn(1);
Mockito.when(processDefinitionMapper.queryAllDefinitionList(1)).thenReturn(new ArrayList<>());
Mockito.when(processDefinitionMapper.queryAllDefinitionList(1L)).thenReturn(new ArrayList<>());
result = projectService.deleteProject(loginUser, 1);
logger.info(result.toString());
Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
@ -356,6 +356,7 @@ public class ProjectServiceTest {
private Project getProject() {
Project project = new Project();
project.setCode(1L);
project.setId(1);
project.setName(projectName);
project.setUserId(1);

298
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskDefinitionServiceImplTest.java

@ -0,0 +1,298 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.api.service;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.service.impl.ProjectServiceImpl;
import org.apache.dolphinscheduler.api.service.impl.TaskDefinitionServiceImpl;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.common.model.TaskNode;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
import org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionLogMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
import org.apache.dolphinscheduler.service.process.ProcessService;
import java.text.MessageFormat;
import java.util.HashMap;
import java.util.Map;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.InjectMocks;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
@RunWith(MockitoJUnitRunner.class)
public class TaskDefinitionServiceImplTest {
String taskDefinitionJson = "{\n"
+ " \"type\": \"SQL\",\n"
+ " \"id\": \"tasks-27297\",\n"
+ " \"name\": \"SQL\",\n"
+ " \"params\": {\n"
+ " \"type\": \"MYSQL\",\n"
+ " \"datasource\": 1,\n"
+ " \"sql\": \"select * from test\",\n"
+ " \"udfs\": \"\",\n"
+ " \"sqlType\": \"1\",\n"
+ " \"title\": \"\",\n"
+ " \"receivers\": \"\",\n"
+ " \"receiversCc\": \"\",\n"
+ " \"showType\": \"TABLE\",\n"
+ " \"localParams\": [\n"
+ " \n"
+ " ],\n"
+ " \"connParams\": \"\",\n"
+ " \"preStatements\": [\n"
+ " \n"
+ " ],\n"
+ " \"postStatements\": [\n"
+ " \n"
+ " ]\n"
+ " },\n"
+ " \"description\": \"\",\n"
+ " \"runFlag\": \"NORMAL\",\n"
+ " \"dependence\": {\n"
+ " \n"
+ " },\n"
+ " \"maxRetryTimes\": \"0\",\n"
+ " \"retryInterval\": \"1\",\n"
+ " \"timeout\": {\n"
+ " \"strategy\": \"\",\n"
+ " \"enable\": false\n"
+ " },\n"
+ " \"taskInstancePriority\": \"MEDIUM\",\n"
+ " \"workerGroupId\": -1,\n"
+ " \"preTasks\": [\n"
+ " \"dependent\"\n"
+ " ]\n"
+ "}\n";
@InjectMocks
private TaskDefinitionServiceImpl taskDefinitionService;
@Mock
private TaskDefinitionMapper taskDefinitionMapper;
@Mock
private TaskDefinitionLogMapper taskDefinitionLogMapper;
@Mock
private ProcessDefinitionMapper processDefineMapper;
@Mock
private ProcessTaskRelationMapper processTaskRelationMapper;
@Mock
private ProjectMapper projectMapper;
@Mock
private ProjectServiceImpl projectService;
@Mock
private ProcessService processService;
@Test
public void createTaskDefinition() {
String projectName = "project_test1";
Project project = getProject(projectName);
Mockito.when(projectMapper.queryByName(projectName)).thenReturn(project);
User loginUser = new User();
loginUser.setId(-1);
loginUser.setUserType(UserType.GENERAL_USER);
Map<String, Object> result = new HashMap<>();
putMsg(result, Status.SUCCESS, projectName);
//project check auth fail
Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result);
TaskNode taskNode = JSONUtils.parseObject(taskDefinitionJson, TaskNode.class);
Mockito.when(processService.saveTaskDefinition(Mockito.eq(loginUser)
, Mockito.eq(project.getCode())
, Mockito.eq(taskNode)
, Mockito.any(TaskDefinition.class)))
.thenReturn(1);
Map<String, Object> relation = taskDefinitionService
.createTaskDefinition(loginUser, projectName, taskDefinitionJson);
Assert.assertEquals(Status.SUCCESS, relation.get(Constants.STATUS));
}
@Test
public void queryTaskDefinitionByName() {
String projectName = "project_test1";
String taskName = "task";
Project project = getProject(projectName);
Mockito.when(projectMapper.queryByName(projectName)).thenReturn(project);
User loginUser = new User();
loginUser.setId(-1);
loginUser.setUserType(UserType.GENERAL_USER);
Map<String, Object> result = new HashMap<>();
putMsg(result, Status.SUCCESS, projectName);
//project check auth fail
Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result);
TaskNode taskNode = JSONUtils.parseObject(taskDefinitionJson, TaskNode.class);
Mockito.when(taskDefinitionMapper.queryByDefinitionName(project.getCode(), taskName))
.thenReturn(new TaskDefinition());
Map<String, Object> relation = taskDefinitionService
.queryTaskDefinitionByName(loginUser, projectName, taskName);
Assert.assertEquals(Status.SUCCESS, relation.get(Constants.STATUS));
}
@Test
public void deleteTaskDefinitionByCode() {
String projectName = "project_test1";
Project project = getProject(projectName);
Mockito.when(projectMapper.queryByName(projectName)).thenReturn(project);
User loginUser = new User();
loginUser.setId(-1);
loginUser.setUserType(UserType.GENERAL_USER);
Map<String, Object> result = new HashMap<>();
putMsg(result, Status.SUCCESS, projectName);
//project check auth fail
Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result);
TaskNode taskNode = JSONUtils.parseObject(taskDefinitionJson, TaskNode.class);
Mockito.when(taskDefinitionMapper.deleteByCode(Mockito.anyLong()))
.thenReturn(1);
Map<String, Object> relation = taskDefinitionService
.deleteTaskDefinitionByCode(loginUser, projectName, 11L);
Assert.assertEquals(Status.SUCCESS, relation.get(Constants.STATUS));
}
@Test
public void updateTaskDefinition() {
String projectName = "project_test1";
Project project = getProject(projectName);
Mockito.when(projectMapper.queryByName(projectName)).thenReturn(project);
User loginUser = new User();
loginUser.setId(-1);
loginUser.setUserType(UserType.GENERAL_USER);
Map<String, Object> result = new HashMap<>();
putMsg(result, Status.SUCCESS, projectName);
//project check auth fail
Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result);
TaskNode taskNode = JSONUtils.parseObject(taskDefinitionJson, TaskNode.class);
Mockito.when(processService.updateTaskDefinition(Mockito.eq(loginUser)
, Mockito.eq(project.getCode())
, Mockito.eq(taskNode)
, Mockito.any(TaskDefinition.class)))
.thenReturn(1);
Mockito.when(taskDefinitionMapper.queryByDefinitionCode(11L))
.thenReturn(new TaskDefinition());
Map<String, Object> relation = taskDefinitionService
.updateTaskDefinition(loginUser, projectName, 11L, taskDefinitionJson);
Assert.assertEquals(Status.SUCCESS, relation.get(Constants.STATUS));
}
@Test
public void switchVersion() {
String projectName = "project_test1";
int version = 1;
Long taskCode = 11L;
Project project = getProject(projectName);
Mockito.when(projectMapper.queryByName(projectName)).thenReturn(project);
User loginUser = new User();
loginUser.setId(-1);
loginUser.setUserType(UserType.GENERAL_USER);
Map<String, Object> result = new HashMap<>();
putMsg(result, Status.SUCCESS, projectName);
//project check auth fail
Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result);
TaskNode taskNode = JSONUtils.parseObject(taskDefinitionJson, TaskNode.class);
Mockito.when(taskDefinitionLogMapper.queryByDefinitionCodeAndVersion(taskCode, version))
.thenReturn(new TaskDefinitionLog());
Mockito.when(taskDefinitionMapper.queryByDefinitionCode(taskCode))
.thenReturn(new TaskDefinition());
Map<String, Object> relation = taskDefinitionService
.switchVersion(loginUser, projectName, taskCode, version);
Assert.assertEquals(Status.SUCCESS, relation.get(Constants.STATUS));
}
private void putMsg(Map<String, Object> result, Status status, Object... statusParams) {
result.put(Constants.STATUS, status);
if (statusParams != null && statusParams.length > 0) {
result.put(Constants.MSG, MessageFormat.format(status.getMsg(), statusParams));
} else {
result.put(Constants.MSG, status.getMsg());
}
}
/**
* get mock Project
*
* @param projectName projectName
* @return Project
*/
private Project getProject(String projectName) {
Project project = new Project();
project.setId(1);
project.setName(projectName);
project.setUserId(1);
return project;
}
}

11
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java

@ -117,7 +117,7 @@ public class TaskInstanceServiceTest {
when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result);
when(usersService.queryUser(loginUser.getId())).thenReturn(loginUser);
when(usersService.getUserIdByName(loginUser.getUserName())).thenReturn(loginUser.getId());
when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getId()), eq(1), eq(""), eq(""), eq(""),
when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1), eq(""), eq(""), eq(""),
eq(0), Mockito.any(), eq("192.168.xx.xx"), eq(start), eq(end))).thenReturn(pageReturn);
when(usersService.queryUser(processInstance.getExecutorId())).thenReturn(loginUser);
when(processService.findProcessInstanceDetailById(taskInstance.getProcessInstanceId())).thenReturn(processInstance);
@ -127,7 +127,7 @@ public class TaskInstanceServiceTest {
Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS));
//executor name empty
when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getId()), eq(1), eq(""), eq(""), eq(""),
when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1), eq(""), eq(""), eq(""),
eq(0), Mockito.any(), eq("192.168.xx.xx"), eq(start), eq(end))).thenReturn(pageReturn);
Map<String, Object> executorEmptyRes = taskInstanceService.queryTaskListPaging(loginUser, projectName, 1, "", "",
"", "2020-01-01 00:00:00", "2020-01-02 00:00:00", "", ExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20);
@ -141,14 +141,14 @@ public class TaskInstanceServiceTest {
Assert.assertEquals(Status.SUCCESS, executorNullRes.get(Constants.STATUS));
//start/end date null
when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getId()), eq(1), eq(""), eq(""), eq(""),
when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1), eq(""), eq(""), eq(""),
eq(0), Mockito.any(), eq("192.168.xx.xx"), any(), any())).thenReturn(pageReturn);
Map<String, Object> executorNullDateRes = taskInstanceService.queryTaskListPaging(loginUser, projectName, 1, "", "",
"", null, null, "", ExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20);
Assert.assertEquals(Status.SUCCESS, executorNullDateRes.get(Constants.STATUS));
//start date error format
when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getId()), eq(1), eq(""), eq(""), eq(""),
when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getCode()), eq(1), eq(""), eq(""), eq(""),
eq(0), Mockito.any(), eq("192.168.xx.xx"), any(), any())).thenReturn(pageReturn);
Map<String, Object> executorErrorStartDateRes = taskInstanceService.queryTaskListPaging(loginUser, projectName, 1, "", "",
"", "error date", null, "", ExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20);
@ -179,6 +179,7 @@ public class TaskInstanceServiceTest {
*/
private Project getProject(String projectName) {
Project project = new Project();
project.setCode(1L);
project.setId(1);
project.setName(projectName);
project.setUserId(1);
@ -267,4 +268,4 @@ public class TaskInstanceServiceTest {
Map<String, Object> successRes = taskInstanceService.forceTaskSuccess(user, projectName, taskId);
Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS));
}
}
}

1
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/UsersServiceTest.java

@ -329,7 +329,6 @@ public class UsersServiceTest {
logger.info(result.toString());
Assert.assertEquals(Status.USER_NOT_EXIST, result.get(Constants.STATUS));
//success
when(projectUserMapper.deleteProjectRelation(Mockito.anyInt(), Mockito.anyInt())).thenReturn(1);
result = usersService.grantProject(loginUser, 1, projectIds);
logger.info(result.toString());
Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));

85
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/WorkFlowLineageServiceTest.java

@ -21,11 +21,15 @@ import static org.mockito.Mockito.when;
import org.apache.dolphinscheduler.api.service.impl.WorkFlowLineageServiceImpl;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.dao.entity.ProcessLineage;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.WorkFlowLineage;
import org.apache.dolphinscheduler.dao.entity.WorkFlowRelation;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.apache.dolphinscheduler.dao.mapper.WorkFlowLineageMapper;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@ -36,6 +40,7 @@ import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.InjectMocks;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
/**
@ -50,30 +55,75 @@ public class WorkFlowLineageServiceTest {
@Mock
private WorkFlowLineageMapper workFlowLineageMapper;
@Mock
private ProjectMapper projectMapper;
/**
* get mock Project
*
* @param projectName projectName
* @return Project
*/
private Project getProject(String projectName) {
Project project = new Project();
project.setCode(1L);
project.setId(1);
project.setName(projectName);
project.setUserId(1);
return project;
}
@Test
public void testQueryWorkFlowLineageByName() {
Project project = getProject("test");
String searchVal = "test";
when(workFlowLineageMapper.queryByName(searchVal, 1)).thenReturn(getWorkFlowLineages());
Map<String, Object> result = workFlowLineageService.queryWorkFlowLineageByName(searchVal,1);
List<WorkFlowLineage> workFlowLineageList = (List<WorkFlowLineage>)result.get(Constants.DATA_LIST);
Assert.assertTrue(workFlowLineageList.size()>0);
when(projectMapper.selectById(1)).thenReturn(project);
when(workFlowLineageMapper.queryByName(Mockito.any(), Mockito.any())).thenReturn(getWorkFlowLineages());
Map<String, Object> result = workFlowLineageService.queryWorkFlowLineageByName(searchVal, 1);
List<WorkFlowLineage> workFlowLineageList = (List<WorkFlowLineage>) result.get(Constants.DATA_LIST);
Assert.assertTrue(workFlowLineageList.size() > 0);
}
@Test
public void testQueryWorkFlowLineageByIds() {
Set<Integer> ids = new HashSet<>();
ids.add(1);
ids.add(2);
when(workFlowLineageMapper.queryByIds(ids, 1)).thenReturn(getWorkFlowLineages());
when(workFlowLineageMapper.querySourceTarget(1)).thenReturn(getWorkFlowRelation());
Map<String, Object> result = workFlowLineageService.queryWorkFlowLineageByIds(ids,1);
Map<String, Object> workFlowLists = (Map<String, Object>)result.get(Constants.DATA_LIST);
List<WorkFlowLineage> workFlowLineages = (List<WorkFlowLineage>)workFlowLists.get("workFlowList");
List<WorkFlowRelation> workFlowRelations = (List<WorkFlowRelation>)workFlowLists.get("workFlowRelationList");
Assert.assertTrue(workFlowLineages.size()>0);
Assert.assertTrue(workFlowRelations.size()>0);
Project project = getProject("test");
List<ProcessLineage> processLineages = new ArrayList<>();
ProcessLineage processLineage = new ProcessLineage();
processLineage.setPreTaskVersion(1);
processLineage.setPreTaskCode(1L);
processLineage.setPostTaskCode(2L);
processLineage.setPostTaskVersion(1);
processLineage.setProcessDefinitionCode(1111L);
processLineage.setProcessDefinitionVersion(1);
processLineage.setProjectCode(1111L);
processLineages.add(processLineage);
WorkFlowLineage workFlowLineage = new WorkFlowLineage();
workFlowLineage.setSourceWorkFlowId("");
when(projectMapper.selectById(1)).thenReturn(project);
when(workFlowLineageMapper.queryRelationByIds(ids, project.getCode())).thenReturn(processLineages);
when(workFlowLineageMapper.queryCodeRelation(processLineage.getPostTaskCode()
, processLineage.getPreTaskVersion()
, processLineage.getProcessDefinitionCode()
, processLineage.getProjectCode()))
.thenReturn(processLineages);
when(workFlowLineageMapper
.queryWorkFlowLineageByCode(processLineage.getProcessDefinitionCode(), processLineage.getProjectCode()))
.thenReturn(workFlowLineage);
Map<String, Object> result = workFlowLineageService.queryWorkFlowLineageByIds(ids, 1);
Map<String, Object> workFlowLists = (Map<String, Object>) result.get(Constants.DATA_LIST);
Collection<WorkFlowLineage> workFlowLineages = (Collection<WorkFlowLineage>) workFlowLists.get(Constants.WORKFLOW_LIST);
Set<WorkFlowRelation> workFlowRelations = (Set<WorkFlowRelation>) workFlowLists.get(Constants.WORKFLOW_RELATION_LIST);
Assert.assertTrue(workFlowLineages.size() > 0);
Assert.assertTrue(workFlowRelations.size() > 0);
}
private List<WorkFlowLineage> getWorkFlowLineages() {
@ -85,13 +135,4 @@ public class WorkFlowLineageServiceTest {
return workFlowLineages;
}
private List<WorkFlowRelation> getWorkFlowRelation(){
List<WorkFlowRelation> workFlowRelations = new ArrayList<>();
WorkFlowRelation workFlowRelation = new WorkFlowRelation();
workFlowRelation.setSourceWorkFlowId(1);
workFlowRelation.setTargetWorkFlowId(2);
workFlowRelations.add(workFlowRelation);
return workFlowRelations;
}
}

137
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/CheckUtilsTest.java

@ -25,6 +25,7 @@ import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.ProgramType;
import org.apache.dolphinscheduler.common.enums.TaskType;
import org.apache.dolphinscheduler.common.model.TaskNode;
import org.apache.dolphinscheduler.common.process.ResourceInfo;
import org.apache.dolphinscheduler.common.task.datax.DataxParameters;
import org.apache.dolphinscheduler.common.task.dependent.DependentParameters;
@ -41,34 +42,17 @@ import org.apache.dolphinscheduler.common.utils.JSONUtils;
import java.util.Map;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class CheckUtilsTest {
private static final Logger logger = LoggerFactory.getLogger(CheckUtilsTest.class);
@Before
public void setUp() throws Exception {
}
@After
public void tearDown() throws Exception {
}
/**
* check username
*/
@Test
public void testCheckUserName() {
assertTrue(CheckUtils.checkUserName("test01"));
assertFalse(CheckUtils.checkUserName(null));
assertFalse(CheckUtils.checkUserName("test01@abc"));
}
@ -77,9 +61,7 @@ public class CheckUtilsTest {
*/
@Test
public void testCheckEmail() {
assertTrue(CheckUtils.checkEmail("test01@gmail.com"));
assertFalse(CheckUtils.checkEmail("test01@gmail"));
}
@ -88,12 +70,9 @@ public class CheckUtilsTest {
*/
@Test
public void testCheckDesc() {
Map<String, Object> objectMap = CheckUtils.checkDesc("I am desc");
Status status = (Status) objectMap.get(Constants.STATUS);
assertEquals(status.getCode(),Status.SUCCESS.getCode());
assertEquals(status.getCode(), Status.SUCCESS.getCode());
}
@Test
@ -104,18 +83,15 @@ public class CheckUtilsTest {
assertFalse(CheckUtils.checkOtherParams("{}"));
assertFalse(CheckUtils.checkOtherParams("{\"key1\":111}"));
}
/**
* check passwd
*/
@Test
public void testCheckPassword() {
assertFalse(CheckUtils.checkPassword(null));
assertFalse(CheckUtils.checkPassword("a"));
assertFalse(CheckUtils.checkPassword("1234567890abcderfasdf2"));
assertTrue(CheckUtils.checkPassword("123456"));
}
@ -124,103 +100,148 @@ public class CheckUtilsTest {
*/
@Test
public void testCheckPhone() {
// phone can be null
assertTrue(CheckUtils.checkPhone(null));
assertFalse(CheckUtils.checkPhone("14567134578654"));
assertTrue(CheckUtils.checkPhone("17362537263"));
}
@Test
public void testCheckTaskNodeParameters() {
TaskNode taskNode = new TaskNode();
assertFalse(CheckUtils.checkTaskNodeParameters(taskNode));
assertFalse(CheckUtils.checkTaskNodeParameters(null,null));
assertFalse(CheckUtils.checkTaskNodeParameters(null,"unKnown"));
assertFalse(CheckUtils.checkTaskNodeParameters("unKnown","unKnown"));
assertFalse(CheckUtils.checkTaskNodeParameters("unKnown",null));
taskNode.setType("unKnown");
assertFalse(CheckUtils.checkTaskNodeParameters(taskNode));
taskNode.setParams("unKnown");
taskNode.setType("unKnown");
assertFalse(CheckUtils.checkTaskNodeParameters(taskNode));
taskNode.setParams("unKnown");
taskNode.setType(null);
assertFalse(CheckUtils.checkTaskNodeParameters(taskNode));
// sub SubProcessParameters
SubProcessParameters subProcessParameters = new SubProcessParameters();
assertFalse(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(subProcessParameters), TaskType.SUB_PROCESS.toString()));
taskNode.setParams(JSONUtils.toJsonString(subProcessParameters));
taskNode.setType(TaskType.SUB_PROCESS.getDesc());
assertFalse(CheckUtils.checkTaskNodeParameters(taskNode));
subProcessParameters.setProcessDefinitionId(1234);
assertTrue(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(subProcessParameters), TaskType.SUB_PROCESS.toString()));
taskNode.setParams(JSONUtils.toJsonString(subProcessParameters));
taskNode.setType(TaskType.SUB_PROCESS.getDesc());
assertTrue(CheckUtils.checkTaskNodeParameters(taskNode));
// ShellParameters
ShellParameters shellParameters = new ShellParameters();
assertFalse(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(shellParameters), TaskType.SHELL.toString()));
taskNode.setParams(JSONUtils.toJsonString(shellParameters));
taskNode.setType(TaskType.SHELL.getDesc());
assertFalse(CheckUtils.checkTaskNodeParameters(taskNode));
shellParameters.setRawScript("");
assertFalse(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(shellParameters), TaskType.SHELL.toString()));
taskNode.setParams(JSONUtils.toJsonString(shellParameters));
assertFalse(CheckUtils.checkTaskNodeParameters(taskNode));
shellParameters.setRawScript("sss");
assertTrue(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(shellParameters), TaskType.SHELL.toString()));
taskNode.setParams(JSONUtils.toJsonString(shellParameters));
assertTrue(CheckUtils.checkTaskNodeParameters(taskNode));
// ProcedureParameters
ProcedureParameters procedureParameters = new ProcedureParameters();
assertFalse(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(procedureParameters), TaskType.PROCEDURE.toString()));
taskNode.setParams(JSONUtils.toJsonString(procedureParameters));
taskNode.setType(TaskType.PROCEDURE.getDesc());
assertFalse(CheckUtils.checkTaskNodeParameters(taskNode));
procedureParameters.setDatasource(1);
procedureParameters.setType("xx");
procedureParameters.setMethod("yy");
assertTrue(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(procedureParameters), TaskType.PROCEDURE.toString()));
taskNode.setParams(JSONUtils.toJsonString(procedureParameters));
assertTrue(CheckUtils.checkTaskNodeParameters(taskNode));
// SqlParameters
SqlParameters sqlParameters = new SqlParameters();
assertFalse(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(sqlParameters), TaskType.SQL.toString()));
taskNode.setParams(JSONUtils.toJsonString(sqlParameters));
taskNode.setType(TaskType.SQL.getDesc());
assertFalse(CheckUtils.checkTaskNodeParameters(taskNode));
sqlParameters.setDatasource(1);
sqlParameters.setType("xx");
sqlParameters.setSql("yy");
assertTrue(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(sqlParameters), TaskType.SQL.toString()));
taskNode.setParams(JSONUtils.toJsonString(sqlParameters));
assertTrue(CheckUtils.checkTaskNodeParameters(taskNode));
// MapReduceParameters
MapReduceParameters mapreduceParameters = new MapReduceParameters();
assertFalse(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(mapreduceParameters), TaskType.MR.toString()));
taskNode.setParams(JSONUtils.toJsonString(mapreduceParameters));
taskNode.setType(TaskType.MR.getDesc());
assertFalse(CheckUtils.checkTaskNodeParameters(taskNode));
ResourceInfo resourceInfoMapreduce = new ResourceInfo();
resourceInfoMapreduce.setId(1);
resourceInfoMapreduce.setRes("");
mapreduceParameters.setMainJar(resourceInfoMapreduce);
mapreduceParameters.setProgramType(ProgramType.JAVA);
assertTrue(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(mapreduceParameters), TaskType.MR.toString()));
taskNode.setParams(JSONUtils.toJsonString(mapreduceParameters));
taskNode.setType(TaskType.MR.getDesc());
assertTrue(CheckUtils.checkTaskNodeParameters(taskNode));
// SparkParameters
SparkParameters sparkParameters = new SparkParameters();
assertFalse(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(sparkParameters), TaskType.SPARK.toString()));
taskNode.setParams(JSONUtils.toJsonString(sparkParameters));
taskNode.setType(TaskType.SPARK.getDesc());
assertFalse(CheckUtils.checkTaskNodeParameters(taskNode));
sparkParameters.setMainJar(new ResourceInfo());
sparkParameters.setProgramType(ProgramType.SCALA);
sparkParameters.setSparkVersion("1.1.1");
assertTrue(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(sparkParameters), TaskType.SPARK.toString()));
taskNode.setParams(JSONUtils.toJsonString(sparkParameters));
assertTrue(CheckUtils.checkTaskNodeParameters(taskNode));
// PythonParameters
PythonParameters pythonParameters = new PythonParameters();
assertFalse(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(pythonParameters), TaskType.PYTHON.toString()));
taskNode.setParams(JSONUtils.toJsonString(pythonParameters));
taskNode.setType(TaskType.PYTHON.getDesc());
assertFalse(CheckUtils.checkTaskNodeParameters(taskNode));
pythonParameters.setRawScript("ss");
assertTrue(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(pythonParameters), TaskType.PYTHON.toString()));
taskNode.setParams(JSONUtils.toJsonString(pythonParameters));
assertTrue(CheckUtils.checkTaskNodeParameters(taskNode));
// DependentParameters
DependentParameters dependentParameters = new DependentParameters();
assertTrue(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(dependentParameters), TaskType.DEPENDENT.toString()));
taskNode.setDependence(JSONUtils.toJsonString(dependentParameters));
taskNode.setType(TaskType.DEPENDENT.getDesc());
assertTrue(CheckUtils.checkTaskNodeParameters(taskNode));
// FlinkParameters
FlinkParameters flinkParameters = new FlinkParameters();
assertFalse(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(flinkParameters), TaskType.FLINK.toString()));
taskNode.setParams(JSONUtils.toJsonString(flinkParameters));
taskNode.setType(TaskType.FLINK.getDesc());
assertFalse(CheckUtils.checkTaskNodeParameters(taskNode));
flinkParameters.setMainJar(new ResourceInfo());
flinkParameters.setProgramType(ProgramType.JAVA);
assertTrue(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(flinkParameters), TaskType.FLINK.toString()));
taskNode.setParams(JSONUtils.toJsonString(flinkParameters));
taskNode.setType(TaskType.FLINK.getDesc());
assertTrue(CheckUtils.checkTaskNodeParameters(taskNode));
// HTTP
HttpParameters httpParameters = new HttpParameters();
assertFalse(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(httpParameters), TaskType.HTTP.toString()));
taskNode.setParams(JSONUtils.toJsonString(httpParameters));
taskNode.setType(TaskType.HTTP.getDesc());
assertFalse(CheckUtils.checkTaskNodeParameters(taskNode));
httpParameters.setUrl("httpUrl");
assertTrue(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(httpParameters), TaskType.HTTP.toString()));
taskNode.setParams(JSONUtils.toJsonString(httpParameters));
taskNode.setType(TaskType.HTTP.getDesc());
assertTrue(CheckUtils.checkTaskNodeParameters(taskNode));
// DataxParameters
DataxParameters dataxParameters = new DataxParameters();
assertFalse(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(dataxParameters), TaskType.DATAX.toString()));
taskNode.setParams(JSONUtils.toJsonString(dataxParameters));
taskNode.setType(TaskType.DATAX.getDesc());
assertFalse(CheckUtils.checkTaskNodeParameters(taskNode));
dataxParameters.setCustomConfig(0);
dataxParameters.setDataSource(111);
dataxParameters.setDataTarget(333);
dataxParameters.setSql("sql");
dataxParameters.setSql(TaskType.SQL.getDesc());
dataxParameters.setTargetTable("tar");
assertTrue(CheckUtils.checkTaskNodeParameters(JSONUtils.toJsonString(dataxParameters), TaskType.DATAX.toString()));
taskNode.setParams(JSONUtils.toJsonString(dataxParameters));
taskNode.setType(TaskType.DATAX.getDesc());
assertTrue(CheckUtils.checkTaskNodeParameters(taskNode));
}
}
}

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

@ -422,6 +422,11 @@ public final class Constants {
*/
public static final String FLOWNODE_RUN_FLAG_FORBIDDEN = "FORBIDDEN";
/**
* normal running task
*/
public static final String FLOWNODE_RUN_FLAG_NORMAL = "NORMAL";
/**
* datasource configuration path
*/
@ -639,6 +644,11 @@ public final class Constants {
*/
public static final int EXIT_CODE_FAILURE = -1;
/**
* process or task definition failure
*/
public static final int DEFINITION_FAILURE = -1;
/**
* date format of yyyyMMdd
*/
@ -769,6 +779,8 @@ public final class Constants {
public static final String SUBPROCESS_INSTANCE_ID = "subProcessInstanceId";
public static final String PROCESS_INSTANCE_STATE = "processInstanceState";
public static final String PARENT_WORKFLOW_INSTANCE = "parentWorkflowInstance";
public static final String CONDITION_RESULT = "conditionResult";
public static final String DEPENDENCE = "dependence";
public static final String TASK_TYPE = "taskType";
public static final String TASK_LIST = "taskList";
public static final String RWXR_XR_X = "rwxr-xr-x";
@ -1076,6 +1088,11 @@ public final class Constants {
*/
public static final String PSTREE = "pstree";
/**
* snow flake, data center id, this id must be greater than 0 and less than 32
*/
public static final String SNOW_FLAKE_DATA_CENTER_ID = "data.center.id";
/**
* docker & kubernetes
*/

69
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/ConditionType.java

@ -0,0 +1,69 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.common.enums;
import java.util.HashMap;
import java.util.Map;
import com.baomidou.mybatisplus.annotation.EnumValue;
/**
* condition type
*/
public enum ConditionType {
/**
* 0 none
* 1 judge
* 2 delay
*/
NONE(0, "none"),
JUDGE(1, "judge"),
DELAY(2, "delay");
ConditionType(int code, String desc) {
this.code = code;
this.desc = desc;
}
@EnumValue
private final int code;
private final String desc;
public int getCode() {
return code;
}
public String getDesc() {
return desc;
}
private static final Map<String, ConditionType> CONDITION_TYPE_MAP = new HashMap<>();
static {
for (ConditionType conditionType : ConditionType.values()) {
CONDITION_TYPE_MAP.put(conditionType.desc, conditionType);
}
}
public static ConditionType of(String desc) {
if (CONDITION_TYPE_MAP.containsKey(desc)) {
return CONDITION_TYPE_MAP.get(desc);
}
throw new IllegalArgumentException("invalid type : " + desc);
}
}

1
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskTimeoutStrategy.java

@ -57,4 +57,5 @@ public enum TaskTimeoutStrategy {
}
throw new IllegalArgumentException("invalid status : " + status);
}
}

43
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskType.java

@ -38,40 +38,35 @@ public enum TaskType {
* 12 SQOOP
* 13 WATERDROP
*/
SHELL(0, "shell"),
SQL(1, "sql"),
SUB_PROCESS(2, "sub_process"),
PROCEDURE(3, "procedure"),
MR(4, "mr"),
SPARK(5, "spark"),
PYTHON(6, "python"),
DEPENDENT(7, "dependent"),
FLINK(8, "flink"),
HTTP(9, "http"),
DATAX(10, "datax"),
CONDITIONS(11, "conditions"),
SQOOP(12, "sqoop"),
WATERDROP(13, "waterdrop");
SHELL(0, "SHELL"),
SQL(1, "SQL"),
SUB_PROCESS(2, "SUB_PROCESS"),
PROCEDURE(3, "PROCEDURE"),
MR(4, "MR"),
SPARK(5, "SPARK"),
PYTHON(6, "PYTHON"),
DEPENDENT(7, "DEPENDENT"),
FLINK(8, "FLINK"),
HTTP(9, "HTTP"),
DATAX(10, "DATAX"),
CONDITIONS(11, "CONDITIONS"),
SQOOP(12, "SQOOP"),
WATERDROP(13, "WATERDROP");
TaskType(int code, String descp){
TaskType(int code, String desc) {
this.code = code;
this.descp = descp;
this.desc = desc;
}
@EnumValue
private final int code;
private final String descp;
public static boolean typeIsNormalTask(String typeName) {
TaskType taskType = TaskType.valueOf(typeName);
return !(taskType == TaskType.SUB_PROCESS || taskType == TaskType.DEPENDENT);
}
private final String desc;
public int getCode() {
return code;
}
public String getDescp() {
return descp;
public String getDesc() {
return desc;
}
}

49
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TimeoutFlag.java

@ -0,0 +1,49 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.common.enums;
import com.baomidou.mybatisplus.annotation.EnumValue;
/**
* timeout flag
*/
public enum TimeoutFlag {
/**
* 0 close
* 1 open
*/
CLOSE(0, "close"),
OPEN(1, "open");
TimeoutFlag(int code, String desc) {
this.code = code;
this.desc = desc;
}
@EnumValue
private final int code;
private final String desc;
public int getCode() {
return code;
}
public String getDesc() {
return desc;
}
}

12
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/DependentItem.java

@ -24,7 +24,7 @@ import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
*/
public class DependentItem {
private int definitionId;
private Long definitionCode;
private String depTasks;
private String cycle;
private String dateValue;
@ -34,18 +34,18 @@ public class DependentItem {
public String getKey(){
return String.format("%d-%s-%s-%s",
getDefinitionId(),
getDefinitionCode(),
getDepTasks(),
getCycle(),
getDateValue());
}
public int getDefinitionId() {
return definitionId;
public Long getDefinitionCode() {
return definitionCode;
}
public void setDefinitionId(int definitionId) {
this.definitionId = definitionId;
public void setDefinitionCode(Long definitionCode) {
this.definitionCode = definitionCode;
}
public String getDepTasks() {

70
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/PreviousTaskNode.java

@ -0,0 +1,70 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.common.model;
public class PreviousTaskNode {
/**
* code
*/
private long code;
/**
* name
*/
private String name;
/**
* version
*/
private int version;
public PreviousTaskNode() {
}
public PreviousTaskNode(long code, String name, int version) {
this.code = code;
this.name = name;
this.version = version;
}
public long getCode() {
return code;
}
public void setCode(long code) {
this.code = code;
}
public String getName() {
return name;
}
public void setName(String name) {
this.name = name;
}
public int getVersion() {
return version;
}
public void setVersion(int version) {
this.version = version;
}
}

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

@ -22,342 +22,306 @@ import org.apache.dolphinscheduler.common.enums.TaskTimeoutStrategy;
import org.apache.dolphinscheduler.common.enums.TaskType;
import org.apache.dolphinscheduler.common.task.TaskTimeoutParameter;
import org.apache.dolphinscheduler.common.utils.CollectionUtils;
import org.apache.dolphinscheduler.common.utils.*;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
import com.fasterxml.jackson.databind.annotation.JsonSerialize;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
import com.fasterxml.jackson.databind.annotation.JsonSerialize;
public class TaskNode {
/**
* task node id
*/
private String id;
/**
* task node name
*/
private String name;
/**
* task node description
*/
private String desc;
/**
* task node type
*/
private String type;
/**
* the run flag has two states, NORMAL or FORBIDDEN
*/
private String runFlag;
/**
* the front field
*/
private String loc;
/**
* maximum number of retries
*/
private int maxRetryTimes;
/**
* Unit of retry interval: points
*/
private int retryInterval;
/**
* params information
*/
@JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class)
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
private String params;
/**
* inner dependency information
*/
@JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class)
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
private String preTasks;
/**
* users store additional information
*/
@JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class)
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
private String extras;
/**
* node dependency list
*/
private List<String> depList;
/**
* outer dependency information
*/
@JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class)
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
private String dependence;
@JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class)
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
private String conditionResult;
/**
* task instance priority
*/
private Priority taskInstancePriority;
/**
* worker group
*/
private String workerGroup;
/**
* worker group id
*/
private Integer workerGroupId;
/**
* task time out
*/
@JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class)
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
private String timeout;
/**
* task node id
*/
private String id;
/**
* task node code
*/
private Long code;
/**
* task node version
*/
private int version;
/**
* task node name
*/
private String name;
/**
* task node description
*/
private String desc;
/**
* task node type
*/
private String type;
/**
* the run flag has two states, NORMAL or FORBIDDEN
*/
private String runFlag;
/**
* the front field
*/
private String loc;
/**
* maximum number of retries
*/
private int maxRetryTimes;
/**
* Unit of retry interval: points
*/
private int retryInterval;
/**
* params information
*/
@JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class)
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
private String params;
/**
* inner dependency information
*/
@JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class)
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
private String preTasks;
/**
* node dependency list
*/
private List<PreviousTaskNode> preTaskNodeList;
/**
* users store additional information
*/
@JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class)
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
private String extras;
/**
* node dependency list
*/
private List<String> depList;
/**
* outer dependency information
*/
@JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class)
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
private String dependence;
@JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class)
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
private String conditionResult;
/**
* task instance priority
*/
private Priority taskInstancePriority;
/**
* worker group
*/
private String workerGroup;
/**
* task time out
*/
@JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class)
@JsonSerialize(using = JSONUtils.JsonDataSerializer.class)
private String timeout;
/**
* delay execution time.
*/
private int delayTime;
public String getId() {
return id;
}
public void setId(String id) {
this.id = id;
}
public String getName() {
return name;
}
public void setName(String name) {
this.name = name;
}
public String getDesc() {
return desc;
}
public void setDesc(String desc) {
this.desc = desc;
}
public String getType() {
return type;
}
public void setType(String type) {
this.type = type;
}
public String getParams() {
return params;
}
public void setParams(String params) {
this.params = params;
}
public String getPreTasks() {
return preTasks;
}
public void setPreTasks(String preTasks) throws IOException {
this.preTasks = preTasks;
this.depList = JSONUtils.toList(preTasks, String.class);
}
public String getExtras() {
return extras;
}
public void setExtras(String extras) {
this.extras = extras;
}
public List<String> getDepList() {
return depList;
}
public void setDepList(List<String> depList) throws JsonProcessingException {
this.depList = depList;
}
public String getLoc() {
return loc;
}
public void setLoc(String loc) {
this.loc = loc;
}
public String getRunFlag(){
return runFlag;
}
public void setRunFlag(String runFlag) {
this.runFlag = runFlag;
}
public Boolean isForbidden(){
return (StringUtils.isNotEmpty(this.runFlag) &&
this.runFlag.equals(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN));
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
TaskNode taskNode = (TaskNode) o;
return Objects.equals(name, taskNode.name) &&
Objects.equals(desc, taskNode.desc) &&
Objects.equals(type, taskNode.type) &&
Objects.equals(params, taskNode.params) &&
Objects.equals(preTasks, taskNode.preTasks) &&
Objects.equals(extras, taskNode.extras) &&
Objects.equals(runFlag, taskNode.runFlag) &&
Objects.equals(dependence, taskNode.dependence) &&
Objects.equals(workerGroup, taskNode.workerGroup) &&
Objects.equals(conditionResult, taskNode.conditionResult) &&
CollectionUtils.equalLists(depList, taskNode.depList);
}
@Override
public int hashCode() {
return Objects.hash(name, desc, type, params, preTasks, extras, depList, runFlag);
}
public String getDependence() {
return dependence;
}
public void setDependence(String dependence) {
this.dependence = dependence;
}
public int getMaxRetryTimes() {
return maxRetryTimes;
}
public void setMaxRetryTimes(int maxRetryTimes) {
this.maxRetryTimes = maxRetryTimes;
}
public int getRetryInterval() {
return retryInterval;
}
public void setRetryInterval(int retryInterval) {
this.retryInterval = retryInterval;
}
public Priority getTaskInstancePriority() {
return taskInstancePriority;
}
public void setTaskInstancePriority(Priority taskInstancePriority) {
this.taskInstancePriority = taskInstancePriority;
}
public String getTimeout() {
return timeout;
}
public void setTimeout(String timeout) {
this.timeout = timeout;
}
/**
* get task time out parameter
* @return task time out parameter
*/
public TaskTimeoutParameter getTaskTimeoutParameter() {
if(StringUtils.isNotEmpty(this.getTimeout())){
String formatStr = String.format("%s,%s", TaskTimeoutStrategy.WARN.name(), TaskTimeoutStrategy.FAILED.name());
String taskTimeout = this.getTimeout().replace(formatStr,TaskTimeoutStrategy.WARNFAILED.name());
return JSONUtils.parseObject(taskTimeout,TaskTimeoutParameter.class);
}
return new TaskTimeoutParameter(false);
}
public boolean isConditionsTask(){
return TaskType.CONDITIONS.toString().equalsIgnoreCase(this.getType());
}
@Override
public String toString() {
return "TaskNode{"
+ "id='" + id + '\''
+ ", name='" + name + '\''
+ ", desc='" + desc + '\''
+ ", type='" + type + '\''
+ ", runFlag='" + runFlag + '\''
+ ", loc='" + loc + '\''
+ ", maxRetryTimes=" + maxRetryTimes
+ ", retryInterval=" + retryInterval
+ ", params='" + params + '\''
+ ", preTasks='" + preTasks + '\''
+ ", extras='" + extras + '\''
+ ", depList=" + depList
+ ", dependence='" + dependence + '\''
+ ", taskInstancePriority=" + taskInstancePriority
+ ", timeout='" + timeout + '\''
+ ", workerGroup='" + workerGroup + '\''
+ ", delayTime=" + delayTime
+ '}';
}
public String getWorkerGroup() {
return workerGroup;
}
public void setWorkerGroup(String workerGroup) {
this.workerGroup = workerGroup;
}
public String getConditionResult() {
return conditionResult;
}
public void setConditionResult(String conditionResult) {
this.conditionResult = conditionResult;
}
public Integer getWorkerGroupId() {
return workerGroupId;
}
public void setWorkerGroupId(Integer workerGroupId) {
this.workerGroupId = workerGroupId;
}
public String getId() {
return id;
}
public void setId(String id) {
this.id = id;
}
public String getName() {
return name;
}
public void setName(String name) {
this.name = name;
}
public String getDesc() {
return desc;
}
public void setDesc(String desc) {
this.desc = desc;
}
public String getType() {
return type;
}
public void setType(String type) {
this.type = type;
}
public String getParams() {
return params;
}
public void setParams(String params) {
this.params = params;
}
public String getPreTasks() {
return preTasks;
}
public void setPreTasks(String preTasks) {
this.preTasks = preTasks;
this.depList = JSONUtils.toList(preTasks, String.class);
}
public String getExtras() {
return extras;
}
public void setExtras(String extras) {
this.extras = extras;
}
public List<String> getDepList() {
return depList;
}
public void setDepList(List<String> depList) {
if (depList != null) {
this.depList = depList;
this.preTasks = JSONUtils.toJsonString(depList);
}
}
public String getLoc() {
return loc;
}
public void setLoc(String loc) {
this.loc = loc;
}
public String getRunFlag() {
return runFlag;
}
public void setRunFlag(String runFlag) {
this.runFlag = runFlag;
}
public Boolean isForbidden() {
return (StringUtils.isNotEmpty(this.runFlag)
&& this.runFlag.equals(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN));
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
TaskNode taskNode = (TaskNode) o;
return Objects.equals(name, taskNode.name)
&& Objects.equals(desc, taskNode.desc)
&& Objects.equals(type, taskNode.type)
&& Objects.equals(params, taskNode.params)
&& Objects.equals(preTasks, taskNode.preTasks)
&& Objects.equals(extras, taskNode.extras)
&& Objects.equals(runFlag, taskNode.runFlag)
&& Objects.equals(dependence, taskNode.dependence)
&& Objects.equals(workerGroup, taskNode.workerGroup)
&& Objects.equals(conditionResult, taskNode.conditionResult)
&& CollectionUtils.equalLists(depList, taskNode.depList);
}
@Override
public int hashCode() {
return Objects.hash(name, desc, type, params, preTasks, extras, depList, runFlag);
}
public String getDependence() {
return dependence;
}
public void setDependence(String dependence) {
this.dependence = dependence;
}
public int getMaxRetryTimes() {
return maxRetryTimes;
}
public void setMaxRetryTimes(int maxRetryTimes) {
this.maxRetryTimes = maxRetryTimes;
}
public int getRetryInterval() {
return retryInterval;
}
public void setRetryInterval(int retryInterval) {
this.retryInterval = retryInterval;
}
public Priority getTaskInstancePriority() {
return taskInstancePriority;
}
public void setTaskInstancePriority(Priority taskInstancePriority) {
this.taskInstancePriority = taskInstancePriority;
}
public String getTimeout() {
return timeout;
}
public void setTimeout(String timeout) {
this.timeout = timeout;
}
public String getWorkerGroup() {
return workerGroup;
}
public void setWorkerGroup(String workerGroup) {
this.workerGroup = workerGroup;
}
public String getConditionResult() {
return conditionResult;
}
public void setConditionResult(String conditionResult) {
this.conditionResult = conditionResult;
}
public int getDelayTime() {
return delayTime;
@ -366,4 +330,91 @@ public class TaskNode {
public void setDelayTime(int delayTime) {
this.delayTime = delayTime;
}
public Long getCode() {
return code;
}
public void setCode(Long code) {
this.code = code;
}
public int getVersion() {
return version;
}
public void setVersion(int version) {
this.version = version;
}
/**
* get task time out parameter
*
* @return task time out parameter
*/
public TaskTimeoutParameter getTaskTimeoutParameter() {
if (StringUtils.isNotEmpty(this.getTimeout())) {
String formatStr = String.format("%s,%s", TaskTimeoutStrategy.WARN.name(), TaskTimeoutStrategy.FAILED.name());
String taskTimeout = this.getTimeout().replace(formatStr, TaskTimeoutStrategy.WARNFAILED.name());
return JSONUtils.parseObject(taskTimeout, TaskTimeoutParameter.class);
}
return new TaskTimeoutParameter(false);
}
public boolean isConditionsTask() {
return TaskType.CONDITIONS.getDesc().equalsIgnoreCase(this.getType());
}
public List<PreviousTaskNode> getPreTaskNodeList() {
return preTaskNodeList;
}
public void setPreTaskNodeList(List<PreviousTaskNode> preTaskNodeList) {
this.preTaskNodeList = preTaskNodeList;
}
public String getTaskParams() {
Map<String, Object> taskParams = JSONUtils.toMap(this.params, String.class, Object.class);
if (taskParams == null) {
taskParams = new HashMap<>();
}
taskParams.put(Constants.CONDITION_RESULT, this.conditionResult);
taskParams.put(Constants.DEPENDENCE, this.dependence);
return JSONUtils.toJsonString(taskParams);
}
public Map<String, Object> taskParamsToJsonObj(String taskParams) {
Map<String, Object> taskParamsMap = JSONUtils.toMap(taskParams, String.class, Object.class);
if (taskParamsMap == null) {
taskParamsMap = new HashMap<>();
}
return taskParamsMap;
}
@Override
public String toString() {
return "TaskNode{"
+ "id='" + id + '\''
+ ", code=" + code
+ ", version=" + version
+ ", name='" + name + '\''
+ ", desc='" + desc + '\''
+ ", type='" + type + '\''
+ ", runFlag='" + runFlag + '\''
+ ", loc='" + loc + '\''
+ ", maxRetryTimes=" + maxRetryTimes
+ ", retryInterval=" + retryInterval
+ ", params='" + params + '\''
+ ", preTasks='" + preTasks + '\''
+ ", preTaskNodeList=" + preTaskNodeList
+ ", extras='" + extras + '\''
+ ", depList=" + depList
+ ", dependence='" + dependence + '\''
+ ", conditionResult='" + conditionResult + '\''
+ ", taskInstancePriority=" + taskInstancePriority
+ ", workerGroup='" + workerGroup + '\''
+ ", timeout='" + timeout + '\''
+ ", delayTime=" + delayTime
+ '}';
}
}

11
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/FileUtils.java

@ -114,15 +114,16 @@ public class FileUtils {
/**
* directory of process execution
*
* @param projectId project id
* @param processDefineId process definition id
* @param projectCode project code
* @param processDefineCode process definition Code
* @param processDefineVersion process definition version
* @param processInstanceId process instance id
* @param taskInstanceId task instance id
* @return directory of process execution
*/
public static String getProcessExecDir(int projectId, int processDefineId, int processInstanceId, int taskInstanceId) {
String fileName = String.format("%s/exec/process/%d/%d/%d/%d", DATA_BASEDIR,
projectId, processDefineId, processInstanceId, taskInstanceId);
public static String getProcessExecDir(long projectCode, long processDefineCode, int processDefineVersion, int processInstanceId, int taskInstanceId) {
String fileName = String.format("%s/exec/process/%d/%s/%d/%d", DATA_BASEDIR,
projectCode, processDefineCode + "_" + processDefineVersion, processInstanceId, taskInstanceId);
File file = new File(fileName);
if (!file.getParentFile().exists()) {
file.getParentFile().mkdirs();

2
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/JSONUtils.java

@ -206,7 +206,7 @@ public class JSONUtils {
return null;
}
return node.toString();
return node.asText();
}
/**

11
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/LoggerUtils.java

@ -67,20 +67,17 @@ public class LoggerUtils {
* build job id
*
* @param affix Task Logger's prefix
* @param processDefId process define id
* @param processInstId process instance id
* @param taskId task id
* @return task id format
*/
public static String buildTaskId(String affix,
int processDefId,
Long processDefineCode,
int processDefineVersion,
int processInstId,
int taskId) {
// - [taskAppId=TASK_79_4084_15210]
return String.format(" - %s%s-%s-%s-%s]", TASK_APPID_LOG_FORMAT, affix,
processDefId,
processInstId,
taskId);
// - [taskAppId=TASK-798_1-4084-15210]
return String.format(" - %s%s-%s_%s-%s-%s]", TASK_APPID_LOG_FORMAT, affix, processDefineCode, processDefineVersion, processInstId, taskId);
}
/**

91
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/SnowFlakeUtils.java

@ -0,0 +1,91 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.common.utils;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.Objects;
public class SnowFlakeUtils {
// start timestamp
private static final long START_TIMESTAMP = 1609430400L; //2021-01-01
// Number of digits
private static final long SEQUENCE_BIT = 13;
private static final long MACHINE_BIT = 2;
private static final long MAX_SEQUENCE = ~(-1L << SEQUENCE_BIT);
// The displacement to the left
private static final long MACHINE_LEFT = SEQUENCE_BIT;
private static final long TIMESTAMP_LEFT = SEQUENCE_BIT + MACHINE_BIT;
private final int machineId;
private long sequence = 0L;
private long lastTimestamp = -1L;
private SnowFlakeUtils() throws SnowFlakeException {
try {
this.machineId = Math.abs(Objects.hash(InetAddress.getLocalHost().getHostName())) % 32;
} catch (UnknownHostException e) {
throw new SnowFlakeException(e.getMessage());
}
}
private static SnowFlakeUtils instance = null;
public static synchronized SnowFlakeUtils getInstance() throws SnowFlakeException {
if (instance == null) {
instance = new SnowFlakeUtils();
}
return instance;
}
public synchronized long nextId() throws SnowFlakeException {
long currStmp = nowTimestamp();
if (currStmp < lastTimestamp) {
throw new SnowFlakeException("Clock moved backwards. Refusing to generate id");
}
if (currStmp == lastTimestamp) {
sequence = (sequence + 1) & MAX_SEQUENCE;
if (sequence == 0L) {
currStmp = getNextMill();
}
} else {
sequence = 0L;
}
lastTimestamp = currStmp;
return (currStmp - START_TIMESTAMP) << TIMESTAMP_LEFT
| machineId << MACHINE_LEFT
| sequence;
}
private long getNextMill() {
long mill = nowTimestamp();
while (mill <= lastTimestamp) {
mill = nowTimestamp();
}
return mill;
}
private long nowTimestamp() {
return System.currentTimeMillis() / 1000;
}
public static class SnowFlakeException extends Exception {
public SnowFlakeException(String message) {
super(message);
}
}
}

43
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/StringUtils.java

@ -30,6 +30,8 @@ public class StringUtils {
*/
public static final String EMPTY = "";
public static final int INDEX_NOT_FOUND = -1;
private StringUtils() {
throw new UnsupportedOperationException("Construct StringUtils");
}
@ -132,6 +134,46 @@ public class StringUtils {
return str1 == null ? str2 == null : str1.equalsIgnoreCase(str2);
}
public static String substringBefore(final String str, final String separator) {
if (isEmpty(str) || separator == null) {
return str;
}
if (separator.isEmpty()) {
return EMPTY;
}
final int pos = str.indexOf(separator);
if (pos == INDEX_NOT_FOUND) {
return str;
}
return str.substring(0, pos);
}
public static String substringAfter(final String str, final String separator) {
if (isEmpty(str)) {
return str;
}
if (separator == null) {
return EMPTY;
}
final int pos = str.indexOf(separator);
if (pos == INDEX_NOT_FOUND) {
return EMPTY;
}
return str.substring(pos + separator.length());
}
public static long strDigitToLong(String str, long defaultValue) {
if (str == null) {
return defaultValue;
} else {
try {
return Long.parseLong(str);
} catch (NumberFormatException var4) {
return defaultValue;
}
}
}
/**
* <p>Joins the elements of the provided Collection into a single String
* containing the provided Collection of elements.</p>
@ -181,5 +223,4 @@ public class StringUtils {
}
}
}
}

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

@ -17,7 +17,6 @@
package org.apache.dolphinscheduler.common.utils;
import org.apache.dolphinscheduler.common.enums.TaskType;
import org.apache.dolphinscheduler.common.task.AbstractParameters;
import org.apache.dolphinscheduler.common.task.conditions.ConditionsParameters;
import org.apache.dolphinscheduler.common.task.datax.DataxParameters;
@ -41,7 +40,7 @@ import org.slf4j.LoggerFactory;
*/
public class TaskParametersUtils {
private static Logger logger = LoggerFactory.getLogger(TaskParametersUtils.class);
private static final Logger logger = LoggerFactory.getLogger(TaskParametersUtils.class);
private TaskParametersUtils() {
throw new UnsupportedOperationException("Construct TaskParametersUtils");
@ -55,40 +54,36 @@ public class TaskParametersUtils {
* @return task parameters
*/
public static AbstractParameters getParameters(String taskType, String parameter) {
TaskType anEnum = EnumUtils.getEnum(TaskType.class, taskType);
if (anEnum == null) {
logger.error("not support task type: {}", taskType);
return null;
}
switch (anEnum) {
case SUB_PROCESS:
switch (taskType) {
case "SUB_PROCESS":
return JSONUtils.parseObject(parameter, SubProcessParameters.class);
case SHELL:
case WATERDROP:
case "SHELL":
case "WATERDROP":
return JSONUtils.parseObject(parameter, ShellParameters.class);
case PROCEDURE:
case "PROCEDURE":
return JSONUtils.parseObject(parameter, ProcedureParameters.class);
case SQL:
case "SQL":
return JSONUtils.parseObject(parameter, SqlParameters.class);
case MR:
case "MR":
return JSONUtils.parseObject(parameter, MapReduceParameters.class);
case SPARK:
case "SPARK":
return JSONUtils.parseObject(parameter, SparkParameters.class);
case PYTHON:
case "PYTHON":
return JSONUtils.parseObject(parameter, PythonParameters.class);
case DEPENDENT:
case "DEPENDENT":
return JSONUtils.parseObject(parameter, DependentParameters.class);
case FLINK:
case "FLINK":
return JSONUtils.parseObject(parameter, FlinkParameters.class);
case HTTP:
case "HTTP":
return JSONUtils.parseObject(parameter, HttpParameters.class);
case DATAX:
case "DATAX":
return JSONUtils.parseObject(parameter, DataxParameters.class);
case CONDITIONS:
case "CONDITIONS":
return JSONUtils.parseObject(parameter, ConditionsParameters.class);
case SQOOP:
case "SQOOP":
return JSONUtils.parseObject(parameter, SqoopParameters.class);
default:
logger.error("not support task type: {}", taskType);
return null;
}

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

@ -39,8 +39,7 @@ public class VarPoolUtils {
*/
public static void setTaskNodeLocalParams(TaskNode taskNode, Map<String, Object> propToValue) {
String taskParamsJson = taskNode.getParams();
Map<String,Object> taskParams = JSONUtils.parseObject(taskParamsJson, HashMap.class);
Map<String,Object> taskParams = JSONUtils.toMap(taskParamsJson, String.class, Object.class);
Object localParamsObject = taskParams.get(LOCALPARAMS);
if (null != localParamsObject && null != propToValue && propToValue.size() > 0) {
ArrayList<Object> localParams = (ArrayList)localParamsObject;

2
dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/plugin/DolphinSchedulerPluginLoaderTest.java

@ -38,7 +38,7 @@ public class DolphinSchedulerPluginLoaderTest {
alertPluginManagerConfig.setPlugins(path + "../../../dolphinscheduler-alert-plugin/dolphinscheduler-alert-email/pom.xml");
DolphinPluginLoader alertPluginLoader = new DolphinPluginLoader(alertPluginManagerConfig, ImmutableList.of(pluginManager));
try {
alertPluginLoader.loadPlugins();
//alertPluginLoader.loadPlugins();
} catch (Exception e) {
throw new RuntimeException("load Alert Plugin Failed !", e);
}

4
dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/FileUtilsTest.java

@ -60,8 +60,8 @@ public class FileUtilsTest {
@Test
public void testGetProcessExecDir() {
String dir = FileUtils.getProcessExecDir(1, 2, 3, 4);
Assert.assertEquals("/tmp/dolphinscheduler/exec/process/1/2/3/4", dir);
String dir = FileUtils.getProcessExecDir(1L, 2L, 1, 3, 4);
Assert.assertEquals("/tmp/dolphinscheduler/exec/process/1/2_1/3/4", dir);
}
@Test

4
dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/LoggerUtilsTest.java

@ -42,9 +42,9 @@ public class LoggerUtilsTest {
@Test
public void buildTaskId() {
String taskId = LoggerUtils.buildTaskId(LoggerUtils.TASK_LOGGER_INFO_PREFIX, 79, 4084, 15210);
String taskId = LoggerUtils.buildTaskId(LoggerUtils.TASK_LOGGER_INFO_PREFIX, 798L,1,4084, 15210);
Assert.assertEquals(" - [taskAppId=TASK-79-4084-15210]", taskId);
Assert.assertEquals(" - [taskAppId=TASK-798_1-4084-15210]", taskId);
}
@Test

33
dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/SnowFlakeUtilsTest.java

@ -0,0 +1,33 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.common.utils;
import org.junit.Test;
public class SnowFlakeUtilsTest {
@Test
public void testNextId() {
try {
for (int i = 0; i < 5; i++) {
System.out.println(SnowFlakeUtils.getInstance().nextId());
}
} catch (Exception e) {
e.printStackTrace();
}
}
}

24
dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtilsTest.java

@ -16,6 +16,8 @@
*/
package org.apache.dolphinscheduler.common.utils;
import org.apache.dolphinscheduler.common.enums.TaskType;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -29,17 +31,15 @@ public class TaskParametersUtilsTest {
@Test
public void testGetParameters() {
Assert.assertNull(TaskParametersUtils.getParameters("xx", "ttt"));
Assert.assertNull(TaskParametersUtils.getParameters("SHELL", "ttt"));
Assert.assertNotNull(TaskParametersUtils.getParameters("SHELL", "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters("SQL", "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters("SUB_PROCESS", "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters("PROCEDURE", "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters("MR", "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters("SPARK", "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters("PYTHON", "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters("DEPENDENT", "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters("FLINK", "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters("HTTP", "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters(TaskType.SHELL.getDesc(), "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters(TaskType.SQL.getDesc(), "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters(TaskType.SUB_PROCESS.getDesc(), "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters(TaskType.PROCEDURE.getDesc(), "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters(TaskType.MR.getDesc(), "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters(TaskType.SPARK.getDesc(), "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters(TaskType.PYTHON.getDesc(), "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters(TaskType.DEPENDENT.getDesc(), "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters(TaskType.FLINK.getDesc(), "{}"));
Assert.assertNotNull(TaskParametersUtils.getParameters(TaskType.HTTP.getDesc(), "{}"));
}
}

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

@ -17,10 +17,6 @@
package org.apache.dolphinscheduler.common.utils;
import org.apache.dolphinscheduler.common.model.TaskNode;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.junit.Assert;
@ -29,7 +25,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class VarPoolUtilsTest {
private static final Logger logger = LoggerFactory.getLogger(VarPoolUtilsTest.class);
@Test
@ -37,54 +33,19 @@ public class VarPoolUtilsTest {
String varPool = "p1,66$VarPool$p2,69$VarPool$";
ConcurrentHashMap<String, Object> propToValue = new ConcurrentHashMap<String, Object>();
VarPoolUtils.convertVarPoolToMap(propToValue, varPool);
Assert.assertEquals((String)propToValue.get("p1"), "66");
Assert.assertEquals((String)propToValue.get("p2"), "69");
Assert.assertEquals((String) propToValue.get("p1"), "66");
Assert.assertEquals((String) propToValue.get("p2"), "69");
logger.info(propToValue.toString());
}
@Test
public void testConvertPythonScriptPlaceholders() throws Exception {
String rawScript = "print(${p1});\n${setShareVar(${p1},3)};\n${setShareVar(${p2},4)};";
rawScript = VarPoolUtils.convertPythonScriptPlaceholders(rawScript);
Assert.assertEquals(rawScript, "print(${p1});\n"
+ "print(\"${{setValue({},{})}}\".format(\"p1\",3));\n"
+ "print(\"${{setValue({},{})}}\".format(\"p2\",4));");
+ "print(\"${{setValue({},{})}}\".format(\"p1\",3));\n"
+ "print(\"${{setValue({},{})}}\".format(\"p2\",4));");
logger.info(rawScript);
}
@Test
public void testSetTaskNodeLocalParams() throws Exception {
String taskJson = "{\"id\":\"tasks-66199\",\"name\":\"file-shell\",\"desc\":null,\"type\":\"SHELL\","
+ "\"runFlag\":\"NORMAL\",\"loc\":null,\"maxRetryTimes\":0,\"retryInterval\":1,\""
+ "params\":{\"rawScript\":\"sh n-1/n-1-1/run.sh\",\""
+ "localParams\":[{\"prop\":\"k1\",\"direct\":\"IN\",\"type\":\"VARCHAR\",\"value\":\"v1\"},{\"prop\":\"k2\",\"direct\":\"IN\",\"type\":\"VARCHAR\",\"value\":\"v2\"},"
+ "{\"prop\":\"k3\",\"direct\":\"IN\",\"type\":\"VARCHAR\",\"value\":\"v3\"}],\""
+ "resourceList\":[{\"id\":\"dolphinschedule-code\",\"res\":\"n-1/n-1-1/dolphinscheduler-api-server.log\"},"
+ "{\"id\":\"mr-code\",\"res\":\"n-1/n-1-1/hadoop-mapreduce-examples-2.7.4.jar\"},"
+ "{\"id\":\"run\",\"res\":\"n-1/n-1-1/run.sh\"}]},\"preTasks\":[],\"extras\":null,\"depList\":[],\""
+ "dependence\":{},\"conditionResult\":{\"successNode\":[\"\"],\"failedNode\":[\"\"]},\"taskInstancePriority\":\"MEDIUM\",\""
+ "workerGroup\":\"default\",\"workerGroupId\":null,\"timeout\":{\"strategy\":\"\",\"interval\":null,\"enable\":false},\"delayTime\":0}";
String changeTaskJson = "{\"id\":\"tasks-66199\",\"name\":\"file-shell\",\"desc\":null,\"type\":\"SHELL\","
+ "\"runFlag\":\"NORMAL\",\"loc\":null,\"maxRetryTimes\":0,\"retryInterval\":1,\""
+ "params\":{\"rawScript\":\"sh n-1/n-1-1/run.sh\",\""
+ "localParams\":[{\"prop\":\"k1\",\"direct\":\"IN\",\"type\":\"VARCHAR\",\"value\":\"k1-value-change\"},"
+ "{\"prop\":\"k2\",\"direct\":\"IN\",\"type\":\"VARCHAR\",\"value\":\"k2-value-change\"},"
+ "{\"prop\":\"k3\",\"direct\":\"IN\",\"type\":\"VARCHAR\",\"value\":\"v3\"}],\""
+ "resourceList\":[{\"id\":\"dolphinschedule-code\",\"res\":\"n-1/n-1-1/dolphinscheduler-api-server.log\"},"
+ "{\"id\":\"mr-code\",\"res\":\"n-1/n-1-1/hadoop-mapreduce-examples-2.7.4.jar\"},"
+ "{\"id\":\"run\",\"res\":\"n-1/n-1-1/run.sh\"}]},\"preTasks\":[],\"extras\":null,\"depList\":[],\""
+ "dependence\":{},\"conditionResult\":{\"successNode\":[\"\"],\"failedNode\":[\"\"]},\"taskInstancePriority\":\"MEDIUM\",\""
+ "workerGroup\":\"default\",\"workerGroupId\":null,\"timeout\":{\"strategy\":\"\",\"interval\":null,\"enable\":false},\"delayTime\":0}";
Map<String, Object> propToValue = new HashMap<String, Object>();
propToValue.put("k1","k1-value-change");
propToValue.put("k2","k2-value-change");
TaskNode taskNode = JSONUtils.parseObject(taskJson,TaskNode.class);
VarPoolUtils.setTaskNodeLocalParams(taskNode,propToValue);
Assert.assertEquals(changeTaskJson,JSONUtils.toJsonString(taskNode));
}
}

44
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinition.java

@ -48,6 +48,11 @@ public class ProcessDefinition {
@TableId(value = "id", type = IdType.AUTO)
private int id;
/**
* code
*/
private Long code;
/**
* name
*/
@ -56,7 +61,7 @@ public class ProcessDefinition {
/**
* version
*/
private long version;
private int version;
/**
* release state : online/offline
@ -65,12 +70,21 @@ public class ProcessDefinition {
/**
* project id
* TODO: delete
*/
@TableField(exist = false)
private int projectId;
/**
* project code
*/
private Long projectCode;
/**
* definition json string
* TODO: delete
*/
@TableField(exist = false)
private String processDefinitionJson;
/**
@ -136,6 +150,7 @@ public class ProcessDefinition {
/**
* connects array for web
* TODO: delete
*/
private String connects;
@ -158,11 +173,13 @@ public class ProcessDefinition {
/**
* modify user name
*/
@TableField(exist = false)
private String modifyBy;
/**
* resource ids
*/
@TableField(exist = false)
private String resourceIds;
/**
@ -171,6 +188,8 @@ public class ProcessDefinition {
@TableField(exist = false)
private int warningGroupId;
public ProcessDefinition(){}
public String getName() {
return name;
}
@ -179,11 +198,11 @@ public class ProcessDefinition {
this.name = name;
}
public long getVersion() {
public int getVersion() {
return version;
}
public void setVersion(long version) {
public void setVersion(int version) {
this.version = version;
}
@ -366,6 +385,22 @@ public class ProcessDefinition {
this.modifyBy = modifyBy;
}
public Long getCode() {
return code;
}
public void setCode(Long code) {
this.code = code;
}
public Long getProjectCode() {
return projectCode;
}
public void setProjectCode(Long projectCode) {
this.projectCode = projectCode;
}
public int getWarningGroupId() {
return warningGroupId;
}
@ -379,9 +414,11 @@ public class ProcessDefinition {
return "ProcessDefinition{"
+ "id=" + id
+ ", name='" + name + '\''
+ ", code=" + code
+ ", version=" + version
+ ", releaseState=" + releaseState
+ ", projectId=" + projectId
+ ", projectCode=" + projectCode
+ ", processDefinitionJson='" + processDefinitionJson + '\''
+ ", description='" + description + '\''
+ ", globalParams='" + globalParams + '\''
@ -403,5 +440,4 @@ public class ProcessDefinition {
+ ", resourceIds='" + resourceIds + '\''
+ '}';
}
}

89
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinitionLog.java

@ -0,0 +1,89 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.entity;
import java.util.Date;
import com.baomidou.mybatisplus.annotation.TableName;
import com.fasterxml.jackson.annotation.JsonFormat;
/**
* process definition log
*/
@TableName("t_ds_process_definition_log")
public class ProcessDefinitionLog extends ProcessDefinition {
/**
* operator
*/
private int operator;
/**
* operateTime
*/
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date operateTime;
public ProcessDefinitionLog() {
super();
}
public ProcessDefinitionLog(ProcessDefinition processDefinition) {
this.setId(processDefinition.getId());
this.setCode(processDefinition.getCode());
this.setName(processDefinition.getName());
this.setVersion(processDefinition.getVersion());
this.setReleaseState(processDefinition.getReleaseState());
this.setProjectCode(processDefinition.getProjectCode());
this.setDescription(processDefinition.getDescription());
this.setGlobalParams(processDefinition.getGlobalParams());
this.setGlobalParamList(processDefinition.getGlobalParamList());
this.setGlobalParamMap(processDefinition.getGlobalParamMap());
this.setCreateTime(processDefinition.getCreateTime());
this.setUpdateTime(processDefinition.getUpdateTime());
this.setFlag(processDefinition.getFlag());
this.setUserId(processDefinition.getUserId());
this.setUserName(processDefinition.getUserName());
this.setProjectName(processDefinition.getProjectName());
this.setLocations(processDefinition.getLocations());
this.setConnects(processDefinition.getConnects());
this.setScheduleReleaseState(processDefinition.getScheduleReleaseState());
this.setTimeout(processDefinition.getTimeout());
this.setTenantId(processDefinition.getTenantId());
this.setModifyBy(processDefinition.getModifyBy());
this.setResourceIds(processDefinition.getResourceIds());
this.setWarningGroupId(processDefinition.getWarningGroupId());
}
public int getOperator() {
return operator;
}
public void setOperator(int operator) {
this.operator = operator;
}
public Date getOperateTime() {
return operateTime;
}
public void setOperateTime(Date operateTime) {
this.operateTime = operateTime;
}
}

311
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinitionVersion.java

@ -1,311 +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.dao.entity;
import java.util.Date;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableField;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
import com.fasterxml.jackson.annotation.JsonFormat;
/**
* process definition version
*/
@TableName("t_ds_process_definition_version")
public class ProcessDefinitionVersion {
/**
* id
*/
@TableId(value = "id", type = IdType.AUTO)
private int id;
/**
* process definition id
*/
private int processDefinitionId;
/**
* version
*/
private long version;
/**
* definition json string
*/
private String processDefinitionJson;
/**
* description
*/
private String description;
/**
* process warning time out. unit: minute
*/
private int timeout;
/**
* resource ids
*/
private String resourceIds;
/**
* create time
*/
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date createTime;
/**
* user defined parameters
*/
private String globalParams;
/**
* locations array for web
*/
private String locations;
/**
* connects array for web
*/
private String connects;
/**
* warningGroupId
*/
@TableField(exist = false)
private int warningGroupId;
public String getGlobalParams() {
return globalParams;
}
public void setGlobalParams(String globalParams) {
this.globalParams = globalParams;
}
public int getId() {
return id;
}
public void setId(int id) {
this.id = id;
}
public int getProcessDefinitionId() {
return processDefinitionId;
}
public void setProcessDefinitionId(int processDefinitionId) {
this.processDefinitionId = processDefinitionId;
}
public long getVersion() {
return version;
}
public void setVersion(long version) {
this.version = version;
}
public String getProcessDefinitionJson() {
return processDefinitionJson;
}
public void setProcessDefinitionJson(String processDefinitionJson) {
this.processDefinitionJson = processDefinitionJson;
}
public String getDescription() {
return description;
}
public void setDescription(String description) {
this.description = description;
}
public Date getCreateTime() {
return createTime;
}
public void setCreateTime(Date createTime) {
this.createTime = createTime;
}
public String getLocations() {
return locations;
}
public void setLocations(String locations) {
this.locations = locations;
}
public String getConnects() {
return connects;
}
public void setConnects(String connects) {
this.connects = connects;
}
public int getTimeout() {
return timeout;
}
public void setTimeout(int timeout) {
this.timeout = timeout;
}
public String getResourceIds() {
return resourceIds;
}
public void setResourceIds(String resourceIds) {
this.resourceIds = resourceIds;
}
public int getWarningGroupId() {
return warningGroupId;
}
public void setWarningGroupId(int warningGroupId) {
this.warningGroupId = warningGroupId;
}
@Override
public String toString() {
return "ProcessDefinitionVersion{"
+ "id=" + id
+ ", processDefinitionId=" + processDefinitionId
+ ", version=" + version
+ ", processDefinitionJson='" + processDefinitionJson + '\''
+ ", description='" + description + '\''
+ ", globalParams='" + globalParams + '\''
+ ", createTime=" + createTime
+ ", locations='" + locations + '\''
+ ", connects='" + connects + '\''
+ ", timeout=" + timeout
+ ", warningGroupId=" + warningGroupId
+ ", resourceIds='" + resourceIds + '\''
+ '}';
}
public static Builder newBuilder() {
return new Builder();
}
public static final class Builder {
private int id;
private int processDefinitionId;
private long version;
private String processDefinitionJson;
private String description;
private String globalParams;
private Date createTime;
private String locations;
private String connects;
private int timeout;
private int warningGroupId;
private String resourceIds;
private Builder() {
}
public Builder id(int id) {
this.id = id;
return this;
}
public Builder processDefinitionId(int processDefinitionId) {
this.processDefinitionId = processDefinitionId;
return this;
}
public Builder version(long version) {
this.version = version;
return this;
}
public Builder processDefinitionJson(String processDefinitionJson) {
this.processDefinitionJson = processDefinitionJson;
return this;
}
public Builder description(String description) {
this.description = description;
return this;
}
public Builder globalParams(String globalParams) {
this.globalParams = globalParams;
return this;
}
public Builder createTime(Date createTime) {
this.createTime = createTime;
return this;
}
public Builder locations(String locations) {
this.locations = locations;
return this;
}
public Builder connects(String connects) {
this.connects = connects;
return this;
}
public Builder timeout(int timeout) {
this.timeout = timeout;
return this;
}
public Builder warningGroupId(int warningGroupId) {
this.warningGroupId = warningGroupId;
return this;
}
public Builder resourceIds(String resourceIds) {
this.resourceIds = resourceIds;
return this;
}
public ProcessDefinitionVersion build() {
ProcessDefinitionVersion processDefinitionVersion = new ProcessDefinitionVersion();
processDefinitionVersion.setId(id);
processDefinitionVersion.setProcessDefinitionId(processDefinitionId);
processDefinitionVersion.setVersion(version);
processDefinitionVersion.setProcessDefinitionJson(processDefinitionJson);
processDefinitionVersion.setDescription(description);
processDefinitionVersion.setGlobalParams(globalParams);
processDefinitionVersion.setCreateTime(createTime);
processDefinitionVersion.setLocations(locations);
processDefinitionVersion.setConnects(connects);
processDefinitionVersion.setTimeout(timeout);
processDefinitionVersion.setWarningGroupId(warningGroupId);
processDefinitionVersion.setResourceIds(resourceIds);
return processDefinitionVersion;
}
}
}

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

@ -47,10 +47,17 @@ public class ProcessInstance {
*/
@TableId(value = "id", type = IdType.AUTO)
private int id;
/**
* process definition code
*/
private Long processDefinitionCode;
/**
* process definition id
* process definition version
*/
private int processDefinitionId;
private int processDefinitionVersion;
/**
* process state
*/
@ -145,7 +152,9 @@ public class ProcessInstance {
/**
* process instance json
* TODO delete
*/
@TableField(exist = false)
private String processInstanceJson;
/**
@ -179,11 +188,13 @@ public class ProcessInstance {
/**
* task locations for web
*/
@TableField(exist = false)
private String locations;
/**
* task connects for web
*/
@TableField(exist = false)
private String connects;
/**
@ -194,6 +205,7 @@ public class ProcessInstance {
/**
* depend processes schedule time
*/
@TableField(exist = false)
private String dependenceScheduleTimes;
/**
@ -273,14 +285,6 @@ public class ProcessInstance {
this.id = id;
}
public int getProcessDefinitionId() {
return processDefinitionId;
}
public void setProcessDefinitionId(int processDefinitionId) {
this.processDefinitionId = processDefinitionId;
}
public ExecutionStatus getState() {
return state;
}
@ -579,11 +583,26 @@ public class ProcessInstance {
this.tenantId = tenantId;
}
public Long getProcessDefinitionCode() {
return processDefinitionCode;
}
public void setProcessDefinitionCode(Long processDefinitionCode) {
this.processDefinitionCode = processDefinitionCode;
}
public int getProcessDefinitionVersion() {
return processDefinitionVersion;
}
public void setProcessDefinitionVersion(int processDefinitionVersion) {
this.processDefinitionVersion = processDefinitionVersion;
}
@Override
public String toString() {
return "ProcessInstance{"
+ "id=" + id
+ ", processDefinitionId=" + processDefinitionId
+ ", state=" + state
+ ", recovery=" + recovery
+ ", startTime=" + startTime
@ -651,6 +670,12 @@ public class ProcessInstance {
+ timeout
+ ", tenantId="
+ tenantId
+ ", processDefinitionCode='"
+ processDefinitionCode
+ '\''
+ ", processDefinitionVersion='"
+ processDefinitionVersion
+ '\''
+ '}';
}
@ -672,4 +697,5 @@ public class ProcessInstance {
public int hashCode() {
return Objects.hash(id);
}
}

120
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessLineage.java

@ -0,0 +1,120 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.entity;
/**
* Process lineage
*/
public class ProcessLineage {
/**
* project code
*/
private Long projectCode;
/**
* post task code
*/
private Long postTaskCode;
/**
* post task version
*/
private int postTaskVersion;
/**
* pre task code
*/
private Long preTaskCode;
/**
* pre task version
*/
private int preTaskVersion;
/**
* process definition code
*/
private Long processDefinitionCode;
/**
* process definition version
*/
private int processDefinitionVersion;
public Long getProjectCode() {
return projectCode;
}
public void setProjectCode(Long projectCode) {
this.projectCode = projectCode;
}
public Long getProcessDefinitionCode() {
return processDefinitionCode;
}
public void setProcessDefinitionCode(Long processDefinitionCode) {
this.processDefinitionCode = processDefinitionCode;
}
public int getProcessDefinitionVersion() {
return processDefinitionVersion;
}
public void setProcessDefinitionVersion(int processDefinitionVersion) {
this.processDefinitionVersion = processDefinitionVersion;
}
public void setPostTaskCode(Long postTaskCode) {
this.postTaskCode = postTaskCode;
}
public Long getPreTaskCode() {
return preTaskCode;
}
public void setPreTaskCode(Long preTaskCode) {
this.preTaskCode = preTaskCode;
}
public int getPreTaskVersion() {
return preTaskVersion;
}
public void setPreTaskVersion(int preTaskVersion) {
this.preTaskVersion = preTaskVersion;
}
public int getPostTaskVersion() {
return postTaskVersion;
}
public void setPostTaskVersion(int postTaskVersion) {
this.postTaskVersion = postTaskVersion;
}
public long getPostTaskCode() {
return postTaskCode;
}
public void setPostTaskCode(long postTaskCode) {
this.postTaskCode = postTaskCode;
}
}

254
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessTaskRelation.java

@ -0,0 +1,254 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.entity;
import org.apache.dolphinscheduler.common.enums.ConditionType;
import java.util.Date;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
import com.fasterxml.jackson.annotation.JsonFormat;
/**
* process task relation
*/
@TableName("t_ds_process_task_relation")
public class ProcessTaskRelation {
/**
* id
*/
@TableId(value = "id", type = IdType.AUTO)
private int id;
/**
* name
*/
private String name;
/**
* process version
*/
private int processDefinitionVersion;
/**
* project code
*/
private long projectCode;
/**
* process code
*/
private long processDefinitionCode;
/**
* pre task code
*/
private long preTaskCode;
/**
* pre node version
*/
private int preTaskVersion;
/**
* post task code
*/
private long postTaskCode;
/**
* post node version
*/
private int postTaskVersion;
/**
* condition type
*/
private ConditionType conditionType;
/**
* condition parameters
*/
private String conditionParams;
/**
* create time
*/
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date createTime;
/**
* update time
*/
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date updateTime;
public ProcessTaskRelation() {
}
public ProcessTaskRelation(String name,
int processDefinitionVersion,
long projectCode,
long processDefinitionCode,
long preTaskCode,
int preTaskVersion,
long postTaskCode,
int postTaskVersion,
ConditionType conditionType,
String conditionParams,
Date createTime,
Date updateTime) {
this.name = name;
this.processDefinitionVersion = processDefinitionVersion;
this.projectCode = projectCode;
this.processDefinitionCode = processDefinitionCode;
this.preTaskCode = preTaskCode;
this.preTaskVersion = preTaskVersion;
this.postTaskCode = postTaskCode;
this.postTaskVersion = postTaskVersion;
this.conditionType = conditionType;
this.conditionParams = conditionParams;
this.createTime = createTime;
this.updateTime = updateTime;
}
public String getName() {
return name;
}
public void setName(String name) {
this.name = name;
}
public int getId() {
return id;
}
public void setId(int id) {
this.id = id;
}
public Date getCreateTime() {
return createTime;
}
public void setCreateTime(Date createTime) {
this.createTime = createTime;
}
public Date getUpdateTime() {
return updateTime;
}
public void setUpdateTime(Date updateTime) {
this.updateTime = updateTime;
}
public String getConditionParams() {
return conditionParams;
}
public void setConditionParams(String conditionParams) {
this.conditionParams = conditionParams;
}
public int getProcessDefinitionVersion() {
return processDefinitionVersion;
}
public void setProcessDefinitionVersion(int processDefinitionVersion) {
this.processDefinitionVersion = processDefinitionVersion;
}
public long getProjectCode() {
return projectCode;
}
public void setProjectCode(long projectCode) {
this.projectCode = projectCode;
}
public long getProcessDefinitionCode() {
return processDefinitionCode;
}
public void setProcessDefinitionCode(long processDefinitionCode) {
this.processDefinitionCode = processDefinitionCode;
}
public long getPreTaskCode() {
return preTaskCode;
}
public void setPreTaskCode(long preTaskCode) {
this.preTaskCode = preTaskCode;
}
public long getPostTaskCode() {
return postTaskCode;
}
public void setPostTaskCode(long postTaskCode) {
this.postTaskCode = postTaskCode;
}
public ConditionType getConditionType() {
return conditionType;
}
public void setConditionType(ConditionType conditionType) {
this.conditionType = conditionType;
}
public int getPreTaskVersion() {
return preTaskVersion;
}
public void setPreTaskVersion(int preTaskVersion) {
this.preTaskVersion = preTaskVersion;
}
public int getPostTaskVersion() {
return postTaskVersion;
}
public void setPostTaskVersion(int postTaskVersion) {
this.postTaskVersion = postTaskVersion;
}
@Override
public String toString() {
return "ProcessTaskRelation{"
+ "id=" + id
+ ", name='" + name + '\''
+ ", processDefinitionVersion=" + processDefinitionVersion
+ ", projectCode=" + projectCode
+ ", processDefinitionCode=" + processDefinitionCode
+ ", preTaskCode=" + preTaskCode
+ ", preTaskVersion=" + preTaskVersion
+ ", postTaskCode=" + postTaskCode
+ ", postTaskVersion=" + postTaskVersion
+ ", conditionType=" + conditionType
+ ", conditionParams='" + conditionParams + '\''
+ ", createTime=" + createTime
+ ", updateTime=" + updateTime
+ '}';
}
}

83
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessTaskRelationLog.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.dao.entity;
import java.util.Date;
import com.baomidou.mybatisplus.annotation.TableName;
import com.fasterxml.jackson.annotation.JsonFormat;
/**
* process task relation log
*/
@TableName("t_ds_process_task_relation_log")
public class ProcessTaskRelationLog extends ProcessTaskRelation {
/**
* operator user id
*/
private int operator;
/**
* operate time
*/
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date operateTime;
public ProcessTaskRelationLog() {
super();
}
public ProcessTaskRelationLog(ProcessTaskRelation processTaskRelation) {
super();
this.setId(processTaskRelation.getId());
this.setName(processTaskRelation.getName());
this.setProcessDefinitionCode(processTaskRelation.getProcessDefinitionCode());
this.setProcessDefinitionVersion(processTaskRelation.getProcessDefinitionVersion());
this.setProjectCode(processTaskRelation.getProjectCode());
this.setPreTaskCode(processTaskRelation.getPreTaskCode());
this.setPreTaskVersion(processTaskRelation.getPreTaskVersion());
this.setPostTaskCode(processTaskRelation.getPostTaskCode());
this.setPostTaskVersion(processTaskRelation.getPostTaskVersion());
this.setConditionType(processTaskRelation.getConditionType());
this.setConditionParams(processTaskRelation.getConditionParams());
this.setCreateTime(processTaskRelation.getCreateTime());
this.setUpdateTime(processTaskRelation.getUpdateTime());
}
public int getOperator() {
return operator;
}
public void setOperator(int operator) {
this.operator = operator;
}
public Date getOperateTime() {
return operateTime;
}
public void setOperateTime(Date operateTime) {
this.operateTime = operateTime;
}
@Override
public String toString() {
return super.toString();
}
}

42
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Project.java

@ -47,6 +47,11 @@ public class Project {
@TableField(exist = false)
private String userName;
/**
* project code
*/
private Long code;
/**
* project name
*/
@ -85,6 +90,14 @@ public class Project {
@TableField(exist = false)
private int instRunningCount;
public Long getCode() {
return code;
}
public void setCode(Long code) {
this.code = code;
}
public int getDefCount() {
return defCount;
}
@ -167,15 +180,19 @@ public class Project {
@Override
public String toString() {
return "Project{" +
"id=" + id +
", userId=" + userId +
", userName='" + userName + '\'' +
", name='" + name + '\'' +
", description='" + description + '\'' +
", createTime=" + createTime +
", updateTime=" + updateTime +
'}';
return "Project{"
+ "id=" + id
+ ", userId=" + userId
+ ", userName='" + userName + '\''
+ ", code=" + code
+ ", name='" + name + '\''
+ ", description='" + description + '\''
+ ", createTime=" + createTime
+ ", updateTime=" + updateTime
+ ", perm=" + perm
+ ", defCount=" + defCount
+ ", instRunningCount=" + instRunningCount
+ '}';
}
@Override
@ -211,6 +228,7 @@ public class Project {
private int id;
private int userId;
private String userName;
private Long code;
private String name;
private String description;
private Date createTime;
@ -222,6 +240,11 @@ public class Project {
private Builder() {
}
public Builder code(Long code) {
this.code = code;
return this;
}
public Builder id(int id) {
this.id = id;
return this;
@ -276,6 +299,7 @@ public class Project {
Project project = new Project();
project.setId(id);
project.setUserId(userId);
project.setCode(code);
project.setUserName(userName);
project.setName(name);
project.setDescription(description);

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

@ -123,17 +123,18 @@ public class ProjectUser {
public void setPerm(int perm) {
this.perm = perm;
}
@Override
public String toString() {
return "ProjectUser{" +
"id=" + id +
", projectId=" + projectId +
", projectName='" + projectName + '\'' +
", userId=" + userId +
", userName='" + userName + '\'' +
", perm=" + perm +
", createTime=" + createTime +
", updateTime=" + updateTime +
'}';
return "ProjectUser{"
+ "id=" + id
+ ", userId=" + userId
+ ", projectId=" + projectId
+ ", projectName='" + projectName + '\''
+ ", userName='" + userName + '\''
+ ", perm=" + perm
+ ", createTime=" + createTime
+ ", updateTime=" + updateTime
+ '}';
}
}

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

@ -0,0 +1,428 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.entity;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.Priority;
import org.apache.dolphinscheduler.common.enums.TaskTimeoutStrategy;
import org.apache.dolphinscheduler.common.enums.TimeoutFlag;
import org.apache.dolphinscheduler.common.process.Property;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import java.util.Date;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableField;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
import com.baomidou.mybatisplus.core.toolkit.StringUtils;
import com.fasterxml.jackson.annotation.JsonFormat;
import com.fasterxml.jackson.databind.JsonNode;
/**
* task definition
*/
@TableName("t_ds_task_definition")
public class TaskDefinition {
/**
* id
*/
@TableId(value = "id", type = IdType.AUTO)
private int id;
/**
* code
*/
private long code;
/**
* name
*/
private String name;
/**
* version
*/
private int version;
/**
* description
*/
private String description;
/**
* project code
*/
private long projectCode;
/**
* task user id
*/
private int userId;
/**
* task type
*/
private String taskType;
/**
* user defined parameters
*/
private String taskParams;
/**
* user defined parameter list
*/
@TableField(exist = false)
private List<Property> taskParamList;
/**
* user define parameter map
*/
@TableField(exist = false)
private Map<String, String> taskParamMap;
/**
* task is valid: yes/no
*/
private Flag flag;
/**
* task priority
*/
private Priority taskPriority;
/**
* user name
*/
@TableField(exist = false)
private String userName;
/**
* project name
*/
@TableField(exist = false)
private String projectName;
/**
* worker group
*/
private String workerGroup;
/**
* fail retry times
*/
private int failRetryTimes;
/**
* fail retry interval
*/
private int failRetryInterval;
/**
* timeout flag
*/
private TimeoutFlag timeoutFlag;
/**
* timeout notify strategy
*/
private TaskTimeoutStrategy timeoutNotifyStrategy;
/**
* task warning time out. unit: minute
*/
private int timeout;
/**
* delay execution time.
*/
private int delayTime;
/**
* resource ids
*/
private String resourceIds;
/**
* create time
*/
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date createTime;
/**
* update time
*/
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date updateTime;
public TaskDefinition() {
}
public TaskDefinition(long code, int version) {
this.code = code;
this.version = version;
}
public String getName() {
return name;
}
public void setName(String name) {
this.name = name;
}
public int getId() {
return id;
}
public void setId(int id) {
this.id = id;
}
public Date getCreateTime() {
return createTime;
}
public void setCreateTime(Date createTime) {
this.createTime = createTime;
}
public Date getUpdateTime() {
return updateTime;
}
public void setUpdateTime(Date updateTime) {
this.updateTime = updateTime;
}
public Flag getFlag() {
return flag;
}
public void setFlag(Flag flag) {
this.flag = flag;
}
public int getUserId() {
return userId;
}
public void setUserId(int userId) {
this.userId = userId;
}
public String getUserName() {
return userName;
}
public void setUserName(String userName) {
this.userName = userName;
}
public String getProjectName() {
return projectName;
}
public void setProjectName(String projectName) {
this.projectName = projectName;
}
public String getTaskParams() {
return taskParams;
}
public void setTaskParams(String taskParams) {
this.taskParams = taskParams;
}
public List<Property> getTaskParamList() {
JsonNode localParams = JSONUtils.parseObject(taskParams).findValue("localParams");
if (localParams != null) {
taskParamList = JSONUtils.toList(localParams.toString(), Property.class);
}
return taskParamList;
}
public void setTaskParamList(List<Property> taskParamList) {
this.taskParamList = taskParamList;
}
public void setTaskParamMap(Map<String, String> taskParamMap) {
this.taskParamMap = taskParamMap;
}
public Map<String, String> getTaskParamMap() {
if (taskParamMap == null && StringUtils.isNotEmpty(taskParams)) {
JsonNode localParams = JSONUtils.parseObject(taskParams).findValue("localParams");
if (localParams != null) {
List<Property> propList = JSONUtils.toList(localParams.toString(), Property.class);
taskParamMap = propList.stream().collect(Collectors.toMap(Property::getProp, Property::getValue));
}
}
return taskParamMap;
}
public int getTimeout() {
return timeout;
}
public void setTimeout(int timeout) {
this.timeout = timeout;
}
public String getDescription() {
return description;
}
public void setDescription(String description) {
this.description = description;
}
public long getCode() {
return code;
}
public void setCode(long code) {
this.code = code;
}
public int getVersion() {
return version;
}
public void setVersion(int version) {
this.version = version;
}
public long getProjectCode() {
return projectCode;
}
public void setProjectCode(long projectCode) {
this.projectCode = projectCode;
}
public String getTaskType() {
return taskType;
}
public void setTaskType(String taskType) {
this.taskType = taskType;
}
public Priority getTaskPriority() {
return taskPriority;
}
public void setTaskPriority(Priority taskPriority) {
this.taskPriority = taskPriority;
}
public String getWorkerGroup() {
return workerGroup;
}
public void setWorkerGroup(String workerGroup) {
this.workerGroup = workerGroup;
}
public int getFailRetryTimes() {
return failRetryTimes;
}
public void setFailRetryTimes(int failRetryTimes) {
this.failRetryTimes = failRetryTimes;
}
public int getFailRetryInterval() {
return failRetryInterval;
}
public void setFailRetryInterval(int failRetryInterval) {
this.failRetryInterval = failRetryInterval;
}
public TaskTimeoutStrategy getTimeoutNotifyStrategy() {
return timeoutNotifyStrategy;
}
public void setTimeoutNotifyStrategy(TaskTimeoutStrategy timeoutNotifyStrategy) {
this.timeoutNotifyStrategy = timeoutNotifyStrategy;
}
public TimeoutFlag getTimeoutFlag() {
return timeoutFlag;
}
public void setTimeoutFlag(TimeoutFlag timeoutFlag) {
this.timeoutFlag = timeoutFlag;
}
public String getResourceIds() {
return resourceIds;
}
public void setResourceIds(String resourceIds) {
this.resourceIds = resourceIds;
}
public int getDelayTime() {
return delayTime;
}
public void setDelayTime(int delayTime) {
this.delayTime = delayTime;
}
@Override
public String toString() {
return "TaskDefinition{"
+ "id=" + id
+ ", code=" + code
+ ", name='" + name + '\''
+ ", version=" + version
+ ", description='" + description + '\''
+ ", projectCode=" + projectCode
+ ", userId=" + userId
+ ", taskType=" + taskType
+ ", taskParams='" + taskParams + '\''
+ ", taskParamList=" + taskParamList
+ ", taskParamMap=" + taskParamMap
+ ", flag=" + flag
+ ", taskPriority=" + taskPriority
+ ", userName='" + userName + '\''
+ ", projectName='" + projectName + '\''
+ ", workerGroup='" + workerGroup + '\''
+ ", failRetryTimes=" + failRetryTimes
+ ", failRetryInterval=" + failRetryInterval
+ ", timeoutFlag=" + timeoutFlag
+ ", timeoutNotifyStrategy=" + timeoutNotifyStrategy
+ ", timeout=" + timeout
+ ", delayTime=" + delayTime
+ ", resourceIds='" + resourceIds + '\''
+ ", createTime=" + createTime
+ ", updateTime=" + updateTime
+ '}';
}
}

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

@ -0,0 +1,95 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.entity;
import java.util.Date;
import com.baomidou.mybatisplus.annotation.TableName;
import com.fasterxml.jackson.annotation.JsonFormat;
/**
* task definition log
*/
@TableName("t_ds_task_definition_log")
public class TaskDefinitionLog extends TaskDefinition {
/**
* operator user id
*/
private int operator;
/**
* operate time
*/
@JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
private Date operateTime;
public TaskDefinitionLog() {
super();
}
public TaskDefinitionLog(TaskDefinition taskDefinition) {
super();
this.setId(taskDefinition.getId());
this.setCode(taskDefinition.getCode());
this.setVersion(taskDefinition.getVersion());
this.setName(taskDefinition.getName());
this.setDescription(taskDefinition.getDescription());
this.setUserId(taskDefinition.getUserId());
this.setUserName(taskDefinition.getUserName());
this.setWorkerGroup(taskDefinition.getWorkerGroup());
this.setProjectCode(taskDefinition.getProjectCode());
this.setProjectName(taskDefinition.getProjectName());
this.setResourceIds(taskDefinition.getResourceIds());
this.setTaskParams(taskDefinition.getTaskParams());
this.setTaskParamList(taskDefinition.getTaskParamList());
this.setTaskParamMap(taskDefinition.getTaskParamMap());
this.setTaskPriority(taskDefinition.getTaskPriority());
this.setTimeoutNotifyStrategy(taskDefinition.getTimeoutNotifyStrategy());
this.setTaskType(taskDefinition.getTaskType());
this.setTimeout(taskDefinition.getTimeout());
this.setDelayTime(taskDefinition.getDelayTime());
this.setTimeoutFlag(taskDefinition.getTimeoutFlag());
this.setUpdateTime(taskDefinition.getUpdateTime());
this.setCreateTime(taskDefinition.getCreateTime());
this.setFailRetryInterval(taskDefinition.getFailRetryInterval());
this.setFailRetryTimes(taskDefinition.getFailRetryTimes());
this.setFlag(taskDefinition.getFlag());
}
public int getOperator() {
return operator;
}
public void setOperator(int operator) {
this.operator = operator;
}
public Date getOperateTime() {
return operateTime;
}
public void setOperateTime(Date operateTime) {
this.operateTime = operateTime;
}
@Override
public String toString() {
return super.toString();
}
}

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

@ -17,11 +17,12 @@
package org.apache.dolphinscheduler.dao.entity;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.Priority;
import org.apache.dolphinscheduler.common.enums.TaskType;
import org.apache.dolphinscheduler.common.model.TaskNode;
import org.apache.dolphinscheduler.common.task.dependent.DependentParameters;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import java.io.Serializable;
@ -58,25 +59,25 @@ public class TaskInstance implements Serializable {
private String taskType;
/**
* process definition id
* process instance id
*/
private int processDefinitionId;
private int processInstanceId;
/**
* process instance id
* task code
*/
private int processInstanceId;
private long taskCode;
/**
* process instance name
* task definition version
*/
@TableField(exist = false)
private String processInstanceName;
private int taskDefinitionVersion;
/**
* task json
* process instance name
*/
private String taskJson;
@TableField(exist = false)
private String processInstanceName;
/**
* state
@ -146,6 +147,12 @@ public class TaskInstance implements Serializable {
@TableField(exist = false)
private ProcessDefinition processDefine;
/**
* task definition
*/
@TableField(exist = false)
private TaskDefinition taskDefine;
/**
* process id
*/
@ -165,7 +172,7 @@ public class TaskInstance implements Serializable {
* dependency
*/
@TableField(exist = false)
private String dependency;
private DependentParameters dependency;
/**
* duration
@ -216,7 +223,7 @@ public class TaskInstance implements Serializable {
* varPool string
*/
private String varPool;
/**
* executor name
*/
@ -232,6 +239,11 @@ public class TaskInstance implements Serializable {
*/
private int delayTime;
/**
* task params
*/
private String taskParams;
public void init(String host, Date startTime, String executePath) {
this.host = host;
this.startTime = startTime;
@ -245,7 +257,7 @@ public class TaskInstance implements Serializable {
public void setVarPool(String varPool) {
this.varPool = varPool;
}
public ProcessInstance getProcessInstance() {
return processInstance;
}
@ -262,6 +274,14 @@ public class TaskInstance implements Serializable {
this.processDefine = processDefine;
}
public TaskDefinition getTaskDefine() {
return taskDefine;
}
public void setTaskDefine(TaskDefinition taskDefine) {
this.taskDefine = taskDefine;
}
public int getId() {
return id;
}
@ -286,14 +306,6 @@ public class TaskInstance implements Serializable {
this.taskType = taskType;
}
public int getProcessDefinitionId() {
return processDefinitionId;
}
public void setProcessDefinitionId(int processDefinitionId) {
this.processDefinitionId = processDefinitionId;
}
public int getProcessInstanceId() {
return processInstanceId;
}
@ -302,14 +314,6 @@ public class TaskInstance implements Serializable {
this.processInstanceId = processInstanceId;
}
public String getTaskJson() {
return taskJson;
}
public void setTaskJson(String taskJson) {
this.taskJson = taskJson;
}
public ExecutionStatus getState() {
return state;
}
@ -410,15 +414,15 @@ public class TaskInstance implements Serializable {
this.appLink = appLink;
}
public String getDependency() {
if (this.dependency != null) {
return this.dependency;
public DependentParameters getDependency() {
if (this.dependency == null) {
Map<String, Object> taskParamsMap = JSONUtils.toMap(this.getTaskParams(), String.class, Object.class);
this.dependency = JSONUtils.parseObject((String) taskParamsMap.get(Constants.DEPENDENCE), DependentParameters.class);
}
TaskNode taskNode = JSONUtils.parseObject(taskJson, TaskNode.class);
return taskNode == null ? null : taskNode.getDependence();
return this.dependency;
}
public void setDependency(String dependency) {
public void setDependency(DependentParameters dependency) {
this.dependency = dependency;
}
@ -495,15 +499,15 @@ public class TaskInstance implements Serializable {
}
public boolean isSubProcess() {
return TaskType.SUB_PROCESS.equals(TaskType.valueOf(this.taskType));
return TaskType.SUB_PROCESS.getDesc().equalsIgnoreCase(this.taskType);
}
public boolean isDependTask() {
return TaskType.DEPENDENT.equals(TaskType.valueOf(this.taskType));
return TaskType.DEPENDENT.getDesc().equalsIgnoreCase(this.taskType);
}
public boolean isConditionsTask() {
return TaskType.CONDITIONS.equals(TaskType.valueOf(this.taskType));
return TaskType.CONDITIONS.getDesc().equalsIgnoreCase(this.taskType);
}
/**
@ -569,10 +573,8 @@ public class TaskInstance implements Serializable {
+ "id=" + id
+ ", name='" + name + '\''
+ ", taskType='" + taskType + '\''
+ ", processDefinitionId=" + processDefinitionId
+ ", processInstanceId=" + processInstanceId
+ ", processInstanceName='" + processInstanceName + '\''
+ ", taskJson='" + taskJson + '\''
+ ", state=" + state
+ ", firstSubmitTime=" + firstSubmitTime
+ ", submitTime=" + submitTime
@ -601,4 +603,28 @@ public class TaskInstance implements Serializable {
+ ", delayTime=" + delayTime
+ '}';
}
public long getTaskCode() {
return taskCode;
}
public void setTaskCode(long taskCode) {
this.taskCode = taskCode;
}
public int getTaskDefinitionVersion() {
return taskDefinitionVersion;
}
public void setTaskDefinitionVersion(int taskDefinitionVersion) {
this.taskDefinitionVersion = taskDefinitionVersion;
}
public String getTaskParams() {
return taskParams;
}
public void setTaskParams(String taskParams) {
this.taskParams = taskParams;
}
}

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

@ -16,6 +16,8 @@
*/
package org.apache.dolphinscheduler.dao.entity;
import java.util.Objects;
public class WorkFlowRelation {
private int sourceWorkFlowId;
private int targetWorkFlowId;
@ -35,4 +37,24 @@ public class WorkFlowRelation {
public void setTargetWorkFlowId(int targetWorkFlowId) {
this.targetWorkFlowId = targetWorkFlowId;
}
public WorkFlowRelation() {
}
public WorkFlowRelation(int sourceWorkFlowId, int targetWorkFlowId) {
this.sourceWorkFlowId = sourceWorkFlowId;
this.targetWorkFlowId = targetWorkFlowId;
}
@Override
public boolean equals(Object obj) {
return obj instanceof WorkFlowRelation
&& this.sourceWorkFlowId == ((WorkFlowRelation) obj).getSourceWorkFlowId()
&& this.targetWorkFlowId == ((WorkFlowRelation) obj).getTargetWorkFlowId();
}
@Override
public int hashCode() {
return Objects.hash(sourceWorkFlowId, targetWorkFlowId);
}
}

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

@ -41,14 +41,14 @@ public interface CommandMapper extends BaseMapper<Command> {
* @param userId userId
* @param startTime startTime
* @param endTime endTime
* @param projectIdArray projectIdArray
* @param projectCodeArray projectCodeArray
* @return CommandCount list
*/
List<CommandCount> countCommandState(
@Param("userId") int userId,
@Param("startTime") Date startTime,
@Param("endTime") Date endTime,
@Param("projectIdArray") Integer[] projectIdArray);
@Param("projectCodeArray") Long[] projectCodeArray);

4
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ErrorCommandMapper.java

@ -33,11 +33,11 @@ public interface ErrorCommandMapper extends BaseMapper<ErrorCommand> {
* count command state
* @param startTime startTime
* @param endTime endTime
* @param projectIdArray projectIdArray
* @param projectCodeArray projectCodeArray
* @return CommandCount list
*/
List<CommandCount> countCommandState(
@Param("startTime") Date startTime,
@Param("endTime") Date endTime,
@Param("projectIdArray") Integer[] projectIdArray);
@Param("projectCodeArray") Long[] projectCodeArray);
}

91
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionLogMapper.java

@ -0,0 +1,91 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.mapper;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinitionLog;
import org.apache.ibatis.annotations.Param;
import java.util.List;
import com.baomidou.mybatisplus.core.mapper.BaseMapper;
import com.baomidou.mybatisplus.core.metadata.IPage;
import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
/**
* process definition log mapper interface
*/
public interface ProcessDefinitionLogMapper extends BaseMapper<ProcessDefinitionLog> {
/**
* query process definition log by name
*
* @param projectCode projectCode
* @param name process name
* @return process definition log list
*/
List<ProcessDefinitionLog> queryByDefinitionName(@Param("projectCode") Long projectCode,
@Param("processDefinitionName") String name);
/**
* query process definition log list
*
* @param processDefinitionCode processDefinitionCode
* @return process definition log list
*/
List<ProcessDefinitionLog> queryByDefinitionCode(@Param("processDefinitionCode") long processDefinitionCode);
/**
* query max version for definition
*/
Integer queryMaxVersionForDefinition(@Param("processDefinitionCode") long processDefinitionCode);
/**
* query max version definition log
*/
ProcessDefinitionLog queryMaxVersionDefinitionLog(@Param("processDefinitionCode") long processDefinitionCode);
/**
* query the certain process definition version info by process definition code and version number
*
* @param processDefinitionCode process definition code
* @param version version number
* @return the process definition version info
*/
ProcessDefinitionLog queryByDefinitionCodeAndVersion(@Param("processDefinitionCode") Long processDefinitionCode,
@Param("version") long version);
/**
* query the paging process definition version list by pagination info
*
* @param page pagination info
* @param processDefinitionCode process definition code
* @return the paging process definition version list
*/
IPage<ProcessDefinitionLog> queryProcessDefinitionVersionsPaging(Page<ProcessDefinitionLog> page,
@Param("processDefinitionCode") Long processDefinitionCode);
/**
* delete the certain process definition version by process definition id and version number
*
* @param processDefinitionCode process definition code
* @param version version number
* @return delete result
*/
int deleteByProcessDefinitionCodeAndVersion(@Param("processDefinitionCode") Long processDefinitionCode, @Param("version") long version);
}

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

@ -19,39 +19,66 @@ package org.apache.dolphinscheduler.dao.mapper;
import org.apache.dolphinscheduler.dao.entity.DefinitionGroupByUser;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinitionLog;
import org.apache.ibatis.annotations.MapKey;
import org.apache.ibatis.annotations.Param;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import com.baomidou.mybatisplus.core.mapper.BaseMapper;
import com.baomidou.mybatisplus.core.metadata.IPage;
import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
/**
* process definition mapper interface
*/
public interface ProcessDefinitionMapper extends BaseMapper<ProcessDefinition> {
/**
* query process definition by code
*
* @param code code
* @return process definition
*/
ProcessDefinition queryByCode(@Param("code") Long code);
/**
* query process definition by code list
*
* @param codes codes
* @return process definition list
*/
List<ProcessDefinition> queryByCodes(@Param("codes") Collection<Long> codes);
/**
* delete process definition by code
*
* @param code code
* @return delete result
*/
int deleteByCode(@Param("code") Long code);
/**
* verify process definition by name
*
* @param projectId projectId
* @param projectCode projectCode
* @param name name
* @return process definition
*/
ProcessDefinition verifyByDefineName(@Param("projectId") int projectId,
ProcessDefinition verifyByDefineName(@Param("projectCode") Long projectCode,
@Param("processDefinitionName") String name);
/**
* query process definition by name
*
* @param projectId projectId
* @param projectCode projectCode
* @param name name
* @return process definition
*/
ProcessDefinition queryByDefineName(@Param("projectId") int projectId,
ProcessDefinition queryByDefineName(@Param("projectCode") Long projectCode,
@Param("processDefinitionName") String name);
/**
@ -68,23 +95,23 @@ public interface ProcessDefinitionMapper extends BaseMapper<ProcessDefinition> {
* @param page page
* @param searchVal searchVal
* @param userId userId
* @param projectId projectId
* @param projectCode projectCode
* @param isAdmin isAdmin
* @return process definition IPage
*/
IPage<ProcessDefinition> queryDefineListPaging(IPage<ProcessDefinition> page,
@Param("searchVal") String searchVal,
@Param("userId") int userId,
@Param("projectId") int projectId,
@Param("projectCode") Long projectCode,
@Param("isAdmin") boolean isAdmin);
/**
* query all process definition list
*
* @param projectId projectId
* @param projectCode projectCode
* @return process definition list
*/
List<ProcessDefinition> queryAllDefinitionList(@Param("projectId") int projectId);
List<ProcessDefinition> queryAllDefinitionList(@Param("projectCode") Long projectCode);
/**
* query process definition by ids
@ -106,13 +133,13 @@ public interface ProcessDefinitionMapper extends BaseMapper<ProcessDefinition> {
* count process definition group by user
*
* @param userId userId
* @param projectIds projectIds
* @param projectCodes projectCodes
* @param isAdmin isAdmin
* @return process definition list
*/
List<DefinitionGroupByUser> countDefinitionGroupByUser(
@Param("userId") Integer userId,
@Param("projectIds") Integer[] projectIds,
@Param("projectCodes") Long[] projectCodes,
@Param("isAdmin") boolean isAdmin);
/**
@ -145,6 +172,16 @@ public interface ProcessDefinitionMapper extends BaseMapper<ProcessDefinition> {
*/
List<Integer> listProjectIds();
/**
* query the paging process definition version list by pagination info
*
* @param page pagination info
* @param processDefinitionCode process definition code
* @return the paging process definition version list
*/
IPage<ProcessDefinitionLog> queryProcessDefinitionVersionsPaging(Page<ProcessDefinitionLog> page,
@Param("processDefinitionCode") Long processDefinitionCode);
/**
* query has associated definition by id and version
* @param processDefinitionId process definition id

69
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionVersionMapper.java

@ -1,69 +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.dao.mapper;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinitionVersion;
import org.apache.ibatis.annotations.Param;
import com.baomidou.mybatisplus.core.mapper.BaseMapper;
import com.baomidou.mybatisplus.core.metadata.IPage;
import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
/**
* process definition mapper interface
*/
public interface ProcessDefinitionVersionMapper extends BaseMapper<ProcessDefinitionVersion> {
/**
* query max version by process definition id
*
* @param processDefinitionId process definition id
* @return the max version of this process definition id
*/
Long queryMaxVersionByProcessDefinitionId(@Param("processDefinitionId") int processDefinitionId);
/**
* query the paging process definition version list by pagination info
*
* @param page pagination info
* @param processDefinitionId process definition id
* @return the paging process definition version list
*/
IPage<ProcessDefinitionVersion> queryProcessDefinitionVersionsPaging(Page<ProcessDefinitionVersion> page,
@Param("processDefinitionId") int processDefinitionId);
/**
* query the certain process definition version info by process definition id and version number
*
* @param processDefinitionId process definition id
* @param version version number
* @return the process definition version info
*/
ProcessDefinitionVersion queryByProcessDefinitionIdAndVersion(@Param("processDefinitionId") int processDefinitionId, @Param("version") long version);
/**
* delete the certain process definition version by process definition id and version number
*
* @param processDefinitionId process definition id
* @param version version number
* @return delete result
*/
int deleteByProcessDefinitionIdAndVersion(@Param("processDefinitionId") int processDefinitionId, @Param("version") long version);
}

78
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.java

@ -37,6 +37,7 @@ public interface ProcessInstanceMapper extends BaseMapper<ProcessInstance> {
/**
* query process instance detail info by id
*
* @param processId processId
* @return process instance
*/
@ -44,6 +45,7 @@ public interface ProcessInstanceMapper extends BaseMapper<ProcessInstance> {
/**
* query process instance by host and stateArray
*
* @param host host
* @param stateArray stateArray
* @return process instance list
@ -53,15 +55,15 @@ public interface ProcessInstanceMapper extends BaseMapper<ProcessInstance> {
/**
* query process instance by tenantId and stateArray
*
* @param tenantId tenantId
* @param states states array
* @return process instance list
*/
List<ProcessInstance> queryByTenantIdAndStatus(@Param("tenantId") int tenantId,
@Param("states") int[] states);
@Param("states") int[] states);
/**
* query process instance by worker group and stateArray
* @param workerGroupName workerGroupName
* @param states states array
* @return process instance list
@ -85,9 +87,10 @@ public interface ProcessInstanceMapper extends BaseMapper<ProcessInstance> {
/**
* process instance page
*
* @param page page
* @param projectId projectId
* @param processDefinitionId processDefinitionId
* @param projectCode projectCode
* @param processDefinitionCode processDefinitionCode
* @param searchVal searchVal
* @param executorId executorId
* @param statusArray statusArray
@ -97,8 +100,8 @@ public interface ProcessInstanceMapper extends BaseMapper<ProcessInstance> {
* @return process instance page
*/
IPage<ProcessInstance> queryProcessInstanceListPaging(Page<ProcessInstance> page,
@Param("projectId") int projectId,
@Param("processDefinitionId") Integer processDefinitionId,
@Param("projectCode") Long projectCode,
@Param("processDefinitionCode") Long processDefinitionCode,
@Param("searchVal") String searchVal,
@Param("executorId") Integer executorId,
@Param("states") int[] statusArray,
@ -108,6 +111,7 @@ public interface ProcessInstanceMapper extends BaseMapper<ProcessInstance> {
/**
* set failover by host and state array
*
* @param host host
* @param stateArray stateArray
* @return set result
@ -117,7 +121,8 @@ public interface ProcessInstanceMapper extends BaseMapper<ProcessInstance> {
/**
* update process instance by state
* @param originState originState
*
* @param originState originState
* @param destState destState
* @return update result
*/
@ -125,7 +130,8 @@ public interface ProcessInstanceMapper extends BaseMapper<ProcessInstance> {
@Param("destState") ExecutionStatus destState);
/**
* update process instance by tenantId
* update process instance by tenantId
*
* @param originTenantId originTenantId
* @param destTenantId destTenantId
* @return update result
@ -134,7 +140,8 @@ public interface ProcessInstanceMapper extends BaseMapper<ProcessInstance> {
@Param("destTenantId") int destTenantId);
/**
* update process instance by worker group name
* update process instance by worker groupId
*
* @param originWorkerGroupName originWorkerGroupName
* @param destWorkerGroupName destWorkerGroupName
* @return update result
@ -144,85 +151,88 @@ public interface ProcessInstanceMapper extends BaseMapper<ProcessInstance> {
/**
* count process instance state by user
*
* @param startTime startTime
* @param endTime endTime
* @param projectIds projectIds
* @param projectCodes projectCodes
* @return ExecuteStatusCount list
*/
List<ExecuteStatusCount> countInstanceStateByUser(
@Param("startTime") Date startTime,
@Param("endTime") Date endTime,
@Param("projectIds") Integer[] projectIds);
@Param("projectCodes") Long[] projectCodes);
/**
* query process instance by processDefinitionId
* @param processDefinitionId processDefinitionId
* query process instance by processDefinitionCode
*
* @param processDefinitionCode processDefinitionCode
* @param size size
* @return process instance list
*/
List<ProcessInstance> queryByProcessDefineId(
@Param("processDefinitionId") int processDefinitionId,
@Param("size") int size);
List<ProcessInstance> queryByProcessDefineCode(@Param("processDefinitionCode") Long processDefinitionCode,
@Param("size") int size);
/**
* query last scheduler process instance
* @param definitionId processDefinitionId
*
* @param definitionCode definitionCode
* @param startTime startTime
* @param endTime endTime
* @return process instance
*/
ProcessInstance queryLastSchedulerProcess(@Param("processDefinitionId") int definitionId,
ProcessInstance queryLastSchedulerProcess(@Param("processDefinitionCode") Long definitionCode,
@Param("startTime") Date startTime,
@Param("endTime") Date endTime);
/**
* query last running process instance
* @param definitionId definitionId
*
* @param definitionCode definitionCode
* @param startTime startTime
* @param endTime endTime
* @param stateArray stateArray
* @return process instance
*/
ProcessInstance queryLastRunningProcess(@Param("processDefinitionId") int definitionId,
ProcessInstance queryLastRunningProcess(@Param("processDefinitionCode") Long definitionCode,
@Param("startTime") Date startTime,
@Param("endTime") Date endTime,
@Param("states") int[] stateArray);
/**
* query last manual process instance
* @param definitionId definitionId
*
* @param definitionCode definitionCode
* @param startTime startTime
* @param endTime endTime
* @return process instance
*/
ProcessInstance queryLastManualProcess(@Param("processDefinitionId") int definitionId,
ProcessInstance queryLastManualProcess(@Param("processDefinitionCode") Long definitionCode,
@Param("startTime") Date startTime,
@Param("endTime") Date endTime);
/**
* query top n process instance order by running duration
* @param size
*
* @param status process instance status
* @param startTime
* @param endTime
* @return ProcessInstance list
*/
List<ProcessInstance> queryTopNProcessInstance(@Param("size") int size,
@Param("startTime") Date startTime,
@Param("endTime") Date endTime,
@Param("status")ExecutionStatus status);
@Param("status") ExecutionStatus status);
/**
* query process instance by processDefinitionId and stateArray
* @param processDefinitionId processDefinitionId
* query process instance by processDefinitionCode and stateArray
*
* @param processDefinitionCode processDefinitionCode
* @param states states array
* @return process instance list
*/
List<ProcessInstance> queryByProcessDefineIdAndStatus(
@Param("processDefinitionId") int processDefinitionId,
@Param("states") int[] states);
List<ProcessInstance> queryByProcessDefineCodeAndStatus(@Param("processDefinitionCode") Long processDefinitionCode,
@Param("states") int[] states);
int updateGlobalParamsById(
@Param("globalParams") String globalParams,
@Param("id") int id);
int updateGlobalParamsById(@Param("globalParams") String globalParams,
@Param("id") int id);
}

47
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessTaskRelationLogMapper.java

@ -0,0 +1,47 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.mapper;
import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelationLog;
import org.apache.ibatis.annotations.Param;
import java.util.List;
import com.baomidou.mybatisplus.core.mapper.BaseMapper;
/**
* process task relation log mapper interface
*/
public interface ProcessTaskRelationLogMapper extends BaseMapper<ProcessTaskRelationLog> {
/**
* query process task relation log
*
* @param processCode process definition code
* @param processVersion process version
* @return process task relation log
*/
List<ProcessTaskRelationLog> queryByProcessCodeAndVersion(@Param("processCode") long processCode,
@Param("processVersion") int processVersion);
List<ProcessTaskRelationLog> queryByTaskRelationList(@Param("processCode") long processCode,
@Param("processVersion") int processVersion,
@Param("taskCode") long taskCode,
@Param("taskVersion") long taskVersion);
}

68
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessTaskRelationMapper.java

@ -0,0 +1,68 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.mapper;
import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelation;
import org.apache.ibatis.annotations.Param;
import java.util.List;
import com.baomidou.mybatisplus.core.mapper.BaseMapper;
/**
* process task relation mapper interface
*/
public interface ProcessTaskRelationMapper extends BaseMapper<ProcessTaskRelation> {
/**
* process task relation by projectCode and processCode
*
* @param projectCode projectCode
* @param processCode processCode
* @return ProcessTaskRelation list
*/
List<ProcessTaskRelation> queryByProcessCode(@Param("projectCode") Long projectCode,
@Param("processCode") Long processCode);
/**
* process task relation by taskCode
*
* @param taskCodes taskCode list
* @return ProcessTaskRelation
*/
List<ProcessTaskRelation> queryByTaskCodes(@Param("taskCodes") Long[] taskCodes);
/**
* process task relation by taskCode
*
* @param taskCode taskCode
* @return ProcessTaskRelation
*/
List<ProcessTaskRelation> queryByTaskCode(@Param("taskCode") Long taskCode);
/**
* delete process task relation by processCode
*
* @param projectCode projectCode
* @param processCode processCode
* @return int
*/
int deleteByCode(@Param("projectCode") Long projectCode,
@Param("processCode") Long processCode);
}

21
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.java

@ -31,14 +31,28 @@ import com.baomidou.mybatisplus.core.metadata.IPage;
* project mapper interface
*/
public interface ProjectMapper extends BaseMapper<Project> {
/**
* query project detail by code
* @param projectCode projectCode
* @return project
*/
Project queryByCode(@Param("projectCode") Long projectCode);
/**
* TODO: delete
* query project detail by id
* @param projectId projectId
* @return project
*/
Project queryDetailById(@Param("projectId") int projectId);
/**
* query project detail by code
* @param projectCode projectCode
* @return project
*/
Project queryDetailByCode(@Param("projectCode") Long projectCode);
/**
* query project by name
* @param projectName projectName
@ -71,6 +85,13 @@ public interface ProjectMapper extends BaseMapper<Project> {
*/
List<Project> queryAuthedProjectListByUserId(@Param("userId") int userId);
/**
* query relation project list by userId
* @param userId userId
* @return project list
*/
List<Project> queryRelationProjectListByUserId(@Param("userId") int userId);
/**
* query project except userId
* @param userId userId

6
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProjectUserMapper.java

@ -17,9 +17,11 @@
package org.apache.dolphinscheduler.dao.mapper;
import org.apache.dolphinscheduler.dao.entity.ProjectUser;
import com.baomidou.mybatisplus.core.mapper.BaseMapper;
import org.apache.ibatis.annotations.Param;
import com.baomidou.mybatisplus.core.mapper.BaseMapper;
/**
* project user mapper interface
*/
@ -27,6 +29,7 @@ public interface ProjectUserMapper extends BaseMapper<ProjectUser> {
/**
* delte prject user relation
*
* @param projectId projectId
* @param userId userId
* @return delete result
@ -36,6 +39,7 @@ public interface ProjectUserMapper extends BaseMapper<ProjectUser> {
/**
* query project relation
*
* @param projectId projectId
* @param userId userId
* @return project user relation

70
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionLogMapper.java

@ -0,0 +1,70 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.mapper;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
import org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog;
import org.apache.ibatis.annotations.Param;
import java.util.Collection;
import java.util.List;
import com.baomidou.mybatisplus.core.mapper.BaseMapper;
/**
* task definition log mapper interface
*/
public interface TaskDefinitionLogMapper extends BaseMapper<TaskDefinitionLog> {
/**
* query task definition log by name
*
* @param projectCode projectCode
* @param name name
* @return task definition log list
*/
List<TaskDefinitionLog> queryByDefinitionName(@Param("projectCode") Long projectCode,
@Param("taskDefinitionName") String name);
/**
* query max version for definition
*
* @param taskDefinitionCode taskDefinitionCode
*/
Integer queryMaxVersionForDefinition(@Param("taskDefinitionCode") long taskDefinitionCode);
/**
* query task definition log
*
* @param taskDefinitionCode taskDefinitionCode
* @param version version
* @return task definition log
*/
TaskDefinitionLog queryByDefinitionCodeAndVersion(@Param("taskDefinitionCode") long taskDefinitionCode,
@Param("version") int version);
/**
*
* @param taskDefinitions
* @return
*/
List<TaskDefinitionLog> queryByTaskDefinitions(@Param("taskDefinitions") Collection<TaskDefinition> taskDefinitions);
}

109
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.java

@ -0,0 +1,109 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.dao.mapper;
import org.apache.dolphinscheduler.dao.entity.DefinitionGroupByUser;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
import org.apache.ibatis.annotations.MapKey;
import org.apache.ibatis.annotations.Param;
import java.util.List;
import java.util.Map;
import com.baomidou.mybatisplus.core.mapper.BaseMapper;
/**
* task definition mapper interface
*/
public interface TaskDefinitionMapper extends BaseMapper<TaskDefinition> {
/**
* query task definition by name
*
* @param projectCode projectCode
* @param name name
* @return task definition
*/
TaskDefinition queryByDefinitionName(@Param("projectCode") Long projectCode,
@Param("taskDefinitionName") String name);
/**
* query task definition by id
*
* @param taskDefinitionId taskDefinitionId
* @return task definition
*/
TaskDefinition queryByDefinitionId(@Param("taskDefinitionId") int taskDefinitionId);
/**
* query task definition by code
*
* @param taskDefinitionCode taskDefinitionCode
* @return task definition
*/
TaskDefinition queryByDefinitionCode(@Param("taskDefinitionCode") Long taskDefinitionCode);
/**
* query all task definition list
*
* @param projectCode projectCode
* @return task definition list
*/
List<TaskDefinition> queryAllDefinitionList(@Param("projectCode") Long projectCode);
/**
* query task definition by ids
*
* @param ids ids
* @return task definition list
*/
List<TaskDefinition> queryDefinitionListByIdList(@Param("ids") Integer[] ids);
/**
* count task definition group by user
*
* @param projectCodes projectCodes
* @return task definition list
*/
List<DefinitionGroupByUser> countDefinitionGroupByUser(@Param("projectCodes") Long[] projectCodes);
/**
* list all resource ids
*
* @return task ids list
*/
@MapKey("id")
List<Map<String, Object>> listResources();
/**
* list all resource ids by user id
*
* @return resource ids list
*/
@MapKey("id")
List<Map<String, Object>> listResourcesByUser(@Param("userId") Integer userId);
/**
* delete task definition by code
*
* @param code code
* @return int
*/
int deleteByCode(@Param("code") Long code);
}

10
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java

@ -51,17 +51,15 @@ public interface TaskInstanceMapper extends BaseMapper<TaskInstance> {
TaskInstance queryByInstanceIdAndName(@Param("processInstanceId") int processInstanceId,
@Param("name") String name);
Integer countTask(
@Param("projectIds") Integer[] projectIds,
Integer countTask(@Param("projectCodes") Long[] projectCodes,
@Param("taskIds") int[] taskIds);
List<ExecuteStatusCount> countTaskInstanceStateByUser(
@Param("startTime") Date startTime,
List<ExecuteStatusCount> countTaskInstanceStateByUser(@Param("startTime") Date startTime,
@Param("endTime") Date endTime,
@Param("projectIds") Integer[] projectIds);
@Param("projectCodes") Long[] projectCodes);
IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
@Param("projectId") int projectId,
@Param("projectCode") Long projectCode,
@Param("processInstanceId") Integer processInstanceId,
@Param("processInstanceName") String processInstanceName,
@Param("searchVal") String searchVal,

44
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/WorkFlowLineageMapper.java

@ -16,17 +16,53 @@
*/
package org.apache.dolphinscheduler.dao.mapper;
import org.apache.dolphinscheduler.dao.entity.ProcessLineage;
import org.apache.dolphinscheduler.dao.entity.WorkFlowLineage;
import org.apache.dolphinscheduler.dao.entity.WorkFlowRelation;
import org.apache.ibatis.annotations.Param;
import java.util.List;
import java.util.Set;
public interface WorkFlowLineageMapper {
List<WorkFlowLineage> queryByName(@Param("searchVal") String searchVal, @Param("projectId") int projectId);
/**
* queryByName
*
* @param searchVal searchVal
* @param projectCode projectCode
* @return WorkFlowLineage list
*/
List<WorkFlowLineage> queryByName(@Param("searchVal") String searchVal, @Param("projectCode") Long projectCode);
/**
* queryCodeRelation
*
* @param taskCode taskCode
* @param taskVersion taskVersion
* @param processDefinitionCode processDefinitionCode
* @return ProcessLineage
*/
List<ProcessLineage> queryCodeRelation(
@Param("taskCode") Long taskCode, @Param("taskVersion") int taskVersion,
@Param("processDefinitionCode") Long processDefinitionCode, @Param("projectCode") Long projectCode);
/**
* queryRelationByIds
*
* @param ids ids
* @param projectCode projectCode
* @return ProcessLineage
*/
List<ProcessLineage> queryRelationByIds(@Param("ids") Set<Integer> ids, @Param("projectCode") Long projectCode);
List<WorkFlowLineage> queryByIds(@Param("ids") Set<Integer> ids, @Param("projectId") int projectId);
/**
* queryWorkFlowLineageByCode
*
* @param processDefinitionCode processDefinitionCode
* @param projectCode projectCode
* @return WorkFlowLineage
*/
WorkFlowLineage queryWorkFlowLineageByCode(@Param("processDefinitionCode") Long processDefinitionCode, @Param("projectCode") Long projectCode);
List<WorkFlowRelation> querySourceTarget(@Param("id") int id);
}

33
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/ProcessDefinitionDao.java

@ -18,14 +18,15 @@
package org.apache.dolphinscheduler.dao.upgrade;
import org.apache.dolphinscheduler.common.utils.ConnectionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.sql.Connection;
import java.sql.PreparedStatement;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.*;
import java.util.HashMap;
import java.util.Map;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class ProcessDefinitionDao {
@ -34,12 +35,13 @@ public class ProcessDefinitionDao {
/**
* queryAllProcessDefinition
*
* @param conn jdbc connection
* @return ProcessDefinition Json List
*/
public Map<Integer,String> queryAllProcessDefinition(Connection conn){
public Map<Integer, String> queryAllProcessDefinition(Connection conn) {
Map<Integer,String> processDefinitionJsonMap = new HashMap<>();
Map<Integer, String> processDefinitionJsonMap = new HashMap<>();
String sql = String.format("SELECT id,process_definition_json FROM t_ds_process_definition");
ResultSet rs = null;
@ -48,14 +50,14 @@ public class ProcessDefinitionDao {
pstmt = conn.prepareStatement(sql);
rs = pstmt.executeQuery();
while (rs.next()){
while (rs.next()) {
Integer id = rs.getInt(1);
String processDefinitionJson = rs.getString(2);
processDefinitionJsonMap.put(id,processDefinitionJson);
processDefinitionJsonMap.put(id, processDefinitionJson);
}
} catch (Exception e) {
logger.error(e.getMessage(),e);
logger.error(e.getMessage(), e);
throw new RuntimeException("sql: " + sql, e);
} finally {
ConnectionUtils.releaseResource(rs, pstmt, conn);
@ -67,23 +69,24 @@ public class ProcessDefinitionDao {
/**
* updateProcessDefinitionJson
*
* @param conn jdbc connection
* @param processDefinitionJsonMap processDefinitionJsonMap
*/
public void updateProcessDefinitionJson(Connection conn,Map<Integer,String> processDefinitionJsonMap){
public void updateProcessDefinitionJson(Connection conn, Map<Integer, String> processDefinitionJsonMap) {
String sql = "UPDATE t_ds_process_definition SET process_definition_json=? where id=?";
try {
for (Map.Entry<Integer, String> entry : processDefinitionJsonMap.entrySet()){
try(PreparedStatement pstmt= conn.prepareStatement(sql)) {
pstmt.setString(1,entry.getValue());
pstmt.setInt(2,entry.getKey());
for (Map.Entry<Integer, String> entry : processDefinitionJsonMap.entrySet()) {
try (PreparedStatement pstmt = conn.prepareStatement(sql)) {
pstmt.setString(1, entry.getValue());
pstmt.setInt(2, entry.getKey());
pstmt.executeUpdate();
}
}
} catch (Exception e) {
logger.error(e.getMessage(),e);
logger.error(e.getMessage(), e);
throw new RuntimeException("sql: " + sql, e);
} finally {
ConnectionUtils.releaseResource(conn);

133
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/DagHelper.java

@ -24,17 +24,21 @@ import org.apache.dolphinscheduler.common.model.TaskNodeRelation;
import org.apache.dolphinscheduler.common.process.ProcessDag;
import org.apache.dolphinscheduler.common.task.conditions.ConditionsParameters;
import org.apache.dolphinscheduler.common.utils.CollectionUtils;
import org.apache.dolphinscheduler.common.utils.*;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessData;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelation;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
/**
* dag tools
*/
@ -70,10 +74,10 @@ public class DagHelper {
/**
* generate task nodes needed by dag
*
* @param taskNodeList taskNodeList
* @param startNodeNameList startNodeNameList
* @param taskNodeList taskNodeList
* @param startNodeNameList startNodeNameList
* @param recoveryNodeNameList recoveryNodeNameList
* @param taskDependType taskDependType
* @param taskDependType taskDependType
* @return task node list
*/
public static List<TaskNode> generateFlowNodeListByStartNode(List<TaskNode> taskNodeList, List<String> startNodeNameList,
@ -131,7 +135,7 @@ public class DagHelper {
/**
* find all the nodes that depended on the start node
*
* @param startNode startNode
* @param startNode startNode
* @param taskNodeList taskNodeList
* @return task node list
*/
@ -156,9 +160,9 @@ public class DagHelper {
/**
* find all nodes that start nodes depend on.
*
* @param startNode startNode
* @param startNode startNode
* @param recoveryNodeNameList recoveryNodeNameList
* @param taskNodeList taskNodeList
* @param taskNodeList taskNodeList
* @return task node list
*/
private static List<TaskNode> getFlowNodeListPre(TaskNode startNode, List<String> recoveryNodeNameList, List<TaskNode> taskNodeList, List<String> visitedNodeNameList) {
@ -191,24 +195,19 @@ public class DagHelper {
/**
* generate dag by start nodes and recovery nodes
*
* @param processDefinitionJson processDefinitionJson
* @param startNodeNameList startNodeNameList
* @param recoveryNodeNameList recoveryNodeNameList
* @param depNodeType depNodeType
* @param totalTaskNodeList totalTaskNodeList
* @param startNodeNameList startNodeNameList
* @param recoveryNodeNameList recoveryNodeNameList
* @param depNodeType depNodeType
* @return process dag
* @throws Exception if error throws Exception
*/
public static ProcessDag generateFlowDag(String processDefinitionJson,
public static ProcessDag generateFlowDag(List<TaskNode> totalTaskNodeList,
List<String> startNodeNameList,
List<String> recoveryNodeNameList,
TaskDependType depNodeType) throws Exception {
ProcessData processData = JSONUtils.parseObject(processDefinitionJson, ProcessData.class);
List<TaskNode> taskNodeList = new ArrayList<>();
if (null != processData) {
taskNodeList = processData.getTasks();
}
List<TaskNode> destTaskNodeList = generateFlowNodeListByStartNode(taskNodeList, startNodeNameList, recoveryNodeNameList, depNodeType);
List<TaskNode> destTaskNodeList = generateFlowNodeListByStartNode(totalTaskNodeList, startNodeNameList, recoveryNodeNameList, depNodeType);
if (destTaskNodeList.isEmpty()) {
return null;
}
@ -219,34 +218,11 @@ public class DagHelper {
return processDag;
}
/**
* parse the forbidden task nodes in process definition.
*
* @param processDefinitionJson processDefinitionJson
* @return task node map
*/
public static Map<String, TaskNode> getForbiddenTaskNodeMaps(String processDefinitionJson) {
Map<String, TaskNode> forbidTaskNodeMap = new ConcurrentHashMap<>();
ProcessData processData = JSONUtils.parseObject(processDefinitionJson, ProcessData.class);
List<TaskNode> taskNodeList = new ArrayList<>();
if (null != processData) {
taskNodeList = processData.getTasks();
}
for (TaskNode node : taskNodeList) {
if (node.isForbidden()) {
forbidTaskNodeMap.putIfAbsent(node.getName(), node);
}
}
return forbidTaskNodeMap;
}
/**
* find node by node name
*
* @param nodeDetails nodeDetails
* @param nodeName nodeName
* @param nodeName nodeName
* @return task node
*/
public static TaskNode findNodeByName(List<TaskNode> nodeDetails, String nodeName) {
@ -261,8 +237,8 @@ public class DagHelper {
/**
* the task can be submit when all the depends nodes are forbidden or complete
*
* @param taskNode taskNode
* @param dag dag
* @param taskNode taskNode
* @param dag dag
* @param completeTaskList completeTaskList
* @return can submit
*/
@ -292,7 +268,6 @@ public class DagHelper {
* this function parse the condition node to find the right branch.
* also check all the depends nodes forbidden or complete
*
* @param preNodeName
* @return successor nodes
*/
public static Set<String> parsePostNodes(String preNodeName,
@ -329,9 +304,6 @@ public class DagHelper {
/**
* if all of the task dependence are skipped, skip it too.
*
* @param taskNode
* @return
*/
private static boolean isTaskNodeNeedSkip(TaskNode taskNode,
Map<String, TaskNode> skipTaskNodeList
@ -351,9 +323,6 @@ public class DagHelper {
/**
* parse condition task find the branch process
* set skip flag for another one.
*
* @param nodeName
* @return
*/
public static List<String> parseConditionTask(String nodeName,
Map<String, TaskNode> skipTaskNodeList,
@ -388,11 +357,6 @@ public class DagHelper {
/**
* set task node and the post nodes skip flag
*
* @param skipNodeName
* @param dag
* @param completeTaskList
* @param skipTaskNodeList
*/
private static void setTaskNodeSkip(String skipNodeName,
DAG<String, TaskNode, TaskNodeRelation> dag,
@ -466,10 +430,39 @@ public class DagHelper {
}
/**
* is there have conditions after the parent node
* get process dag
*
* @param parentNodeName
* @return
* @param taskNodeList task node list
* @return Process dag
*/
public static ProcessDag getProcessDag(List<TaskNode> taskNodeList,
List<ProcessTaskRelation> processTaskRelations) {
Map<Long, TaskNode> taskNodeMap = new HashMap<>();
taskNodeList.forEach(taskNode -> {
taskNodeMap.putIfAbsent(taskNode.getCode(), taskNode);
});
List<TaskNodeRelation> taskNodeRelations = new ArrayList<>();
for (ProcessTaskRelation processTaskRelation : processTaskRelations) {
long preTaskCode = processTaskRelation.getPreTaskCode();
long postTaskCode = processTaskRelation.getPostTaskCode();
if (processTaskRelation.getPreTaskCode() != 0
&& taskNodeMap.containsKey(preTaskCode) && taskNodeMap.containsKey(postTaskCode)) {
TaskNode preNode = taskNodeMap.get(preTaskCode);
TaskNode postNode = taskNodeMap.get(postTaskCode);
taskNodeRelations.add(new TaskNodeRelation(preNode.getName(), postNode.getName()));
}
}
ProcessDag processDag = new ProcessDag();
processDag.setEdges(taskNodeRelations);
processDag.setNodes(taskNodeList);
return processDag;
}
/**
* is there have conditions after the parent node
*/
public static boolean haveConditionsAfterNode(String parentNodeName,
DAG<String, TaskNode, TaskNodeRelation> dag
@ -491,16 +484,10 @@ public class DagHelper {
/**
* is there have conditions after the parent node
*
* @param parentNodeName
* @return
*/
public static boolean haveConditionsAfterNode(String parentNodeName,
List<TaskNode> taskNodes
) {
boolean result = false;
public static boolean haveConditionsAfterNode(String parentNodeName, List<TaskNode> taskNodes) {
if (CollectionUtils.isEmpty(taskNodes)) {
return result;
return false;
}
for (TaskNode taskNode : taskNodes) {
List<String> preTasksList = JSONUtils.toList(taskNode.getPreTasks(), String.class);
@ -508,6 +495,6 @@ public class DagHelper {
return true;
}
}
return result;
return false;
}
}

59
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/ResourceProcessDefinitionUtils.java

@ -18,7 +18,12 @@ package org.apache.dolphinscheduler.dao.utils;
import org.apache.dolphinscheduler.common.utils.CollectionUtils;
import java.util.*;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
/**
@ -26,35 +31,37 @@ import java.util.stream.Collectors;
*/
public class ResourceProcessDefinitionUtils {
/**
* get resource process map key is resource id,value is the set of process definition
* @param list the map key is process definition id and value is resource_ids
* @return resource process definition map
* get resource process map key is resource id,value is the set of process definition code
*
* @param resourceList the map key is process definition code and value is resource_ids
* @return resource process definition map (resourceId -> processDefinitionCodes)
*/
public static Map<Integer, Set<Integer>> getResourceProcessDefinitionMap(List<Map<String, Object>> list) {
Map<Integer, String> map = new HashMap<>();
Map<Integer, Set<Integer>> result = new HashMap<>();
if (CollectionUtils.isNotEmpty(list)) {
for (Map<String, Object> tempMap : list) {
map.put((Integer) tempMap.get("id"), (String)tempMap.get("resource_ids"));
}
}
public static Map<Integer, Set<Long>> getResourceProcessDefinitionMap(List<Map<String, Object>> resourceList) {
// resourceId -> processDefinitionCodes
Map<Integer, Set<Long>> resourceResult = new HashMap<>();
for (Map.Entry<Integer, String> entry : map.entrySet()) {
Integer mapKey = entry.getKey();
String[] arr = entry.getValue().split(",");
Set<Integer> mapValues = Arrays.stream(arr).map(Integer::parseInt).collect(Collectors.toSet());
for (Integer value : mapValues) {
if (result.containsKey(value)) {
Set<Integer> set = result.get(value);
set.add(mapKey);
result.put(value, set);
} else {
Set<Integer> set = new HashSet<>();
set.add(mapKey);
result.put(value, set);
if (CollectionUtils.isNotEmpty(resourceList)) {
for (Map<String, Object> resourceMap : resourceList) {
Long code = (Long) resourceMap.get("code");
String[] resourceIds = ((String) resourceMap.get("resource_ids"))
.split(",");
Set<Integer> resourceIdSet = Arrays.stream(resourceIds).map(Integer::parseInt).collect(Collectors.toSet());
for (Integer resourceId : resourceIdSet) {
Set<Long> codeSet;
if (resourceResult.containsKey(resourceId)) {
codeSet = resourceResult.get(resourceId);
} else {
codeSet = new HashSet<>();
}
codeSet.add(code);
resourceResult.put(resourceId, codeSet);
}
}
}
return result;
return resourceResult;
}
}

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

@ -32,9 +32,9 @@
select cmd.command_type as command_type, count(1) as count
from t_ds_command cmd, t_ds_process_definition process
where cmd.process_definition_id = process.id
<if test="projectIdArray != null and projectIdArray.length != 0">
and process.project_id in
<foreach collection="projectIdArray" index="index" item="i" open="(" close=")" separator=",">
<if test="projectCodeArray != null and projectCodeArray.length != 0">
and process.project_code in
<foreach collection="projectCodeArray" index="index" item="i" open="(" close=")" separator=",">
#{i}
</foreach>
</if>

8
dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ErrorCommandMapper.xml

@ -22,14 +22,14 @@
select cmd.command_type as command_type, count(1) as count
from t_ds_error_command cmd, t_ds_process_definition process
where cmd.process_definition_id = process.id
<if test="projectIdArray != null and projectIdArray.length != 0">
and process.project_id in
<foreach collection="projectIdArray" index="index" item="i" open="(" close=")" separator=",">
<if test="projectCodeArray != null and projectCodeArray.length != 0">
and process.project_code in
<foreach collection="projectCodeArray" index="index" item="i" open="(" close=")" separator=",">
#{i}
</foreach>
</if>
<if test="startTime != null and endTime != null">
and cmd.startTime <![CDATA[ >= ]]> #{startTime} and cmd.update_time <![CDATA[ <= ]]> #{endTime}
and cmd.start_time <![CDATA[ >= ]]> #{startTime} and cmd.update_time <![CDATA[ <= ]]> #{endTime}
</if>
group by cmd.command_type
</select>

82
dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionLogMapper.xml

@ -0,0 +1,82 @@
<?xml version="1.0" encoding="UTF-8" ?>
<!--
~ Licensed to the Apache Software Foundation (ASF) under one or more
~ contributor license agreements. See the NOTICE file distributed with
~ this work for additional information regarding copyright ownership.
~ The ASF licenses this file to You under the Apache License, Version 2.0
~ (the "License"); you may not use this file except in compliance with
~ the License. You may obtain a copy of the License at
~
~ http://www.apache.org/licenses/LICENSE-2.0
~
~ Unless required by applicable law or agreed to in writing, software
~ distributed under the License is distributed on an "AS IS" BASIS,
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
~ See the License for the specific language governing permissions and
~ limitations under the License.
-->
<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" "http://mybatis.org/dtd/mybatis-3-mapper.dtd" >
<mapper namespace="org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionLogMapper">
<sql id="baseSql">
id, code, name, version, description, project_code,
release_state, user_id,global_params, flag, locations, connects,
warning_group_id, timeout, tenant_id,operator, operate_time, create_time,
update_time
</sql>
<select id="queryByDefinitionName" resultType="org.apache.dolphinscheduler.dao.entity.ProcessDefinitionLog">
select pd.id, pd.code, pd.name, pd.version, pd.description, pd.project_code,
pd.release_state, pd.user_id,pd.global_params, pd.flag, pd.locations, pd.connects,
pd.warning_group_id, pd.timeout, pd.tenant_id,pd.operator, pd.operate_time, pd.create_time,
pd.update_time, u.user_name,p.name as project_name
from t_ds_process_definition_log pd
JOIN t_ds_user u ON pd.user_id = u.id
JOIN t_ds_project p ON pd.project_code = p.code
WHERE p.code = #{projectCode}
and pd.name = #{processDefinitionName}
</select>
<select id="queryByDefinitionCode" resultType="org.apache.dolphinscheduler.dao.entity.ProcessDefinitionLog">
select
<include refid="baseSql"/>
from t_ds_process_definition_log
WHERE code = #{processDefinitionCode}
</select>
<select id="queryByDefinitionCodeAndVersion"
resultType="org.apache.dolphinscheduler.dao.entity.ProcessDefinitionLog">
select
<include refid="baseSql"/>
from t_ds_process_definition_log
where code = #{processDefinitionCode}
and version = #{version}
</select>
<select id="queryMaxVersionForDefinition" resultType="java.lang.Integer">
select max(version)
from t_ds_process_definition_log
where code = #{processDefinitionCode}
</select>
<select id="queryMaxVersionDefinitionLog" resultType="org.apache.dolphinscheduler.dao.entity.ProcessDefinitionLog">
select
<include refid="baseSql"/>
from t_ds_process_definition_log
where code = #{processDefinitionCode} order by version desc limit 1
</select>
<select id="queryProcessDefinitionVersionsPaging"
resultType="org.apache.dolphinscheduler.dao.entity.ProcessDefinitionLog">
select
<include refid="baseSql"/>
from t_ds_process_definition_log
where code = #{processDefinitionCode}
order by version desc
</select>
<delete id="deleteByProcessDefinitionCodeAndVersion">
delete
from t_ds_process_definition_log
where code = #{processDefinitionCode}
and version = #{version}
</delete>
</mapper>

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

@ -19,42 +19,66 @@
<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" "http://mybatis.org/dtd/mybatis-3-mapper.dtd" >
<mapper namespace="org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper">
<sql id="baseSql">
id
, name, version, release_state, project_id, user_id, process_definition_json, description,
id, code, name, version, release_state, project_code, user_id, description,
global_params, flag, locations, connects, warning_group_id, create_time, timeout,
tenant_id, update_time, modify_by, resource_ids
tenant_id, update_time
</sql>
<select id="verifyByDefineName" resultType="org.apache.dolphinscheduler.dao.entity.ProcessDefinition">
select pd.id, pd.name, pd.version, pd.release_state, pd.project_id, pd.user_id, pd.process_definition_json, pd.description,
select pd.id, pd.code, pd.name, pd.version, pd.release_state, pd.project_code, pd.user_id, pd.description,
pd.global_params, pd.flag, pd.locations, pd.connects, pd.warning_group_id, pd.create_time, pd.timeout,
pd.tenant_id, pd.update_time, pd.modify_by, pd.resource_ids
pd.tenant_id, pd.update_time
from t_ds_process_definition pd
WHERE pd.project_id = #{projectId}
WHERE pd.project_code = #{projectCode}
and pd.name = #{processDefinitionName}
</select>
<delete id="deleteByCode">
delete from t_ds_process_definition
where code = #{code}
</delete>
<select id="queryByCode" resultType="org.apache.dolphinscheduler.dao.entity.ProcessDefinition">
select
<include refid="baseSql"/>
from t_ds_process_definition
where code = #{code}
</select>
<select id="queryByCodes" resultType="org.apache.dolphinscheduler.dao.entity.ProcessDefinition">
select
<include refid="baseSql"/>
from t_ds_process_definition
where 1 = 1
<if test="codes != null and codes.size() != 0">
and code in
<foreach collection="codes" index="index" item="i" open="(" separator="," close=")">
#{i}
</foreach>
</if>
</select>
<select id="queryByDefineName" resultType="org.apache.dolphinscheduler.dao.entity.ProcessDefinition">
select pd.id, pd.name, pd.version, pd.release_state, pd.project_id, pd.user_id, pd.process_definition_json, pd.description,
select pd.id, pd.code, pd.name, pd.version, pd.release_state, pd.project_code, p.id as project_id, pd.user_id, pd.description,
pd.global_params, pd.flag, pd.locations, pd.connects, pd.warning_group_id, pd.create_time, pd.timeout,
pd.tenant_id, pd.update_time, pd.modify_by, pd.resource_ids,
pd.tenant_id, pd.update_time,
u.user_name,p.name as project_name,t.tenant_code,q.queue,q.queue_name
from t_ds_process_definition pd
JOIN t_ds_user u ON pd.user_id = u.id
JOIN t_ds_project p ON pd.project_id = p.id
JOIN t_ds_project p ON pd.project_code = p.code
JOIN t_ds_tenant t ON t.id = u.tenant_id
JOIN t_ds_queue q ON t.queue_id = q.id
WHERE p.id = #{projectId}
WHERE p.code = #{projectCode}
and pd.name = #{processDefinitionName}
</select>
<select id="queryDefineListPaging" resultType="org.apache.dolphinscheduler.dao.entity.ProcessDefinition">
SELECT td.id, td.name, td.version, td.release_state, td.project_id, td.user_id, td.description,
td.global_params,
td.flag, td.warning_group_id, td.timeout, td.tenant_id, td.modify_by, td.update_time, td.create_time,
SELECT td.id, td.code, td.name, td.version, td.release_state, td.project_code, td.user_id, td.description,
td.global_params, td.flag, td.warning_group_id, td.timeout, td.tenant_id, td.update_time, td.create_time,
sc.schedule_release_state, tu.user_name
FROM t_ds_process_definition td
left join (select process_definition_id,release_state as schedule_release_state from t_ds_schedules group by
process_definition_id,release_state) sc on sc.process_definition_id = td.id
left join t_ds_user tu on td.user_id = tu.id
where td.project_id = #{projectId}
where td.project_code = #{projectCode}
<if test=" searchVal != null and searchVal != ''">
and td.name like concat('%', #{searchVal}, '%')
</if>
@ -68,7 +92,7 @@
select
<include refid="baseSql"/>
from t_ds_process_definition
where project_id = #{projectId}
where project_code = #{projectCode}
order by create_time desc
</select>
<select id="queryDefinitionListByTenant" resultType="org.apache.dolphinscheduler.dao.entity.ProcessDefinition">
@ -91,46 +115,56 @@
FROM t_ds_process_definition td
JOIN t_ds_user tu on tu.id=td.user_id
where 1 = 1
<if test="projectIds != null and projectIds.length != 0">
and td.project_id in
<foreach collection="projectIds" index="index" item="i" open="(" separator="," close=")">
<if test="projectCodes != null and projectCodes.length != 0">
and td.project_code in
<foreach collection="projectCodes" index="index" item="i" open="(" separator="," close=")">
#{i}
</foreach>
</if>
group by td.user_id,tu.user_name
</select>
<select id="queryByDefineId" resultType="org.apache.dolphinscheduler.dao.entity.ProcessDefinition">
SELECT
pd.id, pd.name, pd.version, pd.release_state, pd.project_id, pd.user_id, pd.process_definition_json, pd.description,
pd.id, pd.code, pd.name, pd.version, pd.release_state, pd.project_code, pd.user_id, pd.description,
pd.global_params, pd.flag, pd.locations, pd.connects, pd.warning_group_id, pd.create_time, pd.timeout,
pd.tenant_id, pd.update_time, pd.modify_by, pd.resource_ids,
u.user_name,
p.name AS project_name
pd.tenant_id, pd.update_time, u.user_name,p.name AS project_name
FROM
t_ds_process_definition pd,
t_ds_user u,
t_ds_project p
WHERE
pd.user_id = u.id AND pd.project_id = p.id
pd.user_id = u.id AND pd.project_code = p.code
AND pd.id = #{processDefineId}
</select>
<select id="listResources" resultType="java.util.HashMap">
SELECT id,resource_ids
FROM t_ds_process_definition
WHERE release_state = 1 and resource_ids is not null and resource_ids != ''
SELECT distinct pd.code,td.resource_ids
FROM t_ds_process_task_relation ptr
join t_ds_process_definition pd
on ptr.process_definition_code=pd.code and ptr.process_definition_version = pd.version
and ptr.project_code=pd.project_code and pd.release_state = 1
join t_ds_task_definition td
on (ptr.pre_task_code=td.code and ptr.pre_task_version=td.version)
or (ptr.pre_task_code=td.code and ptr.pre_task_version=td.version)
WHERE td.resource_ids is not null and td.resource_ids != ''
</select>
<select id="listResourcesByUser" resultType="java.util.HashMap">
SELECT id,resource_ids
FROM t_ds_process_definition
WHERE user_id = #{userId} and release_state = 1 and resource_ids is not null and resource_ids != ''
SELECT distinct pd.code,td.resource_ids
FROM t_ds_process_task_relation ptr
join t_ds_process_definition pd
on ptr.process_definition_code=pd.code and ptr.process_definition_version = pd.version
and ptr.project_code=pd.project_code and pd.release_state = 1
join t_ds_task_definition td
on (ptr.pre_task_code=td.code and ptr.pre_task_version=td.version)
or (ptr.pre_task_code=td.code and ptr.pre_task_version=td.version)
WHERE td.resource_ids is not null and td.resource_ids != '' and td.user_id = #{userId}
</select>
<select id="listProjectIds" resultType="java.lang.Integer">
SELECT DISTINCT(project_id) as project_id
FROM t_ds_process_definition
SELECT DISTINCT(id) as project_id
FROM t_ds_project
</select>

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

@ -19,10 +19,10 @@
<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" "http://mybatis.org/dtd/mybatis-3-mapper.dtd" >
<mapper namespace="org.apache.dolphinscheduler.dao.mapper.ProcessInstanceMapper">
<sql id="baseSql">
id, name, process_definition_id, state, recovery, start_time, end_time, run_times,host,
id, name, process_definition_version, process_definition_code, state, recovery, start_time, end_time, run_times,host,
command_type, command_param, task_depend_type, max_try_times, failure_strategy, warning_type,
warning_group_id, schedule_time, command_start_time, global_params, process_instance_json, flag,
update_time, is_sub_process, executor_id, locations, connects, history_cmd, dependence_schedule_times,
warning_group_id, schedule_time, command_start_time, global_params, flag,
update_time, is_sub_process, executor_id, history_cmd,
process_instance_priority, worker_group, timeout, tenant_id, var_pool
</sql>
<select id="queryDetailById" resultType="org.apache.dolphinscheduler.dao.entity.ProcessInstance">
@ -88,15 +88,15 @@
</select>
<select id="queryProcessInstanceListPaging" resultType="org.apache.dolphinscheduler.dao.entity.ProcessInstance">
select instance.id, instance.process_definition_id, instance.command_type, instance.executor_id,
instance.name, instance.state, instance.schedule_time, instance.start_time, instance.end_time,
instance.run_times, instance.recovery, instance.host
select instance.id, instance.command_type, instance.executor_id, instance.process_definition_version,
instance.process_definition_code, instance.name, instance.state, instance.schedule_time, instance.start_time,
instance.end_time, instance.run_times, instance.recovery, instance.host
from t_ds_process_instance instance
join t_ds_process_definition define ON instance.process_definition_id = define.id
join t_ds_process_definition define ON instance.process_definition_code = define.code
where instance.is_sub_process=0
and define.project_id = #{projectId}
<if test="processDefinitionId != 0">
and instance.process_definition_id = #{processDefinitionId}
and define.project_code = #{projectCode}
<if test="processDefinitionCode != 0">
and instance.process_definition_code = #{processDefinitionCode}
</if>
<if test="searchVal != null and searchVal != ''">
and instance.name like concat('%', #{searchVal}, '%')
@ -147,33 +147,33 @@
<select id="countInstanceStateByUser" resultType="org.apache.dolphinscheduler.dao.entity.ExecuteStatusCount">
select t.state, count(0) as count
from t_ds_process_instance t
join t_ds_process_definition d on d.id=t.process_definition_id
join t_ds_project p on p.id=d.project_id
join t_ds_process_definition d on d.code=t.process_definition_code
join t_ds_project p on p.code=d.project_code
where 1 = 1
and t.is_sub_process = 0
<if test="startTime != null and endTime != null">
and t.start_time <![CDATA[ >= ]]> #{startTime} and t.start_time <![CDATA[ <= ]]> #{endTime}
</if>
<if test="projectIds != null and projectIds.length != 0">
and p.id in
<foreach collection="projectIds" index="index" item="i" open="(" close=")" separator=",">
<if test="projectCodes != null and projectCodes.length != 0">
and p.code in
<foreach collection="projectCodes" index="index" item="i" open="(" close=")" separator=",">
#{i}
</foreach>
</if>
group by t.state
</select>
<select id="queryByProcessDefineId" resultType="org.apache.dolphinscheduler.dao.entity.ProcessInstance">
<select id="queryByProcessDefineCode" resultType="org.apache.dolphinscheduler.dao.entity.ProcessInstance">
select
<include refid="baseSql"/>
from t_ds_process_instance
where process_definition_id=#{processDefinitionId}
where process_definition_code=#{processDefinitionCode}
order by start_time desc limit #{size}
</select>
<select id="queryLastSchedulerProcess" resultType="org.apache.dolphinscheduler.dao.entity.ProcessInstance">
select
<include refid="baseSql"/>
from t_ds_process_instance
where process_definition_id=#{processDefinitionId}
where process_definition_code=#{processDefinitionCode}
<if test="startTime!=null and endTime != null ">
and schedule_time <![CDATA[ >= ]]> #{startTime} and schedule_time <![CDATA[ <= ]]> #{endTime}
</if>
@ -183,7 +183,7 @@
select
<include refid="baseSql"/>
from t_ds_process_instance
where process_definition_id=#{processDefinitionId}
where process_definition_code=#{processDefinitionCode}
<if test="states !=null and states.length != 0">
and state in
<foreach collection="states" item="i" index="index" open="(" separator="," close=")">
@ -200,19 +200,18 @@
select
<include refid="baseSql"/>
from t_ds_process_instance
where process_definition_id=#{processDefinitionId}
where process_definition_code=#{processDefinitionCode}
and schedule_time is null
<if test="startTime!=null and endTime != null ">
and start_time <![CDATA[ >= ]]> #{startTime} and start_time <![CDATA[ <= ]]> #{endTime}
</if>
order by end_time desc limit 1
</select>
<select id="queryByProcessDefineIdAndStatus"
resultType="org.apache.dolphinscheduler.dao.entity.ProcessInstance">
<select id="queryByProcessDefineCodeAndStatus" resultType="org.apache.dolphinscheduler.dao.entity.ProcessInstance">
select
<include refid="baseSql"/>
from t_ds_process_instance
where process_definition_id=#{processDefinitionId}
where process_definition_code=#{processDefinitionCode}
and state in
<foreach collection="states" item="i" open="(" close=")" separator=",">
#{i}

43
dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionVersionMapper.xml → dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessTaskRelationLogMapper.xml

@ -17,41 +17,26 @@
-->
<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" "http://mybatis.org/dtd/mybatis-3-mapper.dtd" >
<mapper namespace="org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionVersionMapper">
<mapper namespace="org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationLogMapper">
<sql id="baseSql">
id
, process_definition_id, version, process_definition_json, description, global_params,locations,connects,
warning_group_id, create_time, timeout, resource_ids
id, `name`, process_definition_version, project_code, process_definition_code, pre_task_code, pre_task_version,
post_task_code, post_task_version, condition_type, condition_params, operator, operate_time, create_time, update_time
</sql>
<select id="queryMaxVersionByProcessDefinitionId" resultType="java.lang.Long">
select max(version)
from t_ds_process_definition_version
where process_definition_id = #{processDefinitionId}
</select>
<select id="queryProcessDefinitionVersionsPaging"
resultType="org.apache.dolphinscheduler.dao.entity.ProcessDefinitionVersion">
<select id="queryByProcessCodeAndVersion" resultType="org.apache.dolphinscheduler.dao.entity.ProcessTaskRelationLog">
select
<include refid="baseSql"/>
from t_ds_process_definition_version
where process_definition_id = #{processDefinitionId}
order by version desc
from t_ds_process_task_relation_log
WHERE process_definition_code = #{processCode}
and process_definition_version = #{processVersion}
</select>
<select id="queryByProcessDefinitionIdAndVersion"
resultType="org.apache.dolphinscheduler.dao.entity.ProcessDefinitionVersion">
<select id="queryByTaskRelationList"
resultType="org.apache.dolphinscheduler.dao.entity.ProcessTaskRelationLog">
select
<include refid="baseSql"/>
from t_ds_process_definition_version
where process_definition_id = #{processDefinitionId}
and version = #{version}
from t_ds_process_task_relation_log
WHERE process_definition_code = #{processCode}
and process_definition_version = #{processVersion}
and post_task_code = #{taskCode}
and post_task_version = #{taskVersion}
</select>
<delete id="deleteByProcessDefinitionIdAndVersion">
delete
from t_ds_process_definition_version
where process_definition_id = #{processDefinitionId}
and version = #{version}
</delete>
</mapper>

62
dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessTaskRelationMapper.xml

@ -0,0 +1,62 @@
<?xml version="1.0" encoding="UTF-8" ?>
<!--
~ Licensed to the Apache Software Foundation (ASF) under one or more
~ contributor license agreements. See the NOTICE file distributed with
~ this work for additional information regarding copyright ownership.
~ The ASF licenses this file to You under the Apache License, Version 2.0
~ (the "License"); you may not use this file except in compliance with
~ the License. You may obtain a copy of the License at
~
~ http://www.apache.org/licenses/LICENSE-2.0
~
~ Unless required by applicable law or agreed to in writing, software
~ distributed under the License is distributed on an "AS IS" BASIS,
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
~ See the License for the specific language governing permissions and
~ limitations under the License.
-->
<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" "http://mybatis.org/dtd/mybatis-3-mapper.dtd" >
<mapper namespace="org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationMapper">
<sql id="baseSql">
id, `name`, process_definition_version, project_code, process_definition_code, pre_task_code, pre_task_version,
post_task_code, post_task_version, condition_type, condition_params, create_time, update_time
</sql>
<select id="queryByProcessCode" resultType="org.apache.dolphinscheduler.dao.entity.ProcessTaskRelation">
select
<include refid="baseSql"/>
from t_ds_process_task_relation
WHERE project_code = #{projectCode}
and process_definition_code = #{processCode}
</select>
<select id="queryByTaskCode" resultType="org.apache.dolphinscheduler.dao.entity.ProcessTaskRelation">
select
<include refid="baseSql"/>
from t_ds_process_task_relation
WHERE pre_task_code = #{taskCode}
<if test="taskCode != 0">
or post_task_code = #{taskCode}
</if>
</select>
<select id="queryByTaskCodes" resultType="org.apache.dolphinscheduler.dao.entity.ProcessTaskRelation">
select
<include refid="baseSql"/>
from t_ds_process_task_relation
WHERE 1 = 1
<if test="taskCodes != null and taskCodes.length != 0">
and pre_task_code in
<foreach collection="taskCodes" index="index" item="i" open="(" separator="," close=")">
#{i}
</foreach>
or post_task_code in
<foreach collection="taskCodes" index="index" item="i" open="(" separator="," close=")">
#{i}
</foreach>
</if>
</select>
<delete id="deleteByCode">
delete from t_ds_process_task_relation
WHERE project_code = #{projectCode}
and process_definition_code = #{processCode}
</delete>
</mapper>

58
dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml

@ -19,12 +19,31 @@
<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" "http://mybatis.org/dtd/mybatis-3-mapper.dtd" >
<mapper namespace="org.apache.dolphinscheduler.dao.mapper.ProjectMapper">
<sql id="baseSql">
id, name, description, user_id, flag, create_time, update_time
id, name, code, description, user_id, flag, create_time, update_time
</sql>
<sql id="baseSqlV2">
${alias}.id, ${alias}.name, ${alias}.description, ${alias}.user_id, ${alias}.flag, ${alias}.create_time, ${alias}.update_time
${alias}.id, ${alias}.name, ${alias}.code, ${alias}.description, ${alias}.user_id, ${alias}.flag, ${alias}.create_time, ${alias}.update_time
</sql>
<select id="queryByCode" resultType="org.apache.dolphinscheduler.dao.entity.Project">
select
<include refid="baseSql"/>
from t_ds_project
where code = #{code}
</select>
<select id="queryDetailByCode" resultType="org.apache.dolphinscheduler.dao.entity.Project">
select
<include refid="baseSqlV2">
<property name="alias" value="p"/>
</include>
,
u.user_name as user_name
from t_ds_project p
join t_ds_user u on p.user_id = u.id
where p.code = #{projectCode}
</select>
<select id="queryDetailById" resultType="org.apache.dolphinscheduler.dao.entity.Project">
select
<include refid="baseSqlV2">
@ -55,9 +74,10 @@
</include>
,
u.user_name as user_name,
(SELECT COUNT(*) FROM t_ds_process_definition AS def WHERE def.project_id = p.id) AS def_count,
(SELECT COUNT(*) FROM t_ds_process_definition def, t_ds_process_instance inst WHERE def.id =
inst.process_definition_id AND def.project_id = p.id AND inst.state=1 ) as inst_running_count
(SELECT COUNT(*) FROM t_ds_process_definition AS def WHERE def.project_code = p.code) AS def_count,
(SELECT COUNT(*) FROM t_ds_process_definition_log def, t_ds_process_instance inst WHERE def.code =
inst.process_definition_code and def.version = inst.process_definition_version AND def.project_code = p.code
AND inst.state=1 ) as inst_running_count
from t_ds_project p
left join t_ds_user u on u.id=p.user_id
where 1=1
@ -80,6 +100,18 @@
from t_ds_project p,t_ds_relation_project_user rel
where p.id = rel.project_id and rel.user_id= #{userId}
</select>
<select id="queryRelationProjectListByUserId" resultType="org.apache.dolphinscheduler.dao.entity.Project">
select
<include refid="baseSqlV2">
<property name="alias" value="p"/>
</include>
from t_ds_project p left join t_ds_relation_project_user rel
on p.id = rel.project_id
where 1=1
<if test="userId != 0 ">
and rel.user_id= #{userId}
</if>
</select>
<select id="queryProjectExceptUserId" resultType="org.apache.dolphinscheduler.dao.entity.Project">
select
<include refid="baseSql"/>
@ -93,19 +125,21 @@
where user_id = #{userId}
</select>
<select id="queryProjectCreatedAndAuthorizedByUserId" resultType="org.apache.dolphinscheduler.dao.entity.Project">
select * from t_ds_project where id in
select
<include refid="baseSql"/>
from t_ds_project where id in
(select project_id from t_ds_relation_project_user where user_id=#{userId}
union select id as project_id from t_ds_project where user_id=#{userId})
</select>
<select id="queryProjectWithUserByProcessInstanceId" resultType="org.apache.dolphinscheduler.dao.entity.ProjectUser">
select
dp.id projectId,
dp.name projectName,
u.user_name userName
dp.id project_id,
dp.name project_name,
u.user_name user_name
from t_ds_process_instance di
join t_ds_process_definition dpd on di.process_definition_id = dpd.id
join t_ds_project dp on dpd.project_id = dp.id
join t_ds_process_definition dpd on di.process_definition_code = dpd.code
join t_ds_project dp on dpd.project_code = dp.code
join t_ds_user u on dp.user_id = u.id
where di.id = #{processInstanceId};
</select>

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

@ -33,7 +33,7 @@
</include>
from t_ds_schedules s
join t_ds_process_definition p_f on s.process_definition_id = p_f.id
join t_ds_project as p on p_f.project_id = p.id
join t_ds_project as p on p_f.project_code = p.code
join t_ds_user as u on s.user_id = u.id
where 1=1
<if test="processDefinitionId!= 0">
@ -45,7 +45,7 @@
select p_f.name as process_definition_name, p_f.description as definition_description, p.name as project_name,u.user_name,s.*
from t_ds_schedules s
join t_ds_process_definition p_f on s.process_definition_id = p_f.id
join t_ds_project as p on p_f.project_id = p.id
join t_ds_project as p on p_f.project_code = p.code
join t_ds_user as u on s.user_id = u.id
where p.name = #{projectName}
</select>

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

Loading…
Cancel
Save