增加Topic同步任务&Bug修复

This commit is contained in:
zengqiao
2021-01-16 16:26:38 +08:00
parent 3c091a88d4
commit d5680ffd5d
88 changed files with 2230 additions and 404 deletions

View File

@@ -92,20 +92,4 @@ public class ConsumerMetadataCache {
}
return consumerMetadata.getTopicNameConsumerGroupMap().getOrDefault(topicName, new HashSet<>());
}
public static Map<String, List<String>> getConsumerGroupAppIdListInZk(Long clusterId) {
ConsumerMetadata consumerMetadata = CG_METADATA_IN_ZK_MAP.get(clusterId);
if(consumerMetadata == null){
return new HashMap<>(0);
}
return consumerMetadata.getConsumerGroupAppMap();
}
public static Map<String, List<String>> getConsumerGroupAppIdListInBK(Long clusterId) {
ConsumerMetadata consumerMetadata = CG_METADATA_IN_BK_MAP.get(clusterId);
if(consumerMetadata == null){
return new HashMap<>(0);
}
return consumerMetadata.getConsumerGroupAppMap();
}
}

View File

@@ -1,5 +1,6 @@
package com.xiaojukeji.kafka.manager.service.cache;
import com.google.common.collect.Sets;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.entity.pojo.LogicalClusterDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.RegionDO;
@@ -15,6 +16,7 @@ import org.springframework.stereotype.Service;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
/**
* 逻辑集群元信息
@@ -144,9 +146,16 @@ public class LogicalClusterMetadataManager {
@Scheduled(cron="0/30 * * * * ?")
public void flush() {
List<LogicalClusterDO> logicalClusterDOList = logicalClusterService.listAll();
if (ValidateUtils.isEmptyList(logicalClusterDOList)) {
return;
if (ValidateUtils.isNull(logicalClusterDOList)) {
logicalClusterDOList = Collections.EMPTY_LIST;
}
Set<Long> inDbLogicalClusterIds = logicalClusterDOList.stream()
.map(LogicalClusterDO::getId)
.collect(Collectors.toSet());
// inCache 和 inDb 取差集,差集结果为已删除的、新增的.
Sets.SetView<Long> diffLogicalClusterIds = Sets.difference(LOGICAL_CLUSTER_MAP.keySet(), inDbLogicalClusterIds);
diffLogicalClusterIds.forEach(logicalClusterId -> delLogicalClusterInCache(logicalClusterId));
Map<Long, RegionDO> regionMap = new HashMap<>();
List<RegionDO> regionDOList = regionService.listAll();
@@ -197,4 +206,11 @@ public class LogicalClusterMetadataManager {
}
TOPIC_LOGICAL_MAP.put(logicalClusterDO.getClusterId(), subMap);
}
private void delLogicalClusterInCache(Long logicalClusterId) {
LOGICAL_CLUSTER_ID_TOPIC_NAME_MAP.remove(logicalClusterId);
LOGICAL_CLUSTER_ID_BROKER_ID_MAP.remove(logicalClusterId);
LOGICAL_CLUSTER_MAP.remove(logicalClusterId);
TOPIC_LOGICAL_MAP.remove(logicalClusterId);
}
}

View File

@@ -13,6 +13,8 @@ import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata
import com.xiaojukeji.kafka.manager.common.zookeeper.ZkConfigImpl;
import com.xiaojukeji.kafka.manager.dao.ControllerDao;
import com.xiaojukeji.kafka.manager.common.utils.jmx.JmxConnectorWrap;
import com.xiaojukeji.kafka.manager.dao.TopicDao;
import com.xiaojukeji.kafka.manager.dao.gateway.AuthorityDao;
import com.xiaojukeji.kafka.manager.service.service.JmxService;
import com.xiaojukeji.kafka.manager.service.utils.ConfigUtils;
import com.xiaojukeji.kafka.manager.service.zookeeper.*;
@@ -48,6 +50,12 @@ public class PhysicalClusterMetadataManager {
@Autowired
private ConfigUtils configUtils;
@Autowired
private TopicDao topicDao;
@Autowired
private AuthorityDao authorityDao;
private final static Map<Long, ClusterDO> CLUSTER_MAP = new ConcurrentHashMap<>();
private final static Map<Long, ControllerData> CONTROLLER_DATA_MAP = new ConcurrentHashMap<>();
@@ -116,7 +124,7 @@ public class PhysicalClusterMetadataManager {
zkConfig.watchChildren(ZkPathUtil.BROKER_IDS_ROOT, brokerListener);
//增加Topic监控
TopicStateListener topicListener = new TopicStateListener(clusterDO.getId(), zkConfig);
TopicStateListener topicListener = new TopicStateListener(clusterDO.getId(), zkConfig, topicDao, authorityDao);
topicListener.init();
zkConfig.watchChildren(ZkPathUtil.BROKER_TOPICS_ROOT, topicListener);

View File

@@ -9,6 +9,19 @@ import java.util.List;
import java.util.Properties;
public interface AdminService {
/**
* 创建Topic
* @param clusterDO 集群DO
* @param topicDO TopicDO
* @param partitionNum 分区数
* @param replicaNum 副本数
* @param regionId RegionID
* @param brokerIdList BrokerId
* @param properties Topic属性
* @param applicant 申请人
* @param operator 操作人
* @return 操作状态
*/
ResultStatus createTopic(ClusterDO clusterDO,
TopicDO topicDO,
Integer partitionNum,
@@ -19,19 +32,86 @@ public interface AdminService {
String applicant,
String operator);
/**
* 删除Topic
* @param clusterDO 集群DO
* @param topicName Topic名称
* @param operator 操作人
* @return 操作状态
*/
ResultStatus deleteTopic(ClusterDO clusterDO,
String topicName,
String operator);
/**
* 优先副本选举状态
* @param clusterDO 集群DO
* @return 任务状态
*/
TaskStatusEnum preferredReplicaElectionStatus(ClusterDO clusterDO);
/**
* 集群纬度优先副本选举
* @param clusterDO 集群DO
* @return 任务状态
*/
ResultStatus preferredReplicaElection(ClusterDO clusterDO, String operator);
/**
* Broker纬度优先副本选举
* @param clusterDO 集群DO
* @param brokerId BrokerID
* @param operator 操作人
* @return 任务状态
*/
ResultStatus preferredReplicaElection(ClusterDO clusterDO, Integer brokerId, String operator);
/**
* Topic纬度优先副本选举
* @param clusterDO 集群DO
* @param topicName Topic名称
* @param operator 操作人
* @return 任务状态
*/
ResultStatus preferredReplicaElection(ClusterDO clusterDO, String topicName, String operator);
/**
* 分区纬度优先副本选举
* @param clusterDO 集群DO
* @param topicName Topic名称
* @param partitionId 分区ID
* @param operator 操作人
* @return 任务状态
*/
ResultStatus preferredReplicaElection(ClusterDO clusterDO, String topicName, Integer partitionId, String operator);
/**
* Topic扩分区
* @param clusterDO 集群DO
* @param topicName Topic名称
* @param partitionNum 新增? 分区数
* @param regionId RegionID
* @param brokerIdList 集群ID
* @param operator 操作人
* @return 任务状态
*/
ResultStatus expandPartitions(ClusterDO clusterDO, String topicName, Integer partitionNum, Long regionId, List<Integer> brokerIdList, String operator);
/**
* 获取Topic配置
* @param clusterDO 集群DO
* @param topicName Topic名称
* @return 任务状态
*/
Properties getTopicConfig(ClusterDO clusterDO, String topicName);
/**
* 修改Topic配置
* @param clusterDO 集群DO
* @param topicName Topic名称
* @param properties 新的属性
* @param operator 操作人
* @return 任务状态
*/
ResultStatus modifyTopicConfig(ClusterDO clusterDO, String topicName, Properties properties, String operator);
}

