Browse Source

Fix resource file usage(Delete Resource/ResourceUser which is deprecated) (#15355)

augit-log
Wenjun Ruan 5 months ago committed by GitHub
parent
commit
5b6b0ceb31
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 42
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ResourcesController.java
  2. 24
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/UsersController.java
  3. 45
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/permission/ResourcePermissionCheckServiceImpl.java
  4. 27
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ResourcesService.java
  5. 10
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/UsersService.java
  6. 249
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ResourcesServiceImpl.java
  7. 4
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/UdfFuncServiceImpl.java
  8. 122
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/UsersServiceImpl.java
  9. 3
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ProjectControllerTest.java
  10. 22
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ResourcesControllerTest.java
  11. 4
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/SchedulerControllerTest.java
  12. 18
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/UsersControllerTest.java
  13. 3
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/v2/ProjectV2ControllerTest.java
  14. 106
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/permission/FilePermissionCheckTest.java
  15. 175
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ResourcesServiceTest.java
  16. 16
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/UdfFuncServiceTest.java
  17. 46
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/UsersServiceTest.java
  18. 3
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/AuthorizationType.java
  19. 4
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/JSONUtils.java
  20. 166
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/Resource.java
  21. 61
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ResourcesUser.java
  22. 3
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinition.java
  23. 3
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java
  24. 157
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.java
  25. 60
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ResourceUserMapper.java
  26. 2
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java
  27. 1
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskInstanceUtils.java
  28. 198
      dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml
  29. 2
      dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_h2.sql
  30. 2
      dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql
  31. 4
      dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql
  32. 440
      dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ResourceMapperTest.java
  33. 111
      dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ResourceUserMapperTest.java
  34. 7
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/builder/TaskExecutionContextBuilder.java
  35. 45
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java
  36. 23
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionContextFactory.java
  37. 8
      dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java
  38. 112
      dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java
  39. 5
      dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/process/ProcessServiceTest.java
  40. 9
      dolphinscheduler-storage-plugin/dolphinscheduler-storage-abs/src/main/java/org/apache/dolphinscheduler/plugin/storage/abs/AbsStorageOperator.java
  41. 12
      dolphinscheduler-storage-plugin/dolphinscheduler-storage-api/src/main/java/org/apache/dolphinscheduler/plugin/storage/api/StorageOperate.java
  42. 9
      dolphinscheduler-storage-plugin/dolphinscheduler-storage-gcs/src/main/java/org/apache/dolphinscheduler/plugin/storage/gcs/GcsStorageOperator.java
  43. 9
      dolphinscheduler-storage-plugin/dolphinscheduler-storage-hdfs/src/main/java/org/apache/dolphinscheduler/plugin/storage/hdfs/HdfsStorageOperator.java
  44. 9
      dolphinscheduler-storage-plugin/dolphinscheduler-storage-obs/src/main/java/org/apache/dolphinscheduler/plugin/storage/obs/ObsStorageOperator.java
  45. 8
      dolphinscheduler-storage-plugin/dolphinscheduler-storage-oss/src/main/java/org/apache/dolphinscheduler/plugin/storage/oss/OssStorageOperator.java
  46. 8
      dolphinscheduler-storage-plugin/dolphinscheduler-storage-s3/src/main/java/org/apache/dolphinscheduler/plugin/storage/s3/S3StorageOperator.java
  47. 7
      dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractCommandExecutor.java
  48. 7
      dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java
  49. 4
      dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/model/ResourceInfo.java
  50. 4
      dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/parameters/resource/UdfFuncParameters.java
  51. 67
      dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/resource/ResourceContext.java
  52. 7
      dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/shell/BaseLinuxShellInterceptorBuilder.java
  53. 3
      dolphinscheduler-task-plugin/dolphinscheduler-task-datax/src/test/java/org/apache/dolphinscheduler/plugin/task/datax/DataxParametersTest.java
  54. 15
      dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtilsTest.java
  55. 4
      dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtils.java
  56. 23
      dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtilsTest.java
  57. 4
      dolphinscheduler-task-plugin/dolphinscheduler-task-hivecli/src/main/java/org/apache/dolphinscheduler/plugin/task/hivecli/HiveCliTask.java
  58. 13
      dolphinscheduler-task-plugin/dolphinscheduler-task-hivecli/src/test/java/org/apache/dolphinscheduler/plugin/task/hivecli/HiveCliTaskTest.java
  59. 16
      dolphinscheduler-task-plugin/dolphinscheduler-task-java/src/main/java/org/apache/dolphinscheduler/plugin/task/java/JavaTask.java
  60. 47
      dolphinscheduler-task-plugin/dolphinscheduler-task-java/src/test/java/org/apache/dolphinscheduler/plugin/task/java/JavaTaskTest.java
  61. 4
      dolphinscheduler-task-plugin/dolphinscheduler-task-mr/src/main/java/org/apache/dolphinscheduler/plugin/task/mr/MapReduceArgsUtils.java
  62. 8
      dolphinscheduler-task-plugin/dolphinscheduler-task-spark/src/main/java/org/apache/dolphinscheduler/plugin/task/spark/SparkTask.java
  63. 14
      dolphinscheduler-task-plugin/dolphinscheduler-task-spark/src/test/java/org/apache/dolphinscheduler/plugin/task/spark/SparkTaskTest.java
  64. 9
      dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java
  65. 4
      dolphinscheduler-tools/src/main/java/org/apache/dolphinscheduler/tools/resource/MigrateResource.java
  66. 54
      dolphinscheduler-tools/src/main/java/org/apache/dolphinscheduler/tools/resource/MigrateResourceService.java
  67. 30
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutor.java
  68. 88
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/utils/TaskExecutionContextUtils.java
  69. 2
      dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/utils/TaskFilesTransferUtils.java

42
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ResourcesController.java

@ -17,9 +17,7 @@
package org.apache.dolphinscheduler.api.controller;
import static org.apache.dolphinscheduler.api.enums.Status.AUTHORIZED_FILE_RESOURCE_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.AUTHORIZED_UDF_FUNCTION_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.AUTHORIZE_RESOURCE_TREE;
import static org.apache.dolphinscheduler.api.enums.Status.CREATE_RESOURCE_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.CREATE_RESOURCE_FILE_ON_LINE_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.CREATE_UDF_FUNCTION_ERROR;
@ -625,46 +623,6 @@ public class ResourcesController extends BaseController {
return udfFuncService.delete(loginUser, udfFuncId);
}
/**
* authorized file resource list
*
* @param loginUser login user
* @param userId user id
* @return authorized result
*/
@Operation(summary = "authorizedFile", description = "AUTHORIZED_FILE_NOTES")
@Parameters({
@Parameter(name = "userId", description = "USER_ID", required = true, schema = @Schema(implementation = int.class, example = "100"))
})
@GetMapping(value = "/authed-file")
@ResponseStatus(HttpStatus.CREATED)
@ApiException(AUTHORIZED_FILE_RESOURCE_ERROR)
public Result authorizedFile(@Parameter(hidden = true) @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@RequestParam("userId") Integer userId) {
Map<String, Object> result = resourceService.authorizedFile(loginUser, userId);
return returnDataList(result);
}
/**
* unauthorized file resource list
*
* @param loginUser login user
* @param userId user id
* @return unauthorized result code
*/
@Operation(summary = "authorizeResourceTree", description = "AUTHORIZE_RESOURCE_TREE_NOTES")
@Parameters({
@Parameter(name = "userId", description = "USER_ID", required = true, schema = @Schema(implementation = int.class, example = "100"))
})
@GetMapping(value = "/authed-resource-tree")
@ResponseStatus(HttpStatus.CREATED)
@ApiException(AUTHORIZE_RESOURCE_TREE)
public Result authorizeResourceTree(@Parameter(hidden = true) @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@RequestParam("userId") Integer userId) {
Map<String, Object> result = resourceService.authorizeResourceTree(loginUser, userId);
return returnDataList(result);
}
/**
* unauthorized udf function
*

24
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/UsersController.java

@ -24,7 +24,6 @@ import static org.apache.dolphinscheduler.api.enums.Status.GET_USER_INFO_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.GRANT_DATASOURCE_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.GRANT_K8S_NAMESPACE_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.GRANT_PROJECT_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.GRANT_RESOURCE_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.GRANT_UDF_FUNCTION_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.QUERY_USER_LIST_PAGING_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.REVOKE_PROJECT_ERROR;
@ -318,29 +317,6 @@ public class UsersController extends BaseController {
return returnDataList(result);
}
/**
* grant resource
*
* @param loginUser login user
* @param userId user id
* @param resourceIds resource id array
* @return grant result code
*/
@Operation(summary = "grantResource", description = "GRANT_RESOURCE_NOTES")
@Parameters({
@Parameter(name = "userId", description = "USER_ID", required = true, schema = @Schema(implementation = int.class, example = "100")),
@Parameter(name = "resourceIds", description = "RESOURCE_IDS", required = true, schema = @Schema(implementation = String.class))
})
@PostMapping(value = "/grant-file")
@ResponseStatus(HttpStatus.OK)
@ApiException(GRANT_RESOURCE_ERROR)
public Result grantResource(@Parameter(hidden = true) @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
@RequestParam(value = "userId") int userId,
@RequestParam(value = "resourceIds") String resourceIds) {
Map<String, Object> result = usersService.grantResources(loginUser, userId, resourceIds);
return returnDataList(result);
}
/**
* grant udf function
*

45
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/permission/ResourcePermissionCheckServiceImpl.java

@ -44,7 +44,6 @@ import org.apache.dolphinscheduler.dao.entity.Environment;
import org.apache.dolphinscheduler.dao.entity.K8sNamespace;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.Queue;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.TaskGroup;
import org.apache.dolphinscheduler.dao.entity.Tenant;
import org.apache.dolphinscheduler.dao.entity.UdfFunc;
@ -58,17 +57,12 @@ import org.apache.dolphinscheduler.dao.mapper.EnvironmentMapper;
import org.apache.dolphinscheduler.dao.mapper.K8sNamespaceMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.apache.dolphinscheduler.dao.mapper.QueueMapper;
import org.apache.dolphinscheduler.dao.mapper.ResourceMapper;
import org.apache.dolphinscheduler.dao.mapper.ResourceUserMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskGroupMapper;
import org.apache.dolphinscheduler.dao.mapper.TenantMapper;
import org.apache.dolphinscheduler.dao.mapper.UdfFuncMapper;
import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.commons.collections4.CollectionUtils;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
@ -207,45 +201,6 @@ public class ResourcePermissionCheckServiceImpl
}
}
@Component
public static class FilePermissionCheck implements ResourceAcquisitionAndPermissionCheck<Integer> {
private final ResourceMapper resourceMapper;
private final ResourceUserMapper resourceUserMapper;
public FilePermissionCheck(ResourceMapper resourceMapper, ResourceUserMapper resourceUserMapper) {
this.resourceMapper = resourceMapper;
this.resourceUserMapper = resourceUserMapper;
}
@Override
public List<AuthorizationType> authorizationTypes() {
return Arrays.asList(AuthorizationType.RESOURCE_FILE_ID, AuthorizationType.UDF_FILE);
}
@Override
public Set<Integer> listAuthorizedResourceIds(int userId, Logger logger) {
List<Resource> relationResources;
if (userId == 0) {
relationResources = new ArrayList<>();
} else {
// query resource relation
List<Integer> resIds = resourceUserMapper.queryResourcesIdListByUserIdAndPerm(userId, 0);
relationResources = CollectionUtils.isEmpty(resIds) ? new ArrayList<>()
: resourceMapper.queryResourceListById(resIds);
}
List<Resource> ownResourceList = resourceMapper.queryResourceListAuthored(userId, -1);
relationResources.addAll(ownResourceList);
return relationResources.stream().map(Resource::getId).collect(toSet());
}
@Override
public boolean permissionCheck(int userId, String permissionKey, Logger logger) {
return true;
}
}
@Component
public static class UdfFuncPermissionCheck implements ResourceAcquisitionAndPermissionCheck<Integer> {

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

@ -193,15 +193,6 @@ public interface ResourcesService {
*/
org.springframework.core.io.Resource downloadResource(User loginUser, String fullName) throws IOException;
/**
* list all file
*
* @param loginUser login user
* @param userId user id
* @return unauthorized result code
*/
Map<String, Object> authorizeResourceTree(User loginUser, Integer userId);
/**
* Get resource by given resource type and full name.
* Useful in Python API create task which need processDefinition information.
@ -219,15 +210,6 @@ public interface ResourcesService {
*/
DeleteDataTransferResponse deleteDataTransferData(User loginUser, Integer days);
/**
* unauthorized file
*
* @param loginUser login user
* @param userId user id
* @return unauthorized result code
*/
Map<String, Object> unauthorizedFile(User loginUser, Integer userId);
/**
* unauthorized udf function
*
@ -246,15 +228,6 @@ public interface ResourcesService {
*/
Map<String, Object> authorizedUDFFunction(User loginUser, Integer userId);
/**
* authorized file
*
* @param loginUser login user
* @param userId user id
* @return authorized result
*/
Map<String, Object> authorizedFile(User loginUser, Integer userId);
/**
* get resource by id
* @param fullName resource full name

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

@ -183,16 +183,6 @@ public interface UsersService {
*/
Map<String, Object> revokeProject(User loginUser, int userId, long projectCode);
/**
* grant resource
*
* @param loginUser login user
* @param userId user id
* @param resourceIds resource id array
* @return grant result code
*/
Map<String, Object> grantResources(User loginUser, int userId, String resourceIds);
/**
* grant udf function
*

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

@ -27,7 +27,6 @@ import static org.apache.dolphinscheduler.common.constants.Constants.JAR;
import static org.apache.dolphinscheduler.common.constants.Constants.PERIOD;
import org.apache.dolphinscheduler.api.dto.resources.DeleteDataTransferResponse;
import org.apache.dolphinscheduler.api.dto.resources.ResourceComponent;
import org.apache.dolphinscheduler.api.dto.resources.filter.ResourceFilter;
import org.apache.dolphinscheduler.api.dto.resources.visitor.ResourceTreeVisitor;
import org.apache.dolphinscheduler.api.dto.resources.visitor.Visitor;
@ -45,14 +44,9 @@ import org.apache.dolphinscheduler.common.enums.ResUploadType;
import org.apache.dolphinscheduler.common.utils.FileUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.PropertyUtils;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.Tenant;
import org.apache.dolphinscheduler.dao.entity.UdfFunc;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionLogMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationMapper;
import org.apache.dolphinscheduler.dao.mapper.ResourceMapper;
import org.apache.dolphinscheduler.dao.mapper.ResourceUserMapper;
import org.apache.dolphinscheduler.dao.mapper.TenantMapper;
import org.apache.dolphinscheduler.dao.mapper.UdfFuncMapper;
import org.apache.dolphinscheduler.dao.mapper.UserMapper;
@ -91,8 +85,6 @@ import org.springframework.transaction.annotation.Transactional;
import org.springframework.web.multipart.MultipartFile;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.SerializationFeature;
import com.google.common.base.Joiner;
import com.google.common.io.Files;
/**
@ -102,9 +94,6 @@ import com.google.common.io.Files;
@Slf4j
public class ResourcesServiceImpl extends BaseServiceImpl implements ResourcesService {
@Autowired
private ResourceMapper resourcesMapper;
@Autowired
private UdfFuncMapper udfFunctionMapper;
@ -114,15 +103,6 @@ public class ResourcesServiceImpl extends BaseServiceImpl implements ResourcesSe
@Autowired
private UserMapper userMapper;
@Autowired
private ResourceUserMapper resourceUserMapper;
@Autowired
private ProcessDefinitionLogMapper processDefinitionLogMapper;
@Autowired
private ProcessTaskRelationMapper processTaskRelationMapper;
@Autowired(required = false)
private StorageOperate storageOperate;
@ -267,36 +247,6 @@ public class ResourcesServiceImpl extends BaseServiceImpl implements ResourcesSe
return result;
}
/**
* update the folder's size of the resource
*
* @param resource the current resource
* @param size size
*/
private void updateParentResourceSize(Resource resource, long size) {
if (resource.getSize() > 0) {
String[] splits = resource.getFullName().split("/");
for (int i = 1; i < splits.length; i++) {
String parentFullName = Joiner.on("/").join(Arrays.copyOfRange(splits, 0, i));
if (StringUtils.isNotBlank(parentFullName)) {
List<Resource> resources =
resourcesMapper.queryResource(parentFullName, resource.getType().ordinal());
if (CollectionUtils.isNotEmpty(resources)) {
Resource parentResource = resources.get(0);
if (parentResource.getSize() + size >= 0) {
parentResource.setSize(parentResource.getSize() + size);
} else {
parentResource.setSize(0L);
}
resourcesMapper.updateById(parentResource);
log.info("Resource size update complete, resourceFullName:{}, newSize:{}.",
parentResource.getFullName(), parentResource.getSize());
}
}
}
}
}
/**
* check resource is exists
*
@ -647,14 +597,10 @@ public class ResourcesServiceImpl extends BaseServiceImpl implements ResourcesSe
}
if (!storageOperate.mkdir(tenantCode, fullName)) {
log.error("create resource directory {} failed", fullName);
putMsg(result, Status.STORE_OPERATE_CREATE_ERROR);
// throw new ServiceException(String.format("create resource directory: %s failed.", fullName));
throw new ServiceException(String.format("Create resource directory: %s failed.", fullName));
}
putMsg(result, Status.SUCCESS);
} catch (Exception e) {
log.error("create resource directory {} failed", fullName);
putMsg(result, Status.STORE_OPERATE_CREATE_ERROR);
throw new ServiceException(String.format("create resource directory: %s failed.", fullName));
}
}
@ -817,25 +763,6 @@ public class ResourcesServiceImpl extends BaseServiceImpl implements ResourcesSe
return result;
}
/**
* transform resource object into StorageEntity object
*
* @param resource a resource object
* @return a storageEntity object
*/
private StorageEntity createStorageEntityBasedOnResource(Resource resource) {
StorageEntity entity = new StorageEntity();
entity.setFullName(resource.getFullName());
entity.setPfullName(resource.getPid() == -1 ? ""
: resourcesMapper.selectById(resource.getPid()).getFullName());
entity.setDirectory(resource.isDirectory());
entity.setAlias(resource.getAlias());
entity.setId(resource.getId());
entity.setType(resource.getType());
return entity;
}
/**
* delete resource
*
@ -1358,7 +1285,7 @@ public class ResourcesServiceImpl extends BaseServiceImpl implements ResourcesSe
log.info("Resource path is {}, download local filename is {}", alias, localFileName);
try {
storageOperate.download(tenantCode, fullName, localFileName, true);
storageOperate.download(fullName, localFileName, true);
ApiServerMetrics.recordApiResourceDownloadSize(java.nio.file.Files.size(Paths.get(localFileName)));
return org.apache.dolphinscheduler.api.utils.FileUtils.file2Resource(localFileName);
} catch (IOException e) {
@ -1368,46 +1295,6 @@ public class ResourcesServiceImpl extends BaseServiceImpl implements ResourcesSe
}
}
/**
* list all file
*
* @param loginUser login user
* @param userId user id
* @return unauthorized result code
*/
@Override
public Map<String, Object> authorizeResourceTree(User loginUser, Integer userId) {
Map<String, Object> result = new HashMap<>();
if (resourcePermissionCheckService.functionDisabled()) {
putMsg(result, Status.FUNCTION_DISABLED);
return result;
}
List<Resource> resourceList;
if (isAdmin(loginUser)) {
// admin gets all resources except userId
resourceList = resourcesMapper.queryResourceExceptUserId(userId);
} else {
// non-admins users get their own resources
resourceList = resourcesMapper.queryResourceListAuthored(loginUser.getId(), -1);
}
List<ResourceComponent> list;
if (CollectionUtils.isNotEmpty(resourceList)) {
// Transform into StorageEntity for compatibility
List<StorageEntity> transformedResourceList = resourceList.stream()
.map(this::createStorageEntityBasedOnResource)
.collect(Collectors.toList());
Visitor visitor = new ResourceTreeVisitor(transformedResourceList);
list = visitor.visit("").getChildren();
} else {
list = new ArrayList<>(0);
}
result.put(Constants.DATA_LIST, list);
putMsg(result, Status.SUCCESS);
return result;
}
@Override
public StorageEntity queryFileStatus(String userName, String fileName) throws Exception {
// TODO: It is used in PythonGateway, should be revised
@ -1469,44 +1356,6 @@ public class ResourcesServiceImpl extends BaseServiceImpl implements ResourcesSe
return result;
}
/**
* unauthorized file
*
* @param loginUser login user
* @param userId user id
* @return unauthorized result code
*/
@Override
public Map<String, Object> unauthorizedFile(User loginUser, Integer userId) {
Map<String, Object> result = new HashMap<>();
List<Resource> resourceList;
if (isAdmin(loginUser)) {
// admin gets all resources except userId
resourceList = resourcesMapper.queryResourceExceptUserId(userId);
} else {
// non-admins users get their own resources
resourceList = resourcesMapper.queryResourceListAuthored(loginUser.getId(), -1);
}
List<Resource> list;
if (resourceList != null && !resourceList.isEmpty()) {
Set<Resource> resourceSet = new HashSet<>(resourceList);
List<Resource> authedResourceList = queryResourceList(userId, Constants.AUTHORIZE_WRITABLE_PERM);
getAuthorizedResourceList(resourceSet, authedResourceList);
list = new ArrayList<>(resourceSet);
} else {
list = new ArrayList<>(0);
}
// Transform into StorageEntity for compatibility
List<StorageEntity> transformedResourceList = list.stream()
.map(this::createStorageEntityBasedOnResource)
.collect(Collectors.toList());
Visitor visitor = new ResourceTreeVisitor(transformedResourceList);
result.put(Constants.DATA_LIST, visitor.visit("").getChildren());
putMsg(result, Status.SUCCESS);
return result;
}
/**
* unauthorized udf function
*
@ -1565,37 +1414,6 @@ public class ResourcesServiceImpl extends BaseServiceImpl implements ResourcesSe
return result;
}
/**
* authorized file
*
* @param loginUser login user
* @param userId user id
* @return authorized result
*/
@Override
public Map<String, Object> authorizedFile(User loginUser, Integer userId) {
Map<String, Object> result = new HashMap<>();
if (resourcePermissionCheckService.functionDisabled()) {
putMsg(result, Status.FUNCTION_DISABLED);
return result;
}
List<Resource> authedResources = queryResourceList(userId, Constants.AUTHORIZE_WRITABLE_PERM);
// Transform into StorageEntity for compatibility
List<StorageEntity> transformedResourceList = authedResources.stream()
.map(this::createStorageEntityBasedOnResource)
.collect(Collectors.toList());
Visitor visitor = new ResourceTreeVisitor(transformedResourceList);
String visit = JSONUtils.toJsonString(visitor.visit(""), SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS);
log.info(visit);
String jsonTreeStr =
JSONUtils.toJsonString(visitor.visit("").getChildren(), SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS);
log.info(jsonTreeStr);
result.put(Constants.DATA_LIST, visitor.visit("").getChildren());
putMsg(result, Status.SUCCESS);
return result;
}
/**
* get resource base dir
*
@ -1649,69 +1467,6 @@ public class ResourcesServiceImpl extends BaseServiceImpl implements ResourcesSe
}
}
/**
* list all children id
*
* @param resource resource
* @param containSelf whether add self to children list
* @return all children id
*/
List<Integer> listAllChildren(Resource resource, boolean containSelf) {
List<Integer> childList = new ArrayList<>();
if (resource.getId() != null && containSelf) {
childList.add(resource.getId());
}
if (resource.isDirectory()) {
listAllChildren(resource.getId(), childList);
}
return childList;
}
/**
* list all children id
*
* @param resourceId resource id
* @param childList child list
*/
void listAllChildren(int resourceId, List<Integer> childList) {
List<Integer> children = resourcesMapper.listChildren(resourceId);
for (int childId : children) {
childList.add(childId);
listAllChildren(childId, childList);
}
}
/**
* query authored resource list (own and authorized)
*
* @param loginUser login user
* @param type ResourceType
* @return all authored resource list
*/
private List<Resource> queryAuthoredResourceList(User loginUser, ResourceType type) {
Set<Integer> resourceIds = resourcePermissionCheckService
.userOwnedResourceIdsAcquisition(checkResourceType(type), loginUser.getId(), log);
if (resourceIds.isEmpty()) {
return Collections.emptyList();
}
List<Resource> resources = resourcesMapper.selectBatchIds(resourceIds);
resources = resources.stream().filter(rs -> rs.getType() == type).collect(Collectors.toList());
return resources;
}
/**
* query resource list by userId and perm
*
* @param userId userId
* @param perm perm
* @return resource list
*/
private List<Resource> queryResourceList(Integer userId, int perm) {
List<Integer> resIds = resourceUserMapper.queryResourcesIdListByUserIdAndPerm(userId, perm);
return CollectionUtils.isEmpty(resIds) ? new ArrayList<>() : resourcesMapper.queryResourceListById(resIds);
}
private AuthorizationType checkResourceType(ResourceType type) {
return type.equals(ResourceType.FILE) ? AuthorizationType.RESOURCE_FILE_ID : AuthorizationType.UDF_FILE;
}

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

@ -26,7 +26,6 @@ import org.apache.dolphinscheduler.common.enums.AuthorizationType;
import org.apache.dolphinscheduler.common.enums.UdfType;
import org.apache.dolphinscheduler.dao.entity.UdfFunc;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.ResourceMapper;
import org.apache.dolphinscheduler.dao.mapper.UDFUserMapper;
import org.apache.dolphinscheduler.dao.mapper.UdfFuncMapper;
import org.apache.dolphinscheduler.plugin.storage.api.StorageOperate;
@ -56,9 +55,6 @@ import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
@Slf4j
public class UdfFuncServiceImpl extends BaseServiceImpl implements UdfFuncService {
@Autowired
private ResourceMapper resourceMapper;
@Autowired
private UdfFuncMapper udfFuncMapper;

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

@ -38,8 +38,6 @@ import org.apache.dolphinscheduler.dao.entity.DatasourceUser;
import org.apache.dolphinscheduler.dao.entity.K8sNamespaceUser;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.ProjectUser;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.ResourcesUser;
import org.apache.dolphinscheduler.dao.entity.Tenant;
import org.apache.dolphinscheduler.dao.entity.UDFUser;
import org.apache.dolphinscheduler.dao.entity.User;
@ -47,15 +45,11 @@ import org.apache.dolphinscheduler.dao.mapper.AccessTokenMapper;
import org.apache.dolphinscheduler.dao.mapper.AlertGroupMapper;
import org.apache.dolphinscheduler.dao.mapper.DataSourceUserMapper;
import org.apache.dolphinscheduler.dao.mapper.K8sNamespaceUserMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectUserMapper;
import org.apache.dolphinscheduler.dao.mapper.ResourceMapper;
import org.apache.dolphinscheduler.dao.mapper.ResourceUserMapper;
import org.apache.dolphinscheduler.dao.mapper.TenantMapper;
import org.apache.dolphinscheduler.dao.mapper.UDFUserMapper;
import org.apache.dolphinscheduler.dao.mapper.UserMapper;
import org.apache.dolphinscheduler.dao.utils.ResourceProcessDefinitionUtils;
import org.apache.dolphinscheduler.plugin.storage.api.StorageOperate;
import org.apache.commons.collections4.CollectionUtils;
@ -103,12 +97,6 @@ public class UsersServiceImpl extends BaseServiceImpl implements UsersService {
@Autowired
private ProjectUserMapper projectUserMapper;
@Autowired
private ResourceUserMapper resourceUserMapper;
@Autowired
private ResourceMapper resourceMapper;
@Autowired
private DataSourceUserMapper datasourceUserMapper;
@ -118,9 +106,6 @@ public class UsersServiceImpl extends BaseServiceImpl implements UsersService {
@Autowired
private AlertGroupMapper alertGroupMapper;
@Autowired
private ProcessDefinitionMapper processDefinitionMapper;
@Autowired
private ProjectMapper projectMapper;
@ -754,113 +739,6 @@ public class UsersServiceImpl extends BaseServiceImpl implements UsersService {
return result;
}
/**
* grant resource
*
* @param loginUser login user
* @param userId user id
* @param resourceIds resource id array
* @return grant result code
*/
@Override
@Transactional
public Map<String, Object> grantResources(User loginUser, int userId, String resourceIds) {
Map<String, Object> result = new HashMap<>();
if (resourcePermissionCheckService.functionDisabled()) {
putMsg(result, Status.FUNCTION_DISABLED);
return result;
}
User user = userMapper.selectById(userId);
if (user == null) {
log.error("User does not exist, userId:{}.", userId);
putMsg(result, Status.USER_NOT_EXIST, userId);
return result;
}
Set<Integer> needAuthorizeResIds = new HashSet<>();
if (StringUtils.isNotBlank(resourceIds)) {
String[] resourceFullIdArr = resourceIds.split(",");
// need authorize resource id set
for (String resourceFullId : resourceFullIdArr) {
String[] resourceIdArr = resourceFullId.split("-");
for (int i = 0; i <= resourceIdArr.length - 1; i++) {
int resourceIdValue = Integer.parseInt(resourceIdArr[i]);
needAuthorizeResIds.add(resourceIdValue);
}
}
}
// get the authorized resource id list by user id
List<Integer> resIds =
resourceUserMapper.queryResourcesIdListByUserIdAndPerm(userId, Constants.AUTHORIZE_WRITABLE_PERM);
List<Resource> oldAuthorizedRes =
CollectionUtils.isEmpty(resIds) ? new ArrayList<>() : resourceMapper.queryResourceListById(resIds);
// if resource type is UDF,need check whether it is bound by UDF function
Set<Integer> oldAuthorizedResIds = oldAuthorizedRes.stream().map(Resource::getId).collect(Collectors.toSet());
// get the unauthorized resource id list
oldAuthorizedResIds.removeAll(needAuthorizeResIds);
if (CollectionUtils.isNotEmpty(oldAuthorizedResIds)) {
// get all resource id of process definitions those are released
List<Map<String, Object>> list = processDefinitionMapper.listResourcesByUser(userId);
Map<Integer, Set<Long>> resourceProcessMap =
ResourceProcessDefinitionUtils.getResourceProcessDefinitionMap(list);
Set<Integer> resourceIdSet = resourceProcessMap.keySet();
resourceIdSet.retainAll(oldAuthorizedResIds);
if (CollectionUtils.isNotEmpty(resourceIdSet)) {
for (Integer resId : resourceIdSet) {
log.error("Resource id:{} is used by process definition {}", resId,
resourceProcessMap.get(resId));
}
putMsg(result, Status.RESOURCE_IS_USED);
return result;
}
}
resourceUserMapper.deleteResourceUser(userId, 0);
if (check(result, StringUtils.isEmpty(resourceIds), Status.SUCCESS)) {
log.warn("Parameter resourceIds is empty.");
return result;
}
for (int resourceIdValue : needAuthorizeResIds) {
Resource resource = resourceMapper.selectById(resourceIdValue);
if (resource == null) {
log.error("Resource does not exist, resourceId:{}.", resourceIdValue);
putMsg(result, Status.RESOURCE_NOT_EXIST);
return result;
}
Date now = new Date();
ResourcesUser resourcesUser = new ResourcesUser();
resourcesUser.setUserId(userId);
resourcesUser.setResourcesId(resourceIdValue);
if (resource.isDirectory()) {
resourcesUser.setPerm(Constants.AUTHORIZE_READABLE_PERM);
} else {
resourcesUser.setPerm(Constants.AUTHORIZE_WRITABLE_PERM);
}
resourcesUser.setCreateTime(now);
resourcesUser.setUpdateTime(now);
resourceUserMapper.insert(resourcesUser);
}
log.info("User is granted permission for resources, userId:{}, resourceIds:{}.", user.getId(),
needAuthorizeResIds);
putMsg(result, Status.SUCCESS);
return result;
}
/**
* grant udf function
*

3
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ProjectControllerTest.java

@ -23,7 +23,6 @@ import org.apache.dolphinscheduler.api.utils.PageInfo;
import org.apache.dolphinscheduler.api.utils.Result;
import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
@ -97,7 +96,7 @@ public class ProjectControllerTest {
int pageSize = 10;
String searchVal = "";
Result result = Result.success(new PageInfo<Resource>(1, 10));
Result result = Result.success(new PageInfo<Project>(1, 10));
Mockito.when(projectService.queryProjectListPaging(user, pageSize, pageNo, searchVal)).thenReturn(result);
Result response = projectController.queryProjectListPaging(user, searchVal, pageSize, pageNo);

22
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ResourcesControllerTest.java

@ -397,28 +397,6 @@ public class ResourcesControllerTest extends AbstractControllerTest {
logger.info(mvcResult.getResponse().getContentAsString());
}
@Test
public void testAuthorizedFile() throws Exception {
Map<String, Object> mockResult = new HashMap<>();
mockResult.put(Constants.STATUS, Status.SUCCESS);
Mockito.when(resourcesService.authorizedFile(Mockito.any(), Mockito.anyInt())).thenReturn(mockResult);
MultiValueMap<String, String> paramsMap = new LinkedMultiValueMap<>();
paramsMap.add("userId", "2");
MvcResult mvcResult = mockMvc.perform(get("/resources/authed-file")
.header(SESSION_ID, sessionId)
.params(paramsMap))
.andExpect(status().isCreated())
.andExpect(content().contentType(MediaType.APPLICATION_JSON))
.andReturn();
Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class);
Assertions.assertEquals(Status.SUCCESS.getCode(), result.getCode().intValue());
logger.info(mvcResult.getResponse().getContentAsString());
}
@Test
public void testAuthorizedUDFFunction() throws Exception {
Map<String, Object> mockResult = new HashMap<>();

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

@ -34,7 +34,7 @@ import org.apache.dolphinscheduler.common.enums.FailureStrategy;
import org.apache.dolphinscheduler.common.enums.Priority;
import org.apache.dolphinscheduler.common.enums.WarningType;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.Schedule;
import org.apache.dolphinscheduler.dao.entity.User;
import org.junit.jupiter.api.Assertions;
@ -163,7 +163,7 @@ public class SchedulerControllerTest extends AbstractControllerTest {
paramsMap.add("pageNo", "1");
paramsMap.add("pageSize", "30");
PageInfo<Resource> pageInfo = new PageInfo<>(1, 10);
PageInfo<Schedule> pageInfo = new PageInfo<>(1, 10);
Result mockResult = Result.success(pageInfo);
Mockito.when(schedulerService.querySchedule(isA(User.class), isA(Long.class), isA(Long.class),

18
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/UsersControllerTest.java

@ -145,24 +145,6 @@ public class UsersControllerTest extends AbstractControllerTest {
logger.info(mvcResult.getResponse().getContentAsString());
}
@Test
public void testGrantResource() throws Exception {
MultiValueMap<String, String> paramsMap = new LinkedMultiValueMap<>();
paramsMap.add("userId", "32");
paramsMap.add("resourceIds", "5");
MvcResult mvcResult = mockMvc.perform(post("/users/grant-file")
.header(SESSION_ID, sessionId)
.params(paramsMap))
.andExpect(status().isOk())
.andExpect(content().contentType(MediaType.APPLICATION_JSON))
.andReturn();
Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class);
Assertions.assertEquals(Status.USER_NOT_EXIST.getCode(), result.getCode().intValue());
logger.info(mvcResult.getResponse().getContentAsString());
}
@Test
public void testGrantUDFFunc() throws Exception {
MultiValueMap<String, String> paramsMap = new LinkedMultiValueMap<>();

3
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/v2/ProjectV2ControllerTest.java

@ -27,7 +27,6 @@ import org.apache.dolphinscheduler.api.utils.PageInfo;
import org.apache.dolphinscheduler.api.utils.Result;
import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
@ -101,7 +100,7 @@ public class ProjectV2ControllerTest {
projectQueryReq.setPageNo(1);
projectQueryReq.setPageSize(10);
Result result = Result.success(new PageInfo<Resource>(1, 10));
Result result = Result.success(new PageInfo<Project>(1, 10));
Mockito.when(projectService.queryProjectListPaging(user, projectQueryReq.getPageSize(),
projectQueryReq.getPageNo(), projectQueryReq.getSearchVal())).thenReturn(result);
Result response = projectV2Controller.queryProjectListPaging(user, projectQueryReq);

106
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/permission/FilePermissionCheckTest.java

@ -1,106 +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.permission;
import org.apache.dolphinscheduler.common.enums.AuthorizationType;
import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.ResourceMapper;
import org.apache.dolphinscheduler.dao.mapper.ResourceUserMapper;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.InjectMocks;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ExtendWith(MockitoExtension.class)
public class FilePermissionCheckTest {
private static final Logger logger = LoggerFactory.getLogger(FilePermissionCheckTest.class);
@InjectMocks
private ResourcePermissionCheckServiceImpl.FilePermissionCheck filePermissionCheck;
@Mock
private ResourceMapper resourceMapper;
@Mock
private ResourceUserMapper resourceUserMapper;
@Test
public void testPermissionCheck() {
User user = getLoginUser();
Assertions.assertTrue(filePermissionCheck.permissionCheck(user.getId(), null, logger));
}
@Test
public void testAuthorizationTypes() {
List<AuthorizationType> authorizationTypes = filePermissionCheck.authorizationTypes();
Assertions.assertEquals(Arrays.asList(AuthorizationType.RESOURCE_FILE_ID, AuthorizationType.UDF_FILE),
authorizationTypes);
}
@Test
public void testListAuthorizedResourceIds() {
// ADMIN_USER
User user = getAdminUser();
Resource resource = new Resource();
Set<Integer> ids = new HashSet();
ids.add(resource.getId());
List<Resource> resources = Arrays.asList(resource);
Mockito.when(resourceMapper.queryResourceListAuthored(user.getId(), -1)).thenReturn(resources);
Assertions.assertEquals(ids, filePermissionCheck.listAuthorizedResourceIds(user.getId(), logger));
// GENERAL_USER
user = getLoginUser();
Resource resource1 = new Resource();
ids.add(resource1.getId());
Mockito.when(resourceMapper.queryResourceListAuthored(user.getId(), -1)).thenReturn(resources);
Mockito.when(resourceUserMapper.queryResourcesIdListByUserIdAndPerm(user.getId(), 0))
.thenReturn(Arrays.asList(resource1.getId()));
Assertions.assertEquals(ids, filePermissionCheck.listAuthorizedResourceIds(user.getId(), logger));
}
private User getLoginUser() {
User loginUser = new User();
loginUser.setUserType(UserType.GENERAL_USER);
loginUser.setUserName("test");
loginUser.setId(1);
return loginUser;
}
private User getAdminUser() {
User loginUser = new User();
loginUser.setUserType(UserType.ADMIN_USER);
loginUser.setUserName("test");
loginUser.setId(0);
return loginUser;
}
}

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

@ -20,6 +20,7 @@ package org.apache.dolphinscheduler.api.service;
import static org.mockito.ArgumentMatchers.eq;
import org.apache.dolphinscheduler.api.dto.resources.DeleteDataTransferResponse;
import org.apache.dolphinscheduler.api.dto.resources.ResourceComponent;
import org.apache.dolphinscheduler.api.enums.Status;
import org.apache.dolphinscheduler.api.exceptions.ServiceException;
import org.apache.dolphinscheduler.api.permission.ResourcePermissionCheckService;
@ -31,13 +32,10 @@ import org.apache.dolphinscheduler.common.constants.Constants;
import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.common.utils.FileUtils;
import org.apache.dolphinscheduler.common.utils.PropertyUtils;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.Tenant;
import org.apache.dolphinscheduler.dao.entity.UdfFunc;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
import org.apache.dolphinscheduler.dao.mapper.ResourceMapper;
import org.apache.dolphinscheduler.dao.mapper.ResourceUserMapper;
import org.apache.dolphinscheduler.dao.mapper.TenantMapper;
import org.apache.dolphinscheduler.dao.mapper.UdfFuncMapper;
import org.apache.dolphinscheduler.dao.mapper.UserMapper;
@ -77,8 +75,6 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.mock.web.MockMultipartFile;
import com.baomidou.mybatisplus.core.metadata.IPage;
import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
import com.google.common.io.Files;
/**
@ -93,9 +89,6 @@ public class ResourcesServiceTest {
@InjectMocks
private ResourcesServiceImpl resourcesService;
@Mock
private ResourceMapper resourcesMapper;
@Mock
private TenantMapper tenantMapper;
@ -111,9 +104,6 @@ public class ResourcesServiceTest {
@Mock
private ProcessDefinitionMapper processDefinitionMapper;
@Mock
private ResourceUserMapper resourceUserMapper;
@Mock
private ResourcePermissionCheckService resourcePermissionCheckService;
@ -228,7 +218,6 @@ public class ResourcesServiceTest {
} catch (IOException e) {
logger.error(e.getMessage(), e);
}
Mockito.when(resourcesMapper.existResource("/directoryTest", 0)).thenReturn(true);
Result result = resourcesService.createDirectory(user, "directoryTest", ResourceType.FILE, -1, "/");
logger.info(result.toString());
Assertions.assertEquals(Status.RESOURCE_EXIST.getMsg(), result.getMsg());
@ -325,10 +314,6 @@ public class ResourcesServiceTest {
loginUser.setTenantId(1);
loginUser.setTenantCode("tenant1");
loginUser.setUserType(UserType.ADMIN_USER);
IPage<Resource> resourcePage = new Page<>(1, 10);
resourcePage.setTotal(1);
resourcePage.setRecords(getResourceList());
List<StorageEntity> mockResList = new ArrayList<StorageEntity>();
mockResList.add(getStorageEntityResource());
List<User> mockUserList = new ArrayList<User>();
@ -371,7 +356,7 @@ public class ResourcesServiceTest {
Map<String, Object> result = resourcesService.queryResourceList(loginUser, ResourceType.FILE, "");
logger.info(result.toString());
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
List<Resource> resourceList = (List<Resource>) result.get(Constants.DATA_LIST);
List<ResourceComponent> resourceList = (List<ResourceComponent>) result.get(Constants.DATA_LIST);
Assertions.assertTrue(CollectionUtils.isNotEmpty(resourceList));
// test udf
@ -385,7 +370,7 @@ public class ResourcesServiceTest {
result = resourcesService.queryResourceList(loginUser, ResourceType.UDF, "");
logger.info(result.toString());
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
resourceList = (List<Resource>) result.get(Constants.DATA_LIST);
resourceList = (List<ResourceComponent>) result.get(Constants.DATA_LIST);
Assertions.assertTrue(CollectionUtils.isNotEmpty(resourceList));
}
@ -630,66 +615,6 @@ public class ResourcesServiceTest {
}
@Test
public void testAuthorizeResourceTree() {
User user = getUser();
user.setId(1);
user.setUserType(UserType.ADMIN_USER);
int userId = 3;
// test admin user
List<Integer> resIds = new ArrayList<>();
resIds.add(1);
Mockito.when(resourcePermissionCheckService.functionDisabled()).thenReturn(false);
Mockito.when(resourcesMapper.queryResourceExceptUserId(userId)).thenReturn(getResourceList());
Map<String, Object> result = resourcesService.authorizeResourceTree(user, userId);
logger.info(result.toString());
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
List<Resource> resources = (List<Resource>) result.get(Constants.DATA_LIST);
Assertions.assertTrue(CollectionUtils.isNotEmpty(resources));
// test non-admin user
user.setId(2);
user.setUserType(UserType.GENERAL_USER);
Mockito.when(resourcesMapper.queryResourceListAuthored(user.getId(), -1)).thenReturn(getResourceList());
result = resourcesService.authorizeResourceTree(user, userId);
logger.info(result.toString());
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
resources = (List<Resource>) result.get(Constants.DATA_LIST);
Assertions.assertTrue(CollectionUtils.isNotEmpty(resources));
}
@Test
public void testUnauthorizedFile() {
User user = getUser();
user.setId(1);
user.setUserType(UserType.ADMIN_USER);
int userId = 3;
// test admin user
List<Integer> resIds = new ArrayList<>();
resIds.add(1);
Mockito.when(resourcesMapper.queryResourceExceptUserId(userId)).thenReturn(getResourceList());
Mockito.when(resourceUserMapper.queryResourcesIdListByUserIdAndPerm(Mockito.anyInt(), Mockito.anyInt()))
.thenReturn(resIds);
Mockito.when(resourcesMapper.queryResourceListById(Mockito.any())).thenReturn(getSingleResourceList());
Map<String, Object> result = resourcesService.unauthorizedFile(user, userId);
logger.info(result.toString());
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
List<Resource> resources = (List<Resource>) result.get(Constants.DATA_LIST);
Assertions.assertTrue(CollectionUtils.isNotEmpty(resources));
// test non-admin user
user.setId(2);
user.setUserType(UserType.GENERAL_USER);
Mockito.when(resourcesMapper.queryResourceListAuthored(user.getId(), -1)).thenReturn(getResourceList());
result = resourcesService.unauthorizedFile(user, userId);
logger.info(result.toString());
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
resources = (List<Resource>) result.get(Constants.DATA_LIST);
Assertions.assertTrue(CollectionUtils.isNotEmpty(resources));
}
@Test
public void testUnauthorizedUDFFunction() {
User user = getUser();
@ -745,39 +670,6 @@ public class ResourcesServiceTest {
Assertions.assertTrue(CollectionUtils.isNotEmpty(udfFuncs));
}
@Test
public void testAuthorizedFile() {
User user = getUser();
user.setId(1);
user.setUserType(UserType.ADMIN_USER);
int userId = 3;
// test admin user
List<Integer> resIds = new ArrayList<>();
resIds.add(1);
Mockito.when(resourcePermissionCheckService.functionDisabled()).thenReturn(false);
Mockito.when(resourceUserMapper.queryResourcesIdListByUserIdAndPerm(Mockito.anyInt(), Mockito.anyInt()))
.thenReturn(resIds);
Mockito.when(resourcesMapper.queryResourceListById(Mockito.any())).thenReturn(getResourceList());
Map<String, Object> result = resourcesService.authorizedFile(user, userId);
logger.info(result.toString());
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
List<Resource> resources = (List<Resource>) result.get(Constants.DATA_LIST);
Assertions.assertTrue(CollectionUtils.isNotEmpty(resources));
// test non-admin user
user.setId(2);
user.setUserType(UserType.GENERAL_USER);
Mockito.when(resourceUserMapper.queryResourcesIdListByUserIdAndPerm(Mockito.anyInt(), Mockito.anyInt()))
.thenReturn(resIds);
Mockito.when(resourcesMapper.queryResourceListById(Mockito.any())).thenReturn(getResourceList());
result = resourcesService.authorizedFile(user, userId);
logger.info(result.toString());
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
resources = (List<Resource>) result.get(Constants.DATA_LIST);
Assertions.assertTrue(CollectionUtils.isNotEmpty(resources));
}
@Test
public void testDeleteDataTransferData() throws Exception {
User user = getUser();
@ -853,15 +745,6 @@ public class ResourcesServiceTest {
Assertions.assertEquals(Status.SUCCESS.getMsg(), result.getMsg());
}
private List<Resource> getResourceList() {
List<Resource> resources = new ArrayList<>();
resources.add(getResource(1));
resources.add(getResource(2));
resources.add(getResource(3));
return resources;
}
private Set<Integer> getSetIds() {
Set<Integer> resources = new HashSet<>();
@ -869,27 +752,12 @@ public class ResourcesServiceTest {
return resources;
}
private List<Resource> getSingleResourceList() {
return Collections.singletonList(getResource(1));
}
private Tenant getTenant() {
Tenant tenant = new Tenant();
tenant.setTenantCode("123");
return tenant;
}
private Resource getResource() {
Resource resource = new Resource();
resource.setPid(-1);
resource.setUserId(1);
resource.setDescription("ResourcesServiceTest.jar");
resource.setAlias("ResourcesServiceTest.jar");
resource.setFullName("/ResourcesServiceTest.jar");
resource.setType(ResourceType.FILE);
return resource;
}
private StorageEntity getStorageEntityResource() {
StorageEntity entity = new StorageEntity();
entity.setAlias("ResourcesServiceTest");
@ -902,43 +770,6 @@ public class ResourcesServiceTest {
return entity;
}
private Resource getResource(int resourceId) {
Resource resource = new Resource();
resource.setId(resourceId);
resource.setPid(-1);
resource.setUserId(1);
resource.setDescription("ResourcesServiceTest.jar");
resource.setAlias("ResourcesServiceTest.jar");
resource.setFullName("/ResourcesServiceTest.jar");
resource.setType(ResourceType.FILE);
return resource;
}
private Resource getResource(int resourceId, ResourceType type) {
Resource resource = new Resource();
resource.setId(resourceId);
resource.setPid(-1);
resource.setUserId(1);
resource.setDescription("ResourcesServiceTest.jar");
resource.setAlias("ResourcesServiceTest.jar");
resource.setFullName("/ResourcesServiceTest.jar");
resource.setType(type);
return resource;
}
private Resource getUdfResource() {
Resource resource = new Resource();
resource.setUserId(1);
resource.setDescription("udfTest");
resource.setAlias("udfTest.jar");
resource.setFullName("/udfTest.jar");
resource.setType(ResourceType.UDF);
return resource;
}
private StorageEntity getStorageEntityUdfResource() {
StorageEntity entity = new StorageEntity();
entity.setAlias("ResourcesServiceTest1.jar");

16
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/UdfFuncServiceTest.java

@ -28,10 +28,8 @@ import org.apache.dolphinscheduler.common.enums.AuthorizationType;
import org.apache.dolphinscheduler.common.enums.UdfType;
import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.common.utils.PropertyUtils;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.UdfFunc;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.ResourceMapper;
import org.apache.dolphinscheduler.dao.mapper.UDFUserMapper;
import org.apache.dolphinscheduler.dao.mapper.UdfFuncMapper;
import org.apache.dolphinscheduler.plugin.storage.api.StorageOperate;
@ -78,9 +76,6 @@ public class UdfFuncServiceTest {
@InjectMocks
private UdfFuncServiceImpl udfFuncService;
@Mock
private ResourceMapper resourceMapper;
@Mock
private UdfFuncMapper udfFuncMapper;
@ -291,17 +286,6 @@ public class UdfFuncServiceTest {
return loginUser;
}
/**
* get resourceId
*/
private Resource getResource() {
Resource resource = new Resource();
resource.setId(1);
resource.setAlias("test");
return resource;
}
private List<UdfFunc> getList() {
List<UdfFunc> udfFuncList = new ArrayList<>();
udfFuncList.add(getUdfFunc());

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

@ -37,7 +37,6 @@ import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.common.utils.EncryptionUtils;
import org.apache.dolphinscheduler.dao.entity.AlertGroup;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.Tenant;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.dao.mapper.AccessTokenMapper;
@ -46,13 +45,10 @@ import org.apache.dolphinscheduler.dao.mapper.DataSourceUserMapper;
import org.apache.dolphinscheduler.dao.mapper.K8sNamespaceUserMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectUserMapper;
import org.apache.dolphinscheduler.dao.mapper.ResourceMapper;
import org.apache.dolphinscheduler.dao.mapper.ResourceUserMapper;
import org.apache.dolphinscheduler.dao.mapper.TenantMapper;
import org.apache.dolphinscheduler.dao.mapper.UDFUserMapper;
import org.apache.dolphinscheduler.dao.mapper.UserMapper;
import org.apache.dolphinscheduler.plugin.storage.api.StorageOperate;
import org.apache.dolphinscheduler.spi.enums.ResourceType;
import org.apache.commons.collections4.CollectionUtils;
@ -99,9 +95,6 @@ public class UsersServiceTest {
@Mock
private TenantMapper tenantMapper;
@Mock
private ResourceMapper resourceMapper;
@Mock
private AlertGroupMapper alertGroupMapper;
@ -111,9 +104,6 @@ public class UsersServiceTest {
@Mock
private ProjectUserMapper projectUserMapper;
@Mock
private ResourceUserMapper resourceUserMapper;
@Mock
private MetricsCleanUpService metricsCleanUpService;
@ -526,26 +516,6 @@ public class UsersServiceTest {
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
}
@Test
public void testGrantResources() {
String resourceIds = "100000,120000";
when(userMapper.selectById(1)).thenReturn(getUser());
User loginUser = new User();
// user not exist
loginUser.setUserType(UserType.ADMIN_USER);
Map<String, Object> result = usersService.grantResources(loginUser, 2, resourceIds);
logger.info(result.toString());
Assertions.assertEquals(Status.USER_NOT_EXIST, result.get(Constants.STATUS));
// success
when(resourceMapper.selectById(Mockito.anyInt())).thenReturn(getResource());
when(resourceUserMapper.deleteResourceUser(1, 0)).thenReturn(1);
result = usersService.grantResources(loginUser, 1, resourceIds);
logger.info(result.toString());
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
}
@Test
public void testGrantUDFFunction() {
String udfIds = "100000,120000";
@ -931,22 +901,6 @@ public class UsersServiceTest {
return tenant;
}
/**
* get resource
*
* @return resource
*/
private Resource getResource() {
Resource resource = new Resource();
resource.setPid(-1);
resource.setUserId(1);
resource.setDescription("ResourcesServiceTest.jar");
resource.setAlias("ResourcesServiceTest.jar");
resource.setFullName("/ResourcesServiceTest.jar");
resource.setType(ResourceType.FILE);
return resource;
}
private List<AlertGroup> getAlertGroups() {
List<AlertGroup> alertGroups = new ArrayList<>();
AlertGroup alertGroup = new AlertGroup();

3
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/AuthorizationType.java

@ -44,8 +44,11 @@ public enum AuthorizationType {
* 16 USER;
* 17 Data_Quality;
*/
@Deprecated
RESOURCE_FILE_ID(0, "resource file id"),
@Deprecated
RESOURCE_FILE_NAME(1, "resource file name"),
@Deprecated
UDF_FILE(2, "udf file"),
DATASOURCE(3, "data source"),
UDF(4, "udf function"),

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

@ -21,6 +21,7 @@ import static com.fasterxml.jackson.databind.DeserializationFeature.ACCEPT_EMPTY
import static com.fasterxml.jackson.databind.DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES;
import static com.fasterxml.jackson.databind.DeserializationFeature.READ_UNKNOWN_ENUM_VALUES_AS_NULL;
import static com.fasterxml.jackson.databind.MapperFeature.REQUIRE_SETTERS_FOR_GETTERS;
import static com.fasterxml.jackson.databind.SerializationFeature.FAIL_ON_EMPTY_BEANS;
import static java.nio.charset.StandardCharsets.UTF_8;
import static org.apache.dolphinscheduler.common.constants.DateConstants.YYYY_MM_DD_HH_MM_SS;
@ -65,7 +66,7 @@ import com.google.common.base.Strings;
* json utils
*/
@Slf4j
public class JSONUtils {
public final class JSONUtils {
static {
log.info("init timezone: {}", TimeZone.getDefault());
@ -76,6 +77,7 @@ public class JSONUtils {
.configure(ACCEPT_EMPTY_ARRAY_AS_NULL_OBJECT, true)
.configure(READ_UNKNOWN_ENUM_VALUES_AS_NULL, true)
.configure(REQUIRE_SETTERS_FOR_GETTERS, true)
.configure(FAIL_ON_EMPTY_BEANS, false)
.addModule(new SimpleModule()
.addSerializer(LocalDateTime.class, new LocalDateTimeSerializer())
.addDeserializer(LocalDateTime.class, new LocalDateTimeDeserializer()))

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

@ -1,166 +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 org.apache.dolphinscheduler.spi.enums.ResourceType;
import java.util.Date;
import java.util.Objects;
import lombok.Data;
import lombok.NoArgsConstructor;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableField;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
@Data
@NoArgsConstructor
@TableName("t_ds_resources")
public class Resource {
/**
* id
*/
@TableId(value = "id", type = IdType.AUTO)
private Integer id;
/**
* parent id
*/
private int pid;
/**
* resource alias
*/
private String alias;
/**
* full name
*/
private String fullName;
/**
* is directory
*/
private boolean isDirectory = false;
/**
* description
*/
private String description;
/**
* file alias
*/
private String fileName;
/**
* user id
*/
private int userId;
/**
* resource type
*/
private ResourceType type;
/**
* resource size
*/
private long size;
/**
* create time
*/
private Date createTime;
/**
* update time
*/
private Date updateTime;
/**
* user name
*/
@TableField(exist = false)
private String userName;
public Resource(int id, String alias, String fileName, String description, int userId,
ResourceType type, long size,
Date createTime, Date updateTime) {
this.id = id;
this.alias = alias;
this.fileName = fileName;
this.description = description;
this.userId = userId;
this.type = type;
this.size = size;
this.createTime = createTime;
this.updateTime = updateTime;
}
public Resource(int id, int pid, String alias, String fullName, boolean isDirectory) {
this.id = id;
this.pid = pid;
this.alias = alias;
this.fullName = fullName;
this.isDirectory = isDirectory;
}
public Resource(int pid, String alias, String fullName, boolean isDirectory, String description, String fileName,
int userId, ResourceType type, long size, Date createTime, Date updateTime) {
this.pid = pid;
this.alias = alias;
this.fullName = fullName;
this.isDirectory = isDirectory;
this.description = description;
this.fileName = fileName;
this.userId = userId;
this.type = type;
this.size = size;
this.createTime = createTime;
this.updateTime = updateTime;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Resource resource = (Resource) o;
if (!Objects.equals(id, resource.id)) {
return false;
}
return alias.equals(resource.alias);
}
@Override
public int hashCode() {
int result = id;
result = 31 * result + alias.hashCode();
return result;
}
}

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

@ -1,61 +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 lombok.Data;
import com.baomidou.mybatisplus.annotation.IdType;
import com.baomidou.mybatisplus.annotation.TableId;
import com.baomidou.mybatisplus.annotation.TableName;
@Data
@TableName("t_ds_relation_resources_user")
public class ResourcesUser {
/**
* id
*/
@TableId(value = "id", type = IdType.AUTO)
private Integer id;
/**
* user id
*/
private int userId;
/**
* resource id
*/
private int resourcesId;
/**
* permission
*/
private int perm;
/**
* create time
*/
private Date createTime;
/**
* update time
*/
private Date updateTime;
}

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

@ -179,8 +179,9 @@ public class TaskDefinition {
private int delayTime;
/**
* resource ids
* resource ids we do
*/
@Deprecated
private String resourceIds;
/**

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

@ -258,9 +258,6 @@ public class TaskInstance implements Serializable {
private String executorName;
@TableField(exist = false)
private Map<String, String> resources;
/**
* delay execution time.
*/

157
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.java

@ -1,157 +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.Resource;
import org.apache.ibatis.annotations.Param;
import java.util.List;
import com.baomidou.mybatisplus.core.mapper.BaseMapper;
import com.baomidou.mybatisplus.core.metadata.IPage;
/**
* resource mapper interface
*/
public interface ResourceMapper extends BaseMapper<Resource> {
/**
* query resource list
* @param fullName full name
* @param userId userId
* @param type type
* @return resource list
*/
List<Resource> queryResourceList(@Param("fullName") String fullName,
@Param("userId") int userId,
@Param("type") int type);
/**
* query resource list
* @param userId userId
* @param type type
* @return resource list
*/
List<Resource> queryResourceListAuthored(@Param("userId") int userId,
@Param("type") int type);
/**
* resource page
* @param page page
* @param id id
* @param type type
* @param searchVal searchVal
* @param resIds resIds
* @return resource page
*/
IPage<Resource> queryResourcePaging(IPage<Resource> page,
@Param("id") int id,
@Param("type") int type,
@Param("searchVal") String searchVal,
@Param("resIds") List<Integer> resIds);
/**
* query resource except userId
* @param userId userId
* @return resource list
*/
List<Resource> queryResourceExceptUserId(@Param("userId") int userId);
/**
* list authorized resource
* @param userId userId
* @param resNames resNames
* @param <T> T
* @return resource list
*/
<T> List<Resource> listAuthorizedResource(@Param("userId") int userId, @Param("resNames") T[] resNames);
/**
* list resources by id
* @param resIds resIds
* @return resource list
*/
List<Resource> queryResourceListById(@Param("resIds") List<Integer> resIds);
/**
* list authorized resource
* @param userId userId
* @param resIds resIds
* @param <T> T
* @return resource list
*/
<T> List<Resource> listAuthorizedResourceById(@Param("userId") int userId, @Param("resIds") T[] resIds);
/**
* delete resource by id array
* @param resIds resource id array
* @return delete num
*/
int deleteIds(@Param("resIds") Integer[] resIds);
/**
* list children
* @param direcotyId directory id
* @return resource id array
*/
List<Integer> listChildren(@Param("direcotyId") int direcotyId);
/**
* query resource by full name or pid
* @param fullName full name
* @param type resource type
* @return resource
*/
List<Resource> queryResource(@Param("fullName") String fullName, @Param("type") int type);
/**
* list resource by id array
* @param resIds resource id array
* @return resource list
*/
List<Resource> listResourceByIds(@Param("resIds") Integer[] resIds);
/**
* update resource
* @param resourceList resource list
* @return update num
*/
int batchUpdateResource(@Param("resourceList") List<Resource> resourceList);
/**
* check resource exist
* @param fullName full name
* @param userId userId
* @param type type
* @return true if exist else return null
*/
Boolean existResourceByUser(@Param("fullName") String fullName,
@Param("userId") int userId,
@Param("type") int type);
/**
* check resource exist
* @param fullName full name
* @param type type
* @return true if exist else return null
*/
Boolean existResource(@Param("fullName") String fullName,
@Param("type") int type);
}

60
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ResourceUserMapper.java

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

2
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java

@ -183,7 +183,7 @@ public class TaskCacheUtils {
log.info("{} --- Remote:{} to Local:{}", "CRC file", resourceCRCWholePath, targetPath);
String crcString = "";
try {
storageOperate.download(context.getTenantCode(), resourceCRCWholePath, targetPath, true);
storageOperate.download(resourceCRCWholePath, targetPath, true);
crcString = FileUtils.readFile2Str(new FileInputStream(targetPath));
fileProperty.setValue(crcString);
} catch (IOException e) {

1
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskInstanceUtils.java

@ -71,7 +71,6 @@ public class TaskInstanceUtils {
target.setExecutorId(source.getExecutorId());
target.setVarPool(source.getVarPool());
target.setExecutorName(source.getExecutorName());
target.setResources(source.getResources());
target.setDelayTime(source.getDelayTime());
target.setDryRun(source.getDryRun());
target.setTaskGroupId(source.getTaskGroupId());

198
dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml

@ -1,198 +0,0 @@
<?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.ResourceMapper">
<sql id="baseSqlV2">
${alias}.id, ${alias}.alias, ${alias}.file_name, ${alias}.description, ${alias}.user_id, ${alias}.type, ${alias}.size, ${alias}.create_time, ${alias}.update_time,
${alias}.pid, ${alias}.full_name, ${alias}.is_directory
</sql>
<select id="queryResourceList" resultType="org.apache.dolphinscheduler.dao.entity.Resource">
select
<include refid="baseSqlV2">
<property name="alias" value="r"/>
</include>
from t_ds_resources r
where 1= 1
<if test="fullName != null and fullName != ''">
and r.full_name = #{fullName}
</if>
<if test="type != -1">
and r.type = #{type}
</if>
<if test="userId != 0">
and r.user_id = #{userId}
</if>
</select>
<select id="queryResourceListAuthored" resultType="org.apache.dolphinscheduler.dao.entity.Resource">
select
<include refid="baseSqlV2">
<property name="alias" value="r"/>
</include>
from t_ds_resources r
where 1 = 1
<if test="type != -1">
and r.type=#{type}
</if>
<if test="userId != 0">
and r.user_id=#{userId}
</if>
</select>
<select id="queryResourcePaging" resultType="org.apache.dolphinscheduler.dao.entity.Resource">
select
<include refid="baseSqlV2">
<property name="alias" value="d"/>
</include>
,u.user_name
from t_ds_resources d
left join t_ds_user u on u.id = d.user_id
where 1=1 and d.type=#{type} and d.pid=#{id}
<if test="resIds != null and resIds.size() > 0">
and d.id in
<foreach collection="resIds" item="i" open="(" close=")" separator=",">
#{i}
</foreach>
</if>
<if test="searchVal != null and searchVal != ''">
and d.alias like concat('%', #{searchVal}, '%')
</if>
order by d.update_time desc
</select>
<select id="queryResourceExceptUserId" resultType="org.apache.dolphinscheduler.dao.entity.Resource">
select
<include refid="baseSqlV2">
<property name="alias" value="r"/>
</include>
from t_ds_resources r
where r.user_id <![CDATA[ <> ]]> #{userId}
</select>
<select id="listAuthorizedResource" resultType="org.apache.dolphinscheduler.dao.entity.Resource">
select
<include refid="baseSqlV2">
<property name="alias" value="r"/>
</include>
from t_ds_resources r
where r.type = 0
and r.user_id=#{userId}
<if test="resNames != null and resNames.length > 0">
and full_name in
<foreach collection="resNames" item="i" open="(" close=")" separator=",">
#{i}
</foreach>
</if>
</select>
<select id="queryResourceListById" resultType="org.apache.dolphinscheduler.dao.entity.Resource">
select
<include refid="baseSqlV2">
<property name="alias" value="r"/>
</include>
from t_ds_resources r
where 1 = 1
<if test="resIds != null and resIds.size() > 0">
and r.id in
<foreach collection="resIds" item="i" open="(" close=")" separator=",">
#{i}
</foreach>
</if>
</select>
<select id="listAuthorizedResourceById" resultType="org.apache.dolphinscheduler.dao.entity.Resource">
select
<include refid="baseSqlV2">
<property name="alias" value="r"/>
</include>
from t_ds_resources r
where r.user_id=#{userId}
<if test="resIds != null and resIds.length > 0">
and id in
<foreach collection="resIds" item="i" open="(" close=")" separator=",">
#{i}
</foreach>
</if>
</select>
<delete id="deleteIds" parameterType="java.lang.Integer">
delete from t_ds_resources
<if test="resIds != null and resIds.length != 0">
where id in
<foreach collection="resIds" item="i" open="(" close=")" separator=",">
#{i}
</foreach>
</if>
</delete>
<select id="listChildren" resultType="java.lang.Integer">
select id
from t_ds_resources
where pid = #{direcotyId}
</select>
<select id="queryResource" resultType="org.apache.dolphinscheduler.dao.entity.Resource">
select
<include refid="baseSqlV2">
<property name="alias" value="r"/>
</include>
from t_ds_resources r
where r.type = #{type}
and r.full_name = #{fullName}
</select>
<update id="batchUpdateResource" parameterType="java.util.List">
<foreach collection="resourceList" item="resource" index="index" open="" close="" separator=";">
update t_ds_resources
<set>
full_name=#{resource.fullName},
update_time=#{resource.updateTime}
</set>
<where>
id=#{resource.id}
</where>
</foreach>
</update>
<select id="listResourceByIds" resultType="org.apache.dolphinscheduler.dao.entity.Resource">
select
<include refid="baseSqlV2">
<property name="alias" value="r"/>
</include>
from t_ds_resources r
<if test="resIds != null and resIds.length != 0">
where r.id in
<foreach collection="resIds" item="i" open="(" close=")" separator=",">
#{i}
</foreach>
</if>
</select>
<select id="existResourceByUser" resultType="java.lang.Boolean">
select 1
from t_ds_resources
where full_name = #{fullName}
and type = #{type}
and user_id = #{userId} limit 1
</select>
<select id="existResource" resultType="java.lang.Boolean">
select 1
from t_ds_resources
where full_name = #{fullName}
and type = #{type} limit 1
</select>
</mapper>

2
dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_h2.sql

@ -784,6 +784,7 @@ CREATE TABLE t_ds_relation_project_user
-- ----------------------------
-- Table structure for t_ds_relation_resources_user
-- ----------------------------
-- Deprecated
DROP TABLE IF EXISTS t_ds_relation_resources_user CASCADE;
CREATE TABLE t_ds_relation_resources_user
(
@ -818,6 +819,7 @@ CREATE TABLE t_ds_relation_udfs_user
-- ----------------------------
-- Table structure for t_ds_resources
-- ----------------------------
-- Deprecated
DROP TABLE IF EXISTS t_ds_resources CASCADE;
CREATE TABLE t_ds_resources
(

2
dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql

@ -783,6 +783,7 @@ CREATE TABLE `t_ds_relation_project_user` (
-- ----------------------------
-- Table structure for t_ds_relation_resources_user
-- ----------------------------
-- Deprecated
DROP TABLE IF EXISTS `t_ds_relation_resources_user`;
CREATE TABLE `t_ds_relation_resources_user` (
`id` int(11) NOT NULL AUTO_INCREMENT,
@ -815,6 +816,7 @@ CREATE TABLE `t_ds_relation_udfs_user` (
-- ----------------------------
-- Table structure for t_ds_resources
-- ----------------------------
-- Deprecated
DROP TABLE IF EXISTS `t_ds_resources`;
CREATE TABLE `t_ds_resources` (
`id` int(11) NOT NULL AUTO_INCREMENT COMMENT 'key',

4
dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql

@ -695,7 +695,7 @@ create index relation_project_user_id_index on t_ds_relation_project_user (user_
--
-- Table structure for table t_ds_relation_resources_user
--
-- Deprecated
DROP TABLE IF EXISTS t_ds_relation_resources_user;
CREATE TABLE t_ds_relation_resources_user (
id int NOT NULL ,
@ -726,7 +726,7 @@ CREATE TABLE t_ds_relation_udfs_user (
--
-- Table structure for table t_ds_resources
--
-- Deprecated
DROP TABLE IF EXISTS t_ds_resources;
CREATE TABLE t_ds_resources (
id int NOT NULL ,

440
dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ResourceMapperTest.java

@ -1,440 +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 static java.util.stream.Collectors.toList;
import org.apache.dolphinscheduler.common.constants.Constants;
import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.dao.BaseDaoTest;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.ResourcesUser;
import org.apache.dolphinscheduler.dao.entity.Tenant;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.spi.enums.ResourceType;
import org.apache.commons.collections4.CollectionUtils;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Date;
import java.util.List;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import org.springframework.beans.factory.annotation.Autowired;
import com.baomidou.mybatisplus.core.metadata.IPage;
import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
public class ResourceMapperTest extends BaseDaoTest {
@Autowired
private ResourceMapper resourceMapper;
@Autowired
private ResourceUserMapper resourceUserMapper;
@Autowired
private TenantMapper tenantMapper;
@Autowired
private UserMapper userMapper;
/**
* insert
*
* @return Resource
*/
private Resource insertOne() {
// insertOne
Resource resource = new Resource();
resource.setAlias("ut-resource");
resource.setFullName("/ut-resource");
resource.setPid(-1);
resource.setDirectory(false);
resource.setType(ResourceType.FILE);
resource.setUserId(111);
int status = resourceMapper.insert(resource);
if (status != 1) {
Assertions.fail("insert data error");
}
return resource;
}
/**
* create resource by user
*
* @param user user
* @return Resource
*/
private Resource createResource(User user, boolean isDirectory, ResourceType resourceType, int pid, String alias,
String fullName) {
// insertOne
Resource resource = new Resource();
resource.setDirectory(isDirectory);
resource.setType(resourceType);
resource.setAlias(alias);
resource.setFullName(fullName);
resource.setUserId(user.getId());
int status = resourceMapper.insert(resource);
if (status != 1) {
Assertions.fail("insert data error");
}
return resource;
}
/**
* create resource by user
*
* @param user user
* @return Resource
*/
private Resource createResource(User user) {
// insertOne
String alias = String.format("ut-resource-%s", user.getUserName());
String fullName = String.format("/%s", alias);
Resource resource = createResource(user, false, ResourceType.FILE, -1, alias, fullName);
return resource;
}
/**
* create user
*
* @return User
*/
private User createGeneralUser(String userName) {
User user = new User();
user.setUserName(userName);
user.setUserPassword("1");
user.setEmail("xx@123.com");
user.setUserType(UserType.GENERAL_USER);
user.setCreateTime(new Date());
user.setTenantId(1);
user.setUpdateTime(new Date());
int status = userMapper.insert(user);
if (status != 1) {
Assertions.fail("insert data error");
}
return user;
}
/**
* create resource user
*
* @return ResourcesUser
*/
private ResourcesUser createResourcesUser(Resource resource, User user) {
// insertOne
ResourcesUser resourcesUser = new ResourcesUser();
resourcesUser.setCreateTime(new Date());
resourcesUser.setUpdateTime(new Date());
resourcesUser.setUserId(user.getId());
resourcesUser.setResourcesId(resource.getId());
resourcesUser.setPerm(7);
resourceUserMapper.insert(resourcesUser);
return resourcesUser;
}
@Test
public void testInsert() {
Resource resource = insertOne();
Assertions.assertNotNull(resource.getId());
Assertions.assertTrue(resource.getId() > 0);
}
/**
* test update
*/
@Test
public void testUpdate() {
// insertOne
Resource resource = insertOne();
resource.setCreateTime(new Date());
// update
int update = resourceMapper.updateById(resource);
Assertions.assertEquals(1, update);
}
/**
* test delete
*/
@Test
public void testDelete() {
Resource resourceMap = insertOne();
int delete = resourceMapper.deleteById(resourceMap.getId());
Assertions.assertEquals(1, delete);
}
/**
* test query
*/
@Test
public void testQuery() {
Resource resource = insertOne();
// query
List<Resource> resources = resourceMapper.selectList(null);
Assertions.assertNotEquals(resources.size(), 0);
}
/**
* test query resource list
*/
@Test
public void testQueryResourceList() {
Resource resource = insertOne();
String alias = "";
int userId = resource.getUserId();
int type = resource.getType().ordinal();
List<Resource> resources = resourceMapper.queryResourceList(
alias,
userId,
type);
Assertions.assertNotEquals(resources.size(), 0);
}
/**
* test page
*/
@Test
public void testQueryResourcePaging() {
User user = new User();
user.setUserName("11");
user.setUserPassword("1");
user.setEmail("xx@123.com");
user.setUserType(UserType.GENERAL_USER);
user.setCreateTime(new Date());
user.setTenantId(1);
user.setUpdateTime(new Date());
userMapper.insert(user);
Resource resource = new Resource();
resource.setAlias("ut-resource");
resource.setFullName("/ut-resource");
resource.setPid(-1);
resource.setDirectory(false);
resource.setType(ResourceType.FILE);
resource.setUserId(user.getId());
resourceMapper.insert(resource);
Page<Resource> page = new Page(1, 3);
IPage<Resource> resourceIPage = resourceMapper.queryResourcePaging(
page,
-1,
resource.getType().ordinal(),
"",
new ArrayList<>(resource.getId()));
IPage<Resource> resourceIPage1 = resourceMapper.queryResourcePaging(
page,
-1,
resource.getType().ordinal(),
"",
null);
Assertions.assertEquals(resourceIPage.getTotal(), 1);
Assertions.assertEquals(resourceIPage1.getTotal(), 1);
}
/**
* test authed resource list
*/
@Test
public void testQueryResourceListAuthored() {
Resource resource = insertOne();
List<Integer> resIds = resourceUserMapper.queryResourcesIdListByUserIdAndPerm(resource.getUserId(),
Constants.AUTHORIZE_WRITABLE_PERM);
List<Resource> resources =
CollectionUtils.isEmpty(resIds) ? new ArrayList<>() : resourceMapper.queryResourceListById(resIds);
ResourcesUser resourcesUser = new ResourcesUser();
resourcesUser.setResourcesId(resource.getId());
resourcesUser.setUserId(1110);
resourcesUser.setPerm(Constants.AUTHORIZE_WRITABLE_PERM);
resourceUserMapper.insert(resourcesUser);
List<Integer> resIds1 =
resourceUserMapper.queryResourcesIdListByUserIdAndPerm(1110, Constants.AUTHORIZE_WRITABLE_PERM);
List<Resource> resources1 =
CollectionUtils.isEmpty(resIds1) ? new ArrayList<>() : resourceMapper.queryResourceListById(resIds1);
Assertions.assertEquals(0, resources.size());
Assertions.assertNotEquals(0, resources1.size());
}
/**
* test authed resource list
*/
@Test
public void testQueryAuthorizedResourceList() {
Resource resource = insertOne();
List<Integer> resIds = resourceUserMapper.queryResourcesIdListByUserIdAndPerm(resource.getUserId(),
Constants.AUTHORIZE_WRITABLE_PERM);
List<Resource> resources =
CollectionUtils.isEmpty(resIds) ? new ArrayList<>() : resourceMapper.queryResourceListById(resIds);
resourceMapper.deleteById(resource.getId());
Assertions.assertEquals(0, resources.size());
}
/**
* test query resource expect userId
*/
@Test
public void testQueryResourceExceptUserId() {
Resource resource = insertOne();
List<Resource> resources = resourceMapper.queryResourceExceptUserId(
11111);
Assertions.assertNotEquals(resources.size(), 0);
}
/**
* test query tenant code by resource name
*/
@Test
public void testQueryTenantCodeByResourceName() {
Tenant tenant = new Tenant();
tenant.setTenantCode("ut tenant code for resource");
int tenantInsertStatus = tenantMapper.insert(tenant);
if (tenantInsertStatus != 1) {
Assertions.fail("insert tenant data error");
}
User user = new User();
user.setTenantId(tenant.getId());
user.setUserName("ut user");
int userInsertStatus = userMapper.insert(user);
if (userInsertStatus != 1) {
Assertions.fail("insert user data error");
}
Resource resource = insertOne();
resource.setUserId(user.getId());
int userUpdateStatus = resourceMapper.updateById(resource);
if (userUpdateStatus != 1) {
Assertions.fail("update user data error");
}
List<Resource> resourceList = resourceMapper.queryResource(resource.getFullName(), ResourceType.FILE.ordinal());
int resourceUserId = resourceList.get(0).getUserId();
User resourceUser = userMapper.selectById(resourceUserId);
Tenant resourceTenant = tenantMapper.selectById(resourceUser.getTenantId());
Assertions.assertEquals("ut tenant code for resource", resourceTenant.getTenantCode());
}
@Test
public void testListAuthorizedResource() {
// create a general user
User generalUser1 = createGeneralUser("user1");
User generalUser2 = createGeneralUser("user2");
// create one resource
Resource resource = createResource(generalUser2);
Resource unauthorizedResource = createResource(generalUser1);
// need download resources
String[] resNames = new String[]{resource.getFullName(), unauthorizedResource.getFullName()};
List<Resource> resources = resourceMapper.listAuthorizedResource(generalUser2.getId(), resNames);
Assertions.assertEquals(generalUser2.getId().intValue(), resource.getUserId());
Assertions.assertFalse(
resources.stream().map(t -> t.getFullName()).collect(toList()).containsAll(Arrays.asList(resNames)));
// authorize object unauthorizedResource to generalUser
createResourcesUser(unauthorizedResource, generalUser2);
List<Resource> authorizedResources = resourceMapper.listAuthorizedResource(generalUser2.getId(), resNames);
Assertions.assertTrue(authorizedResources.stream().map(t -> t.getFullName()).collect(toList())
.containsAll(Arrays.asList(resource.getFullName())));
}
@Test
public void deleteIdsTest() {
// create a general user
User generalUser1 = createGeneralUser("user1");
User generalUser = createGeneralUser("user");
Resource resource = createResource(generalUser);
Resource resource1 = createResource(generalUser1);
List<Integer> resourceList = new ArrayList<>();
resourceList.add(resource.getId());
resourceList.add(resource1.getId());
int result = resourceMapper.deleteIds(resourceList.toArray(new Integer[resourceList.size()]));
Assertions.assertEquals(result, 2);
}
@Test
public void queryResourceListAuthoredTest() {
// create a general user
User generalUser1 = createGeneralUser("user1");
User generalUser2 = createGeneralUser("user2");
// create resource
Resource resource = createResource(generalUser1);
createResourcesUser(resource, generalUser2);
List<Resource> resourceList =
resourceMapper.queryResourceListAuthored(generalUser2.getId(), ResourceType.FILE.ordinal());
Assertions.assertNotNull(resourceList);
resourceList = resourceMapper.queryResourceListAuthored(generalUser2.getId(), ResourceType.FILE.ordinal());
Assertions.assertFalse(resourceList.contains(resource));
}
@Test
public void batchUpdateResourceTest() {
// create a general user
User generalUser1 = createGeneralUser("user1");
// create resource
Resource resource = createResource(generalUser1);
resource.setFullName(String.format("%s-update", resource.getFullName()));
resource.setUpdateTime(new Date());
List<Resource> resourceList = new ArrayList<>();
resourceList.add(resource);
int result = resourceMapper.batchUpdateResource(resourceList);
if (result != resourceList.size()) {
Assertions.fail("batch update resource data error");
}
}
@Test
public void existResourceTest() {
String fullName = "/ut-resource";
int userId = 111;
int type = ResourceType.FILE.getCode();
Assertions.assertNull(resourceMapper.existResourceByUser(fullName, userId, type));
Assertions.assertNull(resourceMapper.existResource(fullName, type));
insertOne();
Assertions.assertTrue(resourceMapper.existResourceByUser(fullName, userId, type));
Assertions.assertTrue(resourceMapper.existResource(fullName, type));
}
}

111
dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ResourceUserMapperTest.java

@ -1,111 +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.common.constants.Constants;
import org.apache.dolphinscheduler.dao.BaseDaoTest;
import org.apache.dolphinscheduler.dao.entity.ResourcesUser;
import java.util.Date;
import java.util.List;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import org.springframework.beans.factory.annotation.Autowired;
public class ResourceUserMapperTest extends BaseDaoTest {
@Autowired
private ResourceUserMapper resourceUserMapper;
/**
* insert
* @return ResourcesUser
*/
private ResourcesUser insertOne() {
// insertOne
ResourcesUser resourcesUser = new ResourcesUser();
resourcesUser.setCreateTime(new Date());
resourcesUser.setUpdateTime(new Date());
resourcesUser.setUserId(11111);
resourcesUser.setResourcesId(1110);
resourcesUser.setPerm(Constants.AUTHORIZE_WRITABLE_PERM);
resourceUserMapper.insert(resourcesUser);
return resourcesUser;
}
/**
* test update
*/
@Test
public void testUpdate() {
// insertOne
ResourcesUser queue = insertOne();
queue.setCreateTime(new Date());
// update
int update = resourceUserMapper.updateById(queue);
Assertions.assertEquals(1, update);
}
/**
* test delete
*/
@Test
public void testDelete() {
ResourcesUser queue = insertOne();
int delete = resourceUserMapper.deleteById(queue.getId());
Assertions.assertEquals(1, delete);
}
/**
* test query
*/
@Test
public void testQuery() {
ResourcesUser queue = insertOne();
// query
List<ResourcesUser> queues = resourceUserMapper.selectList(null);
Assertions.assertNotEquals(0, queues.size());
}
/**
* test delete
*/
@Test
public void testDeleteResourceUser() {
ResourcesUser queue = insertOne();
int delete = resourceUserMapper.deleteResourceUser(
queue.getUserId(),
queue.getResourcesId());
Assertions.assertNotEquals(0, delete);
}
/**
* test delete
*/
@Test
public void testDeleteResourceUserArray() {
ResourcesUser resourcesUser = insertOne();
Integer[] resourceIdArray = new Integer[]{resourcesUser.getResourcesId()};
int delete = resourceUserMapper.deleteResourceUserArray(
resourcesUser.getUserId(),
resourceIdArray);
Assertions.assertNotEquals(0, delete);
}
}

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

@ -26,7 +26,6 @@ import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.plugin.task.api.DataQualityTaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.K8sTaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
@ -67,7 +66,6 @@ public class TaskExecutionContextBuilder {
taskExecutionContext.setWorkerGroup(taskInstance.getWorkerGroup());
taskExecutionContext.setEnvironmentConfig(taskInstance.getEnvironmentConfig());
taskExecutionContext.setHost(taskInstance.getHost());
taskExecutionContext.setResources(taskInstance.getResources());
taskExecutionContext.setDelayTime(taskInstance.getDelayTime());
taskExecutionContext.setVarPool(taskInstance.getVarPool());
taskExecutionContext.setDryRun(taskInstance.getDryRun());
@ -122,11 +120,6 @@ public class TaskExecutionContextBuilder {
return this;
}
public TaskExecutionContextBuilder buildDataQualityTaskExecutionContext(DataQualityTaskExecutionContext dataQualityTaskExecutionContext) {
taskExecutionContext.setDataQualityTaskExecutionContext(dataQualityTaskExecutionContext);
return this;
}
public TaskExecutionContextBuilder buildResourceParametersInfo(ResourceParametersHelper parametersHelper) {
taskExecutionContext.setResourceParametersHelper(parametersHelper);
return this;

45
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java

@ -27,7 +27,6 @@ import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.dao.entity.Environment;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelation;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationMapper;
@ -43,7 +42,6 @@ import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
@ -60,9 +58,7 @@ import org.apache.dolphinscheduler.server.master.runner.dispatcher.WorkerTaskDis
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnableFactory;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.spi.enums.ResourceType;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.lang3.StringUtils;
import java.util.Date;
@ -70,10 +66,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import lombok.NonNull;
import lombok.extern.slf4j.Slf4j;
@ -319,8 +312,6 @@ public class StreamTaskExecuteRunnable implements Runnable {
return null;
}
taskInstance.setResources(getResourceFullNames(taskInstance));
TaskChannel taskChannel = taskPluginManager.getTaskChannel(taskInstance.getTaskType());
ResourceParametersHelper resources = taskChannel.getResources(taskInstance.getTaskParams());
@ -350,42 +341,6 @@ public class StreamTaskExecuteRunnable implements Runnable {
return taskExecutionContext;
}
/**
* get resource map key is full name and value is tenantCode
*/
protected Map<String, String> getResourceFullNames(TaskInstance taskInstance) {
Map<String, String> resourcesMap = new HashMap<>();
AbstractParameters baseParam = taskPluginManager.getParameters(ParametersNode.builder()
.taskType(taskInstance.getTaskType()).taskParams(taskInstance.getTaskParams()).build());
if (baseParam != null) {
List<ResourceInfo> projectResourceFiles = baseParam.getResourceFilesList();
if (CollectionUtils.isNotEmpty(projectResourceFiles)) {
// filter the resources that the resource id equals 0
Set<ResourceInfo> oldVersionResources =
projectResourceFiles.stream().filter(t -> t.getId() == null).collect(Collectors.toSet());
if (CollectionUtils.isNotEmpty(oldVersionResources)) {
oldVersionResources.forEach(t -> resourcesMap.put(t.getRes(),
processService.queryTenantCodeByResName(t.getRes(), ResourceType.FILE)));
}
// get the resource id in order to get the resource names in batch
Stream<Integer> resourceIdStream = projectResourceFiles.stream().map(ResourceInfo::getId);
Set<Integer> resourceIdsSet = resourceIdStream.collect(Collectors.toSet());
if (CollectionUtils.isNotEmpty(resourceIdsSet)) {
Integer[] resourceIds = resourceIdsSet.toArray(new Integer[resourceIdsSet.size()]);
List<Resource> resources = processService.listResourceByIds(resourceIds);
resources.forEach(t -> resourcesMap.put(t.getFullName(),
processService.queryTenantCodeByResName(t.getFullName(), ResourceType.FILE)));
}
}
}
return resourcesMap;
}
protected boolean handleTaskEvent(TaskEvent taskEvent) throws StateEventHandleException, StateEventHandleError {
measureTaskState(taskEvent);

23
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionContextFactory.java

@ -54,7 +54,6 @@ import org.apache.dolphinscheduler.plugin.task.api.enums.dp.ConnectorType;
import org.apache.dolphinscheduler.plugin.task.api.enums.dp.ExecuteSqlType;
import org.apache.dolphinscheduler.plugin.task.api.model.JdbcInfo;
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
import org.apache.dolphinscheduler.plugin.task.api.parameters.K8sTaskParameters;
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
@ -72,13 +71,11 @@ import org.apache.dolphinscheduler.server.master.exception.TaskExecutionContextC
import org.apache.dolphinscheduler.service.expand.CuringParamsService;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.spi.enums.DbType;
import org.apache.dolphinscheduler.spi.enums.ResourceType;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.lang3.StringUtils;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
@ -113,7 +110,6 @@ public class TaskExecutionContextFactory {
public TaskExecutionContext createTaskExecutionContext(TaskInstance taskInstance) throws TaskExecutionContextCreateException {
ProcessInstance workflowInstance = taskInstance.getProcessInstance();
taskInstance.setResources(getResourceFullNames(taskInstance));
ResourceParametersHelper resources =
Optional.ofNullable(taskPluginManager.getTaskChannel(taskInstance.getTaskType()))
@ -159,22 +155,6 @@ public class TaskExecutionContextFactory {
taskExecutionContext.setK8sTaskExecutionContext(k8sTaskExecutionContext);
}
private Map<String, String> getResourceFullNames(TaskInstance taskInstance) {
Map<String, String> resourcesMap = new HashMap<>();
AbstractParameters baseParam = taskPluginManager.getParameters(ParametersNode.builder()
.taskType(taskInstance.getTaskType()).taskParams(taskInstance.getTaskParams()).build());
if (baseParam != null) {
List<ResourceInfo> projectResourceFiles = baseParam.getResourceFilesList();
if (CollectionUtils.isNotEmpty(projectResourceFiles)) {
// TODO: Modify this part to accomodate(migrate) oldversionresources in the future.
projectResourceFiles.forEach(file -> resourcesMap.put(file.getResourceName(),
processService.queryTenantCodeByResName(file.getResourceName(), ResourceType.FILE)));
}
}
return resourcesMap;
}
private void setTaskResourceInfo(ResourceParametersHelper resourceParametersHelper) {
if (Objects.isNull(resourceParametersHelper)) {
return;
@ -219,9 +199,6 @@ public class TaskExecutionContextFactory {
udfFuncList.forEach(udfFunc -> {
UdfFuncParameters udfFuncParameters =
JSONUtils.parseObject(JSONUtils.toJsonString(udfFunc), UdfFuncParameters.class);
udfFuncParameters.setDefaultFS(PropertyUtils.getString(Constants.FS_DEFAULT_FS));
String tenantCode = processService.queryTenantCodeByResName(udfFunc.getResourceName(), ResourceType.UDF);
udfFuncParameters.setTenantCode(tenantCode);
map.put(udfFunc.getId(), udfFuncParameters);
});
}

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

@ -39,7 +39,6 @@ import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelation;
import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelationLog;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.ProjectUser;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.Schedule;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
import org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog;
@ -50,7 +49,6 @@ import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.service.exceptions.CronParseException;
import org.apache.dolphinscheduler.service.model.TaskNode;
import org.apache.dolphinscheduler.spi.enums.ResourceType;
import java.util.List;
import java.util.Map;
@ -135,8 +133,6 @@ public interface ProcessService {
List<UdfFunc> queryUdfFunListByIds(Integer[] ids);
String queryTenantCodeByResName(String resName, ResourceType resourceType);
List<Schedule> selectAllByProcessDefineCode(long[] codes);
String queryUserQueueByProcessInstance(ProcessInstance processInstance);
@ -149,10 +145,6 @@ public interface ProcessService {
User getUserById(int userId);
Resource getResourceById(int resourceId);
List<Resource> listResourceByIds(Integer[] resIds);
String formatTaskAppId(TaskInstance taskInstance);
int switchVersion(ProcessDefinition processDefinition, ProcessDefinitionLog processDefinitionLog);

112
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java

@ -69,7 +69,6 @@ import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelation;
import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelationLog;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.ProjectUser;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.Schedule;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
import org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog;
@ -89,7 +88,6 @@ import org.apache.dolphinscheduler.dao.mapper.DqRuleInputEntryMapper;
import org.apache.dolphinscheduler.dao.mapper.DqRuleMapper;
import org.apache.dolphinscheduler.dao.mapper.DqTaskStatisticsValueMapper;
import org.apache.dolphinscheduler.dao.mapper.EnvironmentMapper;
import org.apache.dolphinscheduler.dao.mapper.ErrorCommandMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionLogMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessInstanceMapMapper;
@ -97,8 +95,6 @@ import org.apache.dolphinscheduler.dao.mapper.ProcessInstanceMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationLogMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationMapper;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.apache.dolphinscheduler.dao.mapper.ResourceMapper;
import org.apache.dolphinscheduler.dao.mapper.ResourceUserMapper;
import org.apache.dolphinscheduler.dao.mapper.ScheduleMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionLogMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
@ -125,8 +121,6 @@ import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.enums.dp.DqTaskState;
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
import org.apache.dolphinscheduler.plugin.task.api.parameters.SubProcessParameters;
import org.apache.dolphinscheduler.plugin.task.api.parameters.TaskTimeoutParameter;
import org.apache.dolphinscheduler.service.command.CommandService;
@ -137,7 +131,6 @@ import org.apache.dolphinscheduler.service.expand.CuringParamsService;
import org.apache.dolphinscheduler.service.model.TaskNode;
import org.apache.dolphinscheduler.service.utils.ClusterConfUtils;
import org.apache.dolphinscheduler.service.utils.DagHelper;
import org.apache.dolphinscheduler.spi.enums.ResourceType;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.collections4.MapUtils;
@ -169,7 +162,6 @@ import org.springframework.transaction.annotation.Transactional;
import com.baomidou.mybatisplus.core.conditions.query.QueryWrapper;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.node.ObjectNode;
import com.google.common.base.Joiner;
import com.google.common.base.Strings;
import com.google.common.collect.Lists;
@ -226,15 +218,6 @@ public class ProcessServiceImpl implements ProcessService {
@Autowired
private UdfFuncMapper udfFuncMapper;
@Autowired
private ResourceMapper resourceMapper;
@Autowired
private ResourceUserMapper resourceUserMapper;
@Autowired
private ErrorCommandMapper errorCommandMapper;
@Autowired
private TenantMapper tenantMapper;
@ -1382,7 +1365,6 @@ public class ProcessServiceImpl implements ProcessService {
return new ResourceInfo();
}
resourceInfo = new ResourceInfo();
resourceInfo.setId(-1);
resourceInfo.setResourceName(resourceFullName);
log.info("updated resource info {}",
JSONUtils.toJsonString(resourceInfo));
@ -1617,34 +1599,6 @@ public class ProcessServiceImpl implements ProcessService {
return udfFuncMapper.queryUdfByIdStr(ids, null);
}
/**
* find tenant code by resource name
*
* @param resName resource name
* @param resourceType resource type
* @return tenant code
*/
@Override
public String queryTenantCodeByResName(String resName, ResourceType resourceType) {
// in order to query tenant code successful although the version is older
String fullName = resName.startsWith("/") ? resName : String.format("/%s", resName);
List<Resource> resourceList = resourceMapper.queryResource(fullName, resourceType.ordinal());
if (CollectionUtils.isEmpty(resourceList)) {
return "";
}
int userId = resourceList.get(0).getUserId();
User user = userMapper.selectById(userId);
if (Objects.isNull(user)) {
return "";
}
Tenant tenant = tenantMapper.queryById(user.getTenantId());
if (Objects.isNull(tenant)) {
return "";
}
return tenant.getTenantCode();
}
/**
* find schedule list by process define codes.
*
@ -1723,20 +1677,6 @@ public class ProcessServiceImpl implements ProcessService {
Set<T> originResSet = new HashSet<>(Arrays.asList(needChecks));
switch (authorizationType) {
case RESOURCE_FILE_ID:
case UDF_FILE:
List<Resource> ownUdfResources = resourceMapper.listAuthorizedResourceById(userId, needChecks);
addAuthorizedResources(ownUdfResources, userId);
Set<Integer> authorizedResourceFiles =
ownUdfResources.stream().map(Resource::getId).collect(toSet());
originResSet.removeAll(authorizedResourceFiles);
break;
case RESOURCE_FILE_NAME:
List<Resource> ownResources = resourceMapper.listAuthorizedResource(userId, needChecks);
addAuthorizedResources(ownResources, userId);
Set<String> authorizedResources = ownResources.stream().map(Resource::getFullName).collect(toSet());
originResSet.removeAll(authorizedResources);
break;
case DATASOURCE:
Set<Integer> authorizedDatasources = dataSourceMapper.listAuthorizedDataSource(userId, needChecks)
.stream().map(DataSource::getId).collect(toSet());
@ -1768,28 +1708,6 @@ public class ProcessServiceImpl implements ProcessService {
return userMapper.selectById(userId);
}
/**
* get resource by resource id
*
* @param resourceId resource id
* @return Resource
*/
@Override
public Resource getResourceById(int resourceId) {
return resourceMapper.selectById(resourceId);
}
/**
* list resources by ids
*
* @param resIds resIds
* @return resource list
*/
@Override
public List<Resource> listResourceByIds(Integer[] resIds) {
return resourceMapper.listResourceByIds(resIds);
}
/**
* format task app id in task instance
*/
@ -1880,22 +1798,10 @@ public class ProcessServiceImpl implements ProcessService {
* @param taskDefinition taskDefinition
* @return resource ids
*/
@Deprecated
@Override
public String getResourceIds(TaskDefinition taskDefinition) {
Set<Integer> resourceIds = null;
AbstractParameters params = taskPluginManager.getParameters(ParametersNode.builder()
.taskType(taskDefinition.getTaskType()).taskParams(taskDefinition.getTaskParams()).build());
if (params != null && CollectionUtils.isNotEmpty(params.getResourceFilesList())) {
resourceIds = params.getResourceFilesList().stream()
.map(ResourceInfo::getId)
.filter(Objects::nonNull)
.collect(toSet());
}
if (CollectionUtils.isEmpty(resourceIds)) {
return "";
}
return Joiner.on(",").join(resourceIds);
return "";
}
@Override
@ -2136,20 +2042,6 @@ public class ProcessServiceImpl implements ProcessService {
return processTaskRelationLogList.stream().map(r -> (ProcessTaskRelation) r).collect(Collectors.toList());
}
/**
* add authorized resources
*
* @param ownResources own resources
* @param userId userId
*/
private void addAuthorizedResources(List<Resource> ownResources, int userId) {
List<Integer> relationResourceIds = resourceUserMapper.queryResourcesIdListByUserIdAndPerm(userId, 7);
List<Resource> relationResources = CollectionUtils.isNotEmpty(relationResourceIds)
? resourceMapper.queryResourceListById(relationResourceIds)
: new ArrayList<>();
ownResources.addAll(relationResources);
}
/**
* Use temporarily before refactoring taskNode
*/

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

@ -59,7 +59,6 @@ import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessInstanceMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationLogMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationMapper;
import org.apache.dolphinscheduler.dao.mapper.ResourceMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionLogMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
import org.apache.dolphinscheduler.dao.mapper.TaskGroupMapper;
@ -151,8 +150,6 @@ public class ProcessServiceTest {
@Mock
private ProcessDefinitionLogMapper processDefineLogMapper;
@Mock
private ResourceMapper resourceMapper;
@Mock
private TaskGroupMapper taskGroupMapper;
@Mock
private DataSourceMapper dataSourceMapper;
@ -736,12 +733,10 @@ public class ProcessServiceTest {
// test normal situation
ResourceInfo resourceInfoNormal = new ResourceInfo();
resourceInfoNormal.setId(1);
resourceInfoNormal.setResourceName("/test.txt");
ResourceInfo updatedResourceInfo3 = processService.updateResourceInfo(0, resourceInfoNormal);
Assertions.assertEquals(-1, updatedResourceInfo3.getId().intValue());
Assertions.assertEquals("/test.txt", updatedResourceInfo3.getResourceName());
}

9
dolphinscheduler-storage-plugin/dolphinscheduler-storage-abs/src/main/java/org/apache/dolphinscheduler/plugin/storage/abs/AbsStorageOperator.java

@ -148,12 +148,6 @@ public class AbsStorageOperator implements Closeable, StorageOperate {
return getAbsUdfDir(tenantCode) + FOLDER_SEPARATOR;
}
@Override
public String getResourceFileName(String tenantCode, String fullName) {
String resDir = getResDir(tenantCode);
return fullName.replaceFirst(resDir, "");
}
@Override
public String getResourceFullName(String tenantCode, String fileName) {
if (fileName.startsWith(FOLDER_SEPARATOR)) {
@ -171,8 +165,7 @@ public class AbsStorageOperator implements Closeable, StorageOperate {
}
@Override
public void download(String tenantCode, String srcFilePath, String dstFilePath,
boolean overwrite) throws IOException {
public void download(String srcFilePath, String dstFilePath, boolean overwrite) throws IOException {
File dstFile = new File(dstFilePath);
if (dstFile.isDirectory()) {
Files.delete(dstFile.toPath());

12
dolphinscheduler-storage-plugin/dolphinscheduler-storage-api/src/main/java/org/apache/dolphinscheduler/plugin/storage/api/StorageOperate.java

@ -69,10 +69,11 @@ public interface StorageOperate {
/**
* get the path of the resource file excluding the base path (fileName)
* @param fullName
* @return
*/
String getResourceFileName(String tenantCode, String fullName);
default String getResourceFileName(String tenantCode, String fullName) {
String resDir = getResDir(tenantCode);
return fullName.replaceFirst(resDir, "");
}
/**
* get the path of the file
@ -137,14 +138,13 @@ public interface StorageOperate {
/**
* download the srcPath to local
* @param tenantCode
*
* @param srcFilePath the full path of the srcPath
* @param dstFile
* @param overwrite
* @throws IOException
*/
void download(String tenantCode, String srcFilePath, String dstFile,
boolean overwrite) throws IOException;
void download(String srcFilePath, String dstFile, boolean overwrite) throws IOException;
/**
* vim the context of filePath

9
dolphinscheduler-storage-plugin/dolphinscheduler-storage-gcs/src/main/java/org/apache/dolphinscheduler/plugin/storage/gcs/GcsStorageOperator.java

@ -129,12 +129,6 @@ public class GcsStorageOperator implements Closeable, StorageOperate {
return String.format(FORMAT_S_S, getGcsResDir(tenantCode), fileName);
}
@Override
public String getResourceFileName(String tenantCode, String fullName) {
String resDir = getResDir(tenantCode);
return fullName.replaceFirst(resDir, "");
}
@Override
public String getFileName(ResourceType resourceType, String tenantCode, String fileName) {
if (fileName.startsWith(FOLDER_SEPARATOR)) {
@ -144,8 +138,7 @@ public class GcsStorageOperator implements Closeable, StorageOperate {
}
@Override
public void download(String tenantCode, String srcFilePath, String dstFilePath,
boolean overwrite) throws IOException {
public void download(String srcFilePath, String dstFilePath, boolean overwrite) throws IOException {
File dstFile = new File(dstFilePath);
if (dstFile.isDirectory()) {
Files.delete(dstFile.toPath());

9
dolphinscheduler-storage-plugin/dolphinscheduler-storage-hdfs/src/main/java/org/apache/dolphinscheduler/plugin/storage/hdfs/HdfsStorageOperator.java

@ -281,20 +281,13 @@ public class HdfsStorageOperator implements Closeable, StorageOperate {
return getHdfsResourceFileName(tenantCode, fullName);
}
@Override
public String getResourceFileName(String tenantCode, String fullName) {
String resDir = getResDir(tenantCode);
return fullName.replaceFirst(resDir, "");
}
@Override
public String getFileName(ResourceType resourceType, String tenantCode, String fileName) {
return getHdfsFileName(resourceType, tenantCode, fileName);
}
@Override
public void download(String tenantCode, String srcHdfsFilePath, String dstFile,
boolean overwrite) throws IOException {
public void download(String srcHdfsFilePath, String dstFile, boolean overwrite) throws IOException {
copyHdfsToLocal(srcHdfsFilePath, dstFile, false, overwrite);
}

9
dolphinscheduler-storage-plugin/dolphinscheduler-storage-obs/src/main/java/org/apache/dolphinscheduler/plugin/storage/obs/ObsStorageOperator.java

@ -153,12 +153,6 @@ public class ObsStorageOperator implements Closeable, StorageOperate {
return String.format(FORMAT_S_S, getObsResDir(tenantCode), fileName);
}
@Override
public String getResourceFileName(String tenantCode, String fullName) {
String resDir = getResDir(tenantCode);
return fullName.replaceFirst(resDir, "");
}
@Override
public String getFileName(ResourceType resourceType, String tenantCode, String fileName) {
if (fileName.startsWith(FOLDER_SEPARATOR)) {
@ -188,8 +182,7 @@ public class ObsStorageOperator implements Closeable, StorageOperate {
}
@Override
public void download(String tenantCode, String srcFilePath, String dstFilePath,
boolean overwrite) throws IOException {
public void download(String srcFilePath, String dstFilePath, boolean overwrite) throws IOException {
File dstFile = new File(dstFilePath);
if (dstFile.isDirectory()) {
Files.delete(dstFile.toPath());

8
dolphinscheduler-storage-plugin/dolphinscheduler-storage-oss/src/main/java/org/apache/dolphinscheduler/plugin/storage/oss/OssStorageOperator.java

@ -181,12 +181,6 @@ public class OssStorageOperator implements Closeable, StorageOperate {
return String.format(FORMAT_S_S, getOssResDir(tenantCode), fileName);
}
@Override
public String getResourceFileName(String tenantCode, String fullName) {
String resDir = getResDir(tenantCode);
return fullName.replaceFirst(resDir, "");
}
@Override
public String getFileName(ResourceType resourceType, String tenantCode, String fileName) {
if (fileName.startsWith(FOLDER_SEPARATOR)) {
@ -213,7 +207,7 @@ public class OssStorageOperator implements Closeable, StorageOperate {
}
@Override
public void download(String tenantCode, String srcFilePath, String dstFilePath,
public void download(String srcFilePath, String dstFilePath,
boolean overwrite) throws IOException {
File dstFile = new File(dstFilePath);
if (dstFile.isDirectory()) {

8
dolphinscheduler-storage-plugin/dolphinscheduler-storage-s3/src/main/java/org/apache/dolphinscheduler/plugin/storage/s3/S3StorageOperator.java

@ -185,12 +185,6 @@ public class S3StorageOperator implements Closeable, StorageOperate {
return String.format(FORMAT_S_S, getS3ResDir(tenantCode), fileName);
}
@Override
public String getResourceFileName(String tenantCode, String fullName) {
String resDir = getResDir(tenantCode);
return fullName.replaceFirst(resDir, "");
}
@Override
public String getFileName(ResourceType resourceType, String tenantCode, String fileName) {
if (fileName.startsWith(FOLDER_SEPARATOR)) {
@ -200,7 +194,7 @@ public class S3StorageOperator implements Closeable, StorageOperate {
}
@Override
public void download(String tenantCode, String srcFilePath, String dstFilePath,
public void download(String srcFilePath, String dstFilePath,
boolean overwrite) throws IOException {
File dstFile = new File(dstFilePath);
if (dstFile.isDirectory()) {

7
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractCommandExecutor.java

@ -22,7 +22,6 @@ import static org.apache.dolphinscheduler.common.constants.Constants.SLEEP_TIME_
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.EXIT_CODE_FAILURE;
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.EXIT_CODE_KILL;
import org.apache.dolphinscheduler.common.constants.TenantConstants;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.common.utils.OSUtils;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
@ -131,11 +130,7 @@ public abstract class AbstractCommandExecutor {
// Set sudo (This is only work in Linux)
iShellInterceptorBuilder.sudoMode(OSUtils.isSudoEnable());
// Set tenant (This is only work in Linux)
if (TenantConstants.DEFAULT_TENANT_CODE.equals(taskRequest.getTenantCode())) {
iShellInterceptorBuilder.runUser(TenantConstants.BOOTSTRAPT_SYSTEM_USER);
} else {
iShellInterceptorBuilder.runUser(taskRequest.getTenantCode());
}
iShellInterceptorBuilder.runUser(taskRequest.getTenantCode());
// Set CPU Quota (This is only work in Linux)
if (taskRequest.getCpuQuota() != null) {
iShellInterceptorBuilder.cpuQuota(taskRequest.getCpuQuota());

7
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java

@ -21,6 +21,7 @@ import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy;
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
import org.apache.dolphinscheduler.plugin.task.api.resource.ResourceContext;
import java.io.Serializable;
import java.util.Map;
@ -227,10 +228,8 @@ public class TaskExecutionContext implements Serializable {
* k8s TaskExecutionContext
*/
private K8sTaskExecutionContext k8sTaskExecutionContext;
/**
* resources full name and tenant code
*/
private Map<String, String> resources;
private ResourceContext resourceContext;
/**
* taskInstance varPool

4
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/model/ResourceInfo.java

@ -25,12 +25,14 @@ public class ResourceInfo {
/**
* res the name of the resource that was uploaded
*/
@Deprecated
private Integer id;
/**
* full name of the resource that was uploaded
* ResourceAbsolutePathInStorage
*/
private String resourceName;
@Deprecated
private String res;
}

4
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/parameters/resource/UdfFuncParameters.java

@ -90,10 +90,6 @@ public class UdfFuncParameters extends AbstractResourceParameters {
*/
private UdfType type;
private String tenantCode;
private String defaultFS;
/**
* create time
*/

67
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/resource/ResourceContext.java

@ -0,0 +1,67 @@
/*
* 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.plugin.task.api.resource;
import static com.google.common.base.Preconditions.checkNotNull;
import java.util.HashMap;
import java.util.Map;
import lombok.AllArgsConstructor;
import lombok.Builder;
import lombok.Data;
import lombok.NoArgsConstructor;
import lombok.ToString;
@ToString
public class ResourceContext {
/**
* ResourceAbsolutePathInStorage -> ResourceItem
*/
private final Map<String, ResourceItem> resourceItemMap;
public ResourceContext() {
this.resourceItemMap = new HashMap<>();
}
public void addResourceItem(ResourceItem resourceItem) {
checkNotNull(resourceItem);
resourceItemMap.put(resourceItem.getResourceAbsolutePathInStorage(), resourceItem);
}
public ResourceItem getResourceItem(String resourceAbsolutePathInStorage) {
ResourceItem resourceItem = resourceItemMap.get(resourceAbsolutePathInStorage);
if (resourceItem == null) {
throw new IllegalArgumentException("Cannot find the resourceItem: " + resourceAbsolutePathInStorage);
}
return resourceItem;
}
@Data
@Builder
@AllArgsConstructor
@NoArgsConstructor
public static class ResourceItem {
private String resourceAbsolutePathInStorage;
private String resourceRelativePath;
private String resourceAbsolutePathInLocal;
}
}

7
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/shell/BaseLinuxShellInterceptorBuilder.java

@ -63,7 +63,12 @@ public abstract class BaseLinuxShellInterceptorBuilder<T extends BaseLinuxShellI
Path shellAbsolutePath = shellAbsolutePath();
FileUtils.createFileWith755(shellAbsolutePath);
Files.write(shellAbsolutePath, finalScript.getBytes(), StandardOpenOption.APPEND);
log.info("Final Shell file is : \n{}", finalScript);
log.info("Final Shell file is: ");
log.info(
"****************************** Script Content *****************************************************************");
log.info(finalScript);
log.info(
"****************************** Script Content *****************************************************************");
}
protected List<String> generateBootstrapCommand() throws FileOperateException {

3
dolphinscheduler-task-plugin/dolphinscheduler-task-datax/src/test/java/org/apache/dolphinscheduler/plugin/task/datax/DataxParametersTest.java

@ -58,7 +58,6 @@ public class DataxParametersTest {
DataxParameters dataxParameters = new DataxParameters();
List<ResourceInfo> resourceInfoList = new ArrayList<>();
ResourceInfo resourceInfo = new ResourceInfo();
resourceInfo.setId(2);
resourceInfo.setResourceName("/hdfs.keytab");
resourceInfoList.add(resourceInfo);
@ -90,7 +89,7 @@ public class DataxParametersTest {
+ "jobSpeedRecord=1, "
+ "xms=0, "
+ "xmx=-100, "
+ "resourceList=[{\"id\":2,\"resourceName\":\"/hdfs.keytab\",\"res\":null}]"
+ "resourceList=[{\"id\":null,\"resourceName\":\"/hdfs.keytab\",\"res\":null}]"
+ "}";
Assertions.assertEquals(expected, dataxParameters.toString());

15
dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtilsTest.java

@ -19,8 +19,8 @@ package org.apache.dolphinscheduler.plugin.task.flink;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
import org.apache.dolphinscheduler.plugin.task.api.resource.ResourceContext;
import java.util.HashMap;
import java.util.List;
import org.junit.jupiter.api.Assertions;
@ -38,9 +38,7 @@ public class FlinkArgsUtilsTest {
flinkParameters.setDeployMode(flinkDeployMode);
flinkParameters.setParallelism(4);
ResourceInfo resourceInfo = new ResourceInfo();
resourceInfo.setId(1);
resourceInfo.setResourceName("/opt/job.jar");
resourceInfo.setRes("/opt/job.jar");
flinkParameters.setMainJar(resourceInfo);
flinkParameters.setMainClass("org.example.Main");
flinkParameters.setSlot(4);
@ -54,9 +52,14 @@ public class FlinkArgsUtilsTest {
TaskExecutionContext taskExecutionContext = new TaskExecutionContext();
taskExecutionContext.setTaskAppId("app-id");
taskExecutionContext.setExecutePath("/tmp/execution");
HashMap<String, String> map = new HashMap<>();
map.put("/opt/job.jar", "/opt/job.jar");
taskExecutionContext.setResources(map);
ResourceContext.ResourceItem resourceItem = new ResourceContext.ResourceItem();
resourceItem.setResourceAbsolutePathInLocal("/opt/job.jar");
resourceItem.setResourceAbsolutePathInStorage("/opt/job.jar");
ResourceContext resourceContext = new ResourceContext();
resourceContext.addResourceItem(resourceItem);
taskExecutionContext.setResourceContext(resourceContext);
return taskExecutionContext;
}

4
dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtils.java

@ -20,6 +20,7 @@ package org.apache.dolphinscheduler.plugin.task.flink;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
import org.apache.dolphinscheduler.plugin.task.api.resource.ResourceContext;
import org.apache.dolphinscheduler.plugin.task.api.utils.ArgsUtils;
import org.apache.dolphinscheduler.plugin.task.api.utils.ParameterUtils;
@ -288,7 +289,8 @@ public class FlinkArgsUtils {
if (ProgramType.PYTHON == programType) {
args.add(FlinkConstants.FLINK_PYTHON);
}
args.add(taskExecutionContext.getResources().get(mainJar.getResourceName()));
ResourceContext resourceContext = taskExecutionContext.getResourceContext();
args.add(resourceContext.getResourceItem(mainJar.getResourceName()).getResourceAbsolutePathInLocal());
}
String mainArgs = flinkParameters.getMainArgs();

23
dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/test/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkArgsUtilsTest.java

@ -19,8 +19,8 @@ package org.apache.dolphinscheduler.plugin.task.flink;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
import org.apache.dolphinscheduler.plugin.task.api.resource.ResourceContext;
import java.util.HashMap;
import java.util.List;
import org.junit.jupiter.api.Assertions;
@ -38,9 +38,7 @@ public class FlinkArgsUtilsTest {
flinkParameters.setDeployMode(flinkDeployMode);
flinkParameters.setParallelism(4);
ResourceInfo resourceInfo = new ResourceInfo();
resourceInfo.setId(1);
resourceInfo.setResourceName("/opt/job.jar");
resourceInfo.setRes("/opt/job.jar");
flinkParameters.setMainJar(resourceInfo);
flinkParameters.setMainClass("org.example.Main");
flinkParameters.setSlot(4);
@ -54,9 +52,14 @@ public class FlinkArgsUtilsTest {
TaskExecutionContext taskExecutionContext = new TaskExecutionContext();
taskExecutionContext.setTaskAppId("app-id");
taskExecutionContext.setExecutePath("/tmp/execution");
HashMap<String, String> map = new HashMap<>();
map.put("/opt/job.jar", "/opt/job.jar");
taskExecutionContext.setResources(map);
ResourceContext.ResourceItem resourceItem = new ResourceContext.ResourceItem();
resourceItem.setResourceAbsolutePathInLocal("/opt/job.jar");
resourceItem.setResourceAbsolutePathInStorage("/opt/job.jar");
ResourceContext resourceContext = new ResourceContext();
resourceContext.addResourceItem(resourceItem);
taskExecutionContext.setResourceContext(resourceContext);
return taskExecutionContext;
}
@ -71,7 +74,7 @@ public class FlinkArgsUtilsTest {
}
@Test
public void testRunJarInClusterMode() throws Exception {
public void testRunJarInClusterMode() {
FlinkParameters flinkParameters = buildTestFlinkParametersWithDeployMode(FlinkDeployMode.CLUSTER);
flinkParameters.setFlinkVersion("1.11");
List<String> commandLine1 =
@ -99,7 +102,7 @@ public class FlinkArgsUtilsTest {
}
@Test
public void testRunJarInLocalMode() throws Exception {
public void testRunJarInLocalMode() {
FlinkParameters flinkParameters = buildTestFlinkParametersWithDeployMode(FlinkDeployMode.LOCAL);
List<String> commandLine = FlinkArgsUtils.buildRunCommandLine(buildTestTaskExecutionContext(), flinkParameters);
@ -109,7 +112,7 @@ public class FlinkArgsUtilsTest {
}
@Test
public void testRunSql() throws Exception {
public void testRunSql() {
FlinkParameters flinkParameters = buildTestFlinkParametersWithDeployMode(FlinkDeployMode.CLUSTER);
flinkParameters.setProgramType(ProgramType.SQL);
List<String> commandLine = FlinkArgsUtils.buildRunCommandLine(buildTestTaskExecutionContext(), flinkParameters);
@ -119,7 +122,7 @@ public class FlinkArgsUtilsTest {
}
@Test
public void testInitOptionsInLocalMode() throws Exception {
public void testInitOptionsInLocalMode() {
List<String> initOptions =
FlinkArgsUtils.buildInitOptionsForSql(buildTestFlinkParametersWithDeployMode(FlinkDeployMode.LOCAL));
Assertions.assertEquals(2, initOptions.size());

4
dolphinscheduler-task-plugin/dolphinscheduler-task-hivecli/src/main/java/org/apache/dolphinscheduler/plugin/task/hivecli/HiveCliTask.java

@ -31,6 +31,7 @@ import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
import org.apache.dolphinscheduler.plugin.task.api.model.TaskResponse;
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
import org.apache.dolphinscheduler.plugin.task.api.resource.ResourceContext;
import org.apache.dolphinscheduler.plugin.task.api.shell.IShellInterceptorBuilder;
import org.apache.dolphinscheduler.plugin.task.api.shell.ShellInterceptorBuilderFactory;
import org.apache.dolphinscheduler.plugin.task.api.utils.ParameterUtils;
@ -137,8 +138,9 @@ public class HiveCliTask extends AbstractRemoteTask {
try {
resourceFileName = resourceInfos.get(0).getResourceName();
ResourceContext resourceContext = taskExecutionContext.getResourceContext();
sqlContent = FileUtils.readFileToString(
new File(String.format("%s/%s", taskExecutionContext.getExecutePath(), resourceFileName)),
new File(resourceContext.getResourceItem(resourceFileName).getResourceAbsolutePathInLocal()),
StandardCharsets.UTF_8);
} catch (IOException e) {
log.error("read hive sql content from file {} error ", resourceFileName, e);

13
dolphinscheduler-task-plugin/dolphinscheduler-task-hivecli/src/test/java/org/apache/dolphinscheduler/plugin/task/hivecli/HiveCliTaskTest.java

@ -24,6 +24,7 @@ import static org.mockito.Mockito.when;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
import org.apache.dolphinscheduler.plugin.task.api.resource.ResourceContext;
import org.apache.commons.io.FileUtils;
@ -72,9 +73,17 @@ public class HiveCliTaskTest {
}
@Test
public void hiveCliTaskExecuteSqlFromFile() throws Exception {
public void hiveCliTaskExecuteSqlFromFile() {
String hiveCliTaskParameters = buildHiveCliTaskExecuteSqlFromFileParameters();
HiveCliTask hiveCliTask = prepareHiveCliTaskForTest(hiveCliTaskParameters);
TaskExecutionContext taskExecutionContext = new TaskExecutionContext();
taskExecutionContext.setTaskParams(hiveCliTaskParameters);
ResourceContext resourceContext = new ResourceContext();
resourceContext.addResourceItem(new ResourceContext.ResourceItem("/sql_tasks/hive_task.sql", "123_node.sql",
"/sql_tasks/hive_task.sql"));
taskExecutionContext.setResourceContext(resourceContext);
HiveCliTask hiveCliTask = spy(new HiveCliTask(taskExecutionContext));
doReturn("123_node.sql").when(hiveCliTask).generateSqlScriptFile(Mockito.any());
hiveCliTask.init();
Assertions.assertEquals(hiveCliTask.buildCommand(), EXPECTED_HIVE_CLI_TASK_EXECUTE_FROM_FILE_COMMAND);
}

16
dolphinscheduler-task-plugin/dolphinscheduler-task-java/src/main/java/org/apache/dolphinscheduler/plugin/task/java/JavaTask.java

@ -32,6 +32,7 @@ import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
import org.apache.dolphinscheduler.plugin.task.api.model.TaskResponse;
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
import org.apache.dolphinscheduler.plugin.task.api.resource.ResourceContext;
import org.apache.dolphinscheduler.plugin.task.api.shell.IShellInterceptorBuilder;
import org.apache.dolphinscheduler.plugin.task.api.shell.ShellInterceptorBuilderFactory;
import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
@ -176,7 +177,9 @@ public class JavaTask extends AbstractTask {
* @return String
**/
protected String buildJarCommand() {
String mainJarName = taskRequest.getResources().get(javaParameters.getMainJar().getResourceName());
ResourceContext resourceContext = taskRequest.getResourceContext();
String mainJarName = resourceContext.getResourceItem(javaParameters.getMainJar().getResourceName())
.getResourceAbsolutePathInLocal();
StringBuilder builder = new StringBuilder();
builder.append(getJavaCommandPath())
.append("java").append(" ")
@ -279,14 +282,17 @@ public class JavaTask extends AbstractTask {
} else {
builder.append("--class-path");
}
builder.append(" ").append(JavaConstants.CLASSPATH_CURRENT_DIR)
builder.append(" ")
.append(JavaConstants.CLASSPATH_CURRENT_DIR)
.append(JavaConstants.PATH_SEPARATOR)
.append(taskRequest.getExecutePath());
Map<String, String> resourceMap = taskRequest.getResources();
ResourceContext resourceContext = taskRequest.getResourceContext();
for (ResourceInfo info : javaParameters.getResourceFilesList()) {
builder.append(JavaConstants.PATH_SEPARATOR);
builder.append(taskRequest.getExecutePath()).append(FOLDER_SEPARATOR)
.append(resourceMap.get(info.getResourceName()));
builder
.append(taskRequest.getExecutePath())
.append(FOLDER_SEPARATOR)
.append(resourceContext.getResourceItem(info.getResourceName()).getResourceAbsolutePathInLocal());
}
return builder.toString();
}

47
dolphinscheduler-task-plugin/dolphinscheduler-task-java/src/test/java/org/apache/dolphinscheduler/plugin/task/java/JavaTaskTest.java

@ -28,6 +28,7 @@ import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.model.ApplicationInfo;
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
import org.apache.dolphinscheduler.plugin.task.api.resource.ResourceContext;
import org.apache.dolphinscheduler.plugin.task.java.exception.JavaSourceFileExistException;
import org.apache.dolphinscheduler.plugin.task.java.exception.PublicClassNotFoundException;
import org.apache.dolphinscheduler.plugin.task.java.exception.RunTypeNotFoundException;
@ -39,7 +40,6 @@ import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.HashMap;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
@ -212,9 +212,7 @@ public class JavaTaskTest {
javaParameters.setJvmArgs("-xms:50m");
javaParameters.setMainArgs("-host 127.0.0.1 -port 8080");
ResourceInfo resourceJar = new ResourceInfo();
resourceJar.setId(2);
resourceJar.setResourceName("/opt/share/jar/resource2.jar");
resourceJar.setRes("I'm resource2.jar");
ArrayList<ResourceInfo> resourceInfoArrayList = new ArrayList<>();
resourceInfoArrayList.add(resourceJar);
javaParameters.setResourceList(resourceInfoArrayList);
@ -236,9 +234,7 @@ public class JavaTaskTest {
property.setType(VARCHAR);
javaParameters.setLocalParams(localParams);
ResourceInfo mainJar = new ResourceInfo();
mainJar.setId(1);
mainJar.setResourceName("/opt/share/jar/main.jar");
mainJar.setRes("I'm main.jar");
javaParameters.setMainJar(mainJar);
return javaParameters;
}
@ -253,11 +249,23 @@ public class JavaTaskTest {
taskExecutionContext.setTaskParams(JSONUtils.toJsonString(createJavaParametersObject(RUN_TYPE_JAVA)));
taskExecutionContext.setExecutePath("/tmp/dolphinscheduler/test/executepath");
taskExecutionContext.setTaskAppId("runJavaType");
HashMap<String, String> map = new HashMap<>();
map.put("/opt/share/jar/resource2.jar", "opt/share/jar/resource2.jar");
map.put("/opt/share/jar/main.jar", "opt/share/jar/main.jar");
map.put("/JavaTaskTest.java", "JavaTaskTest.java");
taskExecutionContext.setResources(map);
ResourceContext.ResourceItem resourceItem1 = new ResourceContext.ResourceItem();
resourceItem1.setResourceAbsolutePathInStorage("/opt/share/jar/resource2.jar");
resourceItem1.setResourceAbsolutePathInLocal("opt/share/jar/resource2.jar");
ResourceContext.ResourceItem resourceItem2 = new ResourceContext.ResourceItem();
resourceItem2.setResourceAbsolutePathInStorage("/opt/share/jar/main.jar");
resourceItem2.setResourceAbsolutePathInLocal("opt/share/jar/main.jar");
ResourceContext.ResourceItem resourceItem3 = new ResourceContext.ResourceItem();
resourceItem2.setResourceAbsolutePathInStorage("/JavaTaskTest.java");
resourceItem2.setResourceAbsolutePathInLocal("JavaTaskTest.java");
ResourceContext resourceContext = new ResourceContext();
resourceContext.addResourceItem(resourceItem1);
resourceContext.addResourceItem(resourceItem2);
resourceContext.addResourceItem(resourceItem3);
taskExecutionContext.setResourceContext(resourceContext);
JavaTask javaTask = new JavaTask(taskExecutionContext);
javaTask.init();
return javaTask;
@ -268,15 +276,24 @@ public class JavaTaskTest {
*
* @return JavaTask
**/
public JavaTask runJarType() {
private JavaTask runJarType() {
TaskExecutionContext taskExecutionContext = new TaskExecutionContext();
taskExecutionContext.setTaskParams(JSONUtils.toJsonString(createJavaParametersObject(RUN_TYPE_JAR)));
taskExecutionContext.setExecutePath("/tmp/dolphinscheduler/test/executepath");
taskExecutionContext.setTaskAppId("runJavaType");
HashMap<String, String> map = new HashMap<>();
map.put("/opt/share/jar/resource2.jar", "opt/share/jar/resource2.jar");
map.put("/opt/share/jar/main.jar", "opt/share/jar/main.jar");
taskExecutionContext.setResources(map);
ResourceContext.ResourceItem resourceItem1 = new ResourceContext.ResourceItem();
resourceItem1.setResourceAbsolutePathInStorage("/opt/share/jar/resource2.jar");
resourceItem1.setResourceAbsolutePathInLocal("opt/share/jar/resource2.jar");
ResourceContext.ResourceItem resourceItem2 = new ResourceContext.ResourceItem();
resourceItem2.setResourceAbsolutePathInStorage("/opt/share/jar/main.jar");
resourceItem2.setResourceAbsolutePathInLocal("opt/share/jar/main.jar");
ResourceContext resourceContext = new ResourceContext();
resourceContext.addResourceItem(resourceItem1);
resourceContext.addResourceItem(resourceItem2);
taskExecutionContext.setResourceContext(resourceContext);
JavaTask javaTask = new JavaTask(taskExecutionContext);
javaTask.init();
return javaTask;

4
dolphinscheduler-task-plugin/dolphinscheduler-task-mr/src/main/java/org/apache/dolphinscheduler/plugin/task/mr/MapReduceArgsUtils.java

@ -24,6 +24,7 @@ import static org.apache.dolphinscheduler.plugin.task.mr.MapReduceTaskConstants.
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
import org.apache.dolphinscheduler.plugin.task.api.resource.ResourceContext;
import org.apache.dolphinscheduler.plugin.task.api.utils.ArgsUtils;
import org.apache.commons.lang3.StringUtils;
@ -52,7 +53,8 @@ public class MapReduceArgsUtils {
ResourceInfo mainJar = param.getMainJar();
if (mainJar != null) {
args.add(JAR);
args.add(taskExecutionContext.getResources().get(mainJar.getResourceName()));
ResourceContext resourceContext = taskExecutionContext.getResourceContext();
args.add(resourceContext.getResourceItem(mainJar.getResourceName()).getResourceAbsolutePathInLocal());
}
ProgramType programType = param.getProgramType();

8
dolphinscheduler-task-plugin/dolphinscheduler-task-spark/src/main/java/org/apache/dolphinscheduler/plugin/task/spark/SparkTask.java

@ -31,6 +31,7 @@ import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
import org.apache.dolphinscheduler.plugin.task.api.resource.ResourceContext;
import org.apache.dolphinscheduler.plugin.task.api.utils.ArgsUtils;
import org.apache.dolphinscheduler.plugin.task.api.utils.ParameterUtils;
@ -179,7 +180,8 @@ public class SparkTask extends AbstractYarnTask {
ResourceInfo mainJar = sparkParameters.getMainJar();
if (programType != ProgramType.SQL) {
args.add(taskExecutionContext.getResources().get(mainJar.getResourceName()));
ResourceContext resourceContext = taskExecutionContext.getResourceContext();
args.add(resourceContext.getResourceItem(mainJar.getResourceName()).getResourceAbsolutePathInLocal());
}
String mainArgs = sparkParameters.getMainArgs();
@ -200,8 +202,10 @@ public class SparkTask extends AbstractYarnTask {
try {
resourceFileName = resourceInfos.get(0).getResourceName();
ResourceContext resourceContext = taskExecutionContext.getResourceContext();
sqlContent = FileUtils.readFileToString(
new File(String.format("%s/%s", taskExecutionContext.getExecutePath(), resourceFileName)),
new File(
resourceContext.getResourceItem(resourceFileName).getResourceAbsolutePathInLocal()),
StandardCharsets.UTF_8);
} catch (IOException e) {
log.error("read sql content from file {} error ", resourceFileName, e);

14
dolphinscheduler-task-plugin/dolphinscheduler-task-spark/src/test/java/org/apache/dolphinscheduler/plugin/task/spark/SparkTaskTest.java

@ -20,9 +20,9 @@ package org.apache.dolphinscheduler.plugin.task.spark;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
import org.apache.dolphinscheduler.plugin.task.api.resource.ResourceContext;
import java.util.Collections;
import java.util.HashMap;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
@ -61,10 +61,14 @@ public class SparkTaskTest {
public void testBuildCommandWithSparkSubmit() {
String parameters = buildSparkParametersWithSparkSubmit();
TaskExecutionContext taskExecutionContext = Mockito.mock(TaskExecutionContext.class);
HashMap<String, String> map = new HashMap<>();
map.put("/lib/dolphinscheduler-task-spark.jar", "/lib/dolphinscheduler-task-spark.jar");
Mockito.when(taskExecutionContext.getResources()).thenReturn(map);
ResourceContext.ResourceItem resourceItem = new ResourceContext.ResourceItem();
resourceItem.setResourceAbsolutePathInStorage("/lib/dolphinscheduler-task-spark.jar");
resourceItem.setResourceAbsolutePathInLocal("/lib/dolphinscheduler-task-spark.jar");
ResourceContext resourceContext = new ResourceContext();
resourceContext.addResourceItem(resourceItem);
Mockito.when(taskExecutionContext.getTaskParams()).thenReturn(parameters);
Mockito.when(taskExecutionContext.getResourceContext()).thenReturn(resourceContext);
SparkTask sparkTask = Mockito.spy(new SparkTask(taskExecutionContext));
sparkTask.init();
Assertions.assertEquals(
@ -113,8 +117,6 @@ public class SparkTaskTest {
sparkParameters.setExecutorMemory("1G");
sparkParameters.setExecutorCores(2);
ResourceInfo resourceInfo = new ResourceInfo();
resourceInfo.setId(1);
resourceInfo.setRes("dolphinscheduler-task-spark.jar");
resourceInfo.setResourceName("/lib/dolphinscheduler-task-spark.jar");
sparkParameters.setMainJar(resourceInfo);
return JSONUtils.toJsonString(sparkParameters);

9
dolphinscheduler-task-plugin/dolphinscheduler-task-sql/src/main/java/org/apache/dolphinscheduler/plugin/task/sql/SqlTask.java

@ -20,7 +20,6 @@ package org.apache.dolphinscheduler.plugin.task.sql;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.plugin.datasource.api.plugin.DataSourceClientProvider;
import org.apache.dolphinscheduler.plugin.datasource.api.utils.CommonUtils;
import org.apache.dolphinscheduler.plugin.datasource.api.utils.DataSourceUtils;
import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
import org.apache.dolphinscheduler.plugin.task.api.SQLTaskExecutionContext;
@ -36,6 +35,7 @@ import org.apache.dolphinscheduler.plugin.task.api.model.TaskAlertInfo;
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
import org.apache.dolphinscheduler.plugin.task.api.parameters.SqlParameters;
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.UdfFuncParameters;
import org.apache.dolphinscheduler.plugin.task.api.resource.ResourceContext;
import org.apache.dolphinscheduler.plugin.task.api.utils.ParameterUtils;
import org.apache.dolphinscheduler.plugin.task.sql.utils.SqlSplitUtils;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
@ -518,11 +518,10 @@ public class SqlTask extends AbstractTask {
*/
private List<String> buildJarSql(List<UdfFuncParameters> udfFuncParameters) {
return udfFuncParameters.stream().map(value -> {
String defaultFS = value.getDefaultFS();
String prefixPath = defaultFS.startsWith("file://") ? "file://" : defaultFS;
String uploadPath = CommonUtils.getHdfsUdfDir(value.getTenantCode());
String resourceFullName = value.getResourceName();
return String.format("add jar %s", resourceFullName);
ResourceContext resourceContext = taskExecutionContext.getResourceContext();
return String.format("add jar %s",
resourceContext.getResourceItem(resourceFullName).getResourceAbsolutePathInLocal());
}).collect(Collectors.toList());
}

4
dolphinscheduler-tools/src/main/java/org/apache/dolphinscheduler/tools/resource/MigrateResource.java

@ -17,6 +17,8 @@
package org.apache.dolphinscheduler.tools.resource;
import java.sql.SQLException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
@ -45,7 +47,7 @@ public class MigrateResource {
private MigrateResourceService migrateResourceService;
@Override
public void run(String... args) {
public void run(String... args) throws SQLException {
String targetTenantCode = args[0];
logger.info("Moving all unmanaged resources to tenant: {}", targetTenantCode);
migrateResourceService.migrateResourceOnce(targetTenantCode);

54
dolphinscheduler-tools/src/main/java/org/apache/dolphinscheduler/tools/resource/MigrateResourceService.java

@ -19,9 +19,7 @@ package org.apache.dolphinscheduler.tools.resource;
import static org.apache.dolphinscheduler.common.constants.Constants.FORMAT_S_S;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.UdfFunc;
import org.apache.dolphinscheduler.dao.mapper.ResourceMapper;
import org.apache.dolphinscheduler.dao.mapper.TenantMapper;
import org.apache.dolphinscheduler.dao.mapper.UdfFuncMapper;
import org.apache.dolphinscheduler.plugin.storage.api.StorageOperate;
@ -30,7 +28,16 @@ import org.apache.dolphinscheduler.spi.enums.ResourceType;
import org.apache.commons.lang3.StringUtils;
import java.io.IOException;
import java.sql.Connection;
import java.sql.PreparedStatement;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import javax.sql.DataSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -49,14 +56,14 @@ public class MigrateResourceService {
private TenantMapper tenantMapper;
@Autowired
private ResourceMapper resourceMapper;
private UdfFuncMapper udfFuncMapper;
@Autowired
private UdfFuncMapper udfFuncMapper;
private DataSource dataSource;
private static final String MIGRATE_BASE_DIR = ".migrate";
public void migrateResourceOnce(String targetTenantCode) {
public void migrateResourceOnce(String targetTenantCode) throws SQLException {
if (true != tenantMapper.existTenant(targetTenantCode)) {
logger.error("Tenant not exists!");
return;
@ -67,31 +74,50 @@ public class MigrateResourceService {
if (StringUtils.isEmpty(resMigrateBasePath) || StringUtils.isEmpty(udfMigrateBasePath)) {
return;
}
// migrate all unmanaged resources and udfs once
List<Resource> resources = resourceMapper.queryResourceExceptUserId(-1);
resources.forEach(resource -> {
List<Map<String, Object>> resources = getAllResources();
for (Map<String, Object> item : resources) {
String oriFullName = (String) item.get("full_name");
int type = (int) item.get("type");
int id = (int) item.get("id");
try {
String oriFullName = resource.getFullName();
oriFullName = oriFullName.startsWith("/") ? oriFullName.substring(1) : oriFullName;
if (resource.getType().equals(ResourceType.FILE)) {
if (ResourceType.FILE.getCode() == type) {
storageOperate.copy(oriFullName,
String.format(FORMAT_S_S, resMigrateBasePath, oriFullName), true, true);
} else if (resource.getType().equals(ResourceType.UDF)) {
} else if (ResourceType.UDF.getCode() == type) {
String fullName = String.format(FORMAT_S_S, udfMigrateBasePath, oriFullName);
storageOperate.copy(oriFullName, fullName, true, true);
// change relative udfs resourceName
List<UdfFunc> udfs = udfFuncMapper.listUdfByResourceId(new Integer[]{resource.getId()});
List<UdfFunc> udfs = udfFuncMapper.listUdfByResourceId(new Integer[]{id});
udfs.forEach(udf -> {
udf.setResourceName(fullName);
udfFuncMapper.updateById(udf);
});
}
} catch (IOException e) {
logger.error("Migrate resource failed: {}", e.getMessage());
logger.error("Migrate resource: {} failed: {}", item, e);
}
});
}
}
private List<Map<String, Object>> getAllResources() throws SQLException {
try (
Connection connection = dataSource.getConnection();
PreparedStatement preparedStatement =
connection.prepareStatement("select * from t_ds_resources where user_id != -1");
ResultSet resultSet = preparedStatement.executeQuery()) {
List<Map<String, Object>> result = new ArrayList<>();
while (resultSet.next()) {
Map<String, Object> item = new HashMap<>();
item.put("id", resultSet.getInt("id"));
item.put("full_name", resultSet.getString("full_name"));
item.put("type", resultSet.getInt("type"));
result.add(item);
}
return result;
}
}
public String createMigrateDirByType(String targetTenantCode, ResourceType type) {

30
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerTaskExecutor.java

@ -36,6 +36,7 @@ import org.apache.dolphinscheduler.plugin.datasource.api.utils.CommonUtils;
import org.apache.dolphinscheduler.plugin.storage.api.StorageOperate;
import org.apache.dolphinscheduler.plugin.task.api.AbstractTask;
import org.apache.dolphinscheduler.plugin.task.api.TaskCallBack;
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
import org.apache.dolphinscheduler.plugin.task.api.TaskException;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContextCacheManager;
@ -45,12 +46,13 @@ import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.log.TaskInstanceLogHeader;
import org.apache.dolphinscheduler.plugin.task.api.model.TaskAlertInfo;
import org.apache.dolphinscheduler.plugin.task.api.resource.ResourceContext;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.plugin.task.api.utils.ProcessUtils;
import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
import org.apache.dolphinscheduler.server.worker.registry.WorkerRegistryClient;
import org.apache.dolphinscheduler.server.worker.rpc.WorkerMessageSender;
import org.apache.dolphinscheduler.server.worker.utils.TaskExecutionCheckerUtils;
import org.apache.dolphinscheduler.server.worker.utils.TaskExecutionContextUtils;
import org.apache.dolphinscheduler.server.worker.utils.TaskFilesTransferUtils;
import java.io.File;
@ -208,23 +210,33 @@ public abstract class WorkerTaskExecutor implements Runnable {
log.info("Send task status {} master: {}", TaskExecutionStatus.RUNNING_EXECUTION.name(),
taskExecutionContext.getHost());
TaskExecutionCheckerUtils.checkTenantExist(workerConfig, taskExecutionContext);
// In most of case the origin tenant is the same as the current tenant
// Except `default` tenant. The originTenant is used to download the resources
String originTenant = taskExecutionContext.getTenantCode();
String tenant = TaskExecutionContextUtils.getOrCreateTenant(workerConfig, taskExecutionContext);
taskExecutionContext.setTenantCode(tenant);
log.info("TenantCode: {} check successfully", taskExecutionContext.getTenantCode());
TaskExecutionCheckerUtils.createProcessLocalPathIfAbsent(taskExecutionContext);
TaskExecutionContextUtils.createProcessLocalPathIfAbsent(taskExecutionContext);
log.info("WorkflowInstanceExecDir: {} check successfully", taskExecutionContext.getExecutePath());
TaskExecutionCheckerUtils.downloadResourcesIfNeeded(storageOperate, taskExecutionContext);
log.info("Download resources: {} successfully", taskExecutionContext.getResources());
TaskChannel taskChannel =
Optional.ofNullable(taskPluginManager.getTaskChannelMap().get(taskExecutionContext.getTaskType()))
.orElseThrow(() -> new TaskPluginException(taskExecutionContext.getTaskType()
+ " task plugin not found, please check the task type is correct."));
log.info("Create TaskChannel: {} successfully", taskChannel.getClass().getName());
ResourceContext resourceContext = TaskExecutionContextUtils.downloadResourcesIfNeeded(originTenant, taskChannel,
storageOperate, taskExecutionContext);
taskExecutionContext.setResourceContext(resourceContext);
log.info("Download resources successfully: \n{}", taskExecutionContext.getResourceContext());
TaskFilesTransferUtils.downloadUpstreamFiles(taskExecutionContext, storageOperate);
log.info("Download upstream files: {} successfully",
TaskFilesTransferUtils.getFileLocalParams(taskExecutionContext, Direct.IN));
task = Optional.ofNullable(taskPluginManager.getTaskChannelMap().get(taskExecutionContext.getTaskType()))
.map(taskChannel -> taskChannel.createTask(taskExecutionContext))
.orElseThrow(() -> new TaskPluginException(taskExecutionContext.getTaskType()
+ " task plugin not found, please check the task type is correct."));
task = taskChannel.createTask(taskExecutionContext);
log.info("Task plugin instance: {} create successfully", taskExecutionContext.getTaskType());
// todo: remove the init method, this should initialize in constructor method

88
dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/utils/TaskExecutionCheckerUtils.java → dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/utils/TaskExecutionContextUtils.java

@ -21,36 +21,37 @@ import org.apache.dolphinscheduler.common.constants.TenantConstants;
import org.apache.dolphinscheduler.common.utils.FileUtils;
import org.apache.dolphinscheduler.common.utils.OSUtils;
import org.apache.dolphinscheduler.plugin.storage.api.StorageOperate;
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
import org.apache.dolphinscheduler.plugin.task.api.TaskException;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.model.ResourceInfo;
import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
import org.apache.dolphinscheduler.plugin.task.api.resource.ResourceContext;
import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
import org.apache.dolphinscheduler.server.worker.metrics.WorkerServerMetrics;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.collections4.MapUtils;
import org.apache.commons.lang3.SystemUtils;
import org.apache.commons.lang3.tuple.Pair;
import java.io.File;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import lombok.extern.slf4j.Slf4j;
@Slf4j
public class TaskExecutionCheckerUtils {
public class TaskExecutionContextUtils {
public static void checkTenantExist(WorkerConfig workerConfig, TaskExecutionContext taskExecutionContext) {
public static String getOrCreateTenant(WorkerConfig workerConfig, TaskExecutionContext taskExecutionContext) {
try {
String tenantCode = taskExecutionContext.getTenantCode();
if (TenantConstants.DEFAULT_TENANT_CODE.equals(tenantCode)) {
log.warn("Current tenant is default tenant, will use {} to execute the task",
log.info("Current tenant is default tenant, will use {} to execute the task",
TenantConstants.BOOTSTRAPT_SYSTEM_USER);
return;
return TenantConstants.BOOTSTRAPT_SYSTEM_USER;
}
boolean osUserExistFlag;
// if Using distributed is true and Currently supported systems are linux,Should not let it
@ -70,6 +71,7 @@ public class TaskExecutionCheckerUtils {
throw new TaskException(
String.format("TenantCode: %s doesn't exist", tenantCode));
}
return tenantCode;
} catch (TaskException ex) {
throw ex;
} catch (Exception ex) {
@ -97,51 +99,55 @@ public class TaskExecutionCheckerUtils {
}
}
public static void downloadResourcesIfNeeded(StorageOperate storageOperate,
TaskExecutionContext taskExecutionContext) {
String execLocalPath = taskExecutionContext.getExecutePath();
String tenant = taskExecutionContext.getTenantCode();
String actualTenant =
TenantConstants.DEFAULT_TENANT_CODE.equals(tenant) ? TenantConstants.BOOTSTRAPT_SYSTEM_USER : tenant;
public static ResourceContext downloadResourcesIfNeeded(String tenant,
TaskChannel taskChannel,
StorageOperate storageOperate,
TaskExecutionContext taskExecutionContext) {
AbstractParameters abstractParameters = taskChannel.parseParameters(
ParametersNode.builder()
.taskType(taskExecutionContext.getTaskType())
.taskParams(taskExecutionContext.getTaskParams())
.build());
Map<String, String> projectRes = taskExecutionContext.getResources();
if (MapUtils.isEmpty(projectRes)) {
return;
List<ResourceInfo> resourceFilesList = abstractParameters.getResourceFilesList();
if (CollectionUtils.isEmpty(resourceFilesList)) {
log.debug("There is no resource file need to download");
return new ResourceContext();
}
List<Pair<String, String>> downloadFiles = new ArrayList<>();
projectRes.keySet().forEach(fullName -> {
String fileName = storageOperate.getResourceFileName(actualTenant, fullName);
projectRes.put(fullName, fileName);
File resFile = new File(execLocalPath, fileName);
boolean notExist = !resFile.exists();
if (notExist) {
downloadFiles.add(Pair.of(fullName, fileName));
} else {
log.warn("Resource file : {} already exists will not download again ", resFile.getName());
}
});
if (CollectionUtils.isNotEmpty(downloadFiles)) {
for (Pair<String, String> fileDownload : downloadFiles) {
try {
String fullName = fileDownload.getLeft();
String fileName = fileDownload.getRight();
ResourceContext resourceContext = new ResourceContext();
String taskWorkingDirectory = taskExecutionContext.getExecutePath();
for (ResourceInfo resourceInfo : resourceFilesList) {
String resourceAbsolutePathInStorage = resourceInfo.getResourceName();
String resourceRelativePath = storageOperate.getResourceFileName(tenant, resourceAbsolutePathInStorage);
String resourceAbsolutePathInLocal = Paths.get(taskWorkingDirectory, resourceRelativePath).toString();
File file = new File(resourceAbsolutePathInLocal);
if (!file.exists()) {
try {
long resourceDownloadStartTime = System.currentTimeMillis();
Path localFileAbsolutePath = Paths.get(execLocalPath, fileName);
storageOperate.download(actualTenant, fullName, localFileAbsolutePath.toString(), true);
log.info("Download resource file {} under: {} successfully", fileName, localFileAbsolutePath);
storageOperate.download(resourceAbsolutePathInStorage, resourceAbsolutePathInLocal, true);
log.debug("Download resource file {} under: {} successfully", resourceAbsolutePathInStorage,
resourceAbsolutePathInLocal);
WorkerServerMetrics
.recordWorkerResourceDownloadTime(System.currentTimeMillis() - resourceDownloadStartTime);
WorkerServerMetrics.recordWorkerResourceDownloadSize(Files.size(localFileAbsolutePath));
WorkerServerMetrics
.recordWorkerResourceDownloadSize(Files.size(Paths.get(resourceAbsolutePathInLocal)));
WorkerServerMetrics.incWorkerResourceDownloadSuccessCount();
} catch (Exception e) {
} catch (Exception ex) {
WorkerServerMetrics.incWorkerResourceDownloadFailureCount();
throw new TaskException(String.format("Download resource file: %s error", fileDownload), e);
throw new TaskException(
String.format("Download resource file: %s error", resourceAbsolutePathInStorage), ex);
}
}
ResourceContext.ResourceItem resourceItem = ResourceContext.ResourceItem.builder()
.resourceAbsolutePathInStorage(resourceAbsolutePathInStorage)
.resourceRelativePath(resourceRelativePath)
.resourceAbsolutePathInLocal(resourceAbsolutePathInLocal)
.build();
resourceContext.addResourceItem(resourceItem);
}
return resourceContext;
}
}

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

@ -178,7 +178,7 @@ public class TaskFilesTransferUtils {
String resourceWholePath =
storageOperate.getResourceFullName(taskExecutionContext.getTenantCode(), resourcePath);
log.info("{} --- Remote:{} to Local:{}", property, resourceWholePath, downloadPath);
storageOperate.download(taskExecutionContext.getTenantCode(), resourceWholePath, downloadPath, true);
storageOperate.download(resourceWholePath, downloadPath, true);
} catch (IOException ex) {
throw new TaskException("Download file from storage error", ex);
}

Loading…
Cancel
Save