From e83482c6a0e0d58c06dc6a8256b8fbf09aa10ceb Mon Sep 17 00:00:00 2001 From: SbloodyS <460888207@qq.com> Date: Wed, 4 Dec 2024 09:37:37 +0800 Subject: [PATCH 01/15] feature 16834 --- .../ProjectWorkerGroupRelationService.java | 3 - .../api/service/WorkerGroupService.java | 2 + ...ProjectWorkerGroupRelationServiceImpl.java | 46 +++++---- .../service/impl/WorkerGroupServiceImpl.java | 93 +++++++++++++------ .../api/service/MonitorServiceTest.java | 2 +- ...ProjectWorkerGroupRelationServiceTest.java | 5 - .../common/enums/WorkerGroupSource.java | 38 ++++++++ .../dolphinscheduler/common/model/Server.java | 7 +- .../common/model/WorkerHeartBeat.java | 2 +- .../dao/entity/WorkerGroupInfo.java | 41 ++++++++ .../dao/mapper/ProjectWorkerGroupMapper.java | 11 +++ .../dao/repository/ProjectWorkerGroupDao.java | 36 +++++++ .../impl/ProjectWorkerGroupDaoImpl.java | 61 ++++++++++++ .../dao/utils/WorkerGroupUtils.java | 3 +- .../dao/mapper/ProjectWorkerGroupMapper.xml | 60 ++++++++++++ .../cluster/WorkerGroupChangeNotifier.java | 30 +++++- .../PhysicalTaskExecutorClientDelegator.java | 4 +- .../registry/api/RegistryClient.java | 4 +- .../src/main/resources/application.yaml | 2 + .../src/main/resources/logback-spring.xml | 3 +- .../src/locales/en_US/security.ts | 1 + .../src/locales/zh_CN/security.ts | 1 + .../list/components/use-worker-group.ts | 6 +- .../security/worker-group-manage/use-table.ts | 5 + .../server/worker/config/WorkerConfig.java | 1 + .../worker/task/WorkerHeartBeatTask.java | 1 + .../src/main/resources/application.yaml | 2 + 27 files changed, 393 insertions(+), 77 deletions(-) create mode 100644 dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/WorkerGroupSource.java create mode 100644 dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroupInfo.java create mode 100644 dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProjectWorkerGroupDao.java create mode 100644 dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProjectWorkerGroupDaoImpl.java create mode 100644 dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectWorkerGroupMapper.xml diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationService.java index 5b809f8d6157..bfffcd396080 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationService.java @@ -23,9 +23,6 @@ import java.util.List; import java.util.Map; -/** - * the service of project and worker group - */ public interface ProjectWorkerGroupRelationService { /** diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/WorkerGroupService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/WorkerGroupService.java index 13cd3c3e35c9..2556d3208ef9 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/WorkerGroupService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/WorkerGroupService.java @@ -20,6 +20,7 @@ import org.apache.dolphinscheduler.api.utils.Result; import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; +import org.apache.dolphinscheduler.dao.entity.WorkerGroupInfo; import java.util.List; import java.util.Map; @@ -79,4 +80,5 @@ public interface WorkerGroupService { */ Map queryWorkerGroupByWorkflowDefinitionCodes(List workflowDefinitionCodeList); + List queryAllWorkerGroupList(); } diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProjectWorkerGroupRelationServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProjectWorkerGroupRelationServiceImpl.java index c11915667fc4..99e8bc81a9c2 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProjectWorkerGroupRelationServiceImpl.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProjectWorkerGroupRelationServiceImpl.java @@ -21,6 +21,7 @@ import org.apache.dolphinscheduler.api.exceptions.ServiceException; import org.apache.dolphinscheduler.api.service.ProjectService; import org.apache.dolphinscheduler.api.service.ProjectWorkerGroupRelationService; +import org.apache.dolphinscheduler.api.service.WorkerGroupService; import org.apache.dolphinscheduler.api.utils.Result; import org.apache.dolphinscheduler.common.constants.Constants; import org.apache.dolphinscheduler.dao.entity.Project; @@ -32,12 +33,15 @@ import org.apache.dolphinscheduler.dao.mapper.ScheduleMapper; import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper; import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper; +import org.apache.dolphinscheduler.dao.repository.ProjectWorkerGroupDao; import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.collections4.SetUtils; import org.apache.commons.lang3.StringUtils; +import java.util.ArrayList; +import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.HashSet; @@ -55,9 +59,6 @@ import com.baomidou.mybatisplus.core.conditions.query.QueryWrapper; -/** - * task definition service impl - */ @Service @Slf4j public class ProjectWorkerGroupRelationServiceImpl extends BaseServiceImpl @@ -65,14 +66,11 @@ public class ProjectWorkerGroupRelationServiceImpl extends BaseServiceImpl ProjectWorkerGroupRelationService { @Autowired - private ProjectWorkerGroupMapper projectWorkerGroupMapper; + private ProjectWorkerGroupDao projectWorkerGroupDao; @Autowired private ProjectMapper projectMapper; - @Autowired - private WorkerGroupMapper workerGroupMapper; - @Autowired private TaskDefinitionMapper taskDefinitionMapper; @@ -82,6 +80,9 @@ public class ProjectWorkerGroupRelationServiceImpl extends BaseServiceImpl @Autowired private ProjectService projectService; + @Autowired + private WorkerGroupService workerGroupService; + /** * assign worker groups to a project * @@ -105,7 +106,12 @@ public Result assignWorkerGroupsToProject(User loginUser, Long projectCode, List } if (CollectionUtils.isEmpty(workerGroups)) { - putMsg(result, Status.WORKER_GROUP_TO_PROJECT_IS_EMPTY); + boolean deleted = projectWorkerGroupDao.deleteByProjectCode(projectCode); + if (deleted) { + putMsg(result, Status.SUCCESS); + } else { + putMsg(result, Status.ASSIGN_WORKER_GROUP_TO_PROJECT_ERROR); + } return result; } @@ -116,7 +122,7 @@ public Result assignWorkerGroupsToProject(User loginUser, Long projectCode, List } Set workerGroupNames = - workerGroupMapper.queryAllWorkerGroup().stream().map(WorkerGroup::getName).collect( + workerGroupService.queryAllWorkerGroupList().stream().map(WorkerGroup::getName).collect( Collectors.toSet()); workerGroupNames.add(WorkerGroupUtils.getDefaultWorkerGroup()); @@ -130,12 +136,7 @@ public Result assignWorkerGroupsToProject(User loginUser, Long projectCode, List return result; } - Set projectWorkerGroupNames = projectWorkerGroupMapper.selectList(new QueryWrapper() - .lambda() - .eq(ProjectWorkerGroup::getProjectCode, projectCode)) - .stream() - .map(ProjectWorkerGroup::getWorkerGroup) - .collect(Collectors.toSet()); + Set projectWorkerGroupNames = projectWorkerGroupDao.queryAssignedWorkerGroupNamesByProjectCode(projectCode); difference = SetUtils.difference(projectWorkerGroupNames, assignedWorkerGroupNames); @@ -147,10 +148,8 @@ public Result assignWorkerGroupsToProject(User loginUser, Long projectCode, List SetUtils.intersection(usedWorkerGroups, difference).toSet()); } - int deleted = projectWorkerGroupMapper.delete( - new QueryWrapper().lambda().eq(ProjectWorkerGroup::getProjectCode, projectCode) - .in(ProjectWorkerGroup::getWorkerGroup, difference)); - if (deleted > 0) { + boolean deleted = projectWorkerGroupDao.deleteByProjectCodeAndWorkerGroups(projectCode, new ArrayList<>(difference)); + if (deleted) { log.info("Success to delete worker groups [{}] for the project [{}] .", difference, project.getName()); } else { log.error("Failed to delete worker groups [{}] for the project [{}].", difference, project.getName()); @@ -161,13 +160,13 @@ public Result assignWorkerGroupsToProject(User loginUser, Long projectCode, List difference = SetUtils.difference(assignedWorkerGroupNames, projectWorkerGroupNames); Date now = new Date(); if (CollectionUtils.isNotEmpty(difference)) { - difference.stream().forEach(workerGroupName -> { + difference.forEach(workerGroupName -> { ProjectWorkerGroup projectWorkerGroup = new ProjectWorkerGroup(); projectWorkerGroup.setProjectCode(projectCode); projectWorkerGroup.setWorkerGroup(workerGroupName); projectWorkerGroup.setCreateTime(now); projectWorkerGroup.setUpdateTime(now); - int create = projectWorkerGroupMapper.insert(projectWorkerGroup); + int create = projectWorkerGroupDao.insert(projectWorkerGroup); if (create > 0) { log.info("Success to add worker group [{}] for the project [{}] .", workerGroupName, project.getName()); @@ -201,9 +200,8 @@ public Map queryWorkerGroupsByProject(User loginUser, Long proje Set assignedWorkerGroups = getAllUsedWorkerGroups(project); - projectWorkerGroupMapper.selectList( - new QueryWrapper().lambda().eq(ProjectWorkerGroup::getProjectCode, projectCode)) - .stream().forEach(projectWorkerGroup -> assignedWorkerGroups.add(projectWorkerGroup.getWorkerGroup())); + projectWorkerGroupDao.queryByProjectCode(projectCode) + .forEach(projectWorkerGroup -> assignedWorkerGroups.add(projectWorkerGroup.getWorkerGroup())); List projectWorkerGroups = assignedWorkerGroups.stream().map(workerGroup -> { ProjectWorkerGroup projectWorkerGroup = new ProjectWorkerGroup(); diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkerGroupServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkerGroupServiceImpl.java index 4173a0c9e41a..e0c804ca51ec 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkerGroupServiceImpl.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkerGroupServiceImpl.java @@ -28,14 +28,17 @@ import org.apache.dolphinscheduler.common.constants.Constants; import org.apache.dolphinscheduler.common.enums.AuthorizationType; import org.apache.dolphinscheduler.common.enums.UserType; +import org.apache.dolphinscheduler.common.enums.WorkerGroupSource; import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; import org.apache.dolphinscheduler.common.model.Server; +import org.apache.dolphinscheduler.common.model.WorkerHeartBeat; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.dao.entity.EnvironmentWorkerGroupRelation; import org.apache.dolphinscheduler.dao.entity.Schedule; import org.apache.dolphinscheduler.dao.entity.TaskDefinition; import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; +import org.apache.dolphinscheduler.dao.entity.WorkerGroupInfo; import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; import org.apache.dolphinscheduler.dao.mapper.EnvironmentWorkerGroupRelationMapper; import org.apache.dolphinscheduler.dao.mapper.ScheduleMapper; @@ -43,9 +46,12 @@ import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper; import org.apache.dolphinscheduler.dao.mapper.WorkflowDefinitionMapper; import org.apache.dolphinscheduler.dao.mapper.WorkflowInstanceMapper; +<<<<<<< HEAD import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils; import org.apache.dolphinscheduler.extract.base.client.Clients; import org.apache.dolphinscheduler.extract.master.IMasterContainerService; +======= +>>>>>>> 8aa5dbba0 (feature 16834) import org.apache.dolphinscheduler.registry.api.RegistryClient; import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; @@ -232,7 +238,7 @@ public Result queryAllGroupPaging(User loginUser, Integer pageNo, Integer pageSi int toIndex = (pageNo - 1) * pageSize + pageSize; Result result = new Result(); - List workerGroups; + List workerGroups; if (loginUser.getUserType().equals(UserType.ADMIN_USER)) { workerGroups = getWorkerGroups(null); } else { @@ -240,14 +246,14 @@ public Result queryAllGroupPaging(User loginUser, Integer pageNo, Integer pageSi .userOwnedResourceIdsAcquisition(AuthorizationType.WORKER_GROUP, loginUser.getId(), log); workerGroups = getWorkerGroups(ids.isEmpty() ? Collections.emptyList() : new ArrayList<>(ids)); } - List resultDataList = new ArrayList<>(); + List resultDataList = new ArrayList<>(); int total = 0; if (CollectionUtils.isNotEmpty(workerGroups)) { - List searchValDataList = new ArrayList<>(); + List searchValDataList = new ArrayList<>(); if (!StringUtils.isEmpty(searchVal)) { - for (WorkerGroup workerGroup : workerGroups) { + for (WorkerGroupInfo workerGroup : workerGroups) { if (workerGroup.getName().contains(searchVal)) { searchValDataList.add(workerGroup); } @@ -264,7 +270,7 @@ public Result queryAllGroupPaging(User loginUser, Integer pageNo, Integer pageSi } } - PageInfo pageInfo = new PageInfo<>(pageNo, pageSize); + PageInfo pageInfo = new PageInfo<>(pageNo, pageSize); pageInfo.setTotal(total); pageInfo.setTotalList(resultDataList); @@ -282,7 +288,7 @@ public Result queryAllGroupPaging(User loginUser, Integer pageNo, Integer pageSi @Override public Map queryAllGroup(User loginUser) { Map result = new HashMap<>(); - List workerGroups; + List workerGroups; if (loginUser.getUserType().equals(UserType.ADMIN_USER)) { workerGroups = getWorkerGroups(null); } else { @@ -298,36 +304,69 @@ public Map queryAllGroup(User loginUser) { return result; } - /** - * get worker groups - * - * @return WorkerGroup list - */ - private List getWorkerGroups(List ids) { + private List getWorkerGroups(List ids) { // worker groups from database - List workerGroups; + List workerGroups; if (ids != null) { - workerGroups = ids.isEmpty() ? new ArrayList<>() : workerGroupMapper.selectBatchIds(ids); + workerGroups = ids.isEmpty() ? new ArrayList<>() : workerGroupMapper.selectBatchIds(ids).stream().map(workerGroup -> { + WorkerGroupInfo workerGroupInfo = new WorkerGroupInfo(workerGroup); + workerGroupInfo.setSource(WorkerGroupSource.UI); + return workerGroupInfo; + }).collect(Collectors.toList()); } else { - workerGroups = workerGroupMapper.queryAllWorkerGroup(); + workerGroups = workerGroupMapper.queryAllWorkerGroup().stream().map(workerGroup -> { + WorkerGroupInfo workerGroupInfo = new WorkerGroupInfo(workerGroup); + workerGroupInfo.setSource(WorkerGroupSource.UI); + return workerGroupInfo; + }).collect(Collectors.toList()); } - boolean containDefaultWorkerGroups = workerGroups.stream() - .anyMatch(workerGroup -> WorkerGroupUtils.isWorkerGroupEmpty(workerGroup.getName())); - if (!containDefaultWorkerGroups) { - // there doesn't exist a default WorkerGroup, we will add all worker to the default worker group. - Set activeWorkerNodes = registryClient.getServerNodeSet(RegistryNodeType.WORKER); - WorkerGroup defaultWorkerGroup = new WorkerGroup(); - defaultWorkerGroup.setName(WorkerGroupUtils.getDefaultWorkerGroup()); - defaultWorkerGroup.setAddrList(String.join(Constants.COMMA, activeWorkerNodes)); - defaultWorkerGroup.setCreateTime(new Date()); - defaultWorkerGroup.setUpdateTime(new Date()); - defaultWorkerGroup.setSystemDefault(true); - workerGroups.add(defaultWorkerGroup); + + // get worker groups from registry + List activeWorkerNodes = registryClient.getServerList(RegistryNodeType.WORKER); + Map configWorkerGroups = activeWorkerNodes.stream() + .collect(Collectors.groupingBy(activeWorkerNode -> JSONUtils.parseObject(activeWorkerNode.getHeartBeatInfo(), WorkerHeartBeat.class).getWorkerGroup(), + Collectors.mapping(Server::getHost, Collectors.joining(Constants.COMMA)))); + for (Server activeWorkerNode : activeWorkerNodes) { + WorkerGroupInfo configWorkerGroupInfo = new WorkerGroupInfo(); + WorkerHeartBeat workerHeartBeat = JSONUtils.parseObject(activeWorkerNode.getHeartBeatInfo(), WorkerHeartBeat.class); + configWorkerGroupInfo.setName(workerHeartBeat.getWorkerGroup()); + configWorkerGroupInfo.setAddrList(configWorkerGroups.get(workerHeartBeat.getWorkerGroup())); + configWorkerGroupInfo.setCreateTime(new Date()); + configWorkerGroupInfo.setUpdateTime(new Date()); + configWorkerGroupInfo.setSystemDefault(true); + configWorkerGroupInfo.setSource(WorkerGroupSource.CONFIG); + workerGroups.add(configWorkerGroupInfo); } return workerGroups; } + public List queryAllWorkerGroupList() { + List workerGroupList; + workerGroupList = workerGroupMapper.queryAllWorkerGroup().stream() + .map(workerGroup -> { + WorkerGroupInfo workerGroupInfo = new WorkerGroupInfo(workerGroup); + workerGroupInfo.setSource(WorkerGroupSource.UI); + return workerGroupInfo; + }).collect(Collectors.toList()); + List activeWorkerNodes = registryClient.getServerList(RegistryNodeType.WORKER); + Map configWorkerGroups = activeWorkerNodes.stream() + .collect(Collectors.groupingBy(activeWorkerNode -> JSONUtils.parseObject(activeWorkerNode.getHeartBeatInfo(), WorkerHeartBeat.class).getWorkerGroup(), + Collectors.mapping(Server::getHost, Collectors.joining(Constants.COMMA)))); + for (Server activeWorkerNode : activeWorkerNodes) { + WorkerGroupInfo configWorkerGroupInfo = new WorkerGroupInfo(); + WorkerHeartBeat workerHeartBeat = JSONUtils.parseObject(activeWorkerNode.getHeartBeatInfo(), WorkerHeartBeat.class); + configWorkerGroupInfo.setName(workerHeartBeat.getWorkerGroup()); + configWorkerGroupInfo.setAddrList(configWorkerGroups.get(workerHeartBeat.getWorkerGroup())); + configWorkerGroupInfo.setCreateTime(new Date()); + configWorkerGroupInfo.setUpdateTime(new Date()); + configWorkerGroupInfo.setSystemDefault(true); + configWorkerGroupInfo.setSource(WorkerGroupSource.CONFIG); + workerGroupList.add(configWorkerGroupInfo); + } + return workerGroupList; + } + /** * delete worker group by id * diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/MonitorServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/MonitorServiceTest.java index e1318e936dfb..e9eff1ecee33 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/MonitorServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/MonitorServiceTest.java @@ -131,7 +131,7 @@ private List getServerList() { Server server = new Server(); server.setId(1); server.setHost("127.0.0.1"); - server.setZkDirectory("ws/server"); + server.setServerDirectory("ws/server"); server.setPort(123); server.setCreateTime(new Date()); server.setLastHeartbeatTime(new Date()); diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationServiceTest.java index 6f6b7ca2d6c1..eccbab0c5b44 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationServiceTest.java @@ -95,11 +95,6 @@ public void testAssignWorkerGroupsToProject() { getWorkerGroups()); Assertions.assertEquals(Status.PROJECT_NOT_EXIST.getCode(), result.getCode()); - // worker group is empty - result = projectWorkerGroupRelationService.assignWorkerGroupsToProject(loginUser, projectCode, - Collections.emptyList()); - Assertions.assertEquals(Status.WORKER_GROUP_TO_PROJECT_IS_EMPTY.getCode(), result.getCode()); - // project not exists Mockito.when(projectMapper.queryByCode(projectCode)).thenReturn(null); result = projectWorkerGroupRelationService.assignWorkerGroupsToProject(loginUser, projectCode, diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/WorkerGroupSource.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/WorkerGroupSource.java new file mode 100644 index 000000000000..54e1b301398b --- /dev/null +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/WorkerGroupSource.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.common.enums; + + +import com.baomidou.mybatisplus.annotation.EnumValue; +import lombok.Getter; + +@Getter +public enum WorkerGroupSource { + CONFIG(0, "config"), + UI(1, "ui"); + + @EnumValue + private final int code; + private final String desc; + + WorkerGroupSource(int code, String desc) { + this.code = code; + this.desc = desc; + } + +} diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/Server.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/Server.java index 873a69bcafbc..448779270343 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/Server.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/Server.java @@ -30,12 +30,9 @@ public class Server { private int port; - private String zkDirectory; + private String serverDirectory; - /** - * resource info: CPU and memory - */ - private String resInfo; + private String heartBeatInfo; private Date createTime; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/WorkerHeartBeat.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/WorkerHeartBeat.java index 8c2727d681c1..6ccf01e4abfc 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/WorkerHeartBeat.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/WorkerHeartBeat.java @@ -30,5 +30,5 @@ public class WorkerHeartBeat extends BaseHeartBeat implements HeartBeat { private int workerHostWeight; // worker host weight private double threadPoolUsage; // worker waiting task count - + private String workerGroup; } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroupInfo.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroupInfo.java new file mode 100644 index 000000000000..0e4cc3498fdd --- /dev/null +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroupInfo.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.dao.entity; + +import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.NoArgsConstructor; +import org.apache.dolphinscheduler.common.enums.WorkerGroupSource; + +@EqualsAndHashCode(callSuper = true) +@Data +@NoArgsConstructor +public class WorkerGroupInfo extends WorkerGroup { + private WorkerGroupSource source; + + public WorkerGroupInfo(WorkerGroup workerGroup) { + this.setId(workerGroup.getId()); + this.setName(workerGroup.getName()); + this.setAddrList(workerGroup.getAddrList()); + this.setCreateTime(workerGroup.getCreateTime()); + this.setUpdateTime(workerGroup.getUpdateTime()); + this.setDescription(workerGroup.getDescription()); + this.setSystemDefault(workerGroup.isSystemDefault()); + this.setOtherParamsJson(workerGroup.getOtherParamsJson()); + } +} diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProjectWorkerGroupMapper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProjectWorkerGroupMapper.java index 40353d4c4a07..ae59b5c9fbe3 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProjectWorkerGroupMapper.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProjectWorkerGroupMapper.java @@ -20,7 +20,18 @@ import org.apache.dolphinscheduler.dao.entity.ProjectWorkerGroup; import com.baomidou.mybatisplus.core.mapper.BaseMapper; +import org.apache.ibatis.annotations.Param; + +import java.util.List; +import java.util.Set; public interface ProjectWorkerGroupMapper extends BaseMapper { + int deleteByProjectCode(@Param("projectCode") Long projectCode); + + Set queryAssignedWorkerGroupNamesByProjectCode(@Param("projectCode") Long projectCode); + + int deleteByProjectCodeAndWorkerGroups(@Param("projectCode") Long projectCode, + @Param("workerGroups") List workerGroups); + List queryByProjectCode(@Param("projectCode") Long projectCode); } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProjectWorkerGroupDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProjectWorkerGroupDao.java new file mode 100644 index 000000000000..810005d1c050 --- /dev/null +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProjectWorkerGroupDao.java @@ -0,0 +1,36 @@ +/* + * 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.repository; + +import org.apache.dolphinscheduler.dao.entity.Project; +import org.apache.dolphinscheduler.dao.entity.ProjectWorkerGroup; + +import java.util.Collection; +import java.util.List; +import java.util.Set; + +public interface ProjectWorkerGroupDao extends IDao { + + boolean deleteByProjectCode(Long projectCode); + + Set queryAssignedWorkerGroupNamesByProjectCode(Long projectCode); + + boolean deleteByProjectCodeAndWorkerGroups(Long projectCode, List workerGroups); + + List queryByProjectCode(Long projectCode); +} diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProjectWorkerGroupDaoImpl.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProjectWorkerGroupDaoImpl.java new file mode 100644 index 000000000000..ea127fe8c5c4 --- /dev/null +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProjectWorkerGroupDaoImpl.java @@ -0,0 +1,61 @@ +/* + * 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.repository.impl; + +import lombok.NonNull; +import org.apache.dolphinscheduler.dao.entity.ProjectWorkerGroup; +import org.apache.dolphinscheduler.dao.mapper.ProjectWorkerGroupMapper; +import org.apache.dolphinscheduler.dao.repository.BaseDao; +import org.apache.dolphinscheduler.dao.repository.ProjectWorkerGroupDao; +import org.springframework.stereotype.Repository; + +import java.util.List; +import java.util.Set; + +@Repository +public class ProjectWorkerGroupDaoImpl extends BaseDao implements ProjectWorkerGroupDao { + + public ProjectWorkerGroupDaoImpl(@NonNull ProjectWorkerGroupMapper projectWorkerGroupMapper) { + super(projectWorkerGroupMapper); + } + + @Override + public boolean deleteByProjectCode(Long projectCode) { + int deleted = mybatisMapper.deleteByProjectCode(projectCode); + return deleted > 0; + } + + @Override + public Set queryAssignedWorkerGroupNamesByProjectCode(Long projectCode) { + if (projectCode == null) { + return null; + } + return mybatisMapper.queryAssignedWorkerGroupNamesByProjectCode(projectCode); + } + + @Override + public boolean deleteByProjectCodeAndWorkerGroups(Long projectCode, List workerGroups) { + int deleted = mybatisMapper.deleteByProjectCodeAndWorkerGroups(projectCode, workerGroups); + return deleted > 0; + } + + @Override + public List queryByProjectCode(Long projectCode) { + return mybatisMapper.queryByProjectCode(projectCode); + } +} diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/WorkerGroupUtils.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/WorkerGroupUtils.java index 7535ddb80993..1707425940c8 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/WorkerGroupUtils.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/WorkerGroupUtils.java @@ -17,6 +17,7 @@ package org.apache.dolphinscheduler.dao.utils; +import org.apache.dolphinscheduler.common.constants.Constants; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; import org.apache.commons.lang3.StringUtils; @@ -54,7 +55,7 @@ public static List getWorkerAddressListFromWorkerGroup(WorkerGroup worke if (StringUtils.isEmpty(addrList)) { return Collections.emptyList(); } - return Lists.newArrayList(addrList.split(",")); + return Lists.newArrayList(addrList.split(Constants.COMMA)); } } diff --git a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectWorkerGroupMapper.xml b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectWorkerGroupMapper.xml new file mode 100644 index 000000000000..be7fa35071c5 --- /dev/null +++ b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectWorkerGroupMapper.xml @@ -0,0 +1,60 @@ + + + + + + + delete from t_ds_relation_project_worker_group + where 1=1 + and project_code = #{projectCode} + + + + + + delete from t_ds_relation_project_worker_group + where 1=1 + and project_code = #{projectCode} + and worker_group in ( + + #{workerGroup} + + ) + + + + diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerGroupChangeNotifier.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerGroupChangeNotifier.java index 827ebfc57646..408be878f078 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerGroupChangeNotifier.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerGroupChangeNotifier.java @@ -17,14 +17,24 @@ package org.apache.dolphinscheduler.server.master.cluster; +import org.apache.dolphinscheduler.common.constants.Constants; +import org.apache.dolphinscheduler.common.enums.WorkerGroupSource; +import org.apache.dolphinscheduler.common.model.Server; +import org.apache.dolphinscheduler.common.model.WorkerHeartBeat; +import org.apache.dolphinscheduler.common.utils.DateUtils; +import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.MapComparator; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; +import org.apache.dolphinscheduler.dao.entity.WorkerGroupInfo; import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; +import org.apache.dolphinscheduler.registry.api.RegistryClient; +import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.utils.MasterThreadFactory; import org.apache.commons.collections4.CollectionUtils; +import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -47,6 +57,9 @@ public class WorkerGroupChangeNotifier { @Autowired private MasterConfig masterConfig; + @Autowired + private RegistryClient registryClient; + private final WorkerGroupDao workerGroupDao; private final List listeners = new CopyOnWriteArrayList<>(); @@ -85,9 +98,24 @@ Map getWorkerGroupMap() { } private MapComparator detectChangedWorkerGroups() { - final Map tmpWorkerGroupMap = workerGroupDao.queryAll() + Map tmpWorkerGroupMap = workerGroupDao.queryAll() .stream() .collect(Collectors.toMap(WorkerGroup::getName, workerGroup -> workerGroup)); + + List activeWorkerNodes = registryClient.getServerList(RegistryNodeType.WORKER); + Map configWorkerGroups = activeWorkerNodes.stream() + .collect(Collectors.groupingBy(activeWorkerNode -> JSONUtils.parseObject(activeWorkerNode.getHeartBeatInfo(), WorkerHeartBeat.class).getWorkerGroup(), + Collectors.mapping(Server::getHost, Collectors.joining(Constants.COMMA)))); + for (Server activeWorkerNode : activeWorkerNodes) { + WorkerGroup configWorkerGroup = new WorkerGroup(); + WorkerHeartBeat workerHeartBeat = JSONUtils.parseObject(activeWorkerNode.getHeartBeatInfo(), WorkerHeartBeat.class); + configWorkerGroup.setName(workerHeartBeat.getWorkerGroup()); + configWorkerGroup.setAddrList(configWorkerGroups.get(workerHeartBeat.getWorkerGroup())); + configWorkerGroup.setCreateTime(DateUtils.timeStampToDate(workerHeartBeat.getStartupTime())); + configWorkerGroup.setUpdateTime(DateUtils.timeStampToDate(workerHeartBeat.getReportTime())); + tmpWorkerGroupMap.put(configWorkerGroup.getName(), configWorkerGroup); + } + return new MapComparator<>(workerGroupMap, tmpWorkerGroupMap); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java index f546033436c9..9a579a602582 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java @@ -64,8 +64,8 @@ public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws .map(Host::of) .map(Host::getAddress) .orElseThrow(() -> new TaskDispatchException( - String.format("Cannot find the host to dispatch Task[id=%s, name=%s]", - taskExecutionContext.getTaskInstanceId(), taskName))); + String.format("Cannot find the host to dispatch Task[id=%s, name=%s, workerGroup=%s]", + taskExecutionContext.getTaskInstanceId(), taskName, taskExecutionContext.getWorkerGroup()))); taskExecutionContext.setHost(physicalTaskExecutorAddress); taskExecutionRunnable.getTaskInstance().setHost(physicalTaskExecutorAddress); diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-api/src/main/java/org/apache/dolphinscheduler/registry/api/RegistryClient.java b/dolphinscheduler-registry/dolphinscheduler-registry-api/src/main/java/org/apache/dolphinscheduler/registry/api/RegistryClient.java index df0587019ad3..75b817c2d9de 100644 --- a/dolphinscheduler-registry/dolphinscheduler-registry-api/src/main/java/org/apache/dolphinscheduler/registry/api/RegistryClient.java +++ b/dolphinscheduler-registry/dolphinscheduler-registry-api/src/main/java/org/apache/dolphinscheduler/registry/api/RegistryClient.java @@ -125,9 +125,9 @@ public List getServerList(RegistryNodeType registryNodeType) { log.warn("unknown registry node type: {}", registryNodeType); } - server.setResInfo(heartBeatJson); + server.setHeartBeatInfo(heartBeatJson); // todo: add host, port in heartBeat Info, so that we don't need to parse this again - server.setZkDirectory(registryNodeType.getRegistryPath() + "/" + serverPath); + server.setServerDirectory(registryNodeType.getRegistryPath() + "/" + serverPath); serverList.add(server); } return serverList; diff --git a/dolphinscheduler-standalone-server/src/main/resources/application.yaml b/dolphinscheduler-standalone-server/src/main/resources/application.yaml index 73a307cffbb6..8db9c49dfd3a 100644 --- a/dolphinscheduler-standalone-server/src/main/resources/application.yaml +++ b/dolphinscheduler-standalone-server/src/main/resources/application.yaml @@ -194,6 +194,8 @@ worker: max-heartbeat-interval: 10s # worker host weight to dispatch tasks, default value 100 host-weight: 100 + # worker group name + group: test1 server-load-protection: enabled: true # Worker max system cpu usage, when the worker's system cpu usage is smaller then this value, worker server can be dispatched tasks. diff --git a/dolphinscheduler-standalone-server/src/main/resources/logback-spring.xml b/dolphinscheduler-standalone-server/src/main/resources/logback-spring.xml index 0ad9c6d0cae9..a571cfee9940 100644 --- a/dolphinscheduler-standalone-server/src/main/resources/logback-spring.xml +++ b/dolphinscheduler-standalone-server/src/main/resources/logback-spring.xml @@ -76,7 +76,8 @@ - + + diff --git a/dolphinscheduler-ui/src/locales/en_US/security.ts b/dolphinscheduler-ui/src/locales/en_US/security.ts index a79211bec795..15e1c1fc2c42 100644 --- a/dolphinscheduler-ui/src/locales/en_US/security.ts +++ b/dolphinscheduler-ui/src/locales/en_US/security.ts @@ -64,6 +64,7 @@ export default { group_name_tips: 'Please enter your group name', worker_addresses: 'Worker Addresses', worker_addresses_tips: 'Please select worker addresses', + source: 'Source', create_time: 'Create Time', update_time: 'Update Time' }, diff --git a/dolphinscheduler-ui/src/locales/zh_CN/security.ts b/dolphinscheduler-ui/src/locales/zh_CN/security.ts index 03a4ce7051f3..fc0a00f5e1a1 100644 --- a/dolphinscheduler-ui/src/locales/zh_CN/security.ts +++ b/dolphinscheduler-ui/src/locales/zh_CN/security.ts @@ -64,6 +64,7 @@ export default { group_name_tips: '请输入分组名称', worker_addresses: 'Worker地址', worker_addresses_tips: '请选择Worker地址', + source: '来源', create_time: '创建时间', update_time: '更新时间' }, diff --git a/dolphinscheduler-ui/src/views/projects/list/components/use-worker-group.ts b/dolphinscheduler-ui/src/views/projects/list/components/use-worker-group.ts index 2144bce057b4..da918e5c78ef 100644 --- a/dolphinscheduler-ui/src/views/projects/list/components/use-worker-group.ts +++ b/dolphinscheduler-ui/src/views/projects/list/components/use-worker-group.ts @@ -63,10 +63,8 @@ export function useWorkerGroup( initOptions() const handleValidate = () => { - if (variables.model?.assignedWorkerGroups.length > 0) { - submitModal() - ctx.emit('confirmModal', props.showModalRef) - } + submitModal() + ctx.emit('confirmModal', props.showModalRef) } const submitModal = async () => { diff --git a/dolphinscheduler-ui/src/views/security/worker-group-manage/use-table.ts b/dolphinscheduler-ui/src/views/security/worker-group-manage/use-table.ts index 796fc03c7c43..3213d76fc2be 100644 --- a/dolphinscheduler-ui/src/views/security/worker-group-manage/use-table.ts +++ b/dolphinscheduler-ui/src/views/security/worker-group-manage/use-table.ts @@ -67,6 +67,11 @@ export function useTable() { ) }) }, + { + title: t('security.worker_group.source'), + key: 'source', + className: 'source' + }, { title: t('security.worker_group.create_time'), key: 'createTime' diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/config/WorkerConfig.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/config/WorkerConfig.java index b092199c711b..c92caa79c0f7 100644 --- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/config/WorkerConfig.java +++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/config/WorkerConfig.java @@ -44,6 +44,7 @@ public class WorkerConfig implements Validator { private Duration maxHeartbeatInterval = Duration.ofSeconds(10); private int hostWeight = 100; private WorkerServerLoadProtection serverLoadProtection = new WorkerServerLoadProtection(); + private String group; /** * This field doesn't need to set at config file, it will be calculated by workerIp:listenPort diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/task/WorkerHeartBeatTask.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/task/WorkerHeartBeatTask.java index f5bce3b766d0..739cf95e2250 100644 --- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/task/WorkerHeartBeatTask.java +++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/task/WorkerHeartBeatTask.java @@ -79,6 +79,7 @@ public WorkerHeartBeat getHeartBeat() { .serverStatus(serverStatus) .host(NetUtils.getHost()) .port(workerConfig.getListenPort()) + .workerGroup(workerConfig.getGroup()) .build(); } diff --git a/dolphinscheduler-worker/src/main/resources/application.yaml b/dolphinscheduler-worker/src/main/resources/application.yaml index d9da1ec24457..f463eff33707 100644 --- a/dolphinscheduler-worker/src/main/resources/application.yaml +++ b/dolphinscheduler-worker/src/main/resources/application.yaml @@ -46,6 +46,8 @@ worker: max-heartbeat-interval: 10s # worker host weight to dispatch tasks, default value 100 host-weight: 100 + # worker group name + group: default server-load-protection: # If set true, will open worker overload protection enabled: true From 91c63c5302cbe0b2e13e6507c6ebe01449c233da Mon Sep 17 00:00:00 2001 From: SbloodyS <460888207@qq.com> Date: Wed, 4 Dec 2024 15:14:35 +0800 Subject: [PATCH 02/15] add feature 16834 --- .../api/service/WorkerGroupService.java | 2 - ...ProjectWorkerGroupRelationServiceImpl.java | 20 ++-- .../service/impl/WorkerGroupServiceImpl.java | 100 +++++------------- ...ProjectWorkerGroupRelationServiceTest.java | 34 +++--- .../common/enums/WorkerGroupSource.java | 7 +- .../dao/entity/WorkerGroup.java | 4 + .../dao/entity/WorkerGroupInfo.java | 41 ------- .../dao/mapper/ProjectWorkerGroupMapper.java | 4 +- .../dao/mapper/WorkerGroupMapper.java | 9 +- .../dao/repository/ProjectWorkerGroupDao.java | 2 - .../dao/repository/WorkerGroupDao.java | 11 ++ .../impl/ProjectWorkerGroupDaoImpl.java | 10 +- .../repository/impl/WorkerGroupDaoImpl.java | 41 +++++++ .../dao/mapper/WorkerGroupMapper.xml | 10 ++ .../resources/sql/dolphinscheduler_h2.sql | 1 + .../resources/sql/dolphinscheduler_mysql.sql | 1 + .../sql/dolphinscheduler_postgresql.sql | 1 + .../server/master/cluster/ClusterManager.java | 15 ++- .../master/cluster/MasterServerMetadata.java | 3 +- .../server/master/cluster/WorkerClusters.java | 43 ++++++++ .../cluster/WorkerGroupChangeNotifier.java | 24 ----- .../master/cluster/WorkerServerMetadata.java | 4 +- .../PhysicalTaskExecutorClientDelegator.java | 3 +- .../master/cluster/WorkerClustersTest.java | 16 +-- .../cluster/WorkerServerMetadataTest.java | 2 + .../BaseWorkerLoadBalancerTest.java | 7 +- ...ghtedRoundRobinWorkerLoadBalancerTest.java | 6 +- ...ghtedRoundRobinWorkerLoadBalancerTest.java | 6 +- dolphinscheduler-standalone-server/pom.xml | 7 ++ .../src/main/resources/application.yaml | 2 +- .../src/main/resources/logback-spring.xml | 3 +- 31 files changed, 237 insertions(+), 202 deletions(-) delete mode 100644 dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroupInfo.java diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/WorkerGroupService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/WorkerGroupService.java index 2556d3208ef9..13cd3c3e35c9 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/WorkerGroupService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/WorkerGroupService.java @@ -20,7 +20,6 @@ import org.apache.dolphinscheduler.api.utils.Result; import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; -import org.apache.dolphinscheduler.dao.entity.WorkerGroupInfo; import java.util.List; import java.util.Map; @@ -80,5 +79,4 @@ public interface WorkerGroupService { */ Map queryWorkerGroupByWorkflowDefinitionCodes(List workflowDefinitionCodeList); - List queryAllWorkerGroupList(); } diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProjectWorkerGroupRelationServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProjectWorkerGroupRelationServiceImpl.java index 99e8bc81a9c2..16b50aa4ace8 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProjectWorkerGroupRelationServiceImpl.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProjectWorkerGroupRelationServiceImpl.java @@ -21,19 +21,16 @@ import org.apache.dolphinscheduler.api.exceptions.ServiceException; import org.apache.dolphinscheduler.api.service.ProjectService; import org.apache.dolphinscheduler.api.service.ProjectWorkerGroupRelationService; -import org.apache.dolphinscheduler.api.service.WorkerGroupService; import org.apache.dolphinscheduler.api.utils.Result; import org.apache.dolphinscheduler.common.constants.Constants; import org.apache.dolphinscheduler.dao.entity.Project; import org.apache.dolphinscheduler.dao.entity.ProjectWorkerGroup; import org.apache.dolphinscheduler.dao.entity.User; -import org.apache.dolphinscheduler.dao.entity.WorkerGroup; import org.apache.dolphinscheduler.dao.mapper.ProjectMapper; -import org.apache.dolphinscheduler.dao.mapper.ProjectWorkerGroupMapper; import org.apache.dolphinscheduler.dao.mapper.ScheduleMapper; import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper; -import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper; import org.apache.dolphinscheduler.dao.repository.ProjectWorkerGroupDao; +import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils; import org.apache.commons.collections.CollectionUtils; @@ -41,7 +38,6 @@ import org.apache.commons.lang3.StringUtils; import java.util.ArrayList; -import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.HashSet; @@ -57,8 +53,6 @@ import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Service; -import com.baomidou.mybatisplus.core.conditions.query.QueryWrapper; - @Service @Slf4j public class ProjectWorkerGroupRelationServiceImpl extends BaseServiceImpl @@ -81,7 +75,7 @@ public class ProjectWorkerGroupRelationServiceImpl extends BaseServiceImpl private ProjectService projectService; @Autowired - private WorkerGroupService workerGroupService; + private WorkerGroupDao workerGroupDao; /** * assign worker groups to a project @@ -121,9 +115,7 @@ public Result assignWorkerGroupsToProject(User loginUser, Long projectCode, List return result; } - Set workerGroupNames = - workerGroupService.queryAllWorkerGroupList().stream().map(WorkerGroup::getName).collect( - Collectors.toSet()); + Set workerGroupNames = new HashSet<>(workerGroupDao.queryAllWorkerGroupNames()); workerGroupNames.add(WorkerGroupUtils.getDefaultWorkerGroup()); @@ -136,7 +128,8 @@ public Result assignWorkerGroupsToProject(User loginUser, Long projectCode, List return result; } - Set projectWorkerGroupNames = projectWorkerGroupDao.queryAssignedWorkerGroupNamesByProjectCode(projectCode); + Set projectWorkerGroupNames = + projectWorkerGroupDao.queryAssignedWorkerGroupNamesByProjectCode(projectCode); difference = SetUtils.difference(projectWorkerGroupNames, assignedWorkerGroupNames); @@ -148,7 +141,8 @@ public Result assignWorkerGroupsToProject(User loginUser, Long projectCode, List SetUtils.intersection(usedWorkerGroups, difference).toSet()); } - boolean deleted = projectWorkerGroupDao.deleteByProjectCodeAndWorkerGroups(projectCode, new ArrayList<>(difference)); + boolean deleted = + projectWorkerGroupDao.deleteByProjectCodeAndWorkerGroups(projectCode, new ArrayList<>(difference)); if (deleted) { log.info("Success to delete worker groups [{}] for the project [{}] .", difference, project.getName()); } else { diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkerGroupServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkerGroupServiceImpl.java index e0c804ca51ec..d0a8c47022ac 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkerGroupServiceImpl.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkerGroupServiceImpl.java @@ -31,27 +31,21 @@ import org.apache.dolphinscheduler.common.enums.WorkerGroupSource; import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; import org.apache.dolphinscheduler.common.model.Server; -import org.apache.dolphinscheduler.common.model.WorkerHeartBeat; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.dao.entity.EnvironmentWorkerGroupRelation; import org.apache.dolphinscheduler.dao.entity.Schedule; import org.apache.dolphinscheduler.dao.entity.TaskDefinition; import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; -import org.apache.dolphinscheduler.dao.entity.WorkerGroupInfo; import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; import org.apache.dolphinscheduler.dao.mapper.EnvironmentWorkerGroupRelationMapper; import org.apache.dolphinscheduler.dao.mapper.ScheduleMapper; import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper; -import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper; import org.apache.dolphinscheduler.dao.mapper.WorkflowDefinitionMapper; import org.apache.dolphinscheduler.dao.mapper.WorkflowInstanceMapper; -<<<<<<< HEAD -import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils; +import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; import org.apache.dolphinscheduler.extract.base.client.Clients; import org.apache.dolphinscheduler.extract.master.IMasterContainerService; -======= ->>>>>>> 8aa5dbba0 (feature 16834) import org.apache.dolphinscheduler.registry.api.RegistryClient; import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; @@ -82,7 +76,7 @@ public class WorkerGroupServiceImpl extends BaseServiceImpl implements WorkerGroupService { @Autowired - private WorkerGroupMapper workerGroupMapper; + private WorkerGroupDao workerGroupDao; @Autowired private WorkflowInstanceMapper workflowInstanceMapper; @@ -135,11 +129,12 @@ public WorkerGroup saveWorkerGroup(User loginUser, workerGroup.setCreateTime(now); workerGroup.setName(name); workerGroup.setAddrList(addrList); + workerGroup.setSource(WorkerGroupSource.UI); workerGroup.setUpdateTime(now); workerGroup.setDescription(description); - workerGroupMapper.insert(workerGroup); + workerGroupDao.insert(workerGroup); } else { - workerGroup = workerGroupMapper.selectById(id); + workerGroup = workerGroupDao.queryById(id); if (workerGroup == null) { throw new ServiceException(Status.WORKER_GROUP_NOT_EXIST, id); } @@ -151,7 +146,7 @@ public WorkerGroup saveWorkerGroup(User loginUser, workerGroup.setAddrList(addrList); workerGroup.setUpdateTime(now); workerGroup.setDescription(description); - workerGroupMapper.updateById(workerGroup); + workerGroupDao.updateById(workerGroup); log.info("Update worker group: {} success .", workerGroup); } boardCastToMasterThatWorkerGroupChanged(); @@ -238,7 +233,7 @@ public Result queryAllGroupPaging(User loginUser, Integer pageNo, Integer pageSi int toIndex = (pageNo - 1) * pageSize + pageSize; Result result = new Result(); - List workerGroups; + List workerGroups; if (loginUser.getUserType().equals(UserType.ADMIN_USER)) { workerGroups = getWorkerGroups(null); } else { @@ -246,14 +241,14 @@ public Result queryAllGroupPaging(User loginUser, Integer pageNo, Integer pageSi .userOwnedResourceIdsAcquisition(AuthorizationType.WORKER_GROUP, loginUser.getId(), log); workerGroups = getWorkerGroups(ids.isEmpty() ? Collections.emptyList() : new ArrayList<>(ids)); } - List resultDataList = new ArrayList<>(); + List resultDataList = new ArrayList<>(); int total = 0; if (CollectionUtils.isNotEmpty(workerGroups)) { - List searchValDataList = new ArrayList<>(); + List searchValDataList = new ArrayList<>(); if (!StringUtils.isEmpty(searchVal)) { - for (WorkerGroupInfo workerGroup : workerGroups) { + for (WorkerGroup workerGroup : workerGroups) { if (workerGroup.getName().contains(searchVal)) { searchValDataList.add(workerGroup); } @@ -270,7 +265,7 @@ public Result queryAllGroupPaging(User loginUser, Integer pageNo, Integer pageSi } } - PageInfo pageInfo = new PageInfo<>(pageNo, pageSize); + PageInfo pageInfo = new PageInfo<>(pageNo, pageSize); pageInfo.setTotal(total); pageInfo.setTotalList(resultDataList); @@ -288,7 +283,7 @@ public Result queryAllGroupPaging(User loginUser, Integer pageNo, Integer pageSi @Override public Map queryAllGroup(User loginUser) { Map result = new HashMap<>(); - List workerGroups; + List workerGroups; if (loginUser.getUserType().equals(UserType.ADMIN_USER)) { workerGroups = getWorkerGroups(null); } else { @@ -304,69 +299,22 @@ public Map queryAllGroup(User loginUser) { return result; } - private List getWorkerGroups(List ids) { + private List getWorkerGroups(List ids) { // worker groups from database - List workerGroups; + List workerGroups; if (ids != null) { - workerGroups = ids.isEmpty() ? new ArrayList<>() : workerGroupMapper.selectBatchIds(ids).stream().map(workerGroup -> { - WorkerGroupInfo workerGroupInfo = new WorkerGroupInfo(workerGroup); - workerGroupInfo.setSource(WorkerGroupSource.UI); - return workerGroupInfo; - }).collect(Collectors.toList()); + workerGroups = ids.isEmpty() ? new ArrayList<>() : workerGroupDao.queryByIds(ids); } else { - workerGroups = workerGroupMapper.queryAllWorkerGroup().stream().map(workerGroup -> { - WorkerGroupInfo workerGroupInfo = new WorkerGroupInfo(workerGroup); - workerGroupInfo.setSource(WorkerGroupSource.UI); - return workerGroupInfo; - }).collect(Collectors.toList()); + workerGroups = workerGroupDao.queryAllWorkerGroup(); } - - // get worker groups from registry - List activeWorkerNodes = registryClient.getServerList(RegistryNodeType.WORKER); - Map configWorkerGroups = activeWorkerNodes.stream() - .collect(Collectors.groupingBy(activeWorkerNode -> JSONUtils.parseObject(activeWorkerNode.getHeartBeatInfo(), WorkerHeartBeat.class).getWorkerGroup(), - Collectors.mapping(Server::getHost, Collectors.joining(Constants.COMMA)))); - for (Server activeWorkerNode : activeWorkerNodes) { - WorkerGroupInfo configWorkerGroupInfo = new WorkerGroupInfo(); - WorkerHeartBeat workerHeartBeat = JSONUtils.parseObject(activeWorkerNode.getHeartBeatInfo(), WorkerHeartBeat.class); - configWorkerGroupInfo.setName(workerHeartBeat.getWorkerGroup()); - configWorkerGroupInfo.setAddrList(configWorkerGroups.get(workerHeartBeat.getWorkerGroup())); - configWorkerGroupInfo.setCreateTime(new Date()); - configWorkerGroupInfo.setUpdateTime(new Date()); - configWorkerGroupInfo.setSystemDefault(true); - configWorkerGroupInfo.setSource(WorkerGroupSource.CONFIG); - workerGroups.add(configWorkerGroupInfo); - } - + workerGroups.forEach(workerGroup -> { + if (workerGroup.getSource().equals(WorkerGroupSource.CONFIG)) { + workerGroup.setSystemDefault(true); + } + }); return workerGroups; } - public List queryAllWorkerGroupList() { - List workerGroupList; - workerGroupList = workerGroupMapper.queryAllWorkerGroup().stream() - .map(workerGroup -> { - WorkerGroupInfo workerGroupInfo = new WorkerGroupInfo(workerGroup); - workerGroupInfo.setSource(WorkerGroupSource.UI); - return workerGroupInfo; - }).collect(Collectors.toList()); - List activeWorkerNodes = registryClient.getServerList(RegistryNodeType.WORKER); - Map configWorkerGroups = activeWorkerNodes.stream() - .collect(Collectors.groupingBy(activeWorkerNode -> JSONUtils.parseObject(activeWorkerNode.getHeartBeatInfo(), WorkerHeartBeat.class).getWorkerGroup(), - Collectors.mapping(Server::getHost, Collectors.joining(Constants.COMMA)))); - for (Server activeWorkerNode : activeWorkerNodes) { - WorkerGroupInfo configWorkerGroupInfo = new WorkerGroupInfo(); - WorkerHeartBeat workerHeartBeat = JSONUtils.parseObject(activeWorkerNode.getHeartBeatInfo(), WorkerHeartBeat.class); - configWorkerGroupInfo.setName(workerHeartBeat.getWorkerGroup()); - configWorkerGroupInfo.setAddrList(configWorkerGroups.get(workerHeartBeat.getWorkerGroup())); - configWorkerGroupInfo.setCreateTime(new Date()); - configWorkerGroupInfo.setUpdateTime(new Date()); - configWorkerGroupInfo.setSystemDefault(true); - configWorkerGroupInfo.setSource(WorkerGroupSource.CONFIG); - workerGroupList.add(configWorkerGroupInfo); - } - return workerGroupList; - } - /** * delete worker group by id * @@ -381,7 +329,7 @@ public Map deleteWorkerGroupById(User loginUser, Integer id) { putMsg(result, Status.USER_NO_OPERATION_PERM); return result; } - WorkerGroup workerGroup = workerGroupMapper.selectById(id); + WorkerGroup workerGroup = workerGroupDao.queryById(id); if (workerGroup == null) { log.error("Worker group does not exist, workerGroupId:{}.", id); putMsg(result, Status.DELETE_WORKER_GROUP_NOT_EXIST); @@ -404,7 +352,7 @@ public Map deleteWorkerGroupById(User loginUser, Integer id) { return result; } - workerGroupMapper.deleteById(id); + workerGroupDao.deleteById(id); log.info("Delete worker group complete, workerGroupName:{}.", workerGroup.getName()); putMsg(result, Status.SUCCESS); @@ -441,7 +389,7 @@ private void boardCastToMasterThatWorkerGroupChanged() { for (Server master : masters) { try { Clients.withService(IMasterContainerService.class) - .withHost(master.getHost() + ":" + master.getPort()) + .withHost(master.getHost() + Constants.COLON + master.getPort()) .refreshWorkerGroup(); } catch (Exception e) { log.error("Broadcast to master: {} that worker group changed failed", master, e); diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationServiceTest.java index eccbab0c5b44..216d3183caa8 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationServiceTest.java @@ -31,12 +31,13 @@ import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; import org.apache.dolphinscheduler.dao.mapper.ProjectMapper; -import org.apache.dolphinscheduler.dao.mapper.ProjectWorkerGroupMapper; import org.apache.dolphinscheduler.dao.mapper.ScheduleMapper; import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper; -import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper; +import org.apache.dolphinscheduler.dao.repository.ProjectWorkerGroupDao; +import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -52,6 +53,7 @@ import org.mockito.quality.Strictness; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; @ExtendWith(MockitoExtension.class) @MockitoSettings(strictness = Strictness.LENIENT) @@ -64,10 +66,10 @@ public class ProjectWorkerGroupRelationServiceTest { private ProjectMapper projectMapper; @Mock - private ProjectWorkerGroupMapper projectWorkerGroupMapper; + private ProjectWorkerGroupDao projectWorkerGroupDao; @Mock - private WorkerGroupMapper workerGroupMapper; + private WorkerGroupDao workerGroupDao; @Mock private ProjectService projectService; @@ -105,35 +107,37 @@ public void testAssignWorkerGroupsToProject() { WorkerGroup workerGroup = new WorkerGroup(); workerGroup.setName("test"); Mockito.when(projectMapper.queryByCode(Mockito.anyLong())).thenReturn(getProject()); - Mockito.when(workerGroupMapper.queryAllWorkerGroup()).thenReturn(Collections.singletonList(workerGroup)); + Mockito.when(workerGroupDao.queryAllWorkerGroup()).thenReturn(Collections.singletonList(workerGroup)); result = projectWorkerGroupRelationService.assignWorkerGroupsToProject(loginUser, projectCode, getDiffWorkerGroups()); Assertions.assertEquals(Status.WORKER_GROUP_NOT_EXIST.getCode(), result.getCode()); // db insertion fail - Mockito.when(workerGroupMapper.queryAllWorkerGroup()).thenReturn(Collections.singletonList(workerGroup)); - Mockito.when(projectWorkerGroupMapper.insert(Mockito.any())).thenReturn(-1); + Mockito.when(workerGroupDao.queryAllWorkerGroup()).thenReturn(Collections.singletonList(workerGroup)); + Mockito.when(projectWorkerGroupDao.insert(Mockito.any())).thenReturn(-1); AssertionsHelper.assertThrowsServiceException(Status.ASSIGN_WORKER_GROUP_TO_PROJECT_ERROR, () -> projectWorkerGroupRelationService.assignWorkerGroupsToProject(loginUser, projectCode, getWorkerGroups())); // success - Mockito.when(projectWorkerGroupMapper.insert(Mockito.any())).thenReturn(1); + Mockito.when(projectWorkerGroupDao.insert(Mockito.any())).thenReturn(1); result = projectWorkerGroupRelationService.assignWorkerGroupsToProject(loginUser, projectCode, getWorkerGroups()); Assertions.assertEquals(Status.SUCCESS.getCode(), result.getCode()); // success when there is diff between current wg and assigned wg - Mockito.when(projectWorkerGroupMapper.selectList(Mockito.any())) - .thenReturn(Collections.singletonList(getDiffProjectWorkerGroup())); - Mockito.when(projectWorkerGroupMapper.delete(Mockito.any())).thenReturn(1); + Mockito.when(projectWorkerGroupDao.queryAssignedWorkerGroupNamesByProjectCode(Mockito.any())) + .thenReturn(Sets.newHashSet(getDiffProjectWorkerGroup().getWorkerGroup())); + Mockito.when(projectWorkerGroupDao.deleteByProjectCodeAndWorkerGroups(getProjectWorkerGroup().getProjectCode(), + Collections.singletonList(getDiffProjectWorkerGroup().getWorkerGroup()))).thenReturn(true); result = projectWorkerGroupRelationService.assignWorkerGroupsToProject(loginUser, projectCode, getWorkerGroups()); Assertions.assertEquals(Status.SUCCESS.getCode(), result.getCode()); // db deletion fail - Mockito.when(projectWorkerGroupMapper.delete(Mockito.any())).thenReturn(-1); + Mockito.when(projectWorkerGroupDao.deleteByProjectCodeAndWorkerGroups(Mockito.any(), Mockito.any())) + .thenReturn(false); AssertionsHelper.assertThrowsServiceException(Status.ASSIGN_WORKER_GROUP_TO_PROJECT_ERROR, () -> projectWorkerGroupRelationService.assignWorkerGroupsToProject(loginUser, projectCode, getWorkerGroups())); @@ -164,8 +168,8 @@ public void testQueryWorkerGroupsByProject() { Mockito.when(projectMapper.queryByCode(projectCode)) .thenReturn(getProject()); - Mockito.when(projectWorkerGroupMapper.selectList(Mockito.any())) - .thenReturn(Lists.newArrayList(getProjectWorkerGroup())); + Mockito.when(projectWorkerGroupDao.queryByProjectCode(Mockito.any())) + .thenReturn(Collections.singletonList(getProjectWorkerGroup())); Mockito.when(taskDefinitionMapper.queryAllDefinitionList(Mockito.anyLong())) .thenReturn(new ArrayList<>()); @@ -177,7 +181,7 @@ public void testQueryWorkerGroupsByProject() { ProjectWorkerGroup[] actualValue = ((List) result.get(Constants.DATA_LIST)).toArray(new ProjectWorkerGroup[0]); - + System.out.println(Arrays.toString(actualValue)); Assertions.assertEquals(actualValue[0].getWorkerGroup(), getProjectWorkerGroup().getWorkerGroup()); } diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/WorkerGroupSource.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/WorkerGroupSource.java index 54e1b301398b..9c7f083b2631 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/WorkerGroupSource.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/WorkerGroupSource.java @@ -17,14 +17,15 @@ package org.apache.dolphinscheduler.common.enums; +import lombok.Getter; import com.baomidou.mybatisplus.annotation.EnumValue; -import lombok.Getter; @Getter public enum WorkerGroupSource { - CONFIG(0, "config"), - UI(1, "ui"); + + CONFIG(1, "config"), + UI(2, "ui"); @EnumValue private final int code; diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroup.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroup.java index df9077602ed8..a9ea3f175d52 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroup.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroup.java @@ -17,6 +17,8 @@ package org.apache.dolphinscheduler.dao.entity; +import org.apache.dolphinscheduler.common.enums.WorkerGroupSource; + import java.util.Date; import lombok.AllArgsConstructor; @@ -43,6 +45,8 @@ public class WorkerGroup { private String addrList; + private WorkerGroupSource source; + private Date createTime; private Date updateTime; diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroupInfo.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroupInfo.java deleted file mode 100644 index 0e4cc3498fdd..000000000000 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroupInfo.java +++ /dev/null @@ -1,41 +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 lombok.Data; -import lombok.EqualsAndHashCode; -import lombok.NoArgsConstructor; -import org.apache.dolphinscheduler.common.enums.WorkerGroupSource; - -@EqualsAndHashCode(callSuper = true) -@Data -@NoArgsConstructor -public class WorkerGroupInfo extends WorkerGroup { - private WorkerGroupSource source; - - public WorkerGroupInfo(WorkerGroup workerGroup) { - this.setId(workerGroup.getId()); - this.setName(workerGroup.getName()); - this.setAddrList(workerGroup.getAddrList()); - this.setCreateTime(workerGroup.getCreateTime()); - this.setUpdateTime(workerGroup.getUpdateTime()); - this.setDescription(workerGroup.getDescription()); - this.setSystemDefault(workerGroup.isSystemDefault()); - this.setOtherParamsJson(workerGroup.getOtherParamsJson()); - } -} diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProjectWorkerGroupMapper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProjectWorkerGroupMapper.java index ae59b5c9fbe3..8eb44fb033c6 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProjectWorkerGroupMapper.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProjectWorkerGroupMapper.java @@ -19,13 +19,15 @@ import org.apache.dolphinscheduler.dao.entity.ProjectWorkerGroup; -import com.baomidou.mybatisplus.core.mapper.BaseMapper; import org.apache.ibatis.annotations.Param; import java.util.List; import java.util.Set; +import com.baomidou.mybatisplus.core.mapper.BaseMapper; + public interface ProjectWorkerGroupMapper extends BaseMapper { + int deleteByProjectCode(@Param("projectCode") Long projectCode); Set queryAssignedWorkerGroupNamesByProjectCode(@Param("projectCode") Long projectCode); diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/WorkerGroupMapper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/WorkerGroupMapper.java index fc54b4a0dce7..06a39223efe1 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/WorkerGroupMapper.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/WorkerGroupMapper.java @@ -17,6 +17,7 @@ package org.apache.dolphinscheduler.dao.mapper; +import org.apache.dolphinscheduler.common.enums.WorkerGroupSource; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; import org.apache.ibatis.annotations.Param; @@ -25,9 +26,6 @@ import com.baomidou.mybatisplus.core.mapper.BaseMapper; -/** - * worker group mapper interface - */ public interface WorkerGroupMapper extends BaseMapper { /** @@ -51,4 +49,9 @@ public interface WorkerGroupMapper extends BaseMapper { */ List queryWorkerGroupByName(@Param("name") String name); + int updateAddrListByWorkerGroupName(@Param("name") String name, + @Param("addrList") String addrList, + @Param("source") WorkerGroupSource source); + + int deleteByWorkerGroupName(@Param("name") String name); } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProjectWorkerGroupDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProjectWorkerGroupDao.java index 810005d1c050..46e2cc75366a 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProjectWorkerGroupDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProjectWorkerGroupDao.java @@ -17,10 +17,8 @@ package org.apache.dolphinscheduler.dao.repository; -import org.apache.dolphinscheduler.dao.entity.Project; import org.apache.dolphinscheduler.dao.entity.ProjectWorkerGroup; -import java.util.Collection; import java.util.List; import java.util.Set; diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/WorkerGroupDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/WorkerGroupDao.java index 8ffcff30da66..625bbfae31e5 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/WorkerGroupDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/WorkerGroupDao.java @@ -17,7 +17,18 @@ package org.apache.dolphinscheduler.dao.repository; +import org.apache.dolphinscheduler.common.enums.WorkerGroupSource; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; +import java.util.List; + public interface WorkerGroupDao extends IDao { + + boolean upsertAddrListByWorkerGroupName(String workerGroupName, String addrList, WorkerGroupSource source); + + boolean deleteByWorkerGroupName(String workerGroupName); + + List queryAllWorkerGroupNames(); + + List queryAllWorkerGroup(); } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProjectWorkerGroupDaoImpl.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProjectWorkerGroupDaoImpl.java index ea127fe8c5c4..7feef6368953 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProjectWorkerGroupDaoImpl.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProjectWorkerGroupDaoImpl.java @@ -17,18 +17,22 @@ package org.apache.dolphinscheduler.dao.repository.impl; -import lombok.NonNull; import org.apache.dolphinscheduler.dao.entity.ProjectWorkerGroup; import org.apache.dolphinscheduler.dao.mapper.ProjectWorkerGroupMapper; import org.apache.dolphinscheduler.dao.repository.BaseDao; import org.apache.dolphinscheduler.dao.repository.ProjectWorkerGroupDao; -import org.springframework.stereotype.Repository; import java.util.List; import java.util.Set; +import lombok.NonNull; + +import org.springframework.stereotype.Repository; + @Repository -public class ProjectWorkerGroupDaoImpl extends BaseDao implements ProjectWorkerGroupDao { +public class ProjectWorkerGroupDaoImpl extends BaseDao + implements + ProjectWorkerGroupDao { public ProjectWorkerGroupDaoImpl(@NonNull ProjectWorkerGroupMapper projectWorkerGroupMapper) { super(projectWorkerGroupMapper); diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/WorkerGroupDaoImpl.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/WorkerGroupDaoImpl.java index 2151a6c2167b..70a92ff112f5 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/WorkerGroupDaoImpl.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/WorkerGroupDaoImpl.java @@ -17,14 +17,20 @@ package org.apache.dolphinscheduler.dao.repository.impl; +import org.apache.dolphinscheduler.common.enums.WorkerGroupSource; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper; import org.apache.dolphinscheduler.dao.repository.BaseDao; import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; +import java.util.Date; +import java.util.List; +import java.util.stream.Collectors; + import lombok.NonNull; import org.springframework.stereotype.Repository; +import org.springframework.util.CollectionUtils; @Repository public class WorkerGroupDaoImpl extends BaseDao implements WorkerGroupDao { @@ -33,4 +39,39 @@ public WorkerGroupDaoImpl(@NonNull WorkerGroupMapper workerGroupMapper) { super(workerGroupMapper); } + @Override + public boolean upsertAddrListByWorkerGroupName(String workerGroupName, String addrList, WorkerGroupSource source) { + List currentWorkerGroups = mybatisMapper.queryWorkerGroupByName(workerGroupName); + if (CollectionUtils.isEmpty(currentWorkerGroups)) { + WorkerGroup newWorkerGroup = new WorkerGroup(); + newWorkerGroup.setName(workerGroupName); + newWorkerGroup.setAddrList(addrList); + newWorkerGroup.setSource(source); + newWorkerGroup.setCreateTime(new Date()); + newWorkerGroup.setUpdateTime(newWorkerGroup.getCreateTime()); + int inserted = mybatisMapper.insert(newWorkerGroup); + return inserted > 0; + } else { + int updated = mybatisMapper.updateAddrListByWorkerGroupName(workerGroupName, addrList, source); + return updated > 0; + } + } + + @Override + public boolean deleteByWorkerGroupName(String workerGroupName) { + int deleted = mybatisMapper.deleteByWorkerGroupName(workerGroupName); + return deleted > 0; + } + + @Override + public List queryAllWorkerGroupNames() { + return mybatisMapper.queryAllWorkerGroup().stream() + .map(WorkerGroup::getName) + .collect(Collectors.toList()); + } + + @Override + public List queryAllWorkerGroup() { + return mybatisMapper.queryAllWorkerGroup(); + } } diff --git a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/WorkerGroupMapper.xml b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/WorkerGroupMapper.xml index 79305d146725..7dd29edd0bcc 100644 --- a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/WorkerGroupMapper.xml +++ b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/WorkerGroupMapper.xml @@ -29,4 +29,14 @@ where name = #{name} + + update t_ds_worker_group + set addr_list = #{addrList}, source = #{source} + where name = #{name} + + + + delete from t_ds_worker_group + where name = #{name} + diff --git a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_h2.sql b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_h2.sql index 9beb2ffd9b67..0bbb8de37d14 100644 --- a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_h2.sql +++ b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_h2.sql @@ -1012,6 +1012,7 @@ CREATE TABLE t_ds_worker_group id bigint(11) NOT NULL AUTO_INCREMENT, name varchar(255) NOT NULL, addr_list text NULL DEFAULT NULL, + source int(11) NOT NULL DEFAULT 0, create_time datetime NULL DEFAULT NULL, update_time datetime NULL DEFAULT NULL, description text NULL DEFAULT NULL, diff --git a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql index 7826be4846ac..2f13ee41921a 100644 --- a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql +++ b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql @@ -1009,6 +1009,7 @@ CREATE TABLE `t_ds_worker_group` ( `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'id', `name` varchar(255) NOT NULL COMMENT 'worker group name', `addr_list` text NULL DEFAULT NULL COMMENT 'worker addr list. split by [,]', + `source` int(11) NOT NULL DEFAULT 0 COMMENT 'worker group source', `create_time` datetime NULL DEFAULT NULL COMMENT 'create time', `update_time` datetime NULL DEFAULT NULL COMMENT 'update time', `description` text NULL DEFAULT NULL COMMENT 'description', diff --git a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql index e31ba9184de3..679fa365badf 100644 --- a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql +++ b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql @@ -923,6 +923,7 @@ CREATE TABLE t_ds_worker_group ( id bigint NOT NULL , name varchar(255) NOT NULL , addr_list text DEFAULT NULL , + source bigint NOT NULL DEFAULT 0, create_time timestamp DEFAULT NULL , update_time timestamp DEFAULT NULL , description text DEFAULT NULL, diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/ClusterManager.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/ClusterManager.java index bb491333513f..24a7dcf90c22 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/ClusterManager.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/ClusterManager.java @@ -17,9 +17,12 @@ package org.apache.dolphinscheduler.server.master.cluster; +import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; import org.apache.dolphinscheduler.registry.api.RegistryClient; import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; +import javax.annotation.PostConstruct; + import lombok.Getter; import lombok.extern.slf4j.Slf4j; @@ -31,10 +34,10 @@ public class ClusterManager { @Getter - private final MasterClusters masterClusters; + private MasterClusters masterClusters; @Getter - private final WorkerClusters workerClusters; + private WorkerClusters workerClusters; @Autowired private WorkerGroupChangeNotifier workerGroupChangeNotifier; @@ -42,9 +45,13 @@ public class ClusterManager { @Autowired private RegistryClient registryClient; - public ClusterManager() { + @Autowired + private WorkerGroupDao workerGroupDao; + + @PostConstruct + void init() { this.masterClusters = new MasterClusters(); - this.workerClusters = new WorkerClusters(); + this.workerClusters = new WorkerClusters(workerGroupDao); } public void start() { diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/MasterServerMetadata.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/MasterServerMetadata.java index 1005de4c5084..c49833d5dadd 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/MasterServerMetadata.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/MasterServerMetadata.java @@ -17,6 +17,7 @@ package org.apache.dolphinscheduler.server.master.cluster; +import org.apache.dolphinscheduler.common.constants.Constants; import org.apache.dolphinscheduler.common.model.MasterHeartBeat; import lombok.Data; @@ -33,7 +34,7 @@ public class MasterServerMetadata extends BaseServerMetadata implements Comparab public static MasterServerMetadata parseFromHeartBeat(final MasterHeartBeat masterHeartBeat) { return MasterServerMetadata.builder() .serverStartupTime(masterHeartBeat.getStartupTime()) - .address(masterHeartBeat.getHost() + ":" + masterHeartBeat.getPort()) + .address(masterHeartBeat.getHost() + Constants.COLON + masterHeartBeat.getPort()) .cpuUsage(masterHeartBeat.getCpuUsage()) .memoryUsage(masterHeartBeat.getMemoryUsage()) .serverStatus(masterHeartBeat.getServerStatus()) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerClusters.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerClusters.java index 291b2f1758d0..18b0703e9543 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerClusters.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerClusters.java @@ -17,10 +17,13 @@ package org.apache.dolphinscheduler.server.master.cluster; +import org.apache.dolphinscheduler.common.constants.Constants; import org.apache.dolphinscheduler.common.enums.ServerStatus; +import org.apache.dolphinscheduler.common.enums.WorkerGroupSource; import org.apache.dolphinscheduler.common.model.WorkerHeartBeat; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; +import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils; import org.apache.commons.collections4.list.UnmodifiableList; @@ -35,11 +38,16 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.stream.Collectors; +import lombok.extern.slf4j.Slf4j; + +@Slf4j public class WorkerClusters extends AbstractClusterSubscribeListener implements IClusters, WorkerGroupChangeNotifier.WorkerGroupListener { + private final WorkerGroupDao workerGroupDao; + // WorkerIdentifier(workerAddress) -> worker private final Map workerMapping = new ConcurrentHashMap<>(); @@ -49,6 +57,10 @@ public class WorkerClusters extends AbstractClusterSubscribeListener> workerClusterChangeListeners = new CopyOnWriteArrayList<>(); + public WorkerClusters(WorkerGroupDao workerGroupDao) { + this.workerGroupDao = workerGroupDao; + } + @Override public List getServers() { return UnmodifiableList.unmodifiableList(new ArrayList<>(workerMapping.values())); @@ -126,6 +138,22 @@ WorkerServerMetadata parseServerFromHeartbeat(String serverHeartBeatJson) { @Override public void onServerAdded(WorkerServerMetadata workerServer) { workerMapping.put(workerServer.getAddress(), workerServer); + synchronized (workerGroupMapping) { + List addWorkerGroupAddrList = workerGroupMapping.get(workerServer.getWorkerGroup()); + if (addWorkerGroupAddrList == null) { + List newWorkerGroupAddrList = new ArrayList<>(); + newWorkerGroupAddrList.add(workerServer.getAddress()); + workerGroupMapping.put(workerServer.getWorkerGroup(), newWorkerGroupAddrList); + workerGroupDao.upsertAddrListByWorkerGroupName(workerServer.getWorkerGroup(), + String.join(Constants.COMMA, newWorkerGroupAddrList), + WorkerGroupSource.CONFIG); + } else if (!addWorkerGroupAddrList.contains(workerServer.getAddress())) { + addWorkerGroupAddrList.add(workerServer.getAddress()); + workerGroupDao.upsertAddrListByWorkerGroupName(workerServer.getWorkerGroup(), + String.join(Constants.COMMA, addWorkerGroupAddrList), + WorkerGroupSource.CONFIG); + } + } for (IClustersChangeListener listener : workerClusterChangeListeners) { listener.onServerAdded(workerServer); } @@ -134,6 +162,21 @@ public void onServerAdded(WorkerServerMetadata workerServer) { @Override public void onServerRemove(WorkerServerMetadata workerServer) { workerMapping.remove(workerServer.getAddress(), workerServer); + synchronized (workerGroupMapping) { + List removeWorkerGroupAddrList = workerGroupMapping.get(workerServer.getWorkerGroup()); + if (removeWorkerGroupAddrList != null && removeWorkerGroupAddrList.contains(workerServer.getAddress())) { + removeWorkerGroupAddrList.remove(workerServer.getAddress()); + if (removeWorkerGroupAddrList.isEmpty()) { + workerGroupMapping.remove(workerServer.getWorkerGroup()); + workerGroupDao.deleteByWorkerGroupName(workerServer.getWorkerGroup()); + } else { + workerGroupMapping.put(workerServer.getWorkerGroup(), removeWorkerGroupAddrList); + workerGroupDao.upsertAddrListByWorkerGroupName(workerServer.getWorkerGroup(), + String.join(Constants.COMMA, removeWorkerGroupAddrList), + WorkerGroupSource.CONFIG); + } + } + } for (IClustersChangeListener listener : workerClusterChangeListeners) { listener.onServerRemove(workerServer); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerGroupChangeNotifier.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerGroupChangeNotifier.java index 408be878f078..7c982a7af7f3 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerGroupChangeNotifier.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerGroupChangeNotifier.java @@ -17,24 +17,15 @@ package org.apache.dolphinscheduler.server.master.cluster; -import org.apache.dolphinscheduler.common.constants.Constants; -import org.apache.dolphinscheduler.common.enums.WorkerGroupSource; -import org.apache.dolphinscheduler.common.model.Server; -import org.apache.dolphinscheduler.common.model.WorkerHeartBeat; -import org.apache.dolphinscheduler.common.utils.DateUtils; -import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.MapComparator; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; -import org.apache.dolphinscheduler.dao.entity.WorkerGroupInfo; import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; import org.apache.dolphinscheduler.registry.api.RegistryClient; -import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.utils.MasterThreadFactory; import org.apache.commons.collections4.CollectionUtils; -import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -101,21 +92,6 @@ private MapComparator detectChangedWorkerGroups() { Map tmpWorkerGroupMap = workerGroupDao.queryAll() .stream() .collect(Collectors.toMap(WorkerGroup::getName, workerGroup -> workerGroup)); - - List activeWorkerNodes = registryClient.getServerList(RegistryNodeType.WORKER); - Map configWorkerGroups = activeWorkerNodes.stream() - .collect(Collectors.groupingBy(activeWorkerNode -> JSONUtils.parseObject(activeWorkerNode.getHeartBeatInfo(), WorkerHeartBeat.class).getWorkerGroup(), - Collectors.mapping(Server::getHost, Collectors.joining(Constants.COMMA)))); - for (Server activeWorkerNode : activeWorkerNodes) { - WorkerGroup configWorkerGroup = new WorkerGroup(); - WorkerHeartBeat workerHeartBeat = JSONUtils.parseObject(activeWorkerNode.getHeartBeatInfo(), WorkerHeartBeat.class); - configWorkerGroup.setName(workerHeartBeat.getWorkerGroup()); - configWorkerGroup.setAddrList(configWorkerGroups.get(workerHeartBeat.getWorkerGroup())); - configWorkerGroup.setCreateTime(DateUtils.timeStampToDate(workerHeartBeat.getStartupTime())); - configWorkerGroup.setUpdateTime(DateUtils.timeStampToDate(workerHeartBeat.getReportTime())); - tmpWorkerGroupMap.put(configWorkerGroup.getName(), configWorkerGroup); - } - return new MapComparator<>(workerGroupMap, tmpWorkerGroupMap); } diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerServerMetadata.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerServerMetadata.java index de9a03506d8c..d853c7d061ce 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerServerMetadata.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerServerMetadata.java @@ -17,6 +17,7 @@ package org.apache.dolphinscheduler.server.master.cluster; +import org.apache.dolphinscheduler.common.constants.Constants; import org.apache.dolphinscheduler.common.model.WorkerHeartBeat; import lombok.Builder; @@ -41,7 +42,8 @@ public class WorkerServerMetadata extends BaseServerMetadata { public static WorkerServerMetadata parseFromHeartBeat(final WorkerHeartBeat workerHeartBeat) { return WorkerServerMetadata.builder() .serverStartupTime(workerHeartBeat.getStartupTime()) - .address(workerHeartBeat.getHost() + ":" + workerHeartBeat.getPort()) + .address(workerHeartBeat.getHost() + Constants.COLON + workerHeartBeat.getPort()) + .workerGroup(workerHeartBeat.getWorkerGroup()) .cpuUsage(workerHeartBeat.getCpuUsage()) .memoryUsage(workerHeartBeat.getMemoryUsage()) .serverStatus(workerHeartBeat.getServerStatus()) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java index 9a579a602582..f7566d3133b1 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/client/PhysicalTaskExecutorClientDelegator.java @@ -65,7 +65,8 @@ public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws .map(Host::getAddress) .orElseThrow(() -> new TaskDispatchException( String.format("Cannot find the host to dispatch Task[id=%s, name=%s, workerGroup=%s]", - taskExecutionContext.getTaskInstanceId(), taskName, taskExecutionContext.getWorkerGroup()))); + taskExecutionContext.getTaskInstanceId(), taskName, + taskExecutionContext.getWorkerGroup()))); taskExecutionContext.setHost(physicalTaskExecutorAddress); taskExecutionRunnable.getTaskInstance().setHost(physicalTaskExecutorAddress); diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerClustersTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerClustersTest.java index 045e493d57fe..b0754e280d40 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerClustersTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerClustersTest.java @@ -21,17 +21,21 @@ import org.apache.dolphinscheduler.common.enums.ServerStatus; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; +import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; import org.junit.jupiter.api.Test; +import org.mockito.Mockito; import com.google.common.collect.Lists; import com.google.common.truth.Truth; class WorkerClustersTest { + WorkerGroupDao workerGroupDao = Mockito.mock(WorkerGroupDao.class); + @Test void testOnWorkerGroupDelete() { - WorkerClusters workerClusters = new WorkerClusters(); + WorkerClusters workerClusters = new WorkerClusters(workerGroupDao); WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata(); workerClusters.onServerAdded(normalWorkerServerMetadata); @@ -50,7 +54,7 @@ void testOnWorkerGroupDelete() { @Test void testOnWorkerGroupAdd() { - WorkerClusters workerClusters = new WorkerClusters(); + WorkerClusters workerClusters = new WorkerClusters(workerGroupDao); WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata(); workerClusters.onServerAdded(normalWorkerServerMetadata); @@ -65,7 +69,7 @@ void testOnWorkerGroupAdd() { @Test void testOnWorkerGroupChange() { - WorkerClusters workerClusters = new WorkerClusters(); + WorkerClusters workerClusters = new WorkerClusters(workerGroupDao); WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata(); workerClusters.onServerAdded(normalWorkerServerMetadata); @@ -91,7 +95,7 @@ void testOnServerAdded() { WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata(); WorkerServerMetadata busyWorkerServerMetadata = getBusyWorkerServerMetadata(); - WorkerClusters workerClusters = new WorkerClusters(); + WorkerClusters workerClusters = new WorkerClusters(workerGroupDao); workerClusters.onServerAdded(normalWorkerServerMetadata); workerClusters.onServerAdded(busyWorkerServerMetadata); assertThat(workerClusters.getWorkerServerAddressByGroup("default")) @@ -105,7 +109,7 @@ void testOnServerRemove() { WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata(); WorkerServerMetadata busyWorkerServerMetadata = getBusyWorkerServerMetadata(); - WorkerClusters workerClusters = new WorkerClusters(); + WorkerClusters workerClusters = new WorkerClusters(workerGroupDao); workerClusters.onServerAdded(normalWorkerServerMetadata); workerClusters.onServerAdded(busyWorkerServerMetadata); workerClusters.onServerRemove(busyWorkerServerMetadata); @@ -122,7 +126,7 @@ void testOnServerUpdate() { WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata(); WorkerServerMetadata busyWorkerServerMetadata = getBusyWorkerServerMetadata(); - WorkerClusters workerClusters = new WorkerClusters(); + WorkerClusters workerClusters = new WorkerClusters(workerGroupDao); workerClusters.onServerAdded(normalWorkerServerMetadata); workerClusters.onServerAdded(busyWorkerServerMetadata); diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerServerMetadataTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerServerMetadataTest.java index 16e30b9cd84c..e13555b09d93 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerServerMetadataTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/WorkerServerMetadataTest.java @@ -42,6 +42,7 @@ void parseFromHeartBeat() { .port(12345) .workerHostWeight(2) .threadPoolUsage(0.6) + .workerGroup("test") .build(); WorkerServerMetadata workerServerMetadata = WorkerServerMetadata.parseFromHeartBeat(workerHeartBeat); Truth.assertThat(workerServerMetadata.getCpuUsage()).isEqualTo(0.2); @@ -50,5 +51,6 @@ void parseFromHeartBeat() { Truth.assertThat(workerServerMetadata.getAddress()).isEqualTo("localhost:12345"); Truth.assertThat(workerServerMetadata.getWorkerWeight()).isEqualTo(2); Truth.assertThat(workerServerMetadata.getTaskThreadPoolUsage()).isEqualTo(0.6); + Truth.assertThat(workerServerMetadata.getWorkerGroup()).isEqualTo("test"); } } diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/BaseWorkerLoadBalancerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/BaseWorkerLoadBalancerTest.java index a60cf236dadb..9656e4b608e8 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/BaseWorkerLoadBalancerTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/BaseWorkerLoadBalancerTest.java @@ -19,15 +19,20 @@ import org.apache.dolphinscheduler.common.enums.ServerStatus; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; +import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters; import org.apache.dolphinscheduler.server.master.cluster.WorkerServerMetadata; import java.util.List; +import org.mockito.Mockito; + import com.google.common.collect.Lists; public class BaseWorkerLoadBalancerTest { + WorkerGroupDao workerGroupDao = Mockito.mock(WorkerGroupDao.class); + /** * "default" -> 127.0.0.1:1234, 127.0.0.2:1234, 127.0.0.3:1234, 127.0.0.4:1234 * "sparkCluster" -> 127.0.0.1:1234 @@ -73,7 +78,7 @@ protected WorkerClusters getDefaultWorkerClusters() { .memoryUsage(0.8) .build()); - WorkerClusters workerClusters = new WorkerClusters(); + WorkerClusters workerClusters = new WorkerClusters(workerGroupDao); workerClusters.onWorkerGroupAdd(workerGroups); workerServers.forEach(workerClusters::onServerAdded); return workerClusters; diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/DynamicWeightedRoundRobinWorkerLoadBalancerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/DynamicWeightedRoundRobinWorkerLoadBalancerTest.java index fa870ee8d92d..4f8e38f25df0 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/DynamicWeightedRoundRobinWorkerLoadBalancerTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/DynamicWeightedRoundRobinWorkerLoadBalancerTest.java @@ -19,6 +19,7 @@ import org.apache.dolphinscheduler.common.enums.ServerStatus; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; +import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters; import org.apache.dolphinscheduler.server.master.cluster.WorkerServerMetadata; @@ -28,12 +29,15 @@ import java.util.Optional; import org.junit.jupiter.api.Test; +import org.mockito.Mockito; import com.google.common.collect.Lists; import com.google.common.truth.Truth; class DynamicWeightedRoundRobinWorkerLoadBalancerTest { + WorkerGroupDao workerGroupDao = Mockito.mock(WorkerGroupDao.class); + @Test void select_defaultWorkerGroup() { DynamicWeightedRoundRobinWorkerLoadBalancer loadBalancer = createDynamicWeightedRoundRobinWorkerLoadBalancer(); @@ -107,7 +111,7 @@ private DynamicWeightedRoundRobinWorkerLoadBalancer createDynamicWeightedRoundRo .cpuUsage(0.8) .memoryUsage(0.8) .build()); - WorkerClusters workerClusters = new WorkerClusters(); + WorkerClusters workerClusters = new WorkerClusters(workerGroupDao); DynamicWeightedRoundRobinWorkerLoadBalancer dynamicWeightedRoundRobinWorkerLoadBalancer = new DynamicWeightedRoundRobinWorkerLoadBalancer(workerClusters, new WorkerLoadBalancerConfigurationProperties.DynamicWeightConfigProperties()); diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/FixedWeightedRoundRobinWorkerLoadBalancerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/FixedWeightedRoundRobinWorkerLoadBalancerTest.java index 6e2709802657..eed6f70094ee 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/FixedWeightedRoundRobinWorkerLoadBalancerTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/loadbalancer/FixedWeightedRoundRobinWorkerLoadBalancerTest.java @@ -19,6 +19,7 @@ import org.apache.dolphinscheduler.common.enums.ServerStatus; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; +import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters; import org.apache.dolphinscheduler.server.master.cluster.WorkerServerMetadata; @@ -28,12 +29,15 @@ import java.util.Optional; import org.junit.jupiter.api.Test; +import org.mockito.Mockito; import com.google.common.collect.Lists; import com.google.common.truth.Truth; class FixedWeightedRoundRobinWorkerLoadBalancerTest { + WorkerGroupDao workerGroupDao = Mockito.mock(WorkerGroupDao.class); + @Test void select_defaultWorkerGroup() { FixedWeightedRoundRobinWorkerLoadBalancer loadBalancer = createFixedWeightedRoundRobinWorkerLoadBalancer(); @@ -107,7 +111,7 @@ private FixedWeightedRoundRobinWorkerLoadBalancer createFixedWeightedRoundRobinW .cpuUsage(0.8) .memoryUsage(0.8) .build()); - WorkerClusters workerClusters = new WorkerClusters(); + WorkerClusters workerClusters = new WorkerClusters(workerGroupDao); FixedWeightedRoundRobinWorkerLoadBalancer loadBalancer = new FixedWeightedRoundRobinWorkerLoadBalancer(workerClusters); workerClusters.onWorkerGroupChange(workerGroups); diff --git a/dolphinscheduler-standalone-server/pom.xml b/dolphinscheduler-standalone-server/pom.xml index 97f82dd3da64..b98a7e5ea8e8 100644 --- a/dolphinscheduler-standalone-server/pom.xml +++ b/dolphinscheduler-standalone-server/pom.xml @@ -54,12 +54,19 @@ dolphinscheduler-api provided + org.apache.dolphinscheduler dolphinscheduler-alert-server provided + + org.apache.dolphinscheduler + dolphinscheduler-alert-all + provided + + org.apache.dolphinscheduler dolphinscheduler-task-all diff --git a/dolphinscheduler-standalone-server/src/main/resources/application.yaml b/dolphinscheduler-standalone-server/src/main/resources/application.yaml index 8db9c49dfd3a..31896727dfce 100644 --- a/dolphinscheduler-standalone-server/src/main/resources/application.yaml +++ b/dolphinscheduler-standalone-server/src/main/resources/application.yaml @@ -195,7 +195,7 @@ worker: # worker host weight to dispatch tasks, default value 100 host-weight: 100 # worker group name - group: test1 + group: default server-load-protection: enabled: true # Worker max system cpu usage, when the worker's system cpu usage is smaller then this value, worker server can be dispatched tasks. diff --git a/dolphinscheduler-standalone-server/src/main/resources/logback-spring.xml b/dolphinscheduler-standalone-server/src/main/resources/logback-spring.xml index a571cfee9940..0ad9c6d0cae9 100644 --- a/dolphinscheduler-standalone-server/src/main/resources/logback-spring.xml +++ b/dolphinscheduler-standalone-server/src/main/resources/logback-spring.xml @@ -76,8 +76,7 @@ - - + From 1a45fadc5b7ce1b4c3aa7bfd2d56ead2970356cf Mon Sep 17 00:00:00 2001 From: SbloodyS <460888207@qq.com> Date: Wed, 4 Dec 2024 15:19:08 +0800 Subject: [PATCH 03/15] add feature 16834 --- .../src/test/resources/docker/ldap-login/application.yaml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/docker/ldap-login/application.yaml b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/docker/ldap-login/application.yaml index 51d8e23c2fef..4ee2e9ac1577 100644 --- a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/docker/ldap-login/application.yaml +++ b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/docker/ldap-login/application.yaml @@ -190,6 +190,8 @@ worker: max-heartbeat-interval: 10s # worker host weight to dispatch tasks, default value 100 host-weight: 100 + # worker group name + group: default server-load-protection: enabled: true # Worker max system cpu usage, when the worker's system cpu usage is smaller then this value, worker server can be dispatched tasks. From de06dff8ca7c9348dfdc7136bfd84c713df8831d Mon Sep 17 00:00:00 2001 From: SbloodyS <460888207@qq.com> Date: Wed, 4 Dec 2024 15:21:30 +0800 Subject: [PATCH 04/15] add feature 16834 --- .../mysql/dolphinscheduler_ddl.sql | 21 ++++++++++++++++++- .../postgresql/dolphinscheduler_ddl.sql | 21 ++++++++++++++++++- 2 files changed, 40 insertions(+), 2 deletions(-) diff --git a/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/mysql/dolphinscheduler_ddl.sql b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/mysql/dolphinscheduler_ddl.sql index 935ece7ca82b..8fe7cee1b2ee 100644 --- a/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/mysql/dolphinscheduler_ddl.sql +++ b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/mysql/dolphinscheduler_ddl.sql @@ -243,4 +243,23 @@ END; d// delimiter ; CALL drop_column_t_ds_worker_group_other_params_json; -DROP PROCEDURE drop_column_t_ds_worker_group_other_params_json; \ No newline at end of file +DROP PROCEDURE drop_column_t_ds_worker_group_other_params_json; + +-- add_column_t_ds_worker_group source +DROP PROCEDURE if EXISTS add_column_t_ds_worker_group; +delimiter d// +CREATE PROCEDURE add_column_t_ds_worker_group() +BEGIN + IF EXISTS (SELECT 1 FROM information_schema.COLUMNS + WHERE TABLE_NAME='t_ds_worker_group' + AND TABLE_SCHEMA=(SELECT DATABASE()) + AND COLUMN_NAME ='source') + THEN +ALTER TABLE `t_ds_worker_group` +ADD COLUMN `source` int(11) NOT NULL DEFAULT 0 COMMENT 'worker group source' after `addr_list`; +END IF; +END; +d// +delimiter ; +CALL add_column_t_ds_worker_group; +DROP PROCEDURE add_column_t_ds_worker_group; diff --git a/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/postgresql/dolphinscheduler_ddl.sql b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/postgresql/dolphinscheduler_ddl.sql index 9b9812033aea..0263ac1beada 100644 --- a/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/postgresql/dolphinscheduler_ddl.sql +++ b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/postgresql/dolphinscheduler_ddl.sql @@ -246,4 +246,23 @@ $$ LANGUAGE plpgsql; d// select drop_column_t_ds_worker_group_other_params_json(); -DROP FUNCTION IF EXISTS drop_column_t_ds_worker_group_other_params_json(); \ No newline at end of file +DROP FUNCTION IF EXISTS drop_column_t_ds_worker_group_other_params_json(); + +-- add_column_t_ds_worker_group source +delimiter d// +CREATE OR REPLACE FUNCTION add_column_t_ds_worker_group() RETURNS void AS $$ +BEGIN + IF EXISTS (SELECT 1 + FROM information_schema.columns + WHERE table_name = 't_ds_worker_group' + AND column_name = 'source') + THEN +ALTER TABLE t_ds_worker_group +ADD COLUMN "source" bigint NOT NULL DEFAULT 0; +END IF; +END; +$$ LANGUAGE plpgsql; +d// + +select add_column_t_ds_worker_group(); +DROP FUNCTION IF EXISTS add_column_t_ds_worker_group(); From 9a9864606408e533f0e6ccce37ab9c20b4c2c2c4 Mon Sep 17 00:00:00 2001 From: SbloodyS <460888207@qq.com> Date: Wed, 4 Dec 2024 15:48:00 +0800 Subject: [PATCH 05/15] add feature 16834 --- .../dolphinscheduler/api/test/cases/WorkerGroupAPITest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/WorkerGroupAPITest.java b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/WorkerGroupAPITest.java index 36cb88ccb8ea..a330a7f2208e 100644 --- a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/WorkerGroupAPITest.java +++ b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/WorkerGroupAPITest.java @@ -126,7 +126,7 @@ public void testDeleteWorkerGroupById() { Assertions.assertTrue(queryAllWorkerGroupsResponse.getBody().getSuccess()); Assertions.assertTrue(workerGroupsBeforeDelete.contains("test_worker_group")); - HttpResponse deleteWorkerGroupResponse = workerGroupPage.deleteWorkerGroupById(loginUser, 1); + HttpResponse deleteWorkerGroupResponse = workerGroupPage.deleteWorkerGroupById(loginUser, 2); Assertions.assertTrue(deleteWorkerGroupResponse.getBody().getSuccess()); queryAllWorkerGroupsResponse = workerGroupPage.queryAllWorkerGroups(loginUser); From dc1ea9a8ce90cfe0c8af6a7c1fa0606db92c064e Mon Sep 17 00:00:00 2001 From: SbloodyS <460888207@qq.com> Date: Wed, 4 Dec 2024 16:38:49 +0800 Subject: [PATCH 06/15] fix ut --- .../api/service/WorkerGroupServiceTest.java | 29 +++++++++---------- .../dao/repository/WorkerGroupDao.java | 2 ++ .../repository/impl/WorkerGroupDaoImpl.java | 5 ++++ .../sql/dolphinscheduler_postgresql.sql | 4 +-- 4 files changed, 23 insertions(+), 17 deletions(-) diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/WorkerGroupServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/WorkerGroupServiceTest.java index 86ea324fd64c..01a140635edf 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/WorkerGroupServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/WorkerGroupServiceTest.java @@ -39,8 +39,8 @@ import org.apache.dolphinscheduler.dao.mapper.EnvironmentWorkerGroupRelationMapper; import org.apache.dolphinscheduler.dao.mapper.ScheduleMapper; import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper; -import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper; import org.apache.dolphinscheduler.dao.mapper.WorkflowInstanceMapper; +import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; import org.apache.dolphinscheduler.registry.api.RegistryClient; import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; @@ -76,7 +76,7 @@ public class WorkerGroupServiceTest { private WorkerGroupServiceImpl workerGroupService; @Mock - private WorkerGroupMapper workerGroupMapper; + private WorkerGroupDao workerGroupDao; @Mock private WorkflowInstanceMapper workflowInstanceMapper; @@ -142,7 +142,7 @@ public void giveSameUserName_whenSaveWorkerGroup_expectNAME_EXIST() { serverMaps.put("localhost:0000", ""); when(registryClient.getServerMaps(RegistryNodeType.WORKER)).thenReturn(serverMaps); - when(workerGroupMapper.insert(Mockito.any())).thenThrow(DuplicateKeyException.class); + when(workerGroupDao.insert(Mockito.any())).thenThrow(DuplicateKeyException.class); assertThrowsServiceException(Status.NAME_EXIST, () -> { workerGroupService.saveWorkerGroup(loginUser, 0, GROUP_NAME, "localhost:0000", "test group"); }); @@ -155,8 +155,8 @@ public void giveInvalidAddress_whenSaveWorkerGroup_expectADDRESS_INVALID() { WORKER_GROUP_CREATE, baseServiceLogger)).thenReturn(true); when(resourcePermissionCheckService.resourcePermissionCheck(AuthorizationType.WORKER_GROUP, null, 1, baseServiceLogger)).thenReturn(true); - when(workerGroupMapper.selectById(1)).thenReturn(null); - when(workerGroupMapper.queryWorkerGroupByName(GROUP_NAME)).thenReturn(null); + when(workerGroupDao.queryById(1)).thenReturn(null); + when(workerGroupDao.queryWorkerGroupByName(GROUP_NAME)).thenReturn(null); Map serverMaps = new HashMap<>(); serverMaps.put("localhost1:0000", ""); when(registryClient.getServerMaps(RegistryNodeType.WORKER)).thenReturn(serverMaps); @@ -173,11 +173,11 @@ public void giveValidWorkerGroup_whenSaveWorkerGroup_expectSuccess() { when(resourcePermissionCheckService.resourcePermissionCheck(AuthorizationType.WORKER_GROUP, null, 1, baseServiceLogger)).thenReturn(true); - when(workerGroupMapper.queryWorkerGroupByName(GROUP_NAME)).thenReturn(null); + when(workerGroupDao.queryWorkerGroupByName(GROUP_NAME)).thenReturn(null); Map serverMaps = new HashMap<>(); serverMaps.put("localhost:0000", ""); when(registryClient.getServerMaps(RegistryNodeType.WORKER)).thenReturn(serverMaps); - when(workerGroupMapper.insert(any())).thenReturn(1); + when(workerGroupDao.insert(any())).thenReturn(1); assertDoesNotThrow(() -> { workerGroupService.saveWorkerGroup(loginUser, 0, GROUP_NAME, "localhost:0000", "test group"); }); @@ -192,7 +192,7 @@ public void giveValidParams_whenQueryAllGroupPaging_expectSuccess() { workerGroups.add(getWorkerGroup(1)); when(resourcePermissionCheckService.userOwnedResourceIdsAcquisition(AuthorizationType.WORKER_GROUP, loginUser.getId(), serviceLogger)).thenReturn(ids); - when(workerGroupMapper.selectBatchIds(ids)).thenReturn(workerGroups); + when(workerGroupDao.queryByIds(ids)).thenReturn(workerGroups); Set activeWorkerNodes = new HashSet<>(); activeWorkerNodes.add("localhost:12345"); activeWorkerNodes.add("localhost:23456"); @@ -206,7 +206,7 @@ public void giveValidParams_whenQueryAllGroupPaging_expectSuccess() { public void testQueryAllGroup() { Map result = workerGroupService.queryAllGroup(getLoginUser()); List workerGroups = (List) result.get(Constants.DATA_LIST); - Assertions.assertEquals(workerGroups.size(), 1); + Assertions.assertEquals(workerGroups.size(), 0); } @Test @@ -216,7 +216,7 @@ public void giveNotExistsWorkerGroup_whenDeleteWorkerGroupById_expectNotExists() WORKER_GROUP_DELETE, baseServiceLogger)).thenReturn(true); when(resourcePermissionCheckService.resourcePermissionCheck(AuthorizationType.WORKER_GROUP, null, 1, baseServiceLogger)).thenReturn(true); - when(workerGroupMapper.selectById(1)).thenReturn(null); + when(workerGroupDao.queryById(1)).thenReturn(null); Map notExistResult = workerGroupService.deleteWorkerGroupById(loginUser, 1); Assertions.assertEquals(Status.DELETE_WORKER_GROUP_NOT_EXIST.getCode(), @@ -231,7 +231,7 @@ public void giveRunningProcess_whenDeleteWorkerGroupById_expectFailed() { when(resourcePermissionCheckService.resourcePermissionCheck(AuthorizationType.WORKER_GROUP, null, 1, baseServiceLogger)).thenReturn(true); WorkerGroup workerGroup = getWorkerGroup(1); - when(workerGroupMapper.selectById(1)).thenReturn(workerGroup); + when(workerGroupDao.queryById(1)).thenReturn(workerGroup); WorkflowInstance workflowInstance = new WorkflowInstance(); workflowInstance.setId(1); List workflowInstances = new ArrayList(); @@ -253,11 +253,11 @@ public void giveValidParams_whenDeleteWorkerGroupById_expectSuccess() { when(resourcePermissionCheckService.resourcePermissionCheck(AuthorizationType.WORKER_GROUP, null, 1, baseServiceLogger)).thenReturn(true); WorkerGroup workerGroup = getWorkerGroup(1); - when(workerGroupMapper.selectById(1)).thenReturn(workerGroup); + when(workerGroupDao.queryById(1)).thenReturn(workerGroup); when(workflowInstanceMapper.queryByWorkerGroupNameAndStatus(workerGroup.getName(), WorkflowExecutionStatus.getNotTerminalStatus())).thenReturn(null); - when(workerGroupMapper.deleteById(1)).thenReturn(1); + when(workerGroupDao.deleteById(1)).thenReturn(true); when(environmentWorkerGroupRelationMapper.queryByWorkerGroupName(workerGroup.getName())) .thenReturn(null); @@ -275,8 +275,7 @@ public void giveValidParams_whenDeleteWorkerGroupById_expectSuccess() { public void testQueryAllGroupWithDefault() { Map result = workerGroupService.queryAllGroup(getLoginUser()); List workerGroups = (List) result.get(Constants.DATA_LIST); - Assertions.assertEquals(1, workerGroups.size()); - Assertions.assertEquals("default", workerGroups.toArray()[0]); + Assertions.assertEquals(0, workerGroups.size()); } /** diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/WorkerGroupDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/WorkerGroupDao.java index 625bbfae31e5..acc690ffbc62 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/WorkerGroupDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/WorkerGroupDao.java @@ -31,4 +31,6 @@ public interface WorkerGroupDao extends IDao { List queryAllWorkerGroupNames(); List queryAllWorkerGroup(); + + List queryWorkerGroupByName(String name); } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/WorkerGroupDaoImpl.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/WorkerGroupDaoImpl.java index 70a92ff112f5..70878a27fa1a 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/WorkerGroupDaoImpl.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/WorkerGroupDaoImpl.java @@ -74,4 +74,9 @@ public List queryAllWorkerGroupNames() { public List queryAllWorkerGroup() { return mybatisMapper.queryAllWorkerGroup(); } + + @Override + public List queryWorkerGroupByName(String name) { + return mybatisMapper.queryWorkerGroupByName(name); + } } diff --git a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql index 679fa365badf..547a4486fb78 100644 --- a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql +++ b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql @@ -923,10 +923,10 @@ CREATE TABLE t_ds_worker_group ( id bigint NOT NULL , name varchar(255) NOT NULL , addr_list text DEFAULT NULL , - source bigint NOT NULL DEFAULT 0, create_time timestamp DEFAULT NULL , update_time timestamp DEFAULT NULL , - description text DEFAULT NULL, + description text DEFAULT NULL, + source bigint NOT NULL DEFAULT 0, PRIMARY KEY (id) , CONSTRAINT name_unique UNIQUE (name) ) ; From 6c6c5f7455f4f15aa21d15e2c12a36a0fd175dac Mon Sep 17 00:00:00 2001 From: SbloodyS <460888207@qq.com> Date: Wed, 4 Dec 2024 17:03:33 +0800 Subject: [PATCH 07/15] fix ut --- .../upgrade/3.3.0_schema/postgresql/dolphinscheduler_ddl.sql | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/postgresql/dolphinscheduler_ddl.sql b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/postgresql/dolphinscheduler_ddl.sql index 0263ac1beada..acf4c498ef33 100644 --- a/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/postgresql/dolphinscheduler_ddl.sql +++ b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/postgresql/dolphinscheduler_ddl.sql @@ -252,13 +252,12 @@ DROP FUNCTION IF EXISTS drop_column_t_ds_worker_group_other_params_json(); delimiter d// CREATE OR REPLACE FUNCTION add_column_t_ds_worker_group() RETURNS void AS $$ BEGIN - IF EXISTS (SELECT 1 + IF NOT EXISTS (SELECT 1 FROM information_schema.columns WHERE table_name = 't_ds_worker_group' AND column_name = 'source') THEN -ALTER TABLE t_ds_worker_group -ADD COLUMN "source" bigint NOT NULL DEFAULT 0; +ALTER TABLE "t_ds_worker_group" ADD COLUMN "source" bigint NOT NULL DEFAULT 0; END IF; END; $$ LANGUAGE plpgsql; From e01bb7b1aab4420e78c9e609af141ee368561214 Mon Sep 17 00:00:00 2001 From: SbloodyS <460888207@qq.com> Date: Wed, 4 Dec 2024 18:01:47 +0800 Subject: [PATCH 08/15] fix ut --- .../server/master/cluster/WorkerGroupChangeNotifier.java | 4 ---- .../server/master/cluster/MasterSlotManagerTest.java | 1 + dolphinscheduler-master/src/test/resources/application.yaml | 1 + 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerGroupChangeNotifier.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerGroupChangeNotifier.java index 7c982a7af7f3..cb1eb36bcc10 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerGroupChangeNotifier.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerGroupChangeNotifier.java @@ -20,7 +20,6 @@ import org.apache.dolphinscheduler.common.utils.MapComparator; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; -import org.apache.dolphinscheduler.registry.api.RegistryClient; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.utils.MasterThreadFactory; @@ -48,9 +47,6 @@ public class WorkerGroupChangeNotifier { @Autowired private MasterConfig masterConfig; - @Autowired - private RegistryClient registryClient; - private final WorkerGroupDao workerGroupDao; private final List listeners = new CopyOnWriteArrayList<>(); diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/MasterSlotManagerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/MasterSlotManagerTest.java index 56d119d776e8..2e38aa35f469 100644 --- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/MasterSlotManagerTest.java +++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/cluster/MasterSlotManagerTest.java @@ -38,6 +38,7 @@ public void setUp() { clusterManager = new ClusterManager(); masterConfig = new MasterConfig(); masterConfig.setMasterAddress("127.0.0.1:5678"); + clusterManager.init(); masterSlotManager = new MasterSlotManager(clusterManager, masterConfig); MasterServerMetadata master1 = MasterServerMetadata.builder() .cpuUsage(0.2) diff --git a/dolphinscheduler-master/src/test/resources/application.yaml b/dolphinscheduler-master/src/test/resources/application.yaml index eea966c65932..8fdf521cde1c 100644 --- a/dolphinscheduler-master/src/test/resources/application.yaml +++ b/dolphinscheduler-master/src/test/resources/application.yaml @@ -52,6 +52,7 @@ master: # The number of threads used to execute logic task. task-executor-thread-count: 4 max-heartbeat-interval: 10s + group: default server-load-protection: # If set true, will open master overload protection enabled: true From 93f6accf47cf8b99b5cfd1dd3c70aced7d681944 Mon Sep 17 00:00:00 2001 From: SbloodyS <460888207@qq.com> Date: Thu, 5 Dec 2024 13:59:13 +0800 Subject: [PATCH 09/15] fix ut --- .../impl/WorkerGroupAuditOperatorImpl.java | 6 +- .../ProjectWorkerGroupController.java | 8 +- .../ResourcePermissionCheckServiceImpl.java | 10 +- .../ProjectWorkerGroupRelationService.java | 2 +- .../api/service/WorkerGroupService.java | 4 + ...ProjectWorkerGroupRelationServiceImpl.java | 53 ++++------ .../service/impl/WorkerGroupServiceImpl.java | 96 +++++++++++++------ .../controller/WorkerGroupControllerTest.java | 10 +- ...orkerGroupResourcePermissionCheckTest.java | 6 +- ...ProjectWorkerGroupRelationServiceTest.java | 41 ++++---- .../dao/entity/WorkerGroup.java | 4 - .../dao/entity/WorkerGroupPageDetail.java | 42 ++++++++ .../dao/mapper/TaskDefinitionMapper.java | 5 +- .../dao/repository/TaskDefinitionDao.java | 2 + .../dao/repository/WorkerGroupDao.java | 3 - .../impl/TaskDefinitionDaoImpl.java | 7 +- .../repository/impl/WorkerGroupDaoImpl.java | 21 ---- .../dao/mapper/TaskDefinitionMapper.xml | 6 +- .../resources/sql/dolphinscheduler_h2.sql | 1 - .../resources/sql/dolphinscheduler_mysql.sql | 4 +- .../sql/dolphinscheduler_postgresql.sql | 1 - .../mysql/dolphinscheduler_ddl.sql | 18 ---- .../postgresql/dolphinscheduler_ddl.sql | 17 ---- .../dao/mapper/TaskDefinitionMapperTest.java | 6 +- .../server/master/cluster/ClusterManager.java | 11 +-- .../server/master/cluster/WorkerClusters.java | 23 +---- .../master/cluster/MasterSlotManagerTest.java | 1 - .../master/cluster/WorkerClustersTest.java | 16 ++-- .../BaseWorkerLoadBalancerTest.java | 7 +- ...ghtedRoundRobinWorkerLoadBalancerTest.java | 6 +- ...ghtedRoundRobinWorkerLoadBalancerTest.java | 6 +- .../list/components/use-worker-group.ts | 4 +- .../src/views/projects/list/index.tsx | 1 - .../node/fields/use-worker-group.ts | 2 +- .../projects/task/instance/batch-task.tsx | 1 - .../projects/task/instance/stream-task.tsx | 1 - .../projects/workflow/definition/index.tsx | 1 - .../projects/workflow/instance/index.tsx | 1 - 38 files changed, 207 insertions(+), 247 deletions(-) create mode 100644 dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroupPageDetail.java diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/audit/operator/impl/WorkerGroupAuditOperatorImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/audit/operator/impl/WorkerGroupAuditOperatorImpl.java index f7a264290433..d30bcf396d97 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/audit/operator/impl/WorkerGroupAuditOperatorImpl.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/audit/operator/impl/WorkerGroupAuditOperatorImpl.java @@ -23,7 +23,7 @@ import org.apache.dolphinscheduler.common.enums.AuditOperationType; import org.apache.dolphinscheduler.dao.entity.AuditLog; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; -import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper; +import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; import java.util.List; import java.util.Map; @@ -35,7 +35,7 @@ public class WorkerGroupAuditOperatorImpl extends BaseAuditOperator { @Autowired - private WorkerGroupMapper workerGroupMapper; + private WorkerGroupDao workerGroupDao; @Override public void modifyAuditOperationType(AuditType auditType, Map paramsMap, @@ -54,7 +54,7 @@ public String getObjectNameFromIdentity(Object identity) { return ""; } - WorkerGroup obj = workerGroupMapper.selectById(objId); + WorkerGroup obj = workerGroupDao.queryById(objId); return obj == null ? "" : obj.getName(); } } diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProjectWorkerGroupController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProjectWorkerGroupController.java index 24bd3eddd98b..81eba334e071 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProjectWorkerGroupController.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProjectWorkerGroupController.java @@ -90,15 +90,15 @@ public Result assignWorkerGroups(@Parameter(hidden = true) @RequestAttribute(val * @param projectCode project code * @return worker group list */ - @Operation(summary = "queryWorkerGroups", description = "QUERY_WORKER_GROUP_LIST") + @Operation(summary = "queryAssignedWorkerGroups", description = "QUERY_WORKER_GROUP_LIST") @Parameters({ @Parameter(name = "projectCode", description = "PROJECT_CODE", schema = @Schema(implementation = long.class, example = "123456")) }) @GetMapping() @ResponseStatus(HttpStatus.OK) - public Map queryWorkerGroups(@Parameter(hidden = true) @RequestAttribute(value = Constants.SESSION_USER) User loginUser, - @Parameter(name = "projectCode", description = "PROJECT_CODE", required = true) @PathVariable long projectCode) { - return projectWorkerGroupRelationService.queryWorkerGroupsByProject(loginUser, projectCode); + public Map queryAssignedWorkerGroups(@Parameter(hidden = true) @RequestAttribute(value = Constants.SESSION_USER) User loginUser, + @Parameter(name = "projectCode", description = "PROJECT_CODE", required = true) @PathVariable long projectCode) { + return projectWorkerGroupRelationService.queryAssignedWorkerGroupsByProject(loginUser, projectCode); } } diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/permission/ResourcePermissionCheckServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/permission/ResourcePermissionCheckServiceImpl.java index 91a4abfe4c2b..adfab607c636 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/permission/ResourcePermissionCheckServiceImpl.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/permission/ResourcePermissionCheckServiceImpl.java @@ -42,8 +42,8 @@ import org.apache.dolphinscheduler.dao.mapper.QueueMapper; import org.apache.dolphinscheduler.dao.mapper.TaskGroupMapper; import org.apache.dolphinscheduler.dao.mapper.TenantMapper; -import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper; import org.apache.dolphinscheduler.dao.repository.UserDao; +import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; import org.apache.dolphinscheduler.service.process.ProcessService; import java.util.Arrays; @@ -273,10 +273,10 @@ public Set listAuthorizedResourceIds(int userId, Logger logger) { @Component public static class WorkerGroupResourcePermissionCheck implements ResourceAcquisitionAndPermissionCheck { - private final WorkerGroupMapper workerGroupMapper; + private final WorkerGroupDao workerGroupDao; - public WorkerGroupResourcePermissionCheck(WorkerGroupMapper workerGroupMapper) { - this.workerGroupMapper = workerGroupMapper; + public WorkerGroupResourcePermissionCheck(WorkerGroupDao workerGroupDao) { + this.workerGroupDao = workerGroupDao; } @Override @@ -291,7 +291,7 @@ public boolean permissionCheck(int userId, String url, Logger logger) { @Override public Set listAuthorizedResourceIds(int userId, Logger logger) { - List workerGroups = workerGroupMapper.queryAllWorkerGroup(); + List workerGroups = workerGroupDao.queryAllWorkerGroup(); return workerGroups.stream().map(WorkerGroup::getId).collect(Collectors.toSet()); } } diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationService.java index bfffcd396080..008f8769ffea 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationService.java @@ -40,6 +40,6 @@ public interface ProjectWorkerGroupRelationService { * @param loginUser the login user * @param projectCode project code */ - Map queryWorkerGroupsByProject(User loginUser, Long projectCode); + Map queryAssignedWorkerGroupsByProject(User loginUser, Long projectCode); } diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/WorkerGroupService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/WorkerGroupService.java index 13cd3c3e35c9..6fe05d234149 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/WorkerGroupService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/WorkerGroupService.java @@ -20,6 +20,7 @@ import org.apache.dolphinscheduler.api.utils.Result; import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; +import org.apache.dolphinscheduler.dao.entity.WorkerGroupPageDetail; import java.util.List; import java.util.Map; @@ -79,4 +80,7 @@ public interface WorkerGroupService { */ Map queryWorkerGroupByWorkflowDefinitionCodes(List workflowDefinitionCodeList); + List getConfigWorkerGroupPageDetail(); + + List getAllWorkerGroupPageDetail(); } diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProjectWorkerGroupRelationServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProjectWorkerGroupRelationServiceImpl.java index 16b50aa4ace8..3e6e4617f55d 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProjectWorkerGroupRelationServiceImpl.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/ProjectWorkerGroupRelationServiceImpl.java @@ -21,6 +21,7 @@ import org.apache.dolphinscheduler.api.exceptions.ServiceException; import org.apache.dolphinscheduler.api.service.ProjectService; import org.apache.dolphinscheduler.api.service.ProjectWorkerGroupRelationService; +import org.apache.dolphinscheduler.api.service.WorkerGroupService; import org.apache.dolphinscheduler.api.utils.Result; import org.apache.dolphinscheduler.common.constants.Constants; import org.apache.dolphinscheduler.dao.entity.Project; @@ -28,10 +29,9 @@ import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.mapper.ProjectMapper; import org.apache.dolphinscheduler.dao.mapper.ScheduleMapper; -import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper; import org.apache.dolphinscheduler.dao.repository.ProjectWorkerGroupDao; +import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao; import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; -import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.collections4.SetUtils; @@ -66,7 +66,7 @@ public class ProjectWorkerGroupRelationServiceImpl extends BaseServiceImpl private ProjectMapper projectMapper; @Autowired - private TaskDefinitionMapper taskDefinitionMapper; + private TaskDefinitionDao taskDefinitionDao; @Autowired private ScheduleMapper scheduleMapper; @@ -77,16 +77,11 @@ public class ProjectWorkerGroupRelationServiceImpl extends BaseServiceImpl @Autowired private WorkerGroupDao workerGroupDao; - /** - * assign worker groups to a project - * - * @param loginUser the login user - * @param projectCode the project code - * @param workerGroups assigned worker group names - */ + @Autowired + private WorkerGroupService workerGroupService; + @Override public Result assignWorkerGroupsToProject(User loginUser, Long projectCode, List workerGroups) { - Result result = new Result(); if (!isAdmin(loginUser)) { @@ -115,24 +110,23 @@ public Result assignWorkerGroupsToProject(User loginUser, Long projectCode, List return result; } - Set workerGroupNames = new HashSet<>(workerGroupDao.queryAllWorkerGroupNames()); - - workerGroupNames.add(WorkerGroupUtils.getDefaultWorkerGroup()); - - Set assignedWorkerGroupNames = new HashSet<>(workerGroups); - - Set difference = SetUtils.difference(assignedWorkerGroupNames, workerGroupNames); + Set allWorkerGroupNames = new HashSet<>(workerGroupDao.queryAllWorkerGroupNames()); + workerGroupService.getConfigWorkerGroupPageDetail().forEach( + workerGroupPageDetail -> allWorkerGroupNames.add(workerGroupPageDetail.getName())); + Set unauthorizedWorkerGroupNames = new HashSet<>(workerGroups); + // check if assign worker group exists in the system + Set difference = SetUtils.difference(unauthorizedWorkerGroupNames, allWorkerGroupNames); if (!difference.isEmpty()) { putMsg(result, Status.WORKER_GROUP_NOT_EXIST, difference.toString()); return result; } + // check if assign worker group exists in the project Set projectWorkerGroupNames = projectWorkerGroupDao.queryAssignedWorkerGroupNamesByProjectCode(projectCode); - - difference = SetUtils.difference(projectWorkerGroupNames, assignedWorkerGroupNames); - + difference = SetUtils.difference(unauthorizedWorkerGroupNames, projectWorkerGroupNames); + Date now = new Date(); if (CollectionUtils.isNotEmpty(difference)) { Set usedWorkerGroups = getAllUsedWorkerGroups(project); @@ -149,11 +143,7 @@ public Result assignWorkerGroupsToProject(User loginUser, Long projectCode, List log.error("Failed to delete worker groups [{}] for the project [{}].", difference, project.getName()); throw new ServiceException(Status.ASSIGN_WORKER_GROUP_TO_PROJECT_ERROR); } - } - difference = SetUtils.difference(assignedWorkerGroupNames, projectWorkerGroupNames); - Date now = new Date(); - if (CollectionUtils.isNotEmpty(difference)) { difference.forEach(workerGroupName -> { ProjectWorkerGroup projectWorkerGroup = new ProjectWorkerGroup(); projectWorkerGroup.setProjectCode(projectCode); @@ -176,13 +166,8 @@ public Result assignWorkerGroupsToProject(User loginUser, Long projectCode, List return result; } - /** - * query worker groups that assigned to the project - * - * @param projectCode project code - */ @Override - public Map queryWorkerGroupsByProject(User loginUser, Long projectCode) { + public Map queryAssignedWorkerGroupsByProject(User loginUser, Long projectCode) { Map result = new HashMap<>(); Project project = projectMapper.queryByCode(projectCode); @@ -212,9 +197,9 @@ public Map queryWorkerGroupsByProject(User loginUser, Long proje private Set getAllUsedWorkerGroups(Project project) { Set usedWorkerGroups = new TreeSet<>(); // query all worker groups that tasks depend on - taskDefinitionMapper.queryAllDefinitionList(project.getCode()).stream().forEach(taskDefinition -> { - if (StringUtils.isNotEmpty(taskDefinition.getWorkerGroup())) { - usedWorkerGroups.add(taskDefinition.getWorkerGroup()); + taskDefinitionDao.queryAllTaskDefinitionWorkerGroups(project.getCode()).forEach(workerGroupName -> { + if (StringUtils.isNotEmpty(workerGroupName)) { + usedWorkerGroups.add(workerGroupName); } }); diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkerGroupServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkerGroupServiceImpl.java index d0a8c47022ac..2813c6578f02 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkerGroupServiceImpl.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkerGroupServiceImpl.java @@ -31,12 +31,15 @@ import org.apache.dolphinscheduler.common.enums.WorkerGroupSource; import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; import org.apache.dolphinscheduler.common.model.Server; +import org.apache.dolphinscheduler.common.model.WorkerHeartBeat; +import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.dao.entity.EnvironmentWorkerGroupRelation; import org.apache.dolphinscheduler.dao.entity.Schedule; import org.apache.dolphinscheduler.dao.entity.TaskDefinition; import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; +import org.apache.dolphinscheduler.dao.entity.WorkerGroupPageDetail; import org.apache.dolphinscheduler.dao.entity.WorkflowInstance; import org.apache.dolphinscheduler.dao.mapper.EnvironmentWorkerGroupRelationMapper; import org.apache.dolphinscheduler.dao.mapper.ScheduleMapper; @@ -100,9 +103,9 @@ public class WorkerGroupServiceImpl extends BaseServiceImpl implements WorkerGro * create or update a worker group * * @param loginUser login user - * @param id worker group id - * @param name worker group name - * @param addrList addr list + * @param id worker group id + * @param name worker group name + * @param addrList addr list * @return create or update result code */ @Override @@ -129,7 +132,6 @@ public WorkerGroup saveWorkerGroup(User loginUser, workerGroup.setCreateTime(now); workerGroup.setName(name); workerGroup.setAddrList(addrList); - workerGroup.setSource(WorkerGroupSource.UI); workerGroup.setUpdateTime(now); workerGroup.setDescription(description); workerGroupDao.insert(workerGroup); @@ -220,9 +222,9 @@ private void checkWorkerGroupAddrList(String workerGroupAddress) { * query worker group paging * * @param loginUser login user - * @param pageNo page number + * @param pageNo page number * @param searchVal search value - * @param pageSize page size + * @param pageSize page size * @return worker group list page */ @Override @@ -233,28 +235,29 @@ public Result queryAllGroupPaging(User loginUser, Integer pageNo, Integer pageSi int toIndex = (pageNo - 1) * pageSize + pageSize; Result result = new Result(); - List workerGroups; + List workerGroupPageDetails; if (loginUser.getUserType().equals(UserType.ADMIN_USER)) { - workerGroups = getWorkerGroups(null); + workerGroupPageDetails = getUiWorkerGroupPageDetails(null); } else { Set ids = resourcePermissionCheckService .userOwnedResourceIdsAcquisition(AuthorizationType.WORKER_GROUP, loginUser.getId(), log); - workerGroups = getWorkerGroups(ids.isEmpty() ? Collections.emptyList() : new ArrayList<>(ids)); + workerGroupPageDetails = + getUiWorkerGroupPageDetails(ids.isEmpty() ? Collections.emptyList() : new ArrayList<>(ids)); } - List resultDataList = new ArrayList<>(); + List resultDataList = new ArrayList<>(); int total = 0; - if (CollectionUtils.isNotEmpty(workerGroups)) { - List searchValDataList = new ArrayList<>(); + if (CollectionUtils.isNotEmpty(workerGroupPageDetails)) { + List searchValDataList = new ArrayList<>(); if (!StringUtils.isEmpty(searchVal)) { - for (WorkerGroup workerGroup : workerGroups) { + for (WorkerGroupPageDetail workerGroup : workerGroupPageDetails) { if (workerGroup.getName().contains(searchVal)) { searchValDataList.add(workerGroup); } } } else { - searchValDataList = workerGroups; + searchValDataList = workerGroupPageDetails; } total = searchValDataList.size(); if (fromIndex < searchValDataList.size()) { @@ -264,10 +267,12 @@ public Result queryAllGroupPaging(User loginUser, Integer pageNo, Integer pageSi resultDataList = searchValDataList.subList(fromIndex, toIndex); } } + List configWorkerGroupPageDetails = getConfigWorkerGroupPageDetail(); + configWorkerGroupPageDetails.addAll(resultDataList); - PageInfo pageInfo = new PageInfo<>(pageNo, pageSize); + PageInfo pageInfo = new PageInfo<>(pageNo, pageSize); pageInfo.setTotal(total); - pageInfo.setTotalList(resultDataList); + pageInfo.setTotalList(configWorkerGroupPageDetails); result.setData(pageInfo); putMsg(result, Status.SUCCESS); @@ -283,36 +288,40 @@ public Result queryAllGroupPaging(User loginUser, Integer pageNo, Integer pageSi @Override public Map queryAllGroup(User loginUser) { Map result = new HashMap<>(); - List workerGroups; + List workerGroups; if (loginUser.getUserType().equals(UserType.ADMIN_USER)) { - workerGroups = getWorkerGroups(null); + workerGroups = getUiWorkerGroupPageDetails(null); } else { Set ids = resourcePermissionCheckService .userOwnedResourceIdsAcquisition(AuthorizationType.WORKER_GROUP, loginUser.getId(), log); - workerGroups = getWorkerGroups(ids.isEmpty() ? Collections.emptyList() : new ArrayList<>(ids)); + workerGroups = getUiWorkerGroupPageDetails(ids.isEmpty() ? Collections.emptyList() : new ArrayList<>(ids)); } + List configWorkerGroupNames = getConfigWorkerGroupPageDetail().stream() + .map(WorkerGroupPageDetail::getName) + .collect(Collectors.toList()); List availableWorkerGroupList = workerGroups.stream() .map(WorkerGroup::getName) .collect(Collectors.toList()); + availableWorkerGroupList.addAll(configWorkerGroupNames); result.put(Constants.DATA_LIST, availableWorkerGroupList); putMsg(result, Status.SUCCESS); return result; } - private List getWorkerGroups(List ids) { - // worker groups from database + private List getUiWorkerGroupPageDetails(List ids) { List workerGroups; if (ids != null) { workerGroups = ids.isEmpty() ? new ArrayList<>() : workerGroupDao.queryByIds(ids); } else { workerGroups = workerGroupDao.queryAllWorkerGroup(); } - workerGroups.forEach(workerGroup -> { - if (workerGroup.getSource().equals(WorkerGroupSource.CONFIG)) { - workerGroup.setSystemDefault(true); - } - }); - return workerGroups; + return workerGroups.stream() + .map(workerGroup -> { + WorkerGroupPageDetail workerGroupPageDetail = new WorkerGroupPageDetail(workerGroup); + workerGroupPageDetail.setSource(WorkerGroupSource.UI); + workerGroupPageDetail.setSystemDefault(false); + return workerGroupPageDetail; + }).collect(Collectors.toList()); } /** @@ -397,4 +406,37 @@ private void boardCastToMasterThatWorkerGroupChanged() { } } + @Override + public List getConfigWorkerGroupPageDetail() { + List workerGroupPageDetails = new ArrayList<>(); + registryClient.getServerList(RegistryNodeType.WORKER).forEach(server -> { + WorkerGroupPageDetail workerGroupPageDetail = new WorkerGroupPageDetail(); + WorkerHeartBeat workerHeartBeat = JSONUtils.parseObject(server.getHeartBeatInfo(), WorkerHeartBeat.class); + workerGroupPageDetail.setName(workerHeartBeat.getWorkerGroup()); + workerGroupPageDetail.setAddrList(workerHeartBeat.getHost() + Constants.COLON + workerHeartBeat.getPort()); + workerGroupPageDetail.setSource(WorkerGroupSource.CONFIG); + workerGroupPageDetail.setCreateTime(DateUtils.timeStampToDate(workerHeartBeat.getStartupTime())); + workerGroupPageDetail.setUpdateTime(DateUtils.timeStampToDate(workerHeartBeat.getReportTime())); + workerGroupPageDetail.setSystemDefault(true); + workerGroupPageDetails.add(workerGroupPageDetail); + }); + return workerGroupPageDetails; + } + + @Override + public List getAllWorkerGroupPageDetail() { + List uiWorkerGroupPageDetails = workerGroupDao.queryAllWorkerGroup().stream() + .map( + workerGroup -> { + WorkerGroupPageDetail workerGroupPageDetail = new WorkerGroupPageDetail(workerGroup); + workerGroupPageDetail.setSource(WorkerGroupSource.UI); + workerGroupPageDetail.setSystemDefault(false); + return workerGroupPageDetail; + }) + .collect(Collectors.toList()); + List configWorkerGroupPageDetails = getConfigWorkerGroupPageDetail(); + configWorkerGroupPageDetails.addAll(uiWorkerGroupPageDetails); + return configWorkerGroupPageDetails; + } + } diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/WorkerGroupControllerTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/WorkerGroupControllerTest.java index 6234088ce510..06ac58fde5ce 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/WorkerGroupControllerTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/WorkerGroupControllerTest.java @@ -27,8 +27,8 @@ import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; -import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper; import org.apache.dolphinscheduler.dao.mapper.WorkflowInstanceMapper; +import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; import org.apache.dolphinscheduler.registry.api.RegistryClient; import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; @@ -53,8 +53,8 @@ public class WorkerGroupControllerTest extends AbstractControllerTest { private static final Logger logger = LoggerFactory.getLogger(WorkerGroupControllerTest.class); - @MockBean(name = "workerGroupMapper") - private WorkerGroupMapper workerGroupMapper; + @MockBean(name = "workerGroupDao") + private WorkerGroupDao workerGroupDao; @MockBean(name = "processInstanceMapper") private WorkflowInstanceMapper workflowInstanceMapper; @@ -133,11 +133,11 @@ public void testDeleteById() throws Exception { WorkerGroup workerGroup = new WorkerGroup(); workerGroup.setId(12); workerGroup.setName("测试"); - Mockito.when(workerGroupMapper.selectById(12)).thenReturn(workerGroup); + Mockito.when(workerGroupDao.queryById(12)).thenReturn(workerGroup); Mockito.when(workflowInstanceMapper.queryByWorkerGroupNameAndStatus("测试", WorkflowExecutionStatus.getNotTerminalStatus())) .thenReturn(null); - Mockito.when(workerGroupMapper.deleteById(12)).thenReturn(1); + Mockito.when(workerGroupDao.deleteById(12)).thenReturn(true); Mockito.when(workflowInstanceMapper.updateWorkflowInstanceByWorkerGroupName("测试", "")).thenReturn(1); MvcResult mvcResult = mockMvc.perform(delete("/worker-groups/{id}", "12") diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/permission/WorkerGroupResourcePermissionCheckTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/permission/WorkerGroupResourcePermissionCheckTest.java index 6f4df9ca725b..0f86f2227756 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/permission/WorkerGroupResourcePermissionCheckTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/permission/WorkerGroupResourcePermissionCheckTest.java @@ -21,7 +21,7 @@ import org.apache.dolphinscheduler.common.enums.UserType; import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; -import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper; +import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; import java.util.Arrays; import java.util.Collections; @@ -47,7 +47,7 @@ public class WorkerGroupResourcePermissionCheckTest { private ResourcePermissionCheckServiceImpl.WorkerGroupResourcePermissionCheck workerGroupResourcePermissionCheck; @Mock - private WorkerGroupMapper workerGroupMapper; + private WorkerGroupDao workerGroupDao; @Test public void testPermissionCheck() { @@ -69,7 +69,7 @@ public void testListAuthorizedResourceIds() { ids.add(workerGroup.getId()); List workerGroups = Arrays.asList(workerGroup); - Mockito.when(workerGroupMapper.queryAllWorkerGroup()).thenReturn(workerGroups); + Mockito.when(workerGroupDao.queryAllWorkerGroup()).thenReturn(workerGroups); Assertions.assertEquals(ids, workerGroupResourcePermissionCheck.listAuthorizedResourceIds(user.getId(), logger)); diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationServiceTest.java index 216d3183caa8..70afee9641fc 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProjectWorkerGroupRelationServiceTest.java @@ -30,10 +30,11 @@ import org.apache.dolphinscheduler.dao.entity.TaskDefinition; import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; +import org.apache.dolphinscheduler.dao.entity.WorkerGroupPageDetail; import org.apache.dolphinscheduler.dao.mapper.ProjectMapper; import org.apache.dolphinscheduler.dao.mapper.ScheduleMapper; -import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper; import org.apache.dolphinscheduler.dao.repository.ProjectWorkerGroupDao; +import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao; import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; import java.util.ArrayList; @@ -62,6 +63,9 @@ public class ProjectWorkerGroupRelationServiceTest { @InjectMocks private ProjectWorkerGroupRelationServiceImpl projectWorkerGroupRelationService; + @Mock + private WorkerGroupService workerGroupService; + @Mock private ProjectMapper projectMapper; @@ -75,7 +79,7 @@ public class ProjectWorkerGroupRelationServiceTest { private ProjectService projectService; @Mock - private TaskDefinitionMapper taskDefinitionMapper; + private TaskDefinitionDao taskDefinitionDao; @Mock private ScheduleMapper scheduleMapper; @@ -106,22 +110,21 @@ public void testAssignWorkerGroupsToProject() { // worker group not exists WorkerGroup workerGroup = new WorkerGroup(); workerGroup.setName("test"); + WorkerGroupPageDetail workerGroupPageDetail = new WorkerGroupPageDetail(); + workerGroupPageDetail.setName("test1"); Mockito.when(projectMapper.queryByCode(Mockito.anyLong())).thenReturn(getProject()); Mockito.when(workerGroupDao.queryAllWorkerGroup()).thenReturn(Collections.singletonList(workerGroup)); + Mockito.when(workerGroupService.getConfigWorkerGroupPageDetail()) + .thenReturn(Collections.singletonList(workerGroupPageDetail)); result = projectWorkerGroupRelationService.assignWorkerGroupsToProject(loginUser, projectCode, getDiffWorkerGroups()); Assertions.assertEquals(Status.WORKER_GROUP_NOT_EXIST.getCode(), result.getCode()); - // db insertion fail - Mockito.when(workerGroupDao.queryAllWorkerGroup()).thenReturn(Collections.singletonList(workerGroup)); - Mockito.when(projectWorkerGroupDao.insert(Mockito.any())).thenReturn(-1); - AssertionsHelper.assertThrowsServiceException(Status.ASSIGN_WORKER_GROUP_TO_PROJECT_ERROR, - () -> projectWorkerGroupRelationService.assignWorkerGroupsToProject(loginUser, projectCode, - getWorkerGroups())); - // success + Mockito.when(workerGroupDao.queryAllWorkerGroupNames()).thenReturn(getWorkerGroups()); + Mockito.when(projectWorkerGroupDao.deleteByProjectCodeAndWorkerGroups(Mockito.any(), Mockito.any())) + .thenReturn(true); Mockito.when(projectWorkerGroupDao.insert(Mockito.any())).thenReturn(1); - result = projectWorkerGroupRelationService.assignWorkerGroupsToProject(loginUser, projectCode, getWorkerGroups()); Assertions.assertEquals(Status.SUCCESS.getCode(), result.getCode()); @@ -143,21 +146,21 @@ public void testAssignWorkerGroupsToProject() { getWorkerGroups())); // fail when wg is referenced by task definition - Mockito.when(taskDefinitionMapper.queryAllDefinitionList(Mockito.anyLong())) - .thenReturn(Collections.singletonList(getTaskDefinitionWithDiffWorkerGroup())); + Mockito.when(taskDefinitionDao.queryAllTaskDefinitionWorkerGroups(Mockito.anyLong())) + .thenReturn(Collections.singletonList(getProjectWorkerGroup().getWorkerGroup())); AssertionsHelper.assertThrowsServiceException(Status.USED_WORKER_GROUP_EXISTS, () -> projectWorkerGroupRelationService.assignWorkerGroupsToProject(loginUser, projectCode, getWorkerGroups())); } @Test - public void testQueryWorkerGroupsByProject() { + public void testQueryAssignedWorkerGroupsByProject() { // no permission Mockito.when(projectService.hasProjectAndPerm(Mockito.any(), Mockito.any(), Mockito.anyMap(), Mockito.any())) .thenReturn(false); Map result = - projectWorkerGroupRelationService.queryWorkerGroupsByProject(getGeneralUser(), projectCode); + projectWorkerGroupRelationService.queryAssignedWorkerGroupsByProject(getGeneralUser(), projectCode); Assertions.assertTrue(result.isEmpty()); @@ -171,13 +174,13 @@ public void testQueryWorkerGroupsByProject() { Mockito.when(projectWorkerGroupDao.queryByProjectCode(Mockito.any())) .thenReturn(Collections.singletonList(getProjectWorkerGroup())); - Mockito.when(taskDefinitionMapper.queryAllDefinitionList(Mockito.anyLong())) + Mockito.when(taskDefinitionDao.queryAllTaskDefinitionWorkerGroups(Mockito.anyLong())) .thenReturn(new ArrayList<>()); Mockito.when(scheduleMapper.querySchedulerListByProjectName(Mockito.any())) .thenReturn(Lists.newArrayList()); - result = projectWorkerGroupRelationService.queryWorkerGroupsByProject(getGeneralUser(), projectCode); + result = projectWorkerGroupRelationService.queryAssignedWorkerGroupsByProject(getGeneralUser(), projectCode); ProjectWorkerGroup[] actualValue = ((List) result.get(Constants.DATA_LIST)).toArray(new ProjectWorkerGroup[0]); @@ -186,11 +189,11 @@ public void testQueryWorkerGroupsByProject() { } private List getWorkerGroups() { - return Lists.newArrayList("default"); + return Lists.newArrayList("test"); } private List getDiffWorkerGroups() { - return Lists.newArrayList("default", "new"); + return Lists.newArrayList("test3", "new"); } private Project getProject() { @@ -206,7 +209,7 @@ private ProjectWorkerGroup getProjectWorkerGroup() { ProjectWorkerGroup projectWorkerGroup = new ProjectWorkerGroup(); projectWorkerGroup.setId(1); projectWorkerGroup.setProjectCode(projectCode); - projectWorkerGroup.setWorkerGroup("default"); + projectWorkerGroup.setWorkerGroup("test"); return projectWorkerGroup; } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroup.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroup.java index a9ea3f175d52..df9077602ed8 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroup.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroup.java @@ -17,8 +17,6 @@ package org.apache.dolphinscheduler.dao.entity; -import org.apache.dolphinscheduler.common.enums.WorkerGroupSource; - import java.util.Date; import lombok.AllArgsConstructor; @@ -45,8 +43,6 @@ public class WorkerGroup { private String addrList; - private WorkerGroupSource source; - private Date createTime; private Date updateTime; diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroupPageDetail.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroupPageDetail.java new file mode 100644 index 000000000000..4a1d719bd5f0 --- /dev/null +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/WorkerGroupPageDetail.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.dao.entity; + +import org.apache.dolphinscheduler.common.enums.WorkerGroupSource; + +import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.NoArgsConstructor; + +@EqualsAndHashCode(callSuper = true) +@Data +@NoArgsConstructor +public class WorkerGroupPageDetail extends WorkerGroup { + + private WorkerGroupSource source; + + public WorkerGroupPageDetail(WorkerGroup workerGroup) { + this.setId(workerGroup.getId()); + this.setName(workerGroup.getName()); + this.setAddrList(workerGroup.getAddrList()); + this.setCreateTime(workerGroup.getCreateTime()); + this.setUpdateTime(workerGroup.getUpdateTime()); + this.setDescription(workerGroup.getDescription()); + this.setSystemDefault(workerGroup.isSystemDefault()); + } +} diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.java index 4d8abd150171..d9e322192ca2 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.java @@ -30,9 +30,6 @@ import com.baomidou.mybatisplus.core.mapper.BaseMapper; import com.baomidou.mybatisplus.core.metadata.IPage; -/** - * task definition mapper interface - */ public interface TaskDefinitionMapper extends BaseMapper { /** @@ -61,7 +58,7 @@ TaskDefinition queryByName(@Param("projectCode") long projectCode, * @param projectCode projectCode * @return task definition list */ - List queryAllDefinitionList(@Param("projectCode") long projectCode); + List queryAllTaskDefinitionWorkerGroups(@Param("projectCode") long projectCode); /** * count task definition group by user diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/TaskDefinitionDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/TaskDefinitionDao.java index 308a1d4a8d4e..8eb5856c43f0 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/TaskDefinitionDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/TaskDefinitionDao.java @@ -56,4 +56,6 @@ public interface TaskDefinitionDao extends IDao { * @return task definition */ TaskDefinition queryByCode(long taskCode); + + List queryAllTaskDefinitionWorkerGroups(long projectCode); } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/WorkerGroupDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/WorkerGroupDao.java index acc690ffbc62..7db0c7d10f6c 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/WorkerGroupDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/WorkerGroupDao.java @@ -17,15 +17,12 @@ package org.apache.dolphinscheduler.dao.repository; -import org.apache.dolphinscheduler.common.enums.WorkerGroupSource; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; import java.util.List; public interface WorkerGroupDao extends IDao { - boolean upsertAddrListByWorkerGroupName(String workerGroupName, String addrList, WorkerGroupSource source); - boolean deleteByWorkerGroupName(String workerGroupName); List queryAllWorkerGroupNames(); diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/TaskDefinitionDaoImpl.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/TaskDefinitionDaoImpl.java index 7037e2370a1d..ca3c9b2e6ff6 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/TaskDefinitionDaoImpl.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/TaskDefinitionDaoImpl.java @@ -45,9 +45,6 @@ import com.google.common.collect.Lists; -/** - * Task Definition DAO Implementation - */ @Repository @Slf4j public class TaskDefinitionDaoImpl extends BaseDao implements TaskDefinitionDao { @@ -120,4 +117,8 @@ public TaskDefinition queryByCode(long taskCode) { return mybatisMapper.queryByCode(taskCode); } + @Override + public List queryAllTaskDefinitionWorkerGroups(long projectCode) { + return mybatisMapper.queryAllTaskDefinitionWorkerGroups(projectCode); + } } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/WorkerGroupDaoImpl.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/WorkerGroupDaoImpl.java index 70878a27fa1a..b5203bb661c9 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/WorkerGroupDaoImpl.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/WorkerGroupDaoImpl.java @@ -17,20 +17,17 @@ package org.apache.dolphinscheduler.dao.repository.impl; -import org.apache.dolphinscheduler.common.enums.WorkerGroupSource; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper; import org.apache.dolphinscheduler.dao.repository.BaseDao; import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; -import java.util.Date; import java.util.List; import java.util.stream.Collectors; import lombok.NonNull; import org.springframework.stereotype.Repository; -import org.springframework.util.CollectionUtils; @Repository public class WorkerGroupDaoImpl extends BaseDao implements WorkerGroupDao { @@ -39,24 +36,6 @@ public WorkerGroupDaoImpl(@NonNull WorkerGroupMapper workerGroupMapper) { super(workerGroupMapper); } - @Override - public boolean upsertAddrListByWorkerGroupName(String workerGroupName, String addrList, WorkerGroupSource source) { - List currentWorkerGroups = mybatisMapper.queryWorkerGroupByName(workerGroupName); - if (CollectionUtils.isEmpty(currentWorkerGroups)) { - WorkerGroup newWorkerGroup = new WorkerGroup(); - newWorkerGroup.setName(workerGroupName); - newWorkerGroup.setAddrList(addrList); - newWorkerGroup.setSource(source); - newWorkerGroup.setCreateTime(new Date()); - newWorkerGroup.setUpdateTime(newWorkerGroup.getCreateTime()); - int inserted = mybatisMapper.insert(newWorkerGroup); - return inserted > 0; - } else { - int updated = mybatisMapper.updateAddrListByWorkerGroupName(workerGroupName, addrList, source); - return updated > 0; - } - } - @Override public boolean deleteByWorkerGroupName(String workerGroupName) { int deleted = mybatisMapper.deleteByWorkerGroupName(workerGroupName); diff --git a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.xml b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.xml index 453246e68728..fe154bd6c239 100644 --- a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.xml +++ b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.xml @@ -43,12 +43,12 @@ and td.code = ptr.post_task_code limit 1 - select - + worker_group from t_ds_task_definition where project_code = #{projectCode} - order by create_time desc + group by worker_group diff --git a/dolphinscheduler-e2e/dolphinscheduler-e2e-case/src/test/java/org/apache/dolphinscheduler/e2e/cases/ProjectE2ETest.java b/dolphinscheduler-e2e/dolphinscheduler-e2e-case/src/test/java/org/apache/dolphinscheduler/e2e/cases/ProjectE2ETest.java index 2e2d1d7f2313..18ecdb676830 100644 --- a/dolphinscheduler-e2e/dolphinscheduler-e2e-case/src/test/java/org/apache/dolphinscheduler/e2e/cases/ProjectE2ETest.java +++ b/dolphinscheduler-e2e/dolphinscheduler-e2e-case/src/test/java/org/apache/dolphinscheduler/e2e/cases/ProjectE2ETest.java @@ -23,6 +23,8 @@ import org.apache.dolphinscheduler.e2e.pages.LoginPage; import org.apache.dolphinscheduler.e2e.pages.project.ProjectPage; +import java.util.UUID; + import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Order; import org.junit.jupiter.api.Test; @@ -34,7 +36,9 @@ @DisableIfTestFails class ProjectE2ETest { - private static final String project = "test-project-1"; + private static final String project = "test-project-" + UUID.randomUUID(); + + private static final String workerGroup = "default"; private static RemoteWebDriver browser; @@ -48,7 +52,23 @@ public static void setup() { @Test @Order(1) void testCreateProject() { - new ProjectPage(browser).create(project); + final ProjectPage page = new ProjectPage(browser); + page.create(project); + + Awaitility.await().untilAsserted(() -> { + browser.navigate().refresh(); + assertThat( + page.projectList()).anyMatch( + it -> it.getText().contains(project)); + }); + } + + @Test + @Order(5) + void testAssignWorkerGroup() { + final ProjectPage page = new ProjectPage(browser); + page.assignWorkerGroup(project, workerGroup); + page.verifyAssignedWorkerGroup(project, workerGroup); } @Test diff --git a/dolphinscheduler-e2e/dolphinscheduler-e2e-case/src/test/java/org/apache/dolphinscheduler/e2e/pages/project/ProjectPage.java b/dolphinscheduler-e2e/dolphinscheduler-e2e-case/src/test/java/org/apache/dolphinscheduler/e2e/pages/project/ProjectPage.java index 33f42f92cc36..02de0daef1f5 100644 --- a/dolphinscheduler-e2e/dolphinscheduler-e2e-case/src/test/java/org/apache/dolphinscheduler/e2e/pages/project/ProjectPage.java +++ b/dolphinscheduler-e2e/dolphinscheduler-e2e-case/src/test/java/org/apache/dolphinscheduler/e2e/pages/project/ProjectPage.java @@ -28,7 +28,6 @@ import lombok.Getter; import org.openqa.selenium.By; -import org.openqa.selenium.JavascriptExecutor; import org.openqa.selenium.WebElement; import org.openqa.selenium.remote.RemoteWebDriver; import org.openqa.selenium.support.FindBy; @@ -52,10 +51,13 @@ public final class ProjectPage extends NavBarPage implements NavBarItem { private final CreateProjectForm createProjectForm; + private final AssignWorkerGroupForm assignWorkerGroupForm; + public ProjectPage(RemoteWebDriver driver) { super(driver); this.createProjectForm = new CreateProjectForm(); + this.assignWorkerGroupForm = new AssignWorkerGroupForm(); PageFactory.initElements(driver, this); } @@ -69,6 +71,7 @@ public ProjectPage create(String project) { public ProjectPage createProjectUntilSuccess(String project) { create(project); + assignWorkerGroup(project, "default"); await().untilAsserted(() -> assertThat(projectList()) .as("project list should contain newly-created project") .anyMatch(it -> it.getText().contains(project))); @@ -83,7 +86,47 @@ public ProjectPage delete(String project) { .orElseThrow(() -> new RuntimeException("Cannot find project: " + project)) .findElement(By.className("delete")).click(); - ((JavascriptExecutor) driver).executeScript("arguments[0].click();", buttonConfirm()); + driver.executeScript("arguments[0].click();", buttonConfirm()); + + return this; + } + + public ProjectPage assignWorkerGroup(String project, String workerGroup) { + projectList() + .stream() + .filter(it -> it.getText().contains(project)) + .findFirst() + .orElseThrow(() -> new RuntimeException("Can not find project: " + project)) + .findElement(By.className("assign-worker-group-btn")).click(); + + assignWorkerGroupForm.sourceWorkerGroups() + .stream() + .filter(it -> it.getText().contains(workerGroup)) + .findFirst() + .orElseThrow(() -> new RuntimeException("Can not find source worker group: " + workerGroup)) + .click(); + + assignWorkerGroupForm.buttonSubmit().click(); + + return this; + } + + public ProjectPage verifyAssignedWorkerGroup(String project, String workerGroup) { + projectList() + .stream() + .filter(it -> it.getText().contains(project)) + .findFirst() + .orElseThrow(() -> new RuntimeException("Can not find project: " + project)) + .findElement(By.className("assign-worker-group-btn")).click(); + + assignWorkerGroupForm.targetWorkerGroups() + .stream() + .filter(it -> it.getText().contains(workerGroup)) + .findFirst() + .orElseThrow(() -> new RuntimeException("Can not find target worker group: " + workerGroup)) + .click(); + + assignWorkerGroupForm.buttonCancel().click(); return this; } @@ -115,4 +158,38 @@ public class CreateProjectForm { @FindBy(className = "btn-submit") private WebElement buttonSubmit; } + + @Getter + public class AssignWorkerGroupForm { + + AssignWorkerGroupForm() { + PageFactory.initElements(driver, this); + } + + @FindBys({ + @FindBy(className = "assign-worker-group-modal"), + @FindBy(className = "n-transfer-list--source"), + @FindBy(className = "n-transfer-list-item__label") + }) + private List sourceWorkerGroups; + + @FindBys({ + @FindBy(className = "assign-worker-group-modal"), + @FindBy(className = "n-transfer-list--target"), + @FindBy(className = "n-transfer-list-item__label") + }) + private List targetWorkerGroups; + + @FindBys({ + @FindBy(className = "assign-worker-group-modal"), + @FindBy(className = "btn-submit"), + }) + private WebElement buttonSubmit; + + @FindBys({ + @FindBy(className = "assign-worker-group-modal"), + @FindBy(className = "btn-cancel"), + }) + private WebElement buttonCancel; + } } diff --git a/dolphinscheduler-ui/src/views/projects/list/components/worker-group-modal.tsx b/dolphinscheduler-ui/src/views/projects/list/components/worker-group-modal.tsx index 47b303611b07..1218b00f2ec8 100644 --- a/dolphinscheduler-ui/src/views/projects/list/components/worker-group-modal.tsx +++ b/dolphinscheduler-ui/src/views/projects/list/components/worker-group-modal.tsx @@ -72,6 +72,7 @@ const WorkerGroupModal = defineComponent({ return ( { handleAssign(row) } diff --git a/dolphinscheduler-ui/src/views/projects/task/components/node/fields/use-worker-group.ts b/dolphinscheduler-ui/src/views/projects/task/components/node/fields/use-worker-group.ts index 41eaa6cd3930..401684411354 100644 --- a/dolphinscheduler-ui/src/views/projects/task/components/node/fields/use-worker-group.ts +++ b/dolphinscheduler-ui/src/views/projects/task/components/node/fields/use-worker-group.ts @@ -55,6 +55,6 @@ export function useWorkerGroup(projectCode: number): IJsonItem { required: true, message: t('project.node.worker_group_tips') }, - value: options + value: options.value.length > 0 ? options.value[0].value : '' } } From 506446f89636671766d65697beb874eea2b86813 Mon Sep 17 00:00:00 2001 From: SbloodyS <460888207@qq.com> Date: Thu, 5 Dec 2024 16:35:07 +0800 Subject: [PATCH 14/15] fix ci --- .../sql/upgrade/3.3.0_schema/mysql/dolphinscheduler_ddl.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/mysql/dolphinscheduler_ddl.sql b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/mysql/dolphinscheduler_ddl.sql index f75e707427fa..a57326e680bc 100644 --- a/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/mysql/dolphinscheduler_ddl.sql +++ b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/mysql/dolphinscheduler_ddl.sql @@ -245,3 +245,4 @@ delimiter ; CALL drop_column_t_ds_worker_group_other_params_json; DROP PROCEDURE drop_column_t_ds_worker_group_other_params_json; +ALTER TABLE `t_ds_task_definition` ADD INDEX `idx_project_code` USING BTREE (`project_code`); From 2b937961a812df79ce50998ba3abc4bee6265eb7 Mon Sep 17 00:00:00 2001 From: SbloodyS <460888207@qq.com> Date: Thu, 5 Dec 2024 20:47:18 +0800 Subject: [PATCH 15/15] fix comment --- .../server/master/cluster/WorkerClusters.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerClusters.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerClusters.java index ef55651f1015..509c54f3fba4 100644 --- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerClusters.java +++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/WorkerClusters.java @@ -89,8 +89,10 @@ public void registerListener(IClustersChangeListener liste @Override public void onWorkerGroupDelete(List workerGroups) { - for (WorkerGroup workerGroup : workerGroups) { - workerGroupMapping.remove(workerGroup.getName()); + synchronized (workerGroupMapping) { + for (WorkerGroup workerGroup : workerGroups) { + workerGroupMapping.remove(workerGroup.getName()); + } } } @@ -110,7 +112,9 @@ public void onWorkerGroupChange(List workerGroups) { .filter(Objects::nonNull) .map(WorkerServerMetadata::getAddress) .collect(Collectors.toList()); - workerGroupMapping.put(workerGroup.getName(), activeWorkers); + synchronized (workerGroupMapping) { + workerGroupMapping.put(workerGroup.getName(), activeWorkers); + } } } @@ -151,8 +155,6 @@ public void onServerRemove(WorkerServerMetadata workerServer) { removeWorkerGroupAddrList.remove(workerServer.getAddress()); if (removeWorkerGroupAddrList.isEmpty()) { workerGroupMapping.remove(workerServer.getWorkerGroup()); - } else { - workerGroupMapping.put(workerServer.getWorkerGroup(), removeWorkerGroupAddrList); } } }