View File

@@ -1,7 +1,9 @@
package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.entity.Result;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.ClusterDetailDTO;
import com.xiaojukeji.kafka.manager.common.entity.ao.cluster.ControllerPreferredCandidate;
import com.xiaojukeji.kafka.manager.common.entity.vo.normal.cluster.ClusterNameDTO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterMetricsDO;
@@ -43,5 +45,10 @@ public interface ClusterService {
ResultStatus deleteById(Long clusterId);
ClusterDO selectSuitableCluster(Long clusterId, String dataCenter);
/**
* 获取优先被选举为controller的broker
* @param clusterId 集群ID
* @return void
*/
Result<List<ControllerPreferredCandidate>> getControllerPreferredCandidates(Long clusterId);
}

View File

@@ -2,14 +2,14 @@ package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.bizenum.OffsetLocationEnum;
import com.xiaojukeji.kafka.manager.common.entity.Result;
import com.xiaojukeji.kafka.manager.common.entity.ao.consumer.ConsumerGroupDTO;
import com.xiaojukeji.kafka.manager.common.entity.ao.consumer.ConsumerGroup;
import com.xiaojukeji.kafka.manager.common.entity.ao.PartitionOffsetDTO;
import com.xiaojukeji.kafka.manager.common.entity.ao.consumer.ConsumeDetailDTO;
import com.xiaojukeji.kafka.manager.common.entity.ao.consumer.ConsumerGroupSummary;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterDO;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* consumer相关的服务接口
@@ -20,33 +20,36 @@ public interface ConsumerService {
/**
* 获取消费组列表
*/
List<ConsumerGroupDTO> getConsumerGroupList(Long clusterId);
List<ConsumerGroup> getConsumerGroupList(Long clusterId);
/**
* 查询消费Topic的消费组
*/
List<ConsumerGroupDTO> getConsumerGroupList(Long clusterId, String topicName);
List<ConsumerGroup> getConsumerGroupList(Long clusterId, String topicName);
/**
* 获取消费Topic的消费组概要信息
*/
List<ConsumerGroupSummary> getConsumerGroupSummaries(Long clusterId, String topicName);
/**
* 查询消费详情
*/
List<ConsumeDetailDTO> getConsumeDetail(ClusterDO clusterDO, String topicName, ConsumerGroupDTO consumerGroupDTO);
List<ConsumeDetailDTO> getConsumeDetail(ClusterDO clusterDO, String topicName, ConsumerGroup consumerGroup);
/**
* 获取消费组消费的Topic列表
*/
List<String> getConsumerGroupConsumedTopicList(Long clusterId, String consumerGroup, String location);
Map<Integer, Long> getConsumerOffset(ClusterDO clusterDO,
String topicName,
ConsumerGroupDTO consumerGroupDTO);
Map<Integer, Long> getConsumerOffset(ClusterDO clusterDO, String topicName, ConsumerGroup consumerGroup);
/**
* 重置offset
*/
List<Result> resetConsumerOffset(ClusterDO clusterDO,
String topicName,
ConsumerGroupDTO consumerGroupDTO,
ConsumerGroup consumerGroup,
List<PartitionOffsetDTO> partitionOffsetDTOList);
Map<Long, Integer> getConsumerGroupNumMap(List<ClusterDO> clusterDOList);

View File

@@ -66,6 +66,19 @@ public interface TopicManagerService {
*/
ResultStatus modifyTopic(Long clusterId, String topicName, String description, String operator);
/**
* 修改Topic
* @param clusterId 集群ID
* @param topicName Topic名称
* @param appId 所属应用
* @param description 备注
* @param operator 操作人
* @author zengqiao
* @date 20/5/12
* @return ResultStatus
*/
ResultStatus modifyTopicByOp(Long clusterId, String topicName, String appId, String description, String operator);
/**
* 通过topictopic名称删除
* @param clusterId 集群id

View File

@@ -3,11 +3,27 @@ package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.entity.Result;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.didi.TopicJmxSwitch;
import java.util.List;
/**
* ZK相关的接口
* @author tukun
* @date 2015/11/11.
*/
public interface ZookeeperService {
/**
* 开启JMX
* @param clusterId 集群ID
* @param topicName Topic名称
* @param jmxSwitch JMX开关
* @return 操作结果
*/
Result openTopicJmx(Long clusterId, String topicName, TopicJmxSwitch jmxSwitch);
/**
* 获取优先被选举为controller的broker
* @param clusterId 集群ID
* @return 操作结果
*/
Result<List<Integer>> getControllerPreferredCandidates(Long clusterId);
}

View File

@@ -60,4 +60,6 @@ public interface AuthorityService {
int addAuthorityAndQuota(AuthorityDO authorityDO, TopicQuota quota);
Map<String, Map<Long, Map<String, AuthorityDO>>> getAllAuthority();
int deleteAuthorityByTopic(Long clusterId, String topicName);
}

View File

@@ -1,18 +1,86 @@
package com.xiaojukeji.kafka.manager.service.service.gateway;
import com.xiaojukeji.kafka.manager.common.entity.Result;
import com.xiaojukeji.kafka.manager.common.entity.ao.gateway.*;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.GatewayConfigDO;
import java.util.List;
public interface GatewayConfigService {
/**
* 获取集群服务地址
* @param requestVersion 请求的版本
* @return
*/
KafkaBootstrapServerConfig getKafkaBootstrapServersConfig(Long requestVersion);
/**
* 获取服务发现的请求队列的配置
* @param requestVersion 请求的版本
* @return
*/
RequestQueueConfig getRequestQueueConfig(Long requestVersion);
/**
* 获取服务发现的App请求速度的配置
* @param requestVersion 请求的版本
* @return
*/
AppRateConfig getAppRateConfig(Long requestVersion);
/**
* 获取服务发现的IP请求速度的配置
* @param requestVersion 请求的版本
* @return
*/
IpRateConfig getIpRateConfig(Long requestVersion);
/**
* 获取服务发现的具体IP或者应用纬度的限速配置
* @param requestVersion 请求的版本
* @return
*/
SpRateConfig getSpRateConfig(Long requestVersion);
/**
* 获取配置
* @param configType 配置类型
* @param configName 配置名称
* @return
*/
GatewayConfigDO getByTypeAndName(String configType, String configName);
/**
* 获取配置
* @return
*/
List<GatewayConfigDO> list();
/**
* 新建配置
* @param gatewayConfigDO 配置信息
* @return
*/
Result insert(GatewayConfigDO gatewayConfigDO);
/**
* 删除配置
* @param id 配置ID
* @return
*/
Result deleteById(Long id);
/**
* 更新配置
* @param gatewayConfigDO 配置信息
* @return
*/
Result updateById(GatewayConfigDO gatewayConfigDO);
/**
* 获取配置
* @param id 配置ID
* @return
*/
GatewayConfigDO getById(Long id);
}

View File

@@ -196,8 +196,7 @@ public class AppServiceImpl implements AppService {
}
@Override
public List<AppTopicDTO> getAppTopicDTOList(String appId,
Boolean mine) {
public List<AppTopicDTO> getAppTopicDTOList(String appId, Boolean mine) {
// 查询AppID
AppDO appDO = appDao.getByAppId(appId);
if (ValidateUtils.isNull(appDO)) {
@@ -223,13 +222,17 @@ public class AppServiceImpl implements AppService {
TopicDO topicDO = topicMap
.getOrDefault(authorityDO.getClusterId(), new HashMap<>())
.get(authorityDO.getTopicName());
if (ValidateUtils.isNull(topicDO)) {
continue;
}
if (Boolean.TRUE.equals(mine)
&& (ValidateUtils.isNull(topicDO) || !topicDO.getAppId().equals(appId))) {
&& !topicDO.getAppId().equals(appId)) {
continue;
}
if (Boolean.FALSE.equals(mine)
&& !ValidateUtils.isNull(topicDO)
&& topicDO.getAppId().equals(appId)) {
continue;
}

View File

@@ -192,4 +192,10 @@ public class AuthorityServiceImpl implements AuthorityService {
public Map<String, Map<Long, Map<String, AuthorityDO>>> getAllAuthority() {
return authorityDao.getAllAuthority();
}
@Override
public int deleteAuthorityByTopic(Long clusterId, String topicName) {
return authorityDao.deleteAuthorityByTopic(clusterId, topicName);
}
}

View File

@@ -2,6 +2,8 @@ package com.xiaojukeji.kafka.manager.service.service.gateway.impl;
import com.alibaba.fastjson.JSON;
import com.xiaojukeji.kafka.manager.common.bizenum.gateway.GatewayConfigKeyEnum;
import com.xiaojukeji.kafka.manager.common.entity.Result;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.gateway.*;
import com.xiaojukeji.kafka.manager.common.utils.ListUtils;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
@@ -13,6 +15,7 @@ import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -21,7 +24,7 @@ import java.util.Map;
* @author zengqiao
* @date 20/7/28
*/
@Service("gatewayConfigService")
@Service
public class GatewayConfigServiceImpl implements GatewayConfigService {
private final Logger LOGGER = LoggerFactory.getLogger(GatewayConfigServiceImpl.class);
@@ -52,7 +55,8 @@ public class GatewayConfigServiceImpl implements GatewayConfigService {
? new KafkaBootstrapServerConfig(maxVersion, clusterIdBootstrapServersMap)
: new KafkaBootstrapServerConfig(requestVersion, new HashMap<>(0));
} catch (Exception e) {
LOGGER.error("get kafka bootstrap servers config failed, data:{}.", JSON.toJSONString(doList), e);
LOGGER.error("class=GatewayConfigServiceImpl||method=getKafkaBootstrapServersConfig||data={}||errMsg={}||msg=get kafka bootstrap servers config failed",
JSON.toJSONString(doList), e.getMessage());
}
return null;
}
@@ -71,7 +75,8 @@ public class GatewayConfigServiceImpl implements GatewayConfigService {
return new RequestQueueConfig(configDO.getVersion(), Long.valueOf(configDO.getValue()));
} catch (Exception e) {
LOGGER.error("get request queue config failed, data:{}.", JSON.toJSONString(configDO), e);
LOGGER.error("class=GatewayConfigServiceImpl||method=getRequestQueueConfig||data={}||errMsg={}||msg=get request queue config failed",
JSON.toJSONString(configDO), e.getMessage());
}
return null;
}
@@ -90,7 +95,8 @@ public class GatewayConfigServiceImpl implements GatewayConfigService {
return new AppRateConfig(configDO.getVersion(), Long.valueOf(configDO.getValue()));
} catch (Exception e) {
LOGGER.error("get app rate config failed, data:{}.", JSON.toJSONString(configDO), e);
LOGGER.error("class=GatewayConfigServiceImpl||method=getAppRateConfig||data={}||errMsg={}||msg=get app rate config failed",
JSON.toJSONString(configDO), e.getMessage());
}
return null;
}
@@ -153,4 +159,94 @@ public class GatewayConfigServiceImpl implements GatewayConfigService {
}
return null;
}
@Override
public List<GatewayConfigDO> list() {
try {
return gatewayConfigDao.list();
} catch (Exception e) {
LOGGER.debug("class=GatewayConfigServiceImpl||method=list||errMsg={}||msg=list failed", e.getMessage());
}
return new ArrayList<>();
}
@Override
public Result insert(GatewayConfigDO gatewayConfigDO) {
try {
GatewayConfigKeyEnum configKeyEnum = GatewayConfigKeyEnum.getByConfigType(gatewayConfigDO.getType());
if (ValidateUtils.isNull(configKeyEnum)
&& ValidateUtils.isBlank(gatewayConfigDO.getName())
&& ValidateUtils.isBlank(gatewayConfigDO.getValue())) {
// 参数错误
return Result.buildFrom(ResultStatus.PARAM_ILLEGAL);
}
// 获取当前同类配置, 插入之后需要增大这个version
List<GatewayConfigDO> gatewayConfigDOList = gatewayConfigDao.getByConfigType(gatewayConfigDO.getType());
Long version = 1L;
for (GatewayConfigDO elem: gatewayConfigDOList) {
if (elem.getVersion() > version) {
version = elem.getVersion() + 1L;
}
}
gatewayConfigDO.setVersion(version);
if (gatewayConfigDao.insert(gatewayConfigDO) > 0) {
return Result.buildSuc();
}
return Result.buildFrom(ResultStatus.MYSQL_ERROR);
} catch (Exception e) {
LOGGER.debug("class=GatewayConfigServiceImpl||method=insert||data={}||errMsg={}||msg=insert failed", gatewayConfigDO, e.getMessage());
}
return Result.buildFrom(ResultStatus.MYSQL_ERROR);
}
@Override
public Result deleteById(Long id) {
try {
if (gatewayConfigDao.deleteById(id) > 0) {
return Result.buildSuc();
}
return Result.buildFrom(ResultStatus.RESOURCE_NOT_EXIST);
} catch (Exception e) {
LOGGER.debug("class=GatewayConfigServiceImpl||method=deleteById||id={}||errMsg={}||msg=delete failed", id, e.getMessage());
}
return Result.buildFrom(ResultStatus.MYSQL_ERROR);
}
@Override
public Result updateById(GatewayConfigDO newGatewayConfigDO) {
try {
GatewayConfigDO oldGatewayConfigDO = this.getById(newGatewayConfigDO.getId());
if (ValidateUtils.isNull(oldGatewayConfigDO)) {
return Result.buildFrom(ResultStatus.RESOURCE_NOT_EXIST);
}
if (!oldGatewayConfigDO.getName().equals(newGatewayConfigDO.getName())
|| !oldGatewayConfigDO.getType().equals(newGatewayConfigDO.getType())
|| ValidateUtils.isBlank(newGatewayConfigDO.getValue())) {
return Result.buildFrom(ResultStatus.PARAM_ILLEGAL);
}
newGatewayConfigDO.setVersion(oldGatewayConfigDO.getVersion() + 1);
if (gatewayConfigDao.updateById(oldGatewayConfigDO) > 0) {
return Result.buildSuc();
}
return Result.buildFrom(ResultStatus.MYSQL_ERROR);
} catch (Exception e) {
LOGGER.debug("class=GatewayConfigServiceImpl||method=updateById||data={}||errMsg={}||msg=update failed", newGatewayConfigDO, e.getMessage());
}
return Result.buildFrom(ResultStatus.MYSQL_ERROR);
}
@Override
public GatewayConfigDO getById(Long id) {
if (ValidateUtils.isNull(id)) {
return null;
}
try {
return gatewayConfigDao.getById(id);
} catch (Exception e) {
LOGGER.debug("class=GatewayConfigServiceImpl||method=getById||id={}||errMsg={}||msg=get failed", id, e.getMessage());
}
return null;
}
}

View File

@@ -13,6 +13,7 @@ import com.xiaojukeji.kafka.manager.common.zookeeper.ZkConfigImpl;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.BrokerMetadata;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.TopicDO;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.service.*;
import com.xiaojukeji.kafka.manager.service.service.gateway.AuthorityService;
@@ -139,6 +140,9 @@ public class AdminServiceImpl implements AdminService {
// 3. 数据库中删除topic
topicManagerService.deleteByTopicName(clusterDO.getId(), topicName);
// 4. 数据库中删除authority
authorityService.deleteAuthorityByTopic(clusterDO.getId(), topicName);
return rs;
}
@@ -191,15 +195,55 @@ public class AdminServiceImpl implements AdminService {
@Override
public ResultStatus preferredReplicaElection(ClusterDO clusterDO, Integer brokerId, String operator) {
BrokerMetadata brokerMetadata = PhysicalClusterMetadataManager.getBrokerMetadata(clusterDO.getId(), brokerId);
if (null == brokerMetadata) {
if (ValidateUtils.isNull(brokerMetadata)) {
return ResultStatus.PARAM_ILLEGAL;
}
Map<String, List<Integer>> partitionMap = topicService.getTopicPartitionIdMap(clusterDO.getId(), brokerId);
if (ValidateUtils.isEmptyMap(partitionMap)) {
return ResultStatus.SUCCESS;
}
return preferredReplicaElection(clusterDO, partitionMap, operator);
}
@Override
public ResultStatus preferredReplicaElection(ClusterDO clusterDO, String topicName, String operator) {
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterDO.getId(), topicName);
if (ValidateUtils.isNull(topicMetadata)) {
return ResultStatus.TOPIC_NOT_EXIST;
}
Map<String, List<Integer>> partitionMap = new HashMap<>();
partitionMap.put(topicName, new ArrayList<>(topicMetadata.getPartitionMap().getPartitions().keySet()));
return preferredReplicaElection(clusterDO, partitionMap, operator);
}
@Override
public ResultStatus preferredReplicaElection(ClusterDO clusterDO, String topicName, Integer partitionId, String operator) {
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterDO.getId(), topicName);
if (ValidateUtils.isNull(topicMetadata)) {
return ResultStatus.TOPIC_NOT_EXIST;
}
if (!topicMetadata.getPartitionMap().getPartitions().containsKey(partitionId)) {
return ResultStatus.PARTITION_NOT_EXIST;
}
Map<String, List<Integer>> partitionMap = new HashMap<>();
partitionMap.put(topicName, Arrays.asList(partitionId));
return preferredReplicaElection(clusterDO, partitionMap, operator);
}
private ResultStatus preferredReplicaElection(ClusterDO clusterDO, Map<String, List<Integer>> partitionMap, String operator) {
if (ValidateUtils.isEmptyMap(partitionMap)) {
return ResultStatus.SUCCESS;
}
ZkUtils zkUtils = null;
try {
Map<String, List<Integer>> partitionMap = topicService.getTopicPartitionIdMap(clusterDO.getId(), brokerId);
if (partitionMap == null || partitionMap.isEmpty()) {
return ResultStatus.SUCCESS;
}
String preferredReplicaElectString = convert2preferredReplicaElectString(partitionMap);
zkUtils = ZkUtils.apply(clusterDO.getZookeeper(),

View File

@@ -1,11 +1,16 @@
package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.common.bizenum.DBStatusEnum;
import com.xiaojukeji.kafka.manager.common.constant.Constant;
import com.xiaojukeji.kafka.manager.common.entity.Result;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.ClusterDetailDTO;
import com.xiaojukeji.kafka.manager.common.entity.ao.cluster.ControllerPreferredCandidate;
import com.xiaojukeji.kafka.manager.common.entity.vo.normal.cluster.ClusterNameDTO;
import com.xiaojukeji.kafka.manager.common.utils.ListUtils;
import com.xiaojukeji.kafka.manager.common.entity.pojo.*;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.BrokerMetadata;
import com.xiaojukeji.kafka.manager.dao.ClusterDao;
import com.xiaojukeji.kafka.manager.dao.ClusterMetricsDao;
import com.xiaojukeji.kafka.manager.dao.ControllerDao;
@@ -14,6 +19,7 @@ import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager
import com.xiaojukeji.kafka.manager.service.service.ClusterService;
import com.xiaojukeji.kafka.manager.service.service.ConsumerService;
import com.xiaojukeji.kafka.manager.service.service.RegionService;
import com.xiaojukeji.kafka.manager.service.service.ZookeeperService;
import com.xiaojukeji.kafka.manager.service.utils.ConfigUtils;
import org.apache.zookeeper.ZooKeeper;
import org.slf4j.Logger;
@@ -57,6 +63,9 @@ public class ClusterServiceImpl implements ClusterService {
@Autowired
private ConfigUtils configUtils;
@Autowired
private ZookeeperService zookeeperService;
@Override
public ResultStatus addNew(ClusterDO clusterDO, String operator) {
if (ValidateUtils.isNull(clusterDO) || ValidateUtils.isNull(operator)) {
@@ -262,21 +271,6 @@ public class ClusterServiceImpl implements ClusterService {
return ResultStatus.SUCCESS;
}
@Override
public ClusterDO selectSuitableCluster(Long clusterId, String dataCenter) {
if (!ValidateUtils.isNullOrLessThanZero(clusterId)) {
return getById(clusterId);
}
if (ValidateUtils.isBlank(dataCenter)) {
return null;
}
List<ClusterDO> clusterDOList = this.listAll();
if (ValidateUtils.isEmptyList(clusterDOList)) {
return null;
}
return clusterDOList.get(0);
}
private ClusterDetailDTO getClusterDetailDTO(ClusterDO clusterDO, Boolean needDetail) {
if (ValidateUtils.isNull(clusterDO)) {
return null;
@@ -300,4 +294,31 @@ public class ClusterServiceImpl implements ClusterService {
dto.setControllerId(PhysicalClusterMetadataManager.getControllerId(clusterDO.getId()));
return dto;
}
@Override
public Result<List<ControllerPreferredCandidate>> getControllerPreferredCandidates(Long clusterId) {
Result<List<Integer>> candidateResult = zookeeperService.getControllerPreferredCandidates(clusterId);
if (candidateResult.failed()) {
return new Result<>(candidateResult.getCode(), candidateResult.getMessage());
}
if (ValidateUtils.isEmptyList(candidateResult.getData())) {
return Result.buildSuc(new ArrayList<>());
}
List<ControllerPreferredCandidate> controllerPreferredCandidateList = new ArrayList<>();
for (Integer brokerId: candidateResult.getData()) {
ControllerPreferredCandidate controllerPreferredCandidate = new ControllerPreferredCandidate();
controllerPreferredCandidate.setBrokerId(brokerId);
BrokerMetadata brokerMetadata = PhysicalClusterMetadataManager.getBrokerMetadata(clusterId, brokerId);
if (ValidateUtils.isNull(brokerMetadata)) {
controllerPreferredCandidate.setStatus(DBStatusEnum.DEAD.getStatus());
} else {
controllerPreferredCandidate.setHost(brokerMetadata.getHost());
controllerPreferredCandidate.setStartTime(brokerMetadata.getTimestamp());
controllerPreferredCandidate.setStatus(DBStatusEnum.ALIVE.getStatus());
}
controllerPreferredCandidateList.add(controllerPreferredCandidate);
}
return Result.buildSuc(controllerPreferredCandidateList);
}
}

View File

@@ -2,13 +2,14 @@ package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.common.bizenum.OffsetPosEnum;
import com.xiaojukeji.kafka.manager.common.bizenum.OffsetLocationEnum;
import com.xiaojukeji.kafka.manager.common.bizenum.SinkMonitorSystemEnum;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.Result;
import com.xiaojukeji.kafka.manager.common.entity.ao.consumer.ConsumeDetailDTO;
import com.xiaojukeji.kafka.manager.common.entity.ao.consumer.ConsumerGroup;
import com.xiaojukeji.kafka.manager.common.entity.ao.consumer.ConsumerGroupSummary;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterDO;
import com.xiaojukeji.kafka.manager.common.utils.ListUtils;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata;
import com.xiaojukeji.kafka.manager.common.entity.ao.consumer.ConsumerGroupDTO;
import com.xiaojukeji.kafka.manager.common.entity.ao.PartitionOffsetDTO;
import com.xiaojukeji.kafka.manager.common.exception.ConfigException;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
@@ -23,6 +24,7 @@ import kafka.admin.AdminClient;
import org.apache.commons.lang.StringUtils;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.protocol.types.SchemaException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
@@ -44,70 +46,116 @@ public class ConsumerServiceImpl implements ConsumerService {
private TopicService topicService;
@Override
public List<ConsumerGroupDTO> getConsumerGroupList(Long clusterId) {
List<ConsumerGroupDTO> consumerGroupDTOList = new ArrayList<>();
public List<ConsumerGroup> getConsumerGroupList(Long clusterId) {
List<ConsumerGroup> consumerGroupList = new ArrayList<>();
for (OffsetLocationEnum location: OffsetLocationEnum.values()) {
Map<String, List<String>> consumerGroupAppIdMap = null;
Set<String> consumerGroupSet = null;
if (OffsetLocationEnum.ZOOKEEPER.equals(location)) {
// 获取ZK中的消费组
consumerGroupAppIdMap = ConsumerMetadataCache.getConsumerGroupAppIdListInZk(clusterId);
consumerGroupSet = ConsumerMetadataCache.getGroupInZkMap(clusterId);
} else if (OffsetLocationEnum.BROKER.equals(location)) {
// 获取Broker中的消费组
consumerGroupAppIdMap = ConsumerMetadataCache.getConsumerGroupAppIdListInBK(clusterId);
consumerGroupSet = ConsumerMetadataCache.getGroupInBrokerMap(clusterId);
}
if (consumerGroupSet == null || consumerGroupAppIdMap == null) {
if (ValidateUtils.isEmptySet(consumerGroupSet)) {
continue;
}
for (String consumerGroup : consumerGroupSet) {
consumerGroupDTOList.add(new ConsumerGroupDTO(
clusterId,
consumerGroup,
consumerGroupAppIdMap.getOrDefault(consumerGroup, new ArrayList<>()),
location)
); }
consumerGroupList.add(new ConsumerGroup(clusterId, consumerGroup, location));
}
}
return consumerGroupDTOList;
return consumerGroupList;
}
@Override
public List<ConsumerGroupDTO> getConsumerGroupList(Long clusterId, String topicName) {
List<ConsumerGroupDTO> consumerGroupDTOList = new ArrayList<>();
public List<ConsumerGroup> getConsumerGroupList(Long clusterId, String topicName) {
List<ConsumerGroup> consumerGroupList = new ArrayList<>();
for (OffsetLocationEnum location: OffsetLocationEnum.values()) {
Map<String, List<String>> consumerGroupAppIdMap = null;
Set<String> consumerGroupSet = null;
if (OffsetLocationEnum.ZOOKEEPER.equals(location)) {
// 获取ZK中的消费组
consumerGroupAppIdMap = ConsumerMetadataCache.getConsumerGroupAppIdListInZk(clusterId);
consumerGroupSet = ConsumerMetadataCache.getTopicConsumerGroupInZk(clusterId, topicName);
} else if (OffsetLocationEnum.BROKER.equals(location)) {
// 获取Broker中的消费组
consumerGroupAppIdMap = ConsumerMetadataCache.getConsumerGroupAppIdListInBK(clusterId);
consumerGroupSet = ConsumerMetadataCache.getTopicConsumerGroupInBroker(clusterId, topicName);
}
if (consumerGroupSet == null || consumerGroupAppIdMap == null) {
if (ValidateUtils.isEmptySet(consumerGroupSet)) {
continue;
}
for (String consumerGroup : consumerGroupSet) {
consumerGroupDTOList.add(new ConsumerGroupDTO(
clusterId,
consumerGroup,
consumerGroupAppIdMap.getOrDefault(consumerGroup, new ArrayList<>()),
location
)
);
consumerGroupList.add(new ConsumerGroup(clusterId, consumerGroup, location));
}
}
return consumerGroupDTOList;
return consumerGroupList;
}
@Override
public List<ConsumeDetailDTO> getConsumeDetail(ClusterDO clusterDO,
String topicName,
ConsumerGroupDTO consumeGroupDTO) {
public List<ConsumerGroupSummary> getConsumerGroupSummaries(Long clusterId, String topicName) {
List<ConsumerGroup> consumerGroupList = this.getConsumerGroupList(clusterId, topicName);
if (ValidateUtils.isEmptyList(consumerGroupList)) {
return Collections.emptyList();
}
List<ConsumerGroupSummary> summaryList = new ArrayList<>();
for (ConsumerGroup consumerGroup: consumerGroupList) {
ConsumerGroupSummary consumerGroupSummary = null;
if (OffsetLocationEnum.ZOOKEEPER.equals(consumerGroup.getOffsetStoreLocation())) {
consumerGroupSummary = new ConsumerGroupSummary();
consumerGroupSummary.setClusterId(consumerGroup.getClusterId());
consumerGroupSummary.setConsumerGroup(consumerGroup.getConsumerGroup());
consumerGroupSummary.setOffsetStoreLocation(consumerGroup.getOffsetStoreLocation());
} else {
consumerGroupSummary = getConsumerGroupSummary(clusterId, topicName, consumerGroup.getConsumerGroup());
}
summaryList.add(consumerGroupSummary);
}
return summaryList;
}
private ConsumerGroupSummary getConsumerGroupSummary(Long clusterId, String topicName, String consumerGroup) {
ConsumerGroupSummary summary = new ConsumerGroupSummary();
summary.setClusterId(clusterId);
summary.setConsumerGroup(consumerGroup);
summary.setOffsetStoreLocation(OffsetLocationEnum.BROKER);
summary.setAppIdList(new ArrayList<>());
summary.setState("");
try {
AdminClient adminClient = KafkaClientPool.getAdminClient(clusterId);
AdminClient.ConsumerGroupSummary consumerGroupSummary = adminClient.describeConsumerGroup(consumerGroup);
if (ValidateUtils.isNull(consumerGroupSummary)) {
return summary;
}
summary.setState(consumerGroupSummary.state());
java.util.Iterator<scala.collection.immutable.List<AdminClient.ConsumerSummary>> it = JavaConversions.asJavaIterator(consumerGroupSummary.consumers().iterator());
while (it.hasNext()) {
List<AdminClient.ConsumerSummary> consumerSummaryList = JavaConversions.asJavaList(it.next());
for (AdminClient.ConsumerSummary consumerSummary: consumerSummaryList) {
List<TopicPartition> topicPartitionList = JavaConversions.asJavaList(consumerSummary.assignment());
if (ValidateUtils.isEmptyList(topicPartitionList)) {
continue;
}
if (topicPartitionList.stream().anyMatch(elem -> elem.topic().equals(topicName)) && consumerSummary.clientId().contains(".")) {
String [] splitArray = consumerSummary.clientId().split("\\.");
summary.getAppIdList().add(splitArray[0]);
}
}
}
} catch (SchemaException e) {
logger.error("class=ConsumerServiceImpl||method=getConsumerGroupSummary||clusterId={}||topicName={}||consumerGroup={}||errMsg={}||schema exception",
clusterId, topicName, consumerGroup, e.getMessage());
} catch (Exception e) {
logger.error("class=ConsumerServiceImpl||method=getConsumerGroupSummary||clusterId={}||topicName={}||consumerGroup={}||errMsg={}||throws exception",
clusterId, topicName, consumerGroup, e.getMessage());
}
summary.setAppIdList(new ArrayList<>(new HashSet<>(summary.getAppIdList())));
return summary;
}
@Override
public List<ConsumeDetailDTO> getConsumeDetail(ClusterDO clusterDO, String topicName, ConsumerGroup consumerGroup) {
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterDO.getId(), topicName);
if (topicMetadata == null) {
logger.warn("class=ConsumerServiceImpl||method=getConsumeDetail||clusterId={}||topicName={}||msg=topicMetadata is null!",
@@ -116,10 +164,10 @@ public class ConsumerServiceImpl implements ConsumerService {
}
List<ConsumeDetailDTO> consumerGroupDetailDTOList = null;
if (OffsetLocationEnum.ZOOKEEPER.equals(consumeGroupDTO.getOffsetStoreLocation())) {
consumerGroupDetailDTOList = getConsumerPartitionStateInZK(clusterDO, topicMetadata, consumeGroupDTO);
} else if (OffsetLocationEnum.BROKER.equals(consumeGroupDTO.getOffsetStoreLocation())){
consumerGroupDetailDTOList = getConsumerPartitionStateInBroker(clusterDO, topicMetadata, consumeGroupDTO);
if (OffsetLocationEnum.ZOOKEEPER.equals(consumerGroup.getOffsetStoreLocation())) {
consumerGroupDetailDTOList = getConsumerPartitionStateInZK(clusterDO, topicMetadata, consumerGroup);
} else if (OffsetLocationEnum.BROKER.equals(consumerGroup.getOffsetStoreLocation())){
consumerGroupDetailDTOList = getConsumerPartitionStateInBroker(clusterDO, topicMetadata, consumerGroup);
}
if (consumerGroupDetailDTOList == null) {
logger.info("class=ConsumerServiceImpl||method=getConsumeDetail||msg=consumerGroupDetailDTOList is null!");
@@ -147,7 +195,7 @@ public class ConsumerServiceImpl implements ConsumerService {
}
@Override
public List<Result> resetConsumerOffset(ClusterDO clusterDO, String topicName, ConsumerGroupDTO consumerGroupDTO, List<PartitionOffsetDTO> partitionOffsetDTOList) {
public List<Result> resetConsumerOffset(ClusterDO clusterDO, String topicName, ConsumerGroup consumerGroup, List<PartitionOffsetDTO> partitionOffsetDTOList) {
Map<TopicPartition, Long> offsetMap = partitionOffsetDTOList.stream().collect(Collectors.toMap(elem -> {return new TopicPartition(topicName, elem.getPartitionId());}, PartitionOffsetDTO::getOffset));
List<Result> resultList = new ArrayList<>();
@@ -155,12 +203,12 @@ public class ConsumerServiceImpl implements ConsumerService {
KafkaConsumer<String, String> kafkaConsumer = null;
try {
Properties properties = KafkaClientPool.createProperties(clusterDO, false);
properties.setProperty("group.id", consumerGroupDTO.getConsumerGroup());
properties.setProperty("group.id", consumerGroup.getConsumerGroup());
kafkaConsumer = new KafkaConsumer<>(properties);
checkAndCorrectPartitionOffset(kafkaConsumer, offsetMap);
return resetConsumerOffset(clusterDO, kafkaConsumer, consumerGroupDTO, offsetMap);
return resetConsumerOffset(clusterDO, kafkaConsumer, consumerGroup, offsetMap);
} catch (Exception e) {
logger.error("create kafka consumer failed, clusterId:{} topicName:{} consumerGroup:{} partition:{}.", clusterDO.getId(), topicName, consumerGroupDTO, partitionOffsetDTOList, e);
logger.error("create kafka consumer failed, clusterId:{} topicName:{} consumerGroup:{} partition:{}.", clusterDO.getId(), topicName, consumerGroup, partitionOffsetDTOList, e);
resultList.add(new Result(
ResultStatus.OPERATION_FAILED.getCode(),
"reset failed, create KafkaConsumer or check offset failed"
@@ -173,20 +221,20 @@ public class ConsumerServiceImpl implements ConsumerService {
return resultList;
}
private List<Result> resetConsumerOffset(ClusterDO cluster, KafkaConsumer<String, String> kafkaConsumer, ConsumerGroupDTO consumerGroupDTO, Map<TopicPartition, Long> offsetMap) {
private List<Result> resetConsumerOffset(ClusterDO cluster, KafkaConsumer<String, String> kafkaConsumer, ConsumerGroup consumerGroup, Map<TopicPartition, Long> offsetMap) {
List<Result> resultList = new ArrayList<>();
for(Map.Entry<TopicPartition, Long> entry: offsetMap.entrySet()){
TopicPartition tp = entry.getKey();
Long offset = entry.getValue();
try {
if (consumerGroupDTO.getOffsetStoreLocation().equals(OffsetLocationEnum.ZOOKEEPER)) {
resetConsumerOffsetInZK(cluster, consumerGroupDTO.getConsumerGroup(), tp, offset);
} else if (consumerGroupDTO.getOffsetStoreLocation().equals(OffsetLocationEnum.BROKER)) {
if (consumerGroup.getOffsetStoreLocation().equals(OffsetLocationEnum.ZOOKEEPER)) {
resetConsumerOffsetInZK(cluster, consumerGroup.getConsumerGroup(), tp, offset);
} else if (consumerGroup.getOffsetStoreLocation().equals(OffsetLocationEnum.BROKER)) {
resetConsumerOffsetInBroker(kafkaConsumer, tp, offset);
}
} catch (Exception e) {
logger.error("reset failed, clusterId:{} consumerGroup:{} topic-partition:{}.", cluster.getId(), consumerGroupDTO, tp, e);
logger.error("reset failed, clusterId:{} consumerGroup:{} topic-partition:{}.", cluster.getId(), consumerGroup, tp, e);
resultList.add(new Result(
ResultStatus.OPERATION_FAILED.getCode(),
"reset failed..."));
@@ -232,14 +280,14 @@ public class ConsumerServiceImpl implements ConsumerService {
@Override
public Map<Integer, Long> getConsumerOffset(ClusterDO clusterDO,
String topicName,
ConsumerGroupDTO consumerGroupDTO) {
if (ValidateUtils.isNull(clusterDO) || ValidateUtils.isBlank(topicName) || ValidateUtils.isNull(consumerGroupDTO)) {
ConsumerGroup consumerGroup) {
if (ValidateUtils.isNull(clusterDO) || ValidateUtils.isBlank(topicName) || ValidateUtils.isNull(consumerGroup)) {
return null;
}
if (OffsetLocationEnum.BROKER.equals(consumerGroupDTO.getOffsetStoreLocation())) {
return getConsumerOffsetFromBK(clusterDO, topicName, consumerGroupDTO.getConsumerGroup());
} else if (OffsetLocationEnum.ZOOKEEPER.equals(consumerGroupDTO.getOffsetStoreLocation())) {
return getConsumerOffsetFromZK(clusterDO.getId(), topicName, consumerGroupDTO.getConsumerGroup());
if (OffsetLocationEnum.BROKER.equals(consumerGroup.getOffsetStoreLocation())) {
return getConsumerOffsetFromBK(clusterDO, topicName, consumerGroup.getConsumerGroup());
} else if (OffsetLocationEnum.ZOOKEEPER.equals(consumerGroup.getOffsetStoreLocation())) {
return getConsumerOffsetFromZK(clusterDO.getId(), topicName, consumerGroup.getConsumerGroup());
}
return null;
}
@@ -306,9 +354,9 @@ public class ConsumerServiceImpl implements ConsumerService {
return consumerIdMap;
}
private List<ConsumeDetailDTO> getConsumerPartitionStateInBroker(ClusterDO clusterDO, TopicMetadata topicMetadata, ConsumerGroupDTO consumerGroupDTO) {
Map<Integer, String> consumerIdMap = getConsumeIdMap(clusterDO.getId(), topicMetadata.getTopic(), consumerGroupDTO.getConsumerGroup());
Map<Integer, String> consumeOffsetMap = getOffsetByGroupAndTopicFromBroker(clusterDO, consumerGroupDTO.getConsumerGroup(), topicMetadata.getTopic());
private List<ConsumeDetailDTO> getConsumerPartitionStateInBroker(ClusterDO clusterDO, TopicMetadata topicMetadata, ConsumerGroup consumerGroup) {
Map<Integer, String> consumerIdMap = getConsumeIdMap(clusterDO.getId(), topicMetadata.getTopic(), consumerGroup.getConsumerGroup());
Map<Integer, String> consumeOffsetMap = getOffsetByGroupAndTopicFromBroker(clusterDO, consumerGroup.getConsumerGroup(), topicMetadata.getTopic());
List<ConsumeDetailDTO> consumeDetailDTOList = new ArrayList<>();
for (int partitionId : topicMetadata.getPartitionMap().getPartitions().keySet()) {
@@ -318,7 +366,7 @@ public class ConsumerServiceImpl implements ConsumerService {
try {
consumeDetailDTO.setConsumeOffset(StringUtils.isEmpty(consumeOffsetStr)? null: Long.valueOf(consumeOffsetStr));
} catch (Exception e) {
logger.error("illegal consumer offset, clusterId:{} topicName:{} consumerGroup:{} offset:{}.", clusterDO.getId(), topicMetadata.getTopic(), consumerGroupDTO.getConsumerGroup(), consumeOffsetStr, e);
logger.error("illegal consumer offset, clusterId:{} topicName:{} consumerGroup:{} offset:{}.", clusterDO.getId(), topicMetadata.getTopic(), consumerGroup.getConsumerGroup(), consumeOffsetStr, e);
}
consumeDetailDTO.setConsumerId(consumerIdMap.get(partitionId));
consumeDetailDTOList.add(consumeDetailDTO);
@@ -326,21 +374,19 @@ public class ConsumerServiceImpl implements ConsumerService {
return consumeDetailDTOList;
}
private List<ConsumeDetailDTO> getConsumerPartitionStateInZK(ClusterDO clusterDO,
TopicMetadata topicMetadata,
ConsumerGroupDTO consumerGroupDTO) {
private List<ConsumeDetailDTO> getConsumerPartitionStateInZK(ClusterDO clusterDO, TopicMetadata topicMetadata, ConsumerGroup consumerGroup) {
ZkConfigImpl zkConfig = PhysicalClusterMetadataManager.getZKConfig(clusterDO.getId());
List<ConsumeDetailDTO> consumeDetailDTOList = new ArrayList<>();
for (Integer partitionId : topicMetadata.getPartitionMap().getPartitions().keySet()) {
String consumeGroupPath = ZkPathUtil.getConsumerGroupOffsetTopicPartitionNode(consumerGroupDTO.getConsumerGroup(), topicMetadata.getTopic(), partitionId);
String consumeGroupPath = ZkPathUtil.getConsumerGroupOffsetTopicPartitionNode(consumerGroup.getConsumerGroup(), topicMetadata.getTopic(), partitionId);
String consumeOffset = null;
try {
consumeOffset = zkConfig.get(consumeGroupPath);
} catch (ConfigException e) {
logger.error("get consumeOffset error for zk path:{}", consumeGroupPath, e);
}
String consumeIdZkPath = ZkPathUtil.getConsumerGroupOwnersTopicPartitionNode(consumerGroupDTO.getConsumerGroup(), topicMetadata.getTopic(), partitionId);
String consumeIdZkPath = ZkPathUtil.getConsumerGroupOwnersTopicPartitionNode(consumerGroup.getConsumerGroup(), topicMetadata.getTopic(), partitionId);
String consumerId = null;
try {
consumerId = zkConfig.get(consumeIdZkPath);
@@ -394,7 +440,7 @@ public class ConsumerServiceImpl implements ConsumerService {
@Override
public boolean checkConsumerGroupExist(OffsetLocationEnum offsetLocation, Long clusterId, String topicName, String consumerGroup) {
List<ConsumerGroupDTO> consumerGroupList = getConsumerGroupList(clusterId, topicName).stream()
List<ConsumerGroup> consumerGroupList = getConsumerGroupList(clusterId, topicName).stream()
.filter(group -> offsetLocation.location.equals(group.getOffsetStoreLocation().location) && consumerGroup.equals(group.getConsumerGroup()))
.collect(Collectors.toList());
return !ValidateUtils.isEmptyList(consumerGroupList);

View File

@@ -3,6 +3,7 @@ package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.common.bizenum.KafkaClientEnum;
import com.xiaojukeji.kafka.manager.common.bizenum.TopicAuthorityEnum;
import com.xiaojukeji.kafka.manager.common.constant.KafkaMetricsCollections;
import com.xiaojukeji.kafka.manager.common.constant.TopicCreationConstant;
import com.xiaojukeji.kafka.manager.common.entity.Result;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.RdTopicBasic;
@@ -14,6 +15,7 @@ import com.xiaojukeji.kafka.manager.common.entity.metrics.TopicMetrics;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.AppDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.AuthorityDO;
import com.xiaojukeji.kafka.manager.common.utils.DateUtils;
import com.xiaojukeji.kafka.manager.common.utils.JsonUtils;
import com.xiaojukeji.kafka.manager.common.utils.NumberUtils;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata;
@@ -33,6 +35,7 @@ import com.xiaojukeji.kafka.manager.service.utils.KafkaZookeeperUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.dao.DuplicateKeyException;
import org.springframework.stereotype.Service;
import org.springframework.util.StringUtils;
@@ -345,6 +348,47 @@ public class TopicManagerServiceImpl implements TopicManagerService {
return ResultStatus.MYSQL_ERROR;
}
@Override
public ResultStatus modifyTopicByOp(Long clusterId, String topicName, String appId, String description, String operator) {
try {
if (!PhysicalClusterMetadataManager.isTopicExist(clusterId, topicName)) {
return ResultStatus.TOPIC_NOT_EXIST;
}
AppDO appDO = appService.getByAppId(appId);
if (ValidateUtils.isNull(appDO)) {
return ResultStatus.APP_NOT_EXIST;
}
TopicDO topicDO = topicDao.getByTopicName(clusterId, topicName);
if (ValidateUtils.isNull(topicDO)) {
// 不存在, 则需要插入
topicDO = new TopicDO();
topicDO.setAppId(appId);
topicDO.setClusterId(clusterId);
topicDO.setTopicName(topicName);
topicDO.setPeakBytesIn(TopicCreationConstant.DEFAULT_QUOTA);
topicDO.setDescription(description);
this.addTopic(topicDO);
} else {
// 存在, 则直接更新
topicDO.setAppId(appId);
topicDO.setDescription(description);
topicDao.updateByName(topicDO);
}
AuthorityDO authorityDO = new AuthorityDO();
authorityDO.setAppId(appId);
authorityDO.setClusterId(clusterId);
authorityDO.setTopicName(topicName);
authorityDO.setAccess(TopicAuthorityEnum.READ_WRITE.getCode());
authorityService.addAuthority(authorityDO);
} catch (Exception e) {
LOGGER.error("modify topic failed, clusterId:{} topicName:{} description:{} operator:{} ",
clusterId, topicName, description, operator, e);
}
return ResultStatus.MYSQL_ERROR;
}
@Override
public int deleteByTopicName(Long clusterId, String topicName) {
try {
@@ -359,6 +403,9 @@ public class TopicManagerServiceImpl implements TopicManagerService {
public int addTopic(TopicDO topicDO) {
try {
return topicDao.insert(topicDO);
} catch (DuplicateKeyException duplicateKeyException) {
// 主建重复了, 非重要问题
LOGGER.debug("class=TopicManagerServiceImpl||method=addTopic||data={}||msg=exist duplicate topic", JsonUtils.toJSONString(topicDO));
} catch (Exception e) {
LOGGER.error("insert topic failed, TopicDO:{}", topicDO.toString(), e);
}

View File

@@ -29,6 +29,7 @@ import com.xiaojukeji.kafka.manager.service.cache.LogicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.service.*;
import com.xiaojukeji.kafka.manager.service.service.gateway.AppService;
import com.xiaojukeji.kafka.manager.service.strategy.AbstractHealthScoreStrategy;
import com.xiaojukeji.kafka.manager.service.utils.KafkaZookeeperUtils;
import com.xiaojukeji.kafka.manager.service.utils.MetricsConvertUtils;
import org.apache.kafka.clients.consumer.ConsumerRecord;
@@ -83,6 +84,9 @@ public class TopicServiceImpl implements TopicService {
@Autowired
private RegionService regionService;
@Autowired
private AbstractHealthScoreStrategy healthScoreStrategy;
@Override
public List<TopicMetricsDO> getTopicMetricsFromDB(Long clusterId, String topicName, Date startTime, Date endTime) {
try {
@@ -235,7 +239,7 @@ public class TopicServiceImpl implements TopicService {
basicDTO.setRegionNameList(regionDOList.stream().map(RegionDO::getName).collect(Collectors.toList()));
basicDTO.setTopicCodeC(jmxService.getTopicCodeCValue(clusterId, topicName));
basicDTO.setScore(100);
basicDTO.setScore(healthScoreStrategy.calTopicHealthScore(clusterId, topicName));
return basicDTO;
}

View File

@@ -2,8 +2,10 @@ package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.common.entity.Result;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.utils.ListUtils;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.zookeeper.ZkConfigImpl;
import com.xiaojukeji.kafka.manager.common.zookeeper.ZkPathUtil;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.didi.TopicJmxSwitch;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
@@ -13,6 +15,9 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.stereotype.Service;
import java.util.ArrayList;
import java.util.List;
/**
* @author zengqiao
* @date 20/8/27
@@ -40,4 +45,29 @@ public class ZookeeperServiceImpl implements ZookeeperService {
}
return new Result();
}
@Override
public Result<List<Integer>> getControllerPreferredCandidates(Long clusterId) {
if (ValidateUtils.isNull(clusterId)) {
return Result.buildFrom(ResultStatus.PARAM_ILLEGAL);
}
ZkConfigImpl zkConfig = PhysicalClusterMetadataManager.getZKConfig(clusterId);
if (ValidateUtils.isNull(zkConfig)) {
return Result.buildFrom(ResultStatus.CONNECT_ZOOKEEPER_FAILED);
}
try {
if (!zkConfig.checkPathExists(ZkPathUtil.D_CONTROLLER_CANDIDATES)) {
return Result.buildSuc(new ArrayList<>());
}
List<String> brokerIdList = zkConfig.getChildren(ZkPathUtil.D_CONTROLLER_CANDIDATES);
if (ValidateUtils.isEmptyList(brokerIdList)) {
return Result.buildSuc(new ArrayList<>());
}
return Result.buildSuc(ListUtils.string2IntList(ListUtils.strList2String(brokerIdList)));
} catch (Exception e) {
LOGGER.error("class=ZookeeperServiceImpl||method=getControllerPreferredCandidates||clusterId={}||errMsg={}", clusterId, e.getMessage());
}
return Result.buildFrom(ResultStatus.READ_ZOOKEEPER_FAILED);
}
}

View File

@@ -72,8 +72,8 @@ public class DidiHealthScoreStrategy extends AbstractHealthScoreStrategy {
// 数据获取失败
return Constant.INVALID_CODE;
}
if (((Double) failedFetchRequestsPerSecOneMinuteRate) > 0
|| ((Double) failedProduceRequestsPerSecOneMinuteRate) > 0) {
if (((Double) failedFetchRequestsPerSecOneMinuteRate) > 0.01
|| ((Double) failedProduceRequestsPerSecOneMinuteRate) > 0.01) {
return HEALTH_SCORE_VERY_BAD;
}

View File

@@ -5,6 +5,8 @@ import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata
import com.xiaojukeji.kafka.manager.common.zookeeper.StateChangeListener;
import com.xiaojukeji.kafka.manager.common.zookeeper.ZkConfigImpl;
import com.xiaojukeji.kafka.manager.common.zookeeper.ZkPathUtil;
import com.xiaojukeji.kafka.manager.dao.TopicDao;
import com.xiaojukeji.kafka.manager.dao.gateway.AuthorityDao;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.cache.ThreadPool;
import org.apache.zookeeper.data.Stat;
@@ -28,11 +30,22 @@ public class TopicStateListener implements StateChangeListener {
private ZkConfigImpl zkConfig;
private TopicDao topicDao;
private AuthorityDao authorityDao;
public TopicStateListener(Long clusterId, ZkConfigImpl zkConfig) {
this.clusterId = clusterId;
this.zkConfig = zkConfig;
}
public TopicStateListener(Long clusterId, ZkConfigImpl zkConfig, TopicDao topicDao, AuthorityDao authorityDao) {
this.clusterId = clusterId;
this.zkConfig = zkConfig;
this.topicDao = topicDao;
this.authorityDao = authorityDao;
}
@Override
public void init() {
try {
@@ -79,6 +92,8 @@ public class TopicStateListener implements StateChangeListener {
private void processTopicDelete(String topicName) {
LOGGER.warn("delete topic, clusterId:{} topicName:{}.", clusterId, topicName);
PhysicalClusterMetadataManager.removeTopicMetadata(clusterId, topicName);
topicDao.removeTopicInCache(clusterId, topicName);
authorityDao.removeAuthorityInCache(clusterId, topicName);
}
private void processTopicAdded(String topicName) {