kafka-manager 2.0

This commit is contained in:
zengqiao
2020-09-28 15:46:34 +08:00
parent 28d985aaf1
commit c6e4b60424
1253 changed files with 82183 additions and 37179 deletions

View File

@@ -0,0 +1,111 @@
package com.xiaojukeji.kafka.manager.service.cache;
import com.xiaojukeji.kafka.manager.common.bizenum.OffsetLocationEnum;
import com.xiaojukeji.kafka.manager.common.entity.ConsumerMetadata;
import kafka.admin.AdminClient;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
/**
* 定时任务, 缓存消费组相关信息
* @author zengqiao
* @date 2019/05/01
*/
public class ConsumerMetadataCache {
private static final Map<Long, ConsumerMetadata> CG_METADATA_IN_ZK_MAP = new ConcurrentHashMap<>();
private static final Map<Long, ConsumerMetadata> CG_METADATA_IN_BK_MAP = new ConcurrentHashMap<>();
public static void putConsumerMetadataInZK(Long clusterId, ConsumerMetadata consumerMetadata) {
if (clusterId == null || consumerMetadata == null) {
return;
}
CG_METADATA_IN_ZK_MAP.put(clusterId, consumerMetadata);
}
public static void putConsumerMetadataInBK(Long clusterId, ConsumerMetadata consumerMetadata) {
if (clusterId == null || consumerMetadata == null) {
return;
}
CG_METADATA_IN_BK_MAP.put(clusterId, consumerMetadata);
}
public static Set<String> getGroupInZkMap(Long clusterId) {
ConsumerMetadata consumerMetadata = CG_METADATA_IN_ZK_MAP.get(clusterId);
if (consumerMetadata == null) {
return new HashSet<>();
}
return consumerMetadata.getConsumerGroupSet();
}
public static Set<String> getGroupInBrokerMap(Long clusterId) {
ConsumerMetadata consumerMetadata = CG_METADATA_IN_BK_MAP.get(clusterId);
if (consumerMetadata == null) {
return new HashSet<>();
}
return consumerMetadata.getConsumerGroupSet();
}
public static AdminClient.ConsumerGroupSummary getConsumerGroupSummary(Long clusterId, String consumerGroup) {
ConsumerMetadata consumerMetadata = CG_METADATA_IN_BK_MAP.get(clusterId);
if (consumerMetadata == null) {
return null;
}
return consumerMetadata.getConsumerGroupSummaryMap().get(consumerGroup);
}
public static List<String> getConsumerGroupConsumedTopicList(Long clusterId,
String consumerGroup,
String location) {
ConsumerMetadata consumerMetadata = null;
if(OffsetLocationEnum.ZOOKEEPER.location.equals(location)){
consumerMetadata = CG_METADATA_IN_ZK_MAP.get(clusterId);
} else if (OffsetLocationEnum.BROKER.location.equals(location)) {
consumerMetadata = CG_METADATA_IN_BK_MAP.get(clusterId);
}
if (consumerMetadata == null) {
return new ArrayList<>();
}
List<String> topicNameList = new ArrayList<>();
for(Map.Entry<String, Set<String>> entry: consumerMetadata.getTopicNameConsumerGroupMap().entrySet()){
if(entry.getValue().contains(consumerGroup)){
topicNameList.add(entry.getKey());
}
}
return topicNameList;
}
public static Set<String> getTopicConsumerGroupInZk(Long clusterId, String topicName) {
ConsumerMetadata consumerMetadata = CG_METADATA_IN_ZK_MAP.get(clusterId);
if(consumerMetadata == null){
return new HashSet<>();
}
return consumerMetadata.getTopicNameConsumerGroupMap().getOrDefault(topicName, new HashSet<>());
}
public static Set<String> getTopicConsumerGroupInBroker(Long clusterId, String topicName) {
ConsumerMetadata consumerMetadata = CG_METADATA_IN_BK_MAP.get(clusterId);
if(consumerMetadata == null){
return new HashSet<>();
}
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

@@ -0,0 +1,186 @@
package com.xiaojukeji.kafka.manager.service.cache;
import com.alibaba.fastjson.JSONObject;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterDO;
import com.xiaojukeji.kafka.manager.common.utils.factory.KafkaConsumerFactory;
import kafka.admin.AdminClient;
import org.apache.commons.pool2.impl.GenericObjectPool;
import org.apache.commons.pool2.impl.GenericObjectPoolConfig;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.locks.ReentrantLock;
/**
* Cache Kafka客户端
* @author zengqiao
* @date 19/12/24
*/
public class KafkaClientPool {
private final static Logger LOGGER = LoggerFactory.getLogger(KafkaClientPool.class);
/**
* AdminClient
*/
private static Map<Long, AdminClient> AdminClientMap = new ConcurrentHashMap<>();
private static Map<Long, KafkaProducer<String, String>> KAFKA_PRODUCER_MAP = new ConcurrentHashMap<>();
private static Map<Long, GenericObjectPool<KafkaConsumer>> KAFKA_CONSUMER_POOL = new ConcurrentHashMap<>();
private static ReentrantLock lock = new ReentrantLock();
private static void initKafkaProducerMap(Long clusterId) {
ClusterDO clusterDO = PhysicalClusterMetadataManager.getClusterFromCache(clusterId);
if (clusterDO == null) {
return;
}
lock.lock();
try {
KafkaProducer<String, String> kafkaProducer = KAFKA_PRODUCER_MAP.get(clusterId);
if (!ValidateUtils.isNull(kafkaProducer)) {
return;
}
Properties properties = createProperties(clusterDO, true);
properties.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, "lz4");
properties.setProperty(ProducerConfig.LINGER_MS_CONFIG, "10");
properties.setProperty(ProducerConfig.RETRIES_CONFIG, "3");
KAFKA_PRODUCER_MAP.put(clusterId, new KafkaProducer<String, String>(properties));
} catch (Exception e) {
LOGGER.error("create kafka producer failed, clusterDO:{}.", clusterDO, e);
} finally {
lock.unlock();
}
}
public static boolean produceData2Kafka(Long clusterId, String topicName, String data) {
if (!PhysicalClusterMetadataManager.isTopicExist(clusterId, topicName)) {
// Topic不存在
return false;
}
KafkaProducer<String, String> kafkaProducer = KAFKA_PRODUCER_MAP.get(clusterId);
if (ValidateUtils.isNull(kafkaProducer)) {
initKafkaProducerMap(clusterId);
kafkaProducer = KAFKA_PRODUCER_MAP.get(clusterId);
}
if (ValidateUtils.isNull(kafkaProducer)) {
return false;
}
kafkaProducer.send(new ProducerRecord<String, String>(topicName, data));
return true;
}
private static void initKafkaConsumerPool(ClusterDO clusterDO) {
lock.lock();
try {
GenericObjectPool<KafkaConsumer> objectPool = KAFKA_CONSUMER_POOL.get(clusterDO.getId());
if (objectPool != null) {
return;
}
GenericObjectPoolConfig config = new GenericObjectPoolConfig();
config.setMaxIdle(24);
config.setMinIdle(24);
config.setMaxTotal(24);
KAFKA_CONSUMER_POOL.put(
clusterDO.getId(),
new GenericObjectPool<KafkaConsumer>(new KafkaConsumerFactory(clusterDO), config)
);
} catch (Exception e) {
LOGGER.error("create kafka consumer pool failed, clusterDO:{}.", clusterDO, e);
} finally {
lock.unlock();
}
}
public static KafkaConsumer borrowKafkaConsumerClient(ClusterDO clusterDO) {
if (ValidateUtils.isNull(clusterDO)) {
return null;
}
GenericObjectPool<KafkaConsumer> objectPool = KAFKA_CONSUMER_POOL.get(clusterDO.getId());
if (ValidateUtils.isNull(objectPool)) {
initKafkaConsumerPool(clusterDO);
objectPool = KAFKA_CONSUMER_POOL.get(clusterDO.getId());
}
if (ValidateUtils.isNull(objectPool)) {
return null;
}
try {
return objectPool.borrowObject(3000);
} catch (Exception e) {
LOGGER.error("borrow kafka consumer client failed, clusterDO:{}.", clusterDO, e);
}
return null;
}
public static void returnKafkaConsumerClient(Long physicalClusterId, KafkaConsumer kafkaConsumer) {
if (ValidateUtils.isNull(physicalClusterId) || ValidateUtils.isNull(kafkaConsumer)) {
return;
}
GenericObjectPool<KafkaConsumer> objectPool = KAFKA_CONSUMER_POOL.get(physicalClusterId);
if (ValidateUtils.isNull(objectPool)) {
return;
}
objectPool.returnObject(kafkaConsumer);
}
public static AdminClient getAdminClient(Long clusterId) {
AdminClient adminClient = AdminClientMap.get(clusterId);
if (adminClient != null) {
return adminClient;
}
ClusterDO clusterDO = PhysicalClusterMetadataManager.getClusterFromCache(clusterId);
if (clusterDO == null) {
return null;
}
Properties properties = createProperties(clusterDO, false);
lock.lock();
try {
adminClient = AdminClientMap.get(clusterId);
if (adminClient != null) {
return adminClient;
}
AdminClientMap.put(clusterId, AdminClient.create(properties));
} catch (Exception e) {
LOGGER.error("create kafka admin client failed, clusterId:{}.", clusterId, e);
} finally {
lock.unlock();
}
return AdminClientMap.get(clusterId);
}
public static void closeAdminClient(ClusterDO cluster) {
if (AdminClientMap.containsKey(cluster.getId())) {
AdminClientMap.get(cluster.getId()).close();
}
}
public static Properties createProperties(ClusterDO clusterDO, Boolean serialize) {
Properties properties = new Properties();
properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, clusterDO.getBootstrapServers());
if (serialize) {
properties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer");
properties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer");
} else {
properties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer");
properties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer");
}
if (ValidateUtils.isBlank(clusterDO.getSecurityProperties())) {
return properties;
}
Properties securityProperties = JSONObject.parseObject(clusterDO.getSecurityProperties(), Properties.class);
properties.putAll(securityProperties);
return properties;
}
}

View File

@@ -0,0 +1,45 @@
package com.xiaojukeji.kafka.manager.service.cache;
import com.xiaojukeji.kafka.manager.common.entity.metrics.TopicMetrics;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
/**
* 缓存Metrics数据
* @author zengqiao
* @date 2019-04-30
*/
public class KafkaMetricsCache {
/**
* <clusterId, Metrics List>
*/
private static Map<Long, Map<String, TopicMetrics>> TopicMetricsMap = new ConcurrentHashMap<>();
public static void putTopicMetricsToCache(Long clusterId, List<TopicMetrics> dataList) {
if (clusterId == null || dataList == null) {
return;
}
Map<String, TopicMetrics> subMetricsMap = new HashMap<>(dataList.size());
for (TopicMetrics topicMetrics : dataList) {
subMetricsMap.put(topicMetrics.getTopicName(), topicMetrics);
}
TopicMetricsMap.put(clusterId, subMetricsMap);
}
public static Map<String, TopicMetrics> getTopicMetricsFromCache(Long clusterId) {
return TopicMetricsMap.getOrDefault(clusterId, Collections.emptyMap());
}
public static Map<Long, Map<String, TopicMetrics>> getAllTopicMetricsFromCache() {
return TopicMetricsMap;
}
public static TopicMetrics getTopicMetricsFromCache(Long clusterId, String topicName) {
if (clusterId == null || topicName == null) {
return null;
}
Map<String, TopicMetrics> subMap = TopicMetricsMap.getOrDefault(clusterId, Collections.emptyMap());
return subMap.get(topicName);
}
}

View File

@@ -0,0 +1,196 @@
package com.xiaojukeji.kafka.manager.service.cache;
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;
import com.xiaojukeji.kafka.manager.service.service.LogicalClusterService;
import com.xiaojukeji.kafka.manager.service.service.RegionService;
import com.xiaojukeji.kafka.manager.common.utils.ListUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.scheduling.annotation.Scheduled;
import org.springframework.stereotype.Service;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* 逻辑集群元信息
* @author zengqiao
* @date 20/5/11
*/
@Service
public class LogicalClusterMetadataManager {
private static final Logger LOGGER = LoggerFactory.getLogger(LogicalClusterMetadataManager.class);
@Autowired
private RegionService regionService;
@Autowired
private LogicalClusterService logicalClusterService;
private static final AtomicBoolean LOADED = new AtomicBoolean(false);
/**
* <逻辑集群ID, Topic集合>
*/
private static final Map<Long, Set<String>> LOGICAL_CLUSTER_ID_TOPIC_NAME_MAP = new ConcurrentHashMap<>();
/**
* <逻辑集群ID, BrokerId集合>
*/
private static final Map<Long, Set<Integer>> LOGICAL_CLUSTER_ID_BROKER_ID_MAP = new ConcurrentHashMap<>();
/**
* <逻辑集群ID, 逻辑集群DO>
*/
private static final Map<Long, LogicalClusterDO> LOGICAL_CLUSTER_MAP = new ConcurrentHashMap<>();
/**
* <物理集群ID, <Topic名称, 逻辑集群ID>>
*/
private static final Map<Long, Map<String, Long>> TOPIC_LOGICAL_MAP = new ConcurrentHashMap<>();
public Set<String> getTopicNameSet(Long logicClusterId) {
if (!LOADED.get()) {
flush();
}
return LOGICAL_CLUSTER_ID_TOPIC_NAME_MAP.getOrDefault(logicClusterId, new HashSet<>());
}
public Set<Integer> getBrokerIdSet(Long logicClusterId) {
if (!LOADED.get()) {
flush();
}
return LOGICAL_CLUSTER_ID_BROKER_ID_MAP.getOrDefault(logicClusterId, new HashSet<>());
}
public LogicalClusterDO getTopicLogicalCluster(Long physicalClusterId, String topicName) {
if (!LOADED.get()) {
flush();
}
Map<String, Long> logicalClusterIdMap = TOPIC_LOGICAL_MAP.get(physicalClusterId);
if (ValidateUtils.isNull(logicalClusterIdMap)) {
return null;
}
Long logicalClusterId = logicalClusterIdMap.get(topicName);
if (ValidateUtils.isNull(logicalClusterId)) {
return null;
}
return LOGICAL_CLUSTER_MAP.get(logicalClusterId);
}
public LogicalClusterDO getLogicalCluster(Long logicalClusterId) {
if (!LOADED.get()) {
flush();
}
return LOGICAL_CLUSTER_MAP.get(logicalClusterId);
}
public LogicalClusterDO getLogicalCluster(Long logicalClusterId, Boolean isPhysicalClusterId) {
if (isPhysicalClusterId != null && isPhysicalClusterId) {
return null;
}
return getLogicalCluster(logicalClusterId);
}
public List<LogicalClusterDO> getLogicalClusterList() {
if (!LOADED.get()) {
flush();
}
return new ArrayList<>(LOGICAL_CLUSTER_MAP.values());
}
public Long getPhysicalClusterId(Long logicalClusterId) {
if (ValidateUtils.isNull(logicalClusterId)) {
return null;
}
if (!LOADED.get()) {
flush();
}
LogicalClusterDO logicalClusterDO = LOGICAL_CLUSTER_MAP.get(logicalClusterId);
if (ValidateUtils.isNull(logicalClusterDO)) {
return null;
}
return logicalClusterDO.getClusterId();
}
public Long getPhysicalClusterId(Long clusterId, Boolean isPhysicalClusterId) {
if (isPhysicalClusterId != null && isPhysicalClusterId) {
return clusterId;
}
if (ValidateUtils.isNull(clusterId)) {
return null;
}
if (!LOADED.get()) {
flush();
}
LogicalClusterDO logicalClusterDO = LOGICAL_CLUSTER_MAP.get(clusterId);
if (ValidateUtils.isNull(logicalClusterDO)) {
return null;
}
return logicalClusterDO.getClusterId();
}
@Scheduled(cron="0/30 * * * * ?")
public void flush() {
List<LogicalClusterDO> logicalClusterDOList = logicalClusterService.listAll();
if (ValidateUtils.isEmptyList(logicalClusterDOList)) {
return;
}
Map<Long, RegionDO> regionMap = new HashMap<>();
List<RegionDO> regionDOList = regionService.listAll();
if (ValidateUtils.isNull(regionDOList)) {
regionDOList = new ArrayList<>();
}
for (RegionDO regionDO: regionDOList) {
regionMap.put(regionDO.getId(), regionDO);
}
for (LogicalClusterDO logicalClusterDO: logicalClusterDOList) {
try {
LOGICAL_CLUSTER_MAP.put(logicalClusterDO.getId(), logicalClusterDO);
flush(logicalClusterDO, regionMap);
} catch (Exception e) {
LOGGER.error("flush logical cluster metadata failed, logicalCluster:{}.", logicalClusterDO, e);
}
}
LOADED.set(true);
}
private void flush(LogicalClusterDO logicalClusterDO, Map<Long, RegionDO> regionMap) {
Set<Integer> brokerIdSet = new HashSet<>();
// 计算逻辑集群到brokerId集合的映射
List<Long> regionIdList = ListUtils.string2LongList(logicalClusterDO.getRegionList());
for (Long regionId: regionIdList) {
RegionDO regionDO = regionMap.get(regionId);
if (ValidateUtils.isNull(regionDO) || !logicalClusterDO.getClusterId().equals(regionDO.getClusterId())) {
LOGGER.warn("flush logical cluster metadata failed, exist illegal region, logicalCluster:{} region:{}.",
logicalClusterDO, regionId);
continue;
}
brokerIdSet.addAll(ListUtils.string2IntList(regionDO.getBrokerList()));
}
LOGICAL_CLUSTER_ID_BROKER_ID_MAP.put(logicalClusterDO.getId(), brokerIdSet);
// 计算逻辑集群到Topic名称的映射
Set<String> topicNameSet = PhysicalClusterMetadataManager.getBrokerTopicNum(
logicalClusterDO.getClusterId(),
brokerIdSet);
LOGICAL_CLUSTER_ID_TOPIC_NAME_MAP.put(logicalClusterDO.getId(), topicNameSet);
// 计算Topic名称到逻辑集群的映射
Map<String, Long> subMap =
TOPIC_LOGICAL_MAP.getOrDefault(logicalClusterDO.getClusterId(), new ConcurrentHashMap<>());
for (String topicName: topicNameSet) {
subMap.put(topicName, logicalClusterDO.getId());
}
TOPIC_LOGICAL_MAP.put(logicalClusterDO.getClusterId(), subMap);
}
}

View File

@@ -0,0 +1,494 @@
package com.xiaojukeji.kafka.manager.service.cache;
import com.xiaojukeji.kafka.manager.common.bizenum.KafkaBrokerRoleEnum;
import com.xiaojukeji.kafka.manager.common.constant.Constant;
import com.xiaojukeji.kafka.manager.common.constant.KafkaConstant;
import com.xiaojukeji.kafka.manager.common.entity.KafkaVersion;
import com.xiaojukeji.kafka.manager.common.utils.ListUtils;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterDO;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.BrokerMetadata;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.ControllerData;
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.service.service.JmxService;
import com.xiaojukeji.kafka.manager.service.zookeeper.*;
import com.xiaojukeji.kafka.manager.service.service.ClusterService;
import com.xiaojukeji.kafka.manager.common.zookeeper.ZkPathUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import javax.annotation.PostConstruct;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
/**
* 物理集群元信息
* @author zengqiao
* @date 19/4/3
*/
@Service
public class PhysicalClusterMetadataManager {
private final static Logger LOGGER = LoggerFactory.getLogger(PhysicalClusterMetadataManager.class);
@Autowired
private ControllerDao controllerDao;
@Autowired
private JmxService jmxService;
@Autowired
private ClusterService clusterService;
private final static Map<Long, ClusterDO> CLUSTER_MAP = new ConcurrentHashMap<>();
private final static Map<Long, ControllerData> CONTROLLER_DATA_MAP = new ConcurrentHashMap<>();
private final static Map<Long, ZkConfigImpl> ZK_CONFIG_MAP = new ConcurrentHashMap<>();
private final static Map<Long, Map<String, TopicMetadata>> TOPIC_METADATA_MAP = new ConcurrentHashMap<>();
private final static Map<Long, Map<String, Long>> TOPIC_RETENTION_TIME_MAP = new ConcurrentHashMap<>();
private final static Map<Long, Map<Integer, BrokerMetadata>> BROKER_METADATA_MAP = new ConcurrentHashMap<>();
/**
* JXM连接, 延迟连接
*/
private final static Map<Long, Map<Integer, JmxConnectorWrap>> JMX_CONNECTOR_MAP = new ConcurrentHashMap<>();
/**
* KafkaBroker版本, 延迟获取
*/
private static final Map<Long, Map<Integer, KafkaVersion>> KAFKA_VERSION_MAP = new ConcurrentHashMap<>();
@PostConstruct
public void init() {
LOGGER.info("cluster metadata initialization start.");
List<ClusterDO> doList = clusterService.list();
for (ClusterDO elem : doList) {
LOGGER.info("cluster metadata initializing, clusterId:{}.", elem.getId());
addNew(elem);
}
LOGGER.info("cluster metadata initialization finished.");
}
public synchronized void addNew(ClusterDO clusterDO) {
try {
if (ZK_CONFIG_MAP.containsKey(clusterDO.getId())) {
return;
}
ZkConfigImpl zkConfig = new ZkConfigImpl(clusterDO.getZookeeper());
//增加Broker监控
BROKER_METADATA_MAP.put(clusterDO.getId(), new ConcurrentHashMap<>());
JMX_CONNECTOR_MAP.put(clusterDO.getId(), new ConcurrentHashMap<>());
KAFKA_VERSION_MAP.put(clusterDO.getId(), new ConcurrentHashMap<>());
BrokerStateListener brokerListener = new BrokerStateListener(clusterDO.getId(), zkConfig);
brokerListener.init();
zkConfig.watchChildren(ZkPathUtil.BROKER_IDS_ROOT, brokerListener);
//增加Topic监控
TOPIC_METADATA_MAP.put(clusterDO.getId(), new ConcurrentHashMap<>());
TopicStateListener topicListener = new TopicStateListener(clusterDO.getId(), zkConfig);
topicListener.init();
zkConfig.watchChildren(ZkPathUtil.BROKER_TOPICS_ROOT, topicListener);
//增加Controller监控
ControllerStateListener controllerListener =
new ControllerStateListener(clusterDO.getId(), zkConfig, controllerDao);
controllerListener.init();
zkConfig.watch(ZkPathUtil.CONTROLLER_ROOT_NODE, controllerListener);
//增加Config变更监控
TOPIC_RETENTION_TIME_MAP.put(clusterDO.getId(), new ConcurrentHashMap<>());
CLUSTER_MAP.put(clusterDO.getId(), clusterDO);
ZK_CONFIG_MAP.put(clusterDO.getId(), zkConfig);
} catch (Exception e) {
LOGGER.error("add cluster failed, cluster:{}.", clusterDO, e);
}
}
public void remove(Long clusterId) {
try {
ZkConfigImpl zkConfig = ZK_CONFIG_MAP.remove(clusterId);
if (zkConfig != null) {
zkConfig.cancelWatchChildren(ZkPathUtil.BROKER_IDS_ROOT);
zkConfig.cancelWatchChildren(ZkPathUtil.BROKER_TOPICS_ROOT);
zkConfig.cancelWatchChildren(ZkPathUtil.CONTROLLER_ROOT_NODE);
zkConfig.close();
}
} catch (Exception e) {
LOGGER.error("remove cluster metadata failed, clusterId:{}.", clusterId, e);
}
CONTROLLER_DATA_MAP.remove(clusterId);
BROKER_METADATA_MAP.remove(clusterId);
JMX_CONNECTOR_MAP.remove(clusterId);
KAFKA_VERSION_MAP.remove(clusterId);
TOPIC_METADATA_MAP.remove(clusterId);
TOPIC_RETENTION_TIME_MAP.remove(clusterId);
CLUSTER_MAP.remove(clusterId);
}
public Set<Long> getClusterIdSet() {
return CLUSTER_MAP.keySet();
}
public static ClusterDO getClusterFromCache(Long clusterId) {
return CLUSTER_MAP.get(clusterId);
}
//---------------------------Controller元信息相关--------------
public static ControllerData removeControllerData(Long clusterId) {
return CONTROLLER_DATA_MAP.remove(clusterId);
}
public static void putControllerData(Long clusterId, ControllerData controllerData) {
CONTROLLER_DATA_MAP.put(clusterId, controllerData);
}
public static Integer getControllerId(Long clusterId) {
ControllerData data = CONTROLLER_DATA_MAP.get(clusterId);
if (data == null) {
return null;
}
return data.getBrokerid();
}
//---------------------------Topic元信息相关--------------
public static void putTopicMetadata(Long clusterId, String topicName, TopicMetadata topicMetadata) {
Map<String, TopicMetadata> metadataMap = TOPIC_METADATA_MAP.get(clusterId);
if (metadataMap == null) {
return;
}
metadataMap.put(topicName, topicMetadata);
}
public static TopicMetadata removeTopicMetadata(Long clusterId, String topicName) {
Map<String, TopicMetadata> metadataMap = TOPIC_METADATA_MAP.get(clusterId);
if (metadataMap == null) {
return null;
}
return metadataMap.remove(topicName);
}
public static TopicMetadata getTopicMetadata(Long clusterId, String topicName) {
Map<String, TopicMetadata> metadataMap = TOPIC_METADATA_MAP.get(clusterId);
if (metadataMap == null) {
return null;
}
return metadataMap.get(topicName);
}
public static List<String> getTopicNameList(Long clusterId) {
Map<String, TopicMetadata> metadataMap = TOPIC_METADATA_MAP.get(clusterId);
if (metadataMap == null) {
return new ArrayList<>();
}
return new ArrayList<>(metadataMap.keySet());
}
public static boolean isTopicExist(Long clusterId, String topicName) {
Map<String, TopicMetadata> metadataMap = TOPIC_METADATA_MAP.get(clusterId);
return metadataMap != null && metadataMap.containsKey(topicName);
}
public static boolean isTopicExistStrictly(Long clusterId, String topicName) {
Map<String, TopicMetadata> metadataMap = TOPIC_METADATA_MAP.get(clusterId);
if (metadataMap == null) {
// 集群不存在, 直接false
return false;
}
if (metadataMap.containsKey(topicName)) {
// 存在则一定存在
return true;
}
// 不存在则可能是因为重启导致的
try {
ZkConfigImpl zkConfig = ZK_CONFIG_MAP.get(clusterId);
if (zkConfig == null) {
return false;
}
if (zkConfig.checkPathExists(ZkPathUtil.getConfigTopicNode(topicName))) {
return true;
}
} catch (Exception e) {
LOGGER.error("unknown whether topic exist, clusterId:{} topicName:{}.", clusterId, topicName, e);
}
return false;
}
//---------------------------配置相关元信息--------------
public static void putTopicRetentionTime(Long clusterId, String topicName, Long retentionTime) {
Map<String, Long> timeMap = TOPIC_RETENTION_TIME_MAP.get(clusterId);
if (timeMap == null) {
return;
}
timeMap.put(topicName, retentionTime);
}
public static Long getTopicRetentionTime(Long clusterId, String topicName) {
Map<String, Long> timeMap = TOPIC_RETENTION_TIME_MAP.get(clusterId);
if (timeMap == null) {
return null;
}
return timeMap.get(topicName);
}
//---------------------------Broker元信息相关--------------
public static void putBrokerMetadata(Long clusterId, Integer brokerId, BrokerMetadata brokerMetadata) {
Map<Integer, BrokerMetadata> metadataMap = BROKER_METADATA_MAP.get(clusterId);
if (metadataMap == null) {
return;
}
metadataMap.put(brokerId, brokerMetadata);
Map<Integer, JmxConnectorWrap> jmxMap = JMX_CONNECTOR_MAP.getOrDefault(clusterId, new ConcurrentHashMap<>());
jmxMap.put(brokerId, new JmxConnectorWrap(brokerMetadata.getHost(), brokerMetadata.getJmxPort()));
JMX_CONNECTOR_MAP.put(clusterId, jmxMap);
Map<Integer, KafkaVersion> versionMap = KAFKA_VERSION_MAP.getOrDefault(clusterId, new ConcurrentHashMap<>());
versionMap.put(brokerId, new KafkaVersion());
KAFKA_VERSION_MAP.put(clusterId, versionMap);
}
public static void removeBrokerMetadata(Long clusterId, Integer brokerId) {
Map<Integer, JmxConnectorWrap> jmxMap = JMX_CONNECTOR_MAP.get(clusterId);
JmxConnectorWrap jmxConnectorWrap = jmxMap == null? null: jmxMap.remove(brokerId);
Map<Integer, KafkaVersion> versionMap = KAFKA_VERSION_MAP.get(clusterId);
if (versionMap != null) {
versionMap.remove(brokerId);
}
Map<Integer, BrokerMetadata> metadataMap = BROKER_METADATA_MAP.get(clusterId);
if (metadataMap == null) {
return;
}
metadataMap.remove(brokerId);
if (jmxConnectorWrap != null) {
jmxConnectorWrap.close();
}
}
public static boolean isBrokerAlive(Long clusterId, Integer brokerId) {
Map<Integer, BrokerMetadata> metadataMap = BROKER_METADATA_MAP.get(clusterId);
if (metadataMap == null) {
return false;
}
return metadataMap.containsKey(brokerId);
}
public static BrokerMetadata getBrokerMetadata(Long clusterId, Integer brokerId) {
Map<Integer, BrokerMetadata> metadataMap = BROKER_METADATA_MAP.get(clusterId);
if (metadataMap == null) {
return null;
}
return metadataMap.get(brokerId);
}
public static Map<String, List<String>> getBrokerHostKafkaRoleMap(Long clusterId) {
Map<String, List<String>> hostRoleMap = new HashMap<>();
ControllerData controllerData = CONTROLLER_DATA_MAP.get(clusterId);
if (controllerData != null) {
supplyHostRoleData(hostRoleMap, clusterId, controllerData.getBrokerid(), KafkaBrokerRoleEnum.CONTROLLER);
}
TopicMetadata topicMetadata =
PhysicalClusterMetadataManager.getTopicMetadata(clusterId, KafkaConstant.COORDINATOR_TOPIC_NAME);
if (topicMetadata != null) {
for (Integer brokerId: topicMetadata.getBrokerIdSet()) {
supplyHostRoleData(hostRoleMap, clusterId, brokerId, KafkaBrokerRoleEnum.COORDINATOR);
}
}
List<Integer> brokerIdList = PhysicalClusterMetadataManager.getBrokerIdList(clusterId);
for (Integer brokerId: brokerIdList) {
supplyHostRoleData(hostRoleMap, clusterId, brokerId, KafkaBrokerRoleEnum.NORMAL);
}
return hostRoleMap;
}
private static void supplyHostRoleData(Map<String, List<String>> hostRoleMap,
Long clusterId,
Integer brokerId,
KafkaBrokerRoleEnum roleEnum) {
BrokerMetadata brokerMetadata =
PhysicalClusterMetadataManager.getBrokerMetadata(clusterId, brokerId);
if (ValidateUtils.isNull(brokerMetadata)) {
return;
}
String hostname = brokerMetadata.getHost().replace(KafkaConstant.BROKER_HOST_NAME_SUFFIX, "");
if (KafkaBrokerRoleEnum.NORMAL.equals(roleEnum) && hostRoleMap.containsKey(hostname)) {
return;
}
List<String> roleList = hostRoleMap.getOrDefault(brokerMetadata.getHost(), new ArrayList<>());
roleList.add(roleEnum.getRole());
hostRoleMap.put(hostname, roleList);
}
public static Map<String, List<String>> getKafkaRoleBrokerHostMap(Long clusterId) {
Map<String, List<String>> roleHostMap = new HashMap<>(3);
ControllerData controllerData = CONTROLLER_DATA_MAP.get(clusterId);
if (controllerData != null) {
supplyRoleHostData(roleHostMap, clusterId, controllerData.getBrokerid(), KafkaBrokerRoleEnum.CONTROLLER);
}
TopicMetadata topicMetadata =
PhysicalClusterMetadataManager.getTopicMetadata(clusterId, KafkaConstant.COORDINATOR_TOPIC_NAME);
if (topicMetadata != null) {
for (Integer brokerId: topicMetadata.getBrokerIdSet()) {
supplyRoleHostData(roleHostMap, clusterId, brokerId, KafkaBrokerRoleEnum.COORDINATOR);
}
}
List<Integer> brokerIdList = PhysicalClusterMetadataManager.getBrokerIdList(clusterId);
for (Integer brokerId: brokerIdList) {
supplyRoleHostData(roleHostMap, clusterId, brokerId, KafkaBrokerRoleEnum.NORMAL);
}
return roleHostMap;
}
private static void supplyRoleHostData(Map<String, List<String>> roleHostMap,
Long clusterId,
Integer brokerId,
KafkaBrokerRoleEnum roleEnum) {
BrokerMetadata brokerMetadata =
PhysicalClusterMetadataManager.getBrokerMetadata(clusterId, brokerId);
if (ValidateUtils.isNull(brokerMetadata)) {
return;
}
List<String> hostList = roleHostMap.getOrDefault(roleEnum.getRole(), new ArrayList<>());
hostList.add(brokerMetadata.getHost().replace(KafkaConstant.BROKER_HOST_NAME_SUFFIX, ""));
roleHostMap.put(roleEnum.getRole(), hostList);
}
public static List<Integer> getBrokerIdList(Long clusterId) {
Map<Integer, BrokerMetadata> metadataMap = BROKER_METADATA_MAP.get(clusterId);
if (metadataMap == null) {
return new ArrayList<>();
}
return new ArrayList<>(metadataMap.keySet());
}
public static Set<String> getBrokerHostnameSet(Long clusterId) {
Map<Integer, BrokerMetadata> metadataMap = BROKER_METADATA_MAP.get(clusterId);
if (metadataMap == null) {
return new HashSet<>();
}
Set<String> hostnameSet = new HashSet<>();
for (Map.Entry<Integer, BrokerMetadata> entry: metadataMap.entrySet()) {
hostnameSet.add(entry.getValue().getHost().replace(KafkaConstant.BROKER_HOST_NAME_SUFFIX, ""));
}
return hostnameSet;
}
public static JmxConnectorWrap getJmxConnectorWrap(Long clusterId, Integer brokerId) {
Map<Integer, JmxConnectorWrap> jmxConnectorWrapMap = JMX_CONNECTOR_MAP.get(clusterId);
if (jmxConnectorWrapMap == null) {
return null;
}
return jmxConnectorWrapMap.get(brokerId);
}
public KafkaVersion getKafkaVersion(Long clusterId, Integer brokerId) {
Map<Integer, KafkaVersion> versionMap = KAFKA_VERSION_MAP.get(clusterId);
if (versionMap == null) {
return new KafkaVersion();
}
KafkaVersion kafkaVersion = versionMap.get(brokerId);
if (kafkaVersion == null) {
return new KafkaVersion();
}
if (kafkaVersion.initialized()) {
return kafkaVersion;
}
kafkaVersion.init(jmxService.getBrokerVersion(clusterId, brokerId));
return kafkaVersion;
}
public String getKafkaVersion(Long clusterId) {
return getKafkaVersion(clusterId, PhysicalClusterMetadataManager.getBrokerIdList(clusterId));
}
public String getKafkaVersion(Long clusterId, List<Integer> brokerIdList) {
Set<String> kafkaVersionSet = new HashSet<>();
for (Integer brokerId: brokerIdList) {
KafkaVersion kafkaVersion = this.getKafkaVersion(clusterId, brokerId);
if (kafkaVersion == null) {
kafkaVersionSet.add(Constant.UNKNOWN_VERSION);
continue;
}
kafkaVersionSet.add(kafkaVersion.getVersion());
}
return ListUtils.strList2String(new ArrayList<>(kafkaVersionSet));
}
public String getKafkaVersionFromCache(Long clusterId, Integer brokerId) {
Map<Integer, KafkaVersion> versionMap = KAFKA_VERSION_MAP.get(clusterId);
if (versionMap == null) {
return null;
}
KafkaVersion kafkaVersion = versionMap.get(brokerId);
if (kafkaVersion == null) {
return null;
}
if (kafkaVersion.initialized()) {
return kafkaVersion.getVersion();
}
return null;
}
public static ZkConfigImpl getZKConfig(Long clusterId) {
if (!ZK_CONFIG_MAP.containsKey(clusterId)) {
return null;
}
return ZK_CONFIG_MAP.get(clusterId);
}
public static Set<String> getBrokerTopicNum(Long clusterId, Set<Integer> brokerIdSet) {
Set<String> topicNameSet = new HashSet<>();
Map<String, TopicMetadata> metadataMap = TOPIC_METADATA_MAP.get(clusterId);
for (String topicName: metadataMap.keySet()) {
try {
TopicMetadata tm = metadataMap.get(topicName);
for (Integer brokerId: tm.getBrokerIdSet()) {
if (!brokerIdSet.contains(brokerId)) {
continue;
}
topicNameSet.add(topicName);
}
} catch (Exception e) {
}
}
return topicNameSet;
}
public static long getNotAliveBrokerNum(Long clusterId, List<Integer> brokerIdList) {
Set<Integer> aliveBrokerIdSet = new HashSet<>(getBrokerIdList(clusterId));
return brokerIdList.stream()
.filter(brokerId -> !aliveBrokerIdSet.contains(brokerId))
.count();
}
}

View File

@@ -0,0 +1,37 @@
package com.xiaojukeji.kafka.manager.service.cache;
import com.xiaojukeji.kafka.manager.common.utils.factory.DefaultThreadFactory;
import java.util.concurrent.*;
/**
* @author zengqiao
* @date 20/8/24
*/
public class ThreadPool {
private static final ExecutorService COLLECT_METRICS_THREAD_POOL = new ThreadPoolExecutor(
256,
256,
120L,
TimeUnit.SECONDS,
new LinkedBlockingQueue<Runnable>(),
new DefaultThreadFactory("Collect-Metrics-Thread")
);
private static final ExecutorService API_CALL_THREAD_POOL = new ThreadPoolExecutor(
16,
16,
120L,
TimeUnit.SECONDS,
new LinkedBlockingQueue<Runnable>(),
new DefaultThreadFactory("Api-Call-Thread")
);
public static void submitCollectMetricsTask(Runnable collectMetricsTask) {
COLLECT_METRICS_THREAD_POOL.submit(collectMetricsTask);
}
public static void submitApiCallTask(Runnable apiCallTask) {
API_CALL_THREAD_POOL.submit(apiCallTask);
}
}

View File

@@ -0,0 +1,37 @@
package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.bizenum.TaskStatusEnum;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.TopicDO;
import java.util.List;
import java.util.Properties;
public interface AdminService {
ResultStatus createTopic(ClusterDO clusterDO,
TopicDO topicDO,
Integer partitionNum,
Integer replicaNum,
Long regionId,
List<Integer> brokerIdList,
Properties properties,
String applicant,
String operator);
ResultStatus deleteTopic(ClusterDO clusterDO,
String topicName,
String operator);
TaskStatusEnum preferredReplicaElectionStatus(ClusterDO clusterDO);
ResultStatus preferredReplicaElection(ClusterDO clusterDO, String operator);
ResultStatus preferredReplicaElection(ClusterDO clusterDO, Integer brokerId, String operator);
ResultStatus expandPartitions(ClusterDO clusterDO, String topicName, Integer partitionNum, Long regionId, List<Integer> brokerIdList, String operator);
Properties getTopicConfig(ClusterDO clusterDO, String topicName);
ResultStatus modifyTopicConfig(ClusterDO clusterDO, String topicName, Properties properties, String operator);
}

View File

@@ -0,0 +1,11 @@
package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.entity.ao.analysis.AnalysisBrokerDTO;
/**
* @author huangyiminghappy@163.com, zengqiao_cn@163.com
* @date 2019-06-14
*/
public interface AnalysisService {
AnalysisBrokerDTO doAnalysisBroker(Long clusterId , Integer brokerId);
}

View File

@@ -0,0 +1,67 @@
package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.BrokerBasicDTO;
import com.xiaojukeji.kafka.manager.common.entity.ao.TopicDiskLocation;
import com.xiaojukeji.kafka.manager.common.entity.ao.cluster.ClusterBrokerStatus;
import com.xiaojukeji.kafka.manager.common.entity.metrics.BrokerMetrics;
import com.xiaojukeji.kafka.manager.common.entity.ao.BrokerOverviewDTO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.BrokerDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.BrokerMetricsDO;
import java.util.Date;
import java.util.List;
import java.util.Set;
/**
* Broker模块的service接口
* @author tukun, zengqiao
* @date 2015/11/9
*/
public interface BrokerService {
ClusterBrokerStatus getClusterBrokerStatus(Long clusterId);
/**
* 获取Broker列表信息
*/
List<BrokerOverviewDTO> getBrokerOverviewList(Long clusterId, Set<Integer> brokerIdSet);
/**
* 获取BrokerMetrics信息
*/
List<BrokerMetrics> getBrokerMetricsFromJmx(Long clusterId, Set<Integer> brokerIdSet, Integer metricsCode);
/**
* 获取BrokerMetrics信息
*/
BrokerMetrics getBrokerMetricsFromJmx(Long clusterId, Integer brokerId, Integer metricsCode);
/**
* 根据时间区间获取Broker监控数据
*/
List<BrokerMetricsDO> getBrokerMetricsFromDB(Long clusterId, Integer brokerId, Date startTime, Date endTime);
List<TopicDiskLocation> getBrokerTopicLocation(Long clusterId, Integer brokerId);
/**
* 计算Broker的峰值均值流量
*/
Double calBrokerMaxAvgBytesIn(Long clusterId,
Integer brokerId,
Integer duration,
Date startTime,
Date endTime);
/**
* 根据Cluster和brokerId获取broker的具体信息
*/
BrokerBasicDTO getBrokerBasicDTO(Long clusterId, Integer brokerId);
String getBrokerVersion(Long clusterId, Integer brokerId);
List<BrokerDO> listAll();
int replace(BrokerDO brokerDO);
ResultStatus delete(Long clusterId, Integer brokerId);
}

View File

@@ -0,0 +1,47 @@
package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.ClusterDetailDTO;
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;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ControllerDO;
import java.util.Date;
import java.util.List;
import java.util.Map;
/**
* Cluster Service
* @author zengqiao
* @date 19/4/3
*/
public interface ClusterService {
ClusterDO getById(Long clusterId);
ResultStatus addNew(ClusterDO clusterDO, String operator);
ResultStatus updateById(ClusterDO clusterDO, String operator);
ResultStatus modifyStatus(Long clusterId, Integer status, String operator);
List<ClusterDO> list();
Map<Long, ClusterDO> listMap();
List<ClusterDO> listAll();
List<ClusterMetricsDO> getClusterMetricsFromDB(Long clusterId, Date startTime, Date endTime);
List<ControllerDO> getKafkaControllerHistory(Long clusterId);
ClusterDetailDTO getClusterDetailDTO(Long clusterId, Boolean needDetail);
List<ClusterDetailDTO> getClusterDetailDTOList(Boolean needDetail);
ClusterNameDTO getClusterName(Long logicClusterId);
ResultStatus deleteById(Long clusterId);
ClusterDO selectSuitableCluster(Long clusterId, String dataCenter);
}

View File

@@ -0,0 +1,37 @@
package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.config.CreateTopicElemConfig;
import com.xiaojukeji.kafka.manager.common.entity.dto.config.ConfigDTO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ConfigDO;
import java.util.List;
/**
* @author zengqiao
* @date 20/3/19
*/
public interface ConfigService {
ResultStatus insert(ConfigDTO dto);
ResultStatus deleteByKey(String configKey);
ResultStatus updateByKey(ConfigDTO dto);
ResultStatus updateByKey(String configKey, String configValue);
ConfigDO getByKey(String configKey);
<T> T getByKey(String configKey, Class<T> clazz);
<T> List<T> getArrayByKey(String configKey, Class<T> clazz);
Long getLongValue(String configKey, Long defaultValue);
List<ConfigDO> listAll();
CreateTopicElemConfig getCreateTopicConfig(Long clusterId, String systemCode);
ClusterDO getClusterDO(Long clusterId);
}

View File

@@ -0,0 +1,55 @@
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.PartitionOffsetDTO;
import com.xiaojukeji.kafka.manager.common.entity.ao.consumer.ConsumeDetailDTO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterDO;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* consumer相关的服务接口
* @author tukun
* @date 2015/11/12.
*/
public interface ConsumerService {
/**
* 获取消费组列表
*/
List<ConsumerGroupDTO> getConsumerGroupList(Long clusterId);
/**
* 查询消费Topic的消费组
*/
List<ConsumerGroupDTO> getConsumerGroupList(Long clusterId, String topicName);
/**
* 查询消费详情
*/
List<ConsumeDetailDTO> getConsumeDetail(ClusterDO clusterDO, String topicName, ConsumerGroupDTO consumerGroupDTO);
/**
* 获取消费组消费的Topic列表
*/
List<String> getConsumerGroupConsumedTopicList(Long clusterId, String consumerGroup, String location);
Map<Integer, Long> getConsumerOffset(ClusterDO clusterDO,
String topicName,
ConsumerGroupDTO consumerGroupDTO);
/**
* 重置offset
*/
List<Result> resetConsumerOffset(ClusterDO clusterDO,
String topicName,
ConsumerGroupDTO consumerGroupDTO,
List<PartitionOffsetDTO> partitionOffsetDTOList);
Map<Long, Integer> getConsumerGroupNumMap(List<ClusterDO> clusterDOList);
boolean checkConsumerGroupExist(OffsetLocationEnum offsetLocation, Long id, String topicName, String consumerGroup);
}

View File

@@ -0,0 +1,45 @@
package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.entity.ao.expert.TopicAnomalyFlow;
import com.xiaojukeji.kafka.manager.common.entity.ao.expert.TopicInsufficientPartition;
import com.xiaojukeji.kafka.manager.common.entity.ao.expert.TopicRegionHot;
import com.xiaojukeji.kafka.manager.common.entity.pojo.TopicExpiredDO;
import java.util.List;
/**
* 专家服务
* @author zengqiao
* @date 20/3/20
*/
public interface ExpertService {
/**
* Region内热点Topic
* @author zengqiao
* @date 20/3/30
* @return java.util.List<com.xiaojukeji.kafka.manager.common.entity.dto.expert.RegionHotTopicDTO>
*/
List<TopicRegionHot> getRegionHotTopics();
/**
* 分区不足Topic
*/
List<TopicInsufficientPartition> getPartitionInsufficientTopics();
/**
* 流量陡增
* @param timestamp 时间戳
* @author zengqiao
* @date 20/3/30
* @return java.util.List<com.xiaojukeji.kafka.manager.common.entity.dto.expert.AnomalyFlowTopicDTO>
*/
List<TopicAnomalyFlow> getAnomalyFlowTopics(Long timestamp);
/**
* 过期Topic列表
* @author zengqiao
* @date 20/3/30
* @return java.util.List<com.xiaojukeji.kafka.manager.common.entity.po.TopicExpiredDO>
*/
List<TopicExpiredDO> getExpiredTopics();
}

View File

@@ -0,0 +1,58 @@
package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.bizenum.KafkaClientEnum;
import com.xiaojukeji.kafka.manager.common.entity.ao.PartitionAttributeDTO;
import com.xiaojukeji.kafka.manager.common.entity.metrics.TopicMetrics;
import com.xiaojukeji.kafka.manager.common.entity.metrics.BrokerMetrics;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.PartitionState;
import org.apache.kafka.common.TopicPartition;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* 从Jmx获取相关数据的服务接口
*
* @author tukun, zengqiao
* @date 2015/11/11.
*/
public interface JmxService {
/**
* 获取Broker指标
* @param clusterId 集群ID
* @param brokerId brokerId
* @param metricsCode 指标集
* @return 如果broker不存在, 返回NULL, 其他情况返回BrokerMetrics, BrokerMetrics
*/
BrokerMetrics getBrokerMetrics(Long clusterId, Integer brokerId, Integer metricsCode);
TopicMetrics getTopicMetrics(Long clusterId, String topicName, Integer metricsCode, Boolean byAdd);
TopicMetrics getTopicMetrics(Long clusterId, Integer brokerId, String topicName, Integer metricsCode, Boolean byAdd);
List<TopicMetrics> getTopicMetrics(Long clusterId, Integer metricsCode, Boolean byAdd);
/**
* 从JMX中获取appId维度的的流量信息
*/
List<TopicMetrics> getTopicAppMetrics(Long clusterId, Integer metricsCode);
Map<TopicPartition, String> getBrokerTopicLocation(Long clusterId, Integer brokerId);
/**
* 获取分区位置和日志大小
*/
Map<Integer, PartitionAttributeDTO> getPartitionAttribute(Long clusterId,
String topicName,
List<PartitionState> partitionStateList);
/**
* 获取被限流客户端
*/
Set<String> getBrokerThrottleClients(Long clusterId, Integer brokerId, KafkaClientEnum kafkaClientEnum);
String getBrokerVersion(Long clusterId, Integer brokerId);
double getTopicAppThrottle(Long clusterId, Integer brokerId, String clientId, KafkaClientEnum kafkaClientEnum);
}

View File

@@ -0,0 +1,22 @@
package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.entity.pojo.KafkaBillDO;
import java.util.Date;
import java.util.List;
/**
* @author zengqiao
* @date 20/5/12
*/
public interface KafkaBillService {
int replace(KafkaBillDO kafkaBillDO);
List<KafkaBillDO> getByTopicName(Long clusterId, String topicName, Date startTime, Date endTime);
List<KafkaBillDO> getByPrincipal(String principal, Date startTime, Date endTime);
List<KafkaBillDO> getByTimeBetween(Date startTime, Date endTime);
List<KafkaBillDO> getByGmtDay(String gmtDay);
}

View File

@@ -0,0 +1,73 @@
package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.cluster.LogicalCluster;
import com.xiaojukeji.kafka.manager.common.entity.ao.cluster.LogicalClusterMetrics;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.BrokerMetadata;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata;
import com.xiaojukeji.kafka.manager.common.entity.pojo.LogicalClusterDO;
import java.util.Date;
import java.util.List;
/**
* @author zengqiao
* @date 20/6/29
*/
public interface LogicalClusterService {
List<LogicalCluster> getAllLogicalCluster();
/**
* 获取逻辑集群信息
*/
List<LogicalCluster> getLogicalClusterListByPrincipal(String principal);
/**
* 获取逻辑集群信息
*/
LogicalCluster getLogicalCluster(Long logicalClusterId);
/**
* 逻辑集群下Topic元信息
*/
List<TopicMetadata> getTopicMetadatas(LogicalClusterDO logicalClusterDO);
/**
* 逻辑集群下broker元信息
*/
List<BrokerMetadata> getBrokerMetadatas(LogicalClusterDO logicalClusterDO);
/**
* 获取逻辑集群流量
*/
List<LogicalClusterMetrics> getLogicalClusterMetricsFromDB(LogicalClusterDO logicalClusterDO,
Date startTime,
Date endTime);
List<LogicalClusterDO> listAll();
/**
* 创建逻辑集群
*/
ResultStatus createLogicalCluster(LogicalClusterDO logicalClusterDO);
/**
* 通过物理集群ID查找
*/
List<LogicalClusterDO> getByPhysicalClusterId(Long physicalClusterId);
LogicalClusterDO getById(Long id);
/**
* 删除逻辑集群
*/
ResultStatus deleteById(Long logicalClusterId);
/**
* 修改逻辑集群
*/
ResultStatus updateById(LogicalClusterDO logicalClusterDO);
ResultStatus updateById(Long logicalClusterId, List<Long> regionIdList);
}

View File

@@ -0,0 +1,16 @@
package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.entity.dto.rd.OperateRecordDTO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.OperateRecordDO;
import java.util.List;
/**
* @author zhongyuankai
* @date 2020/09/03
*/
public interface OperateRecordService {
int insert(OperateRecordDO operateRecordDO);
List<OperateRecordDO> queryByCondt(OperateRecordDTO dto);
}

View File

@@ -0,0 +1,41 @@
package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.bizenum.TaskStatusReassignEnum;
import com.xiaojukeji.kafka.manager.common.bizenum.TopicReassignActionEnum;
import com.xiaojukeji.kafka.manager.common.entity.Result;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.reassign.ReassignStatus;
import com.xiaojukeji.kafka.manager.common.entity.dto.op.reassign.ReassignExecDTO;
import com.xiaojukeji.kafka.manager.common.entity.dto.op.reassign.ReassignExecSubDTO;
import com.xiaojukeji.kafka.manager.common.entity.dto.op.reassign.ReassignTopicDTO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ReassignTaskDO;
import kafka.common.TopicAndPartition;
import kafka.utils.ZkUtils;
import java.util.List;
import java.util.Map;
/**
* migrate topic service
* @author zengqiao_cn@163.com
* @date 19/4/16
*/
public interface ReassignService {
ResultStatus createTask(List<ReassignTopicDTO> dtoList, String operator);
ResultStatus modifyTask(ReassignExecDTO dto, TopicReassignActionEnum actionEnum);
ResultStatus modifySubTask(ReassignExecSubDTO dto);
List<ReassignTaskDO> getReassignTaskList();
List<ReassignTaskDO> getTask(Long taskId);
Result<List<ReassignStatus>> getReassignStatus(Long taskId);
Map<TopicAndPartition, TaskStatusReassignEnum> verifyAssignment(String zkAddr, String reassignmentJson);
Map<TopicAndPartition, TaskStatusReassignEnum> verifyAssignment(ZkUtils zkUtils, String reassignmentJson);
}

View File

@@ -0,0 +1,76 @@
package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.dto.rd.RegionDTO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.RegionDO;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* @author zengqiao
* @date 19/4/21
*/
public interface RegionService {
List<RegionDO> listAll();
/**
* 创建Region
*/
ResultStatus createRegion(RegionDO regionDO);
/**
* 通过id查找
*/
RegionDO getById(Long id);
/**
* 删除Region
*/
ResultStatus deleteById(Long regionId);
/**
* 修改Region信息
*/
ResultStatus updateRegion(RegionDO regionDO);
int updateCapacityById(RegionDO regionDO);
/**
* 查询Region详情
*/
List<RegionDO> getByClusterId(Long clusterId);
/**
* 获取集群有几个Region
*/
Map<Long, Integer> getRegionNum();
/**
* 合并regionId和brokerIdList中的brokerId
*/
List<Integer> getFullBrokerIdList(Long clusterId, Long regionId, List<Integer> brokerIdList);
Map<Integer, RegionDO> convert2BrokerIdRegionMap(List<RegionDO> regionDOList);
/**
* 更新逻辑集群容量
* @param clusterId 集群id
* @param newBrokerList 新的broker列表
* @return ResultStatus
*/
ResultStatus updateRegion(Long clusterId, String newBrokerList);
/**
* 获取空闲的region的broker列表
*/
List<Integer> getIdleRegionBrokerList(Long physicalClusterId, List<Long> regionIdList);
Map<String, Set<Integer>> getTopicNameRegionBrokerIdMap(Long clusterId);
/**
* 获取topic所在的region
*/
List<RegionDO> getRegionListByTopicName(Long clusterId, String topicName);
}

View File

@@ -0,0 +1,48 @@
package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.bizenum.KafkaClientEnum;
import com.xiaojukeji.kafka.manager.common.entity.metrics.TopicThrottledMetrics;
import com.xiaojukeji.kafka.manager.common.entity.pojo.TopicThrottledMetricsDO;
import java.util.Date;
import java.util.List;
import java.util.Set;
/**
* @author zhongyuankai
* @date 20/4/3
*/
public interface ThrottleService {
/**
* 批量插入
* @param topicThrottleDOList
* @return
*/
int insertBatch(List<TopicThrottledMetricsDO> topicThrottleDOList);
/**
* 查询Topic限流历史信息
* @param clusterId
* @param topicName
* @param appId
* @param startTime
* @param endTime
* @return
*/
List<TopicThrottledMetricsDO> getTopicThrottleFromDB(Long clusterId,
String topicName,
String appId,
Date startTime,
Date endTime);
/**
* 查询topic限流历史信息
* @param clusterId 集群ID
* @param brokerIdSet BrokerId集合
* @param kafkaClientList 查询字段
* @return 限流信息
*/
List<TopicThrottledMetrics> getThrottledTopicsFromJmx(Long clusterId,
Set<Integer> brokerIdSet,
List<KafkaClientEnum> kafkaClientList);
}

View File

@@ -0,0 +1,16 @@
package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.topic.TopicExpiredData;
import java.util.List;
/**
* @author zengqiao
* @date 20/9/2
*/
public interface TopicExpiredService {
List<TopicExpiredData> getExpiredTopicDataList(String username);
ResultStatus retainExpiredTopic(Long physicalClusterId, String topicName, Integer retainDays);
}

View File

@@ -0,0 +1,102 @@
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.RdTopicBasic;
import com.xiaojukeji.kafka.manager.common.entity.ao.topic.TopicAppData;
import com.xiaojukeji.kafka.manager.common.entity.ao.topic.TopicBusinessInfo;
import com.xiaojukeji.kafka.manager.common.entity.ao.topic.TopicDTO;
import com.xiaojukeji.kafka.manager.common.entity.ao.topic.MineTopicSummary;
import com.xiaojukeji.kafka.manager.common.entity.pojo.TopicDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.TopicExpiredDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.TopicStatisticsDO;
import java.util.Date;
import java.util.List;
import java.util.Map;
/**
* @author arthur
* @date 2017/7/20.
*/
public interface TopicManagerService {
List<TopicDO> listAll();
List<TopicDO> getByClusterId(Long clusterId);
TopicDO getByTopicName(Long clusterId, String topicName);
int replaceTopicStatistics(TopicStatisticsDO topicStatisticsDO);
Map<String, List<Double>> getTopicMaxAvgBytesIn(Long clusterId, Integer latestDay, Double minMaxAvgBytesIn);
Double getTopicMaxAvgBytesIn(Long clusterId, String topicName, Date startTime, Date endTime, Integer maxAvgDay);
TopicStatisticsDO getByTopicAndDay(Long clusterId, String topicName, String gmtDay);
List<TopicExpiredDO> getExpiredTopics(Integer expiredDay);
/**
* 获取 username 的 Topic
* @param username 用户名
* @author zengqiao
* @date 20/5/12
* @return java.util.List<TopicMineDTO>
*/
List<MineTopicSummary> getMyTopics(String username);
/**
* 获取 username 可见的 Topic
* @param username 用户名
* @author zengqiao
* @date 20/5/12
* @return java.util.List<TopicDTO>
*/
List<TopicDTO> getTopics(String username);
/**
* 修改Topic
* @param clusterId 集群ID
* @param topicName Topic名称
* @param description 备注
* @param operator 操作人
* @author zengqiao
* @date 20/5/12
* @return ResultStatus
*/
ResultStatus modifyTopic(Long clusterId, String topicName, String description, String operator);
/**
* 通过topictopic名称删除
* @param clusterId 集群id
* @param topicName topic名称
* @return int
*/
int deleteByTopicName(Long clusterId, String topicName);
/**
* 新增topic
* @param topicDO topicDO
* @return int
*/
int addTopic(TopicDO topicDO);
List<TopicAppData> getTopicAuthorizedApps(Long physicalClusterId, String topicName);
List<TopicAppData> getTopicMineApps(Long physicalClusterId, String topicName, String username);
/**
* RD视角获取Topic基本信息
* @param physicalClusterId 物理集群ID
* @param topicName Topic名称
* @author zengqiao
* @date 20/6/10
* @return Result<RdTopicBasic>
*/
Result<RdTopicBasic> getRdTopicBasic(Long physicalClusterId, String topicName);
List<TopicStatisticsDO> getTopicStatistic(Long clusterId, String topicName, Date startTime, Date endTime);
TopicBusinessInfo getTopicBusinessInfo(Long physicalClusterId, String topicName);
}

View File

@@ -0,0 +1,108 @@
package com.xiaojukeji.kafka.manager.service.service;
import com.xiaojukeji.kafka.manager.common.bizenum.OffsetPosEnum;
import com.xiaojukeji.kafka.manager.common.bizenum.TopicOffsetChangedEnum;
import com.xiaojukeji.kafka.manager.common.entity.Result;
import com.xiaojukeji.kafka.manager.common.entity.ao.*;
import com.xiaojukeji.kafka.manager.common.entity.dto.normal.TopicDataSampleDTO;
import com.xiaojukeji.kafka.manager.common.entity.ao.topic.*;
import com.xiaojukeji.kafka.manager.common.entity.metrics.BaseMetrics;
import com.xiaojukeji.kafka.manager.common.entity.pojo.TopicMetricsDO;
import com.xiaojukeji.kafka.manager.common.entity.ao.topic.TopicBrokerDTO;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.PartitionState;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterDO;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.common.TopicPartition;
import java.util.Date;
import java.util.List;
import java.util.Map;
/**
* Topic相关的接口
* @author tukun
* @date 2015/11/11.
*/
public interface TopicService {
/**
* 从DB获取监控数据
*/
List<TopicMetricsDO> getTopicMetricsFromDB(Long clusterId, String topicName, Date startTime, Date endTime);
List<TopicMetricsDTO> getTopicMetricsFromDB(String appId,
Long clusterId,
String topicName,
Date startTime,
Date endTime);
/**
* 获取指定时间段内的峰值的均值流量
*/
Double getMaxAvgBytesInFromDB(Long clusterId, String topicName, Date startTime, Date endTime);
/**
* 获取brokerId下所有的Topic及其对应的PartitionId
*/
Map<String, List<Integer>> getTopicPartitionIdMap(Long clusterId, Integer brokerId);
/**
* 获取 Topic 的 basic-info 信息
*/
TopicBasicDTO getTopicBasicDTO(Long clusterId, String topicName);
/**
* 获取Topic的PartitionState信息
*/
List<TopicPartitionDTO> getTopicPartitionDTO(ClusterDO clusterDO, String topicName, Boolean needDetail);
/**
* 得到topic流量信息
*/
BaseMetrics getTopicMetricsFromJMX(Long clusterId, String topicName, Integer metricsCode, Boolean byAdd);
/**
* 获取Topic的分区的offset
*/
Map<TopicPartition, Long> getPartitionOffset(ClusterDO cluster, String topicName, OffsetPosEnum offsetPosEnum);
/**
* 获取Topic概览信息
*/
List<TopicOverview> getTopicOverviewList(Long clusterId, Integer brokerId);
List<TopicOverview> getTopicOverviewList(Long clusterId, List<String> topicNameList);
/**
* 获取指定时间的offset信息
*/
List<PartitionOffsetDTO> getPartitionOffsetList(ClusterDO cluster, String topicName, Long timestamp);
Map<String, List<PartitionState>> getTopicPartitionState(Long clusterId, Integer filterBrokerId);
/**
* 数据采样
*/
List<String> fetchTopicData(ClusterDO clusterDO, String topicName, TopicDataSampleDTO reqObj);
List<String> fetchTopicData(KafkaConsumer kafkaConsumer, Integer maxMsgNum, Integer timeout, Boolean truncated);
/**
* 采样指定分区最新的数据
*/
List<String> fetchTopicData(KafkaConsumer kafkaConsumer,
Integer maxMsgNum,
Long maxWaitMs,
Boolean truncated,
List<TopicPartition> tpList);
/**
* 获取Topic历史耗时
*/
List<TopicMetricsDO> getTopicRequestMetricsFromDB(Long clusterId, String topicName, Date startTime, Date endTime);
/**
* 获取topic的broker列表
*/
List<TopicBrokerDTO> getTopicBrokerList(Long clusterId, String topicName);
Result<TopicOffsetChangedEnum> checkTopicOffsetChanged(Long physicalClusterId, String topicName, Long latestTime);
}

View File

@@ -0,0 +1,13 @@
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;
/**
* ZK相关的接口
* @author tukun
* @date 2015/11/11.
*/
public interface ZookeeperService {
Result openTopicJmx(Long clusterId, String topicName, TopicJmxSwitch jmxSwitch);
}

View File

@@ -0,0 +1,70 @@
package com.xiaojukeji.kafka.manager.service.service.gateway;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.AppTopicDTO;
import com.xiaojukeji.kafka.manager.common.entity.dto.normal.AppDTO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.AppDO;
import java.util.List;
/**
* @author zhongyuankai
* @date 2020/4/28
*/
public interface AppService {
/**
* 插入数据
* @param appDO appDO
* @return int
*/
ResultStatus addApp(AppDO appDO);
/**
* 删除数据
* @param appDO appDO
* @return int
*/
int deleteApp(AppDO appDO, String operator);
/**
* 通过负责人来查找
* @param name app名称
* @return List<AppDO>
*/
AppDO getByName(String name);
/**
* 更新App信息
* @param dto app信息
* @param operator 操作人
* @param adminApi admin操作
* @author zengqiao
* @date 20/5/4
* @return int
*/
ResultStatus updateByAppId(AppDTO dto, String operator, Boolean adminApi);
/**
* 通过负责人来查找
* @param principal 负责人
* @return List<AppDO>
*/
List<AppDO> getByPrincipal(String principal);
/**
* 通过appId来查
* @param appId appId
* @return AppDO
*/
AppDO getByAppId(String appId);
/**
* 查找所有
* @return List<TopicDO>
*/
List<AppDO> listAll();
List<AppTopicDTO> getAppTopicDTOList(String appId, Boolean mine);
boolean verifyAppIdByPassword(String appId, String password);
}

View File

@@ -0,0 +1,63 @@
package com.xiaojukeji.kafka.manager.service.service.gateway;
import com.xiaojukeji.kafka.manager.common.entity.ao.gateway.TopicQuota;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.AuthorityDO;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import java.util.List;
import java.util.Map;
/**
* @author zhongyuankai
* @date 2020/4/28
*/
public interface AuthorityService {
/**
* 添加权限
* @param authorityDO authorityDO
* @return int
*/
int addAuthority(AuthorityDO authorityDO);
/**
* 删除指定权限
* @param appId 应用ID
* @param clusterId 集群ID
* @param topicName Topic名称
* @param access 权限
* @return int
*/
ResultStatus deleteSpecifiedAccess(String appId, Long clusterId, String topicName, Integer access, String operator);
/**
* 获取权限
* @param clusterId 集群id
* @param topicName topic名称
* @param appId 应用id
* @return AuthorityDO
*/
AuthorityDO getAuthority(Long clusterId, String topicName, String appId);
/**
* 获取权限
* @param clusterId 集群id
* @param topicName topic名称
* @return List<AuthorityDO>
*/
List<AuthorityDO> getAuthorityByTopic(Long clusterId, String topicName);
List<AuthorityDO> getAuthority(String appId);
/**
* 查找所有
* @return List<TopicDO>
*/
List<AuthorityDO> listAll();
/**
* 添加权限和quota
*/
int addAuthorityAndQuota(AuthorityDO authorityDO, TopicQuota quota);
Map<String, Map<Long, Map<String, AuthorityDO>>> getAllAuthority();
}

View File

@@ -0,0 +1,18 @@
package com.xiaojukeji.kafka.manager.service.service.gateway;
import com.xiaojukeji.kafka.manager.common.entity.ao.gateway.*;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.GatewayConfigDO;
public interface GatewayConfigService {
KafkaBootstrapServerConfig getKafkaBootstrapServersConfig(Long requestVersion);
RequestQueueConfig getRequestQueueConfig(Long requestVersion);
AppRateConfig getAppRateConfig(Long requestVersion);
IpRateConfig getIpRateConfig(Long requestVersion);
SpRateConfig getSpRateConfig(Long requestVersion);
GatewayConfigDO getByTypeAndName(String configType, String configName);
}

View File

@@ -0,0 +1,37 @@
package com.xiaojukeji.kafka.manager.service.service.gateway;
import com.xiaojukeji.kafka.manager.common.entity.ao.gateway.TopicQuota;
/**
* @author zengqiao
* @date 20/4/28
*/
public interface QuotaService {
/**
* 添加quota
* @param topicQuotaDO topicQuotaDO
* @return int
*/
int addTopicQuota(TopicQuota topicQuotaDO);
/**
* 添加quota
* @param topicQuotaDO topicQuotaDO
* @param access 权限
* @return int
*/
int addTopicQuota(TopicQuota topicQuotaDO, Integer access);
/**
* 从ZK读取Quota
* @param clusterId 集群ID
* @param topicName Topic名称
* @param appId AppID
* @author zengqiao
* @date 20/5/12
* @return TopicQuotaDO
*/
TopicQuota getQuotaFromZk(Long clusterId, String topicName, String appId);
Boolean modifyProduceQuota(Long clusterId, String topicName, String appId, Long produceQuota);
}

View File

@@ -0,0 +1,16 @@
package com.xiaojukeji.kafka.manager.service.service.gateway;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.KafkaAclDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.KafkaUserDO;
import java.util.List;
/**
* @author zengqiao
* @date 20/7/27
*/
public interface SecurityService {
List<KafkaUserDO> getKafkaUsers(Long startTime, Long endTime);
List<KafkaAclDO> getKafkaAcls(Long clusterId, Long startTime, Long endTime);
}

View File

@@ -0,0 +1,56 @@
package com.xiaojukeji.kafka.manager.service.service.gateway;
import com.xiaojukeji.kafka.manager.common.entity.ao.topic.TopicConnection;
import com.xiaojukeji.kafka.manager.common.entity.dto.gateway.TopicConnectionDTO;
import java.util.Date;
import java.util.List;
/**
* @author zhongyuankai
* @date 20/4/13
*/
public interface TopicConnectionService {
int batchAdd(List<TopicConnectionDTO> dtoList);
/**
* 查询连接信息
*/
List<TopicConnection> getByTopicName(Long clusterId,
String topicName,
Date startTime,
Date endTime);
/**
* 查询连接信息
*/
List<TopicConnection> getByTopicName(Long clusterId,
String topicName,
String appId,
Date startTime,
Date endTime);
/**
* 查询连接信息
*/
List<TopicConnection> getByAppId(String appId,
Date startTime,
Date endTime);
/**
* 判断topic是否存在连接
*/
boolean isExistConnection(Long clusterId,
String topicName,
Date startTime,
Date endTime);
/**
* 判断app是否对topic存在连接
*/
boolean isExistConnection(Long clusterId,
String topicName,
String appId,
Date startTime,
Date endTime);
}

View File

@@ -0,0 +1,13 @@
package com.xiaojukeji.kafka.manager.service.service.gateway;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.TopicReportDO;
import java.util.List;
/**
* @author zengqiao
* @date 20/7/29
*/
public interface TopicReportService {
List<TopicReportDO> getNeedReportTopic(Long clusterId);
}

View File

@@ -0,0 +1,245 @@
package com.xiaojukeji.kafka.manager.service.service.gateway.impl;
import com.alibaba.fastjson.JSONObject;
import com.xiaojukeji.kafka.manager.common.bizenum.ModuleEnum;
import com.xiaojukeji.kafka.manager.common.bizenum.OperateEnum;
import com.xiaojukeji.kafka.manager.common.bizenum.OperationStatusEnum;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.AppTopicDTO;
import com.xiaojukeji.kafka.manager.common.entity.dto.normal.AppDTO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.OperateRecordDO;
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.entity.pojo.gateway.KafkaUserDO;
import com.xiaojukeji.kafka.manager.common.utils.ListUtils;
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.TopicDO;
import com.xiaojukeji.kafka.manager.dao.gateway.AppDao;
import com.xiaojukeji.kafka.manager.dao.gateway.KafkaUserDao;
import com.xiaojukeji.kafka.manager.service.cache.LogicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.service.OperateRecordService;
import com.xiaojukeji.kafka.manager.service.service.gateway.AppService;
import com.xiaojukeji.kafka.manager.service.service.gateway.AuthorityService;
import com.xiaojukeji.kafka.manager.service.service.TopicManagerService;
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 java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* @author zhongyuankai
* @date 20/4/28
*/
@Service("appService")
public class AppServiceImpl implements AppService {
private static final Logger LOGGER = LoggerFactory.getLogger(AppServiceImpl.class);
@Autowired
private AppDao appDao;
@Autowired
private KafkaUserDao kafkaUserDao;
@Autowired
private AuthorityService authorityService;
@Autowired
private TopicManagerService topicManagerService;
@Autowired
private LogicalClusterMetadataManager logicClusterMetadataManager;
@Autowired
private OperateRecordService operateRecordService;
@Override
public ResultStatus addApp(AppDO appDO) {
try {
if (appDao.insert(appDO) < 1) {
return ResultStatus.MYSQL_ERROR;
}
KafkaUserDO kafkaUserDO = new KafkaUserDO();
kafkaUserDO.setAppId(appDO.getAppId());
kafkaUserDO.setPassword(appDO.getPassword());
kafkaUserDO.setOperation(OperationStatusEnum.CREATE.getCode());
kafkaUserDO.setUserType(0);
kafkaUserDao.insert(kafkaUserDO);
} catch (DuplicateKeyException e) {
return ResultStatus.RESOURCE_ALREADY_EXISTED;
} catch (Exception e) {
LOGGER.error("add app failed, appDO:{}.", appDO, e);
return ResultStatus.MYSQL_ERROR;
}
return ResultStatus.SUCCESS;
}
@Override
public int deleteApp(AppDO appDO, String operator) {
int result = 0;
try {
result = appDao.deleteByName(appDO.getName());
if (result < 1) {
return result;
}
KafkaUserDO kafkaUserDO = new KafkaUserDO();
kafkaUserDO.setAppId(appDO.getAppId());
kafkaUserDO.setOperation(OperationStatusEnum.DELETE.getCode());
kafkaUserDO.setUserType(0);
kafkaUserDO.setPassword(appDO.getPassword());
kafkaUserDao.insert(kafkaUserDO);
// 记录操作
Map<String, Object> content = new HashMap<>(1);
content.put("appId", appDO.getAppId());
OperateRecordDO operateRecordDO = new OperateRecordDO();
operateRecordDO.setModuleId(ModuleEnum.APP.getCode());
operateRecordDO.setOperateId(OperateEnum.DELETE.getCode());
operateRecordDO.setResource(appDO.getAppId());
operateRecordDO.setContent(JSONObject.toJSONString(content));
operateRecordDO.setOperator(operator);
operateRecordService.insert(operateRecordDO);
} catch (Exception e) {
LOGGER.error("delete app failed, appDO:{}.", appDO, e);
}
return result;
}
@Override
public AppDO getByName(String name) {
try {
return appDao.getByName(name);
} catch (Exception e) {
LOGGER.error("get app failed, name:{}.", name, e);
}
return null;
}
@Override
public ResultStatus updateByAppId(AppDTO dto, String operator, Boolean adminApi) {
try {
AppDO appDO = appDao.getByAppId(dto.getAppId());
if (ValidateUtils.isNull(appDO)) {
return ResultStatus.APP_NOT_EXIST;
}
if (!adminApi && !ListUtils.string2StrList(appDO.getPrincipals()).contains(operator)) {
return ResultStatus.USER_WITHOUT_AUTHORITY;
}
appDO.setName(dto.getName());
appDO.setPrincipals(dto.getPrincipals());
appDO.setDescription(dto.getDescription());
if (appDao.updateById(appDO) > 0) {
return ResultStatus.SUCCESS;
}
} catch (DuplicateKeyException e) {
return ResultStatus.RESOURCE_NAME_DUPLICATED;
} catch (Exception e) {
LOGGER.error("update app failed, dto:{}, operator:{}, adminApi:{}.", dto, operator, adminApi, e);
}
return ResultStatus.MYSQL_ERROR;
}
@Override
public List<AppDO> getByPrincipal(String principals) {
try {
return appDao.getByPrincipal(principals);
} catch (Exception e) {
LOGGER.error("get app list failed, principals:{}.", principals);
}
return new ArrayList<>();
}
@Override
public AppDO getByAppId(String appId) {
try {
return appDao.getByAppId(appId);
} catch (Exception e) {
LOGGER.error("get app failed, appId:{}.", appId, e);
}
return null;
}
@Override
public List<AppDO> listAll() {
return appDao.listAll();
}
@Override
public List<AppTopicDTO> getAppTopicDTOList(String appId,
Boolean mine) {
// 查询AppID
AppDO appDO = appDao.getByAppId(appId);
if (ValidateUtils.isNull(appDO)) {
return new ArrayList<>();
}
List<TopicDO> topicDOList = topicManagerService.listAll();
Map<Long, Map<String, TopicDO>> topicMap = new HashMap<>();
for (TopicDO topicDO: topicDOList) {
Map<String, TopicDO> subTopicMap = topicMap.getOrDefault(topicDO.getClusterId(), new HashMap<>());
subTopicMap.put(topicDO.getTopicName(), topicDO);
topicMap.put(topicDO.getClusterId(), subTopicMap);
}
// 查询AppID有权限的Topic
List<AuthorityDO> authorityDOList = authorityService.getAuthority(appId);
if (ValidateUtils.isEmptyList(authorityDOList)) {
return new ArrayList<>();
}
List<AppTopicDTO> dtoList = new ArrayList<>();
for (AuthorityDO authorityDO: authorityDOList) {
TopicDO topicDO = topicMap
.getOrDefault(authorityDO.getClusterId(), new HashMap<>())
.get(authorityDO.getTopicName());
if (Boolean.TRUE.equals(mine)
&& (ValidateUtils.isNull(topicDO) || !topicDO.getAppId().equals(appId))) {
continue;
}
if (Boolean.FALSE.equals(mine)
&& !ValidateUtils.isNull(topicDO)
&& topicDO.getAppId().equals(appId)) {
continue;
}
LogicalClusterDO logicalClusterDO = logicClusterMetadataManager.getTopicLogicalCluster(
authorityDO.getClusterId(),
authorityDO.getTopicName()
);
AppTopicDTO appTopicDTO = new AppTopicDTO();
if (!ValidateUtils.isNull(logicalClusterDO)) {
appTopicDTO.setPhysicalClusterId(logicalClusterDO.getClusterId());
appTopicDTO.setLogicalClusterId(logicalClusterDO.getId());
appTopicDTO.setLogicalClusterName(logicalClusterDO.getName());
} else {
continue;
}
appTopicDTO.setOperator("");
appTopicDTO.setTopicName(authorityDO.getTopicName());
appTopicDTO.setAccess(authorityDO.getAccess());
appTopicDTO.setGmtCreate(authorityDO.getCreateTime().getTime());
dtoList.add(appTopicDTO);
}
return dtoList;
}
@Override
public boolean verifyAppIdByPassword(String appId, String password) {
if (ValidateUtils.isBlank(appId) || ValidateUtils.isBlank(password)) {
return false;
}
AppDO appDO = getByAppId(appId);
if (ValidateUtils.isNull(appDO) || !password.equals(appDO.getPassword())) {
return false;
}
return true;
}
}

View File

@@ -0,0 +1,195 @@
package com.xiaojukeji.kafka.manager.service.service.gateway.impl;
import com.alibaba.fastjson.JSONObject;
import com.xiaojukeji.kafka.manager.common.bizenum.ModuleEnum;
import com.xiaojukeji.kafka.manager.common.bizenum.OperateEnum;
import com.xiaojukeji.kafka.manager.common.bizenum.OperationStatusEnum;
import com.xiaojukeji.kafka.manager.common.entity.pojo.OperateRecordDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.AuthorityDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.KafkaAclDO;
import com.xiaojukeji.kafka.manager.dao.gateway.AuthorityDao;
import com.xiaojukeji.kafka.manager.common.entity.ao.gateway.TopicQuota;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.dao.gateway.KafkaAclDao;
import com.xiaojukeji.kafka.manager.service.service.OperateRecordService;
import com.xiaojukeji.kafka.manager.service.service.gateway.AuthorityService;
import com.xiaojukeji.kafka.manager.service.service.gateway.QuotaService;
import org.slf4j.Logger;
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;
/**
* @author zhongyuankai
* @date 20/4/28
*/
@Service("authorityService")
public class AuthorityServiceImpl implements AuthorityService {
private static final Logger LOGGER = LoggerFactory.getLogger(AuthorityServiceImpl.class);
@Autowired
private AuthorityDao authorityDao;
@Autowired
private KafkaAclDao kafkaAclDao;
@Autowired
private QuotaService quotaService;
@Autowired
private OperateRecordService operateRecordService;
@Override
public int addAuthority(AuthorityDO authorityDO) {
int result = 0;
Integer newAccess = authorityDO.getAccess();
try {
// 权限只会增加, 不会减少, 这里做了新旧权限的merge
AuthorityDO originAuthority = getAuthority(
authorityDO.getClusterId(),
authorityDO.getTopicName(),
authorityDO.getAppId()
);
if (!ValidateUtils.isNull(originAuthority)) {
newAccess |= originAuthority.getAccess();
authorityDO.setAccess(newAccess);
if (newAccess.equals(originAuthority.getAccess())) {
// 新旧权限一致, 不需要做任何调整
return result;
}
}
if (authorityDao.insert(authorityDO) < 1) {
return result;
}
KafkaAclDO kafkaAclDO = new KafkaAclDO();
kafkaAclDO.setTopicName(authorityDO.getTopicName());
kafkaAclDO.setClusterId(authorityDO.getClusterId());
kafkaAclDO.setAppId(authorityDO.getAppId());
kafkaAclDO.setAccess(authorityDO.getAccess());
kafkaAclDO.setOperation(OperationStatusEnum.CREATE.getCode());
return kafkaAclDao.insert(kafkaAclDO);
} catch (Exception e) {
LOGGER.error("add authority failed, authorityDO:{}.", authorityDO, e);
}
return result;
}
@Override
public ResultStatus deleteSpecifiedAccess(String appId, Long clusterId, String topicName, Integer access, String operator) {
AuthorityDO authorityDO = getAuthority(clusterId, topicName, appId);
if (ValidateUtils.isNull(authorityDO)) {
return ResultStatus.AUTHORITY_NOT_EXIST;
}
if ((authorityDO.getAccess() & access) != access) {
// 并不具备所要删除的权限, 返回错误
return ResultStatus.PARAM_ILLEGAL;
}
int newAccess = authorityDO.getAccess() ^ access;
authorityDO.setAccess(newAccess);
try {
if (authorityDao.insert(authorityDO) < 1) {
return ResultStatus.OPERATION_FAILED;
}
// kafka_acl表, 删除权限时, 只需要存储所要删除的权限, 不需要存储权限的终态或者什么的
KafkaAclDO kafkaAclDO = new KafkaAclDO();
kafkaAclDO.setOperation(OperationStatusEnum.DELETE.getCode());
kafkaAclDO.setAccess(access);
kafkaAclDO.setAppId(appId);
kafkaAclDO.setClusterId(clusterId);
kafkaAclDO.setTopicName(topicName);
if (kafkaAclDao.insert(kafkaAclDO) < 1) {
return ResultStatus.OPERATION_FAILED;
}
// 记录操作
Map<String, Object> content = new HashMap<>(4);
content.put("clusterId", clusterId);
content.put("topicName", topicName);
content.put("access", access);
content.put("appId", appId);
OperateRecordDO operateRecordDO = new OperateRecordDO();
operateRecordDO.setModuleId(ModuleEnum.AUTHORITY.getCode());
operateRecordDO.setOperateId(OperateEnum.DELETE.getCode());
operateRecordDO.setResource(topicName);
operateRecordDO.setContent(JSONObject.toJSONString(content));
operateRecordDO.setOperator(operator);
operateRecordService.insert(operateRecordDO);
} catch (Exception e) {
LOGGER.error("delete authority failed, authorityDO:{}.", authorityDO, e);
}
return ResultStatus.SUCCESS;
}
@Override
public AuthorityDO getAuthority(Long clusterId, String topicName, String appId) {
List<AuthorityDO> authorityDOList = null;
try {
authorityDOList = authorityDao.getAuthority(clusterId, topicName, appId);
} catch (Exception e) {
LOGGER.error("get authority failed, clusterId:{}, topicName:{}, appId:{}.", clusterId, topicName, appId, e);
}
if (ValidateUtils.isEmptyList(authorityDOList)) {
return null;
}
return authorityDOList.get(0);
}
@Override
public List<AuthorityDO> getAuthorityByTopic(Long clusterId, String topicName) {
try {
return authorityDao.getAuthorityByTopic(clusterId, topicName);
} catch (Exception e) {
LOGGER.error("get authority failed, clusterId:{} topicName:{}.", clusterId, topicName, e);
}
return null;
}
@Override
public List<AuthorityDO> getAuthority(String appId) {
List<AuthorityDO> doList = null;
try {
doList = authorityDao.getByAppId(appId);
} catch (Exception e) {
LOGGER.error("get authority failed, appId:{}.", appId, e);
}
if (ValidateUtils.isEmptyList(doList)) {
return new ArrayList<>();
}
return doList;
}
@Override
public List<AuthorityDO> listAll() {
return authorityDao.listAll();
}
@Override
public int addAuthorityAndQuota(AuthorityDO authorityDO, TopicQuota topicQuotaDO) {
int result = 0;
try {
result = addAuthority(authorityDO);
if (result < 1) {
return result;
}
return quotaService.addTopicQuota(topicQuotaDO, authorityDO.getAccess());
} catch (Exception e) {
LOGGER.error("add authority and quota failed, authorityDO:{} topicQuotaDO:{}.",
authorityDO, topicQuotaDO, e);
return result;
}
}
@Override
public Map<String, Map<Long, Map<String, AuthorityDO>>> getAllAuthority() {
return authorityDao.getAllAuthority();
}
}

View File

@@ -0,0 +1,155 @@
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.ao.gateway.*;
import com.xiaojukeji.kafka.manager.common.utils.ListUtils;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.dao.gateway.GatewayConfigDao;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.GatewayConfigDO;
import com.xiaojukeji.kafka.manager.service.service.gateway.GatewayConfigService;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* @author zengqiao
* @date 20/7/28
*/
@Service("gatewayConfigService")
public class GatewayConfigServiceImpl implements GatewayConfigService {
private final Logger LOGGER = LoggerFactory.getLogger(GatewayConfigServiceImpl.class);
@Autowired
private GatewayConfigDao gatewayConfigDao;
@Override
public KafkaBootstrapServerConfig getKafkaBootstrapServersConfig(Long requestVersion) {
List<GatewayConfigDO> doList = null;
try {
doList = gatewayConfigDao.getByConfigType(GatewayConfigKeyEnum.SD_CLUSTER_ID.getConfigType());
if (ValidateUtils.isEmptyList(doList)) {
return new KafkaBootstrapServerConfig(Long.MIN_VALUE, new HashMap<>(0));
}
Long maxVersion = Long.MIN_VALUE;
Map<Long, List<String>> clusterIdBootstrapServersMap = new HashMap<>(doList.size());
for (GatewayConfigDO configDO: doList) {
clusterIdBootstrapServersMap.put(
Long.valueOf(configDO.getName()),
ListUtils.string2StrList(configDO.getValue())
);
if (configDO.getVersion().compareTo(maxVersion) > 0) {
maxVersion = configDO.getVersion();
}
}
return maxVersion > requestVersion
? 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);
}
return null;
}
@Override
public RequestQueueConfig getRequestQueueConfig(Long requestVersion) {
GatewayConfigDO configDO = null;
try {
configDO = gatewayConfigDao.getByConfigTypeAndName(
GatewayConfigKeyEnum.SD_QUEUE_SIZE.getConfigType(),
GatewayConfigKeyEnum.SD_QUEUE_SIZE.getConfigName()
);
if (ValidateUtils.isNull(configDO) || configDO.getVersion() <= requestVersion) {
return new RequestQueueConfig(Long.MIN_VALUE, null);
}
return new RequestQueueConfig(configDO.getVersion(), Long.valueOf(configDO.getValue()));
} catch (Exception e) {
LOGGER.error("get request queue config failed, data:{}.", JSON.toJSONString(configDO), e);
}
return null;
}
@Override
public AppRateConfig getAppRateConfig(Long requestVersion) {
GatewayConfigDO configDO = null;
try {
configDO = gatewayConfigDao.getByConfigTypeAndName(
GatewayConfigKeyEnum.SD_APP_ID_RATE.getConfigType(),
GatewayConfigKeyEnum.SD_APP_ID_RATE.getConfigName()
);
if (ValidateUtils.isNull(configDO) || configDO.getVersion() <= requestVersion) {
return new AppRateConfig(Long.MIN_VALUE, null);
}
return new AppRateConfig(configDO.getVersion(), Long.valueOf(configDO.getValue()));
} catch (Exception e) {
LOGGER.error("get app rate config failed, data:{}.", JSON.toJSONString(configDO), e);
}
return null;
}
@Override
public IpRateConfig getIpRateConfig(Long requestVersion) {
GatewayConfigDO configDO = null;
try {
configDO = gatewayConfigDao.getByConfigTypeAndName(
GatewayConfigKeyEnum.SD_IP_RATE.getConfigType(),
GatewayConfigKeyEnum.SD_IP_RATE.getConfigName()
);
if (ValidateUtils.isNull(configDO) || configDO.getVersion() <= requestVersion) {
return new IpRateConfig(Long.MIN_VALUE, null);
}
return new IpRateConfig(configDO.getVersion(), Long.valueOf(configDO.getValue()));
} catch (Exception e) {
LOGGER.error("get ip rate config failed, data:{}.", JSON.toJSONString(configDO), e);
}
return null;
}
@Override
public SpRateConfig getSpRateConfig(Long requestVersion) {
List<GatewayConfigDO> doList = null;
try {
doList = gatewayConfigDao.getByConfigType(GatewayConfigKeyEnum.SD_SP_RATE.getConfigType());
if (ValidateUtils.isEmptyList(doList)) {
return new SpRateConfig(Long.MIN_VALUE, new HashMap<>(0));
}
Long maxVersion = Long.MIN_VALUE;
Map<String, Long> spRateMap = new HashMap<>(doList.size());
for (GatewayConfigDO configDO: doList) {
spRateMap.put(
configDO.getName(),
Long.valueOf(configDO.getValue())
);
if (configDO.getVersion().compareTo(maxVersion) > 0) {
maxVersion = configDO.getVersion();
}
}
return maxVersion > requestVersion
? new SpRateConfig(maxVersion, spRateMap)
: new SpRateConfig(requestVersion, new HashMap<>(0));
} catch (Exception e) {
LOGGER.error("get sp rate config failed, data:{}.", JSON.toJSONString(doList), e);
}
return null;
}
@Override
public GatewayConfigDO getByTypeAndName(String configType, String configName) {
try {
return gatewayConfigDao.getByConfigTypeAndName(configType, configName);
} catch (Exception e) {
LOGGER.error("get gateway config failed, configType:{} configName:{}.", configType, configName, e);
}
return null;
}
}

View File

@@ -0,0 +1,81 @@
package com.xiaojukeji.kafka.manager.service.service.gateway.impl;
import com.xiaojukeji.kafka.manager.common.entity.ao.gateway.TopicQuota;
import com.xiaojukeji.kafka.manager.common.utils.NumberUtils;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.config.TopicQuotaData;
import com.xiaojukeji.kafka.manager.common.entity.pojo.TopicDO;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.service.gateway.QuotaService;
import com.xiaojukeji.kafka.manager.service.service.TopicManagerService;
import com.xiaojukeji.kafka.manager.service.strategy.AbstractAllocateQuotaStrategy;
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.stereotype.Service;
/**
* @author zengqiao
* @date 20/4/28
*/
@Service("quotaService")
public class QuotaServiceImpl implements QuotaService {
private static final Logger LOGGER = LoggerFactory.getLogger(QuotaServiceImpl.class);
@Autowired
private TopicManagerService topicManagerService;
@Autowired
private AbstractAllocateQuotaStrategy allocateQuotaStrategy;
@Override
public int addTopicQuota(TopicQuota topicQuotaDO) {
return KafkaZookeeperUtils.setTopicQuota(
PhysicalClusterMetadataManager.getZKConfig(topicQuotaDO.getClusterId()),
topicQuotaDO
) ? 1: 0;
}
@Override
public int addTopicQuota(TopicQuota topicQuota, Integer access) {
TopicDO topicDO = topicManagerService.getByTopicName(topicQuota.getClusterId(), topicQuota.getTopicName());
if (ValidateUtils.isNull(topicDO)) {
topicDO = new TopicDO();
}
TopicQuota newTopicQuota = allocateQuotaStrategy.getNewTopicQuota(topicQuota, access, topicDO.getPeakBytesIn());
return addTopicQuota(newTopicQuota);
}
@Override
public TopicQuota getQuotaFromZk(Long clusterId, String topicName, String appId) {
TopicQuotaData quotaData = KafkaZookeeperUtils.getTopicQuota(
PhysicalClusterMetadataManager.getZKConfig(clusterId),
appId,
topicName
);
if (ValidateUtils.isNull(quotaData)) {
return null;
}
TopicQuota topicQuotaDO = new TopicQuota();
topicQuotaDO.setClusterId(clusterId);
topicQuotaDO.setTopicName(topicName);
topicQuotaDO.setAppId(appId);
topicQuotaDO.setConsumeQuota(NumberUtils.string2Long(quotaData.getConsumer_byte_rate()));
topicQuotaDO.setProduceQuota(NumberUtils.string2Long(quotaData.getProducer_byte_rate()));
return topicQuotaDO;
}
@Override
public Boolean modifyProduceQuota(Long clusterId, String topicName, String appId, Long produceQuota) {
TopicQuota topicQuotaDO = this.getQuotaFromZk(clusterId, topicName, appId);
if (ValidateUtils.isNull(topicQuotaDO)) {
return Boolean.FALSE;
}
topicQuotaDO.setProduceQuota(produceQuota);
if (this.addTopicQuota(topicQuotaDO) < 1) {
return Boolean.FALSE;
}
return Boolean.TRUE;
}
}

View File

@@ -0,0 +1,35 @@
package com.xiaojukeji.kafka.manager.service.service.gateway.impl;
import com.xiaojukeji.kafka.manager.dao.gateway.KafkaAclDao;
import com.xiaojukeji.kafka.manager.dao.gateway.KafkaUserDao;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.KafkaAclDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.KafkaUserDO;
import com.xiaojukeji.kafka.manager.service.service.gateway.SecurityService;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import java.util.Date;
import java.util.List;
/**
* @author zengqiao
* @date 20/7/27
*/
@Service("securityService")
public class SecurityServiceImpl implements SecurityService {
@Autowired
private KafkaUserDao kafkaUserDao;
@Autowired
private KafkaAclDao kafkaAclDao;
@Override
public List<KafkaUserDO> getKafkaUsers(Long startTime, Long endTime) {
return kafkaUserDao.getKafkaUsers(new Date(startTime), new Date(endTime));
}
@Override
public List<KafkaAclDO> getKafkaAcls(Long clusterId, Long startTime, Long endTime) {
return kafkaAclDao.getKafkaAcls(clusterId, new Date(startTime), new Date(endTime));
}
}

View File

@@ -0,0 +1,224 @@
package com.xiaojukeji.kafka.manager.service.service.gateway.impl;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.TopicConnectionDO;
import com.xiaojukeji.kafka.manager.common.entity.ao.topic.TopicConnection;
import com.xiaojukeji.kafka.manager.common.entity.dto.gateway.TopicConnectionDTO;
import com.xiaojukeji.kafka.manager.common.constant.KafkaConstant;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.dao.gateway.TopicConnectionDao;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.service.gateway.TopicConnectionService;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import java.net.InetAddress;
import java.util.*;
import java.util.stream.Collectors;
/**
* @author zyk
*/
@Service("topicConnectionService")
public class TopicConnectionServiceImpl implements TopicConnectionService {
private final static Logger LOGGER = LoggerFactory.getLogger(TopicConnectionServiceImpl.class);
@Autowired
private TopicConnectionDao topicConnectionDao;
@Override
public int batchAdd(List<TopicConnectionDTO> dtoList) {
if (ValidateUtils.isEmptyList(dtoList)) {
return 0;
}
int count = 0;
for (TopicConnectionDTO dto: dtoList) {
try {
TopicConnectionDO topicConnectionDO = new TopicConnectionDO();
topicConnectionDO.setClusterId(dto.getClusterId());
topicConnectionDO.setTopicName(dto.getTopicName());
topicConnectionDO.setType(dto.getType());
topicConnectionDO.setAppId(dto.getAppId());
topicConnectionDO.setIp(dto.getIp());
topicConnectionDO.setClientVersion(dto.getClientVersion());
count += topicConnectionDao.replace(topicConnectionDO);
} catch (Exception e) {
LOGGER.error("replace topic connections failed, data:{}.", dto);
}
}
return count;
}
@Override
public List<TopicConnection> getByTopicName(Long clusterId,
String topicName,
Date startTime,
Date endTime) {
List<TopicConnectionDO> doList = null;
try {
doList = topicConnectionDao.getByTopicName(clusterId, topicName, startTime, endTime);
} catch (Exception e) {
LOGGER.error("get topic connections failed, clusterId:{} topicName:{}.", clusterId, topicName, e);
}
if (ValidateUtils.isEmptyList(doList)) {
return new ArrayList<>();
}
return getByTopicName(clusterId, doList);
}
@Override
public List<TopicConnection> getByTopicName(Long clusterId,
String topicName,
String appId,
Date startTime,
Date endTime) {
List<TopicConnectionDO> doList = null;
try {
doList = topicConnectionDao.getByTopicName(clusterId, topicName, startTime, endTime);
} catch (Exception e) {
LOGGER.error("get topic connections failed, clusterId:{} topicName:{}.", clusterId, topicName, e);
}
if (ValidateUtils.isEmptyList(doList)) {
return new ArrayList<>();
}
return getByTopicName(
clusterId,
doList.stream().filter(elem -> elem.getAppId().equals(appId)).collect(Collectors.toList())
);
}
@Override
public List<TopicConnection> getByAppId(String appId, Date startTime, Date endTime) {
List<TopicConnectionDO> doList = null;
try {
doList = topicConnectionDao.getByAppId(appId, startTime, endTime);
} catch (Exception e) {
LOGGER.error("get topic connections failed, appId:{} .", appId, e);
}
if (ValidateUtils.isEmptyList(doList)) {
return new ArrayList<>();
}
return getByTopicName(null, doList);
}
@Override
public boolean isExistConnection(Long clusterId,
String topicName,
Date startTime,
Date endTime) {
List<TopicConnection> connectionList = this.getByTopicName(
clusterId,
topicName,
startTime,
endTime);
if (!ValidateUtils.isEmptyList(connectionList)) {
return true;
}
return false;
}
@Override
public boolean isExistConnection(Long clusterId,
String topicName,
String appId,
Date startTime,
Date endTime) {
List<TopicConnection> connectionList = this.getByTopicName(
clusterId,
topicName,
appId,
startTime,
endTime);
if (!ValidateUtils.isEmptyList(connectionList)) {
return true;
}
return false;
}
private List<TopicConnection> getByTopicName(Long clusterId, List<TopicConnectionDO> doList) {
if (ValidateUtils.isEmptyList(doList)) {
return new ArrayList<>();
}
Set<String> brokerHostnameSet = new HashSet<>();
if (!ValidateUtils.isNull(clusterId)) {
brokerHostnameSet = PhysicalClusterMetadataManager.getBrokerHostnameSet(clusterId);
}
// 去重复及版本信息无效的数据
Map<String, List<TopicConnection>> dtoListMap = new HashMap<>();
for (TopicConnectionDO connectionDO: doList) {
//如果存在其他版本信息, 则过滤掉版本为-1的用户
String deDuplicateKey = connectionDO.uniqueKey();
List<TopicConnection> dtoList = dtoListMap.getOrDefault(deDuplicateKey, new ArrayList<>());
if (!dtoList.isEmpty() && (
KafkaConstant.CLIENT_VERSION_CODE_UNKNOWN.equals(connectionDO.getClientVersion())
|| KafkaConstant.CLIENT_VERSION_NAME_UNKNOWN.equals(connectionDO.getClientVersion()))
) {
// 非空 && connectionDO是Version_UNKNOWN
continue;
}
if (!dtoList.isEmpty() && (
KafkaConstant.CLIENT_VERSION_NAME_UNKNOWN.equals(dtoList.get(0).getClientVersion())
|| KafkaConstant.CLIENT_VERSION_CODE_UNKNOWN.equals(dtoList.get(0).getClientVersion())
)) {
// 非空 && dtoList里面是Version_UNKNOWN
dtoList.remove(0);
}
TopicConnection dto = convert2TopicConnectionDTO(connectionDO);
// 过滤掉broker的机器
if (brokerHostnameSet.contains(dto.getHostname()) || brokerHostnameSet.contains(dto.getIp())) {
// 发现消费的机器是broker, 则直接跳过. brokerHostnameSet有的集群存储的是IP
continue;
}
dtoList.add(dto);
dtoListMap.put(deDuplicateKey, dtoList);
}
List<TopicConnection> dtoList = new ArrayList<>();
for (Map.Entry<String, List<TopicConnection>> entry: dtoListMap.entrySet()) {
dtoList.addAll(entry.getValue());
}
return dtoList;
}
private TopicConnection convert2TopicConnectionDTO(TopicConnectionDO connectionDO) {
TopicConnection dto = new TopicConnection();
dto.setClusterId(connectionDO.getClusterId());
dto.setTopicName(connectionDO.getTopicName());
switch (connectionDO.getType()) {
case "produce": dto.setClientType("producer"); break;
case "fetch": dto.setClientType("consumer"); break;
default: dto.setClientType("");
}
dto.setAppId(connectionDO.getAppId());
dto.setClientVersion(
connectionDO.getClientVersion().equals(KafkaConstant.CLIENT_VERSION_CODE_UNKNOWN) ?
KafkaConstant.CLIENT_VERSION_NAME_UNKNOWN : connectionDO.getClientVersion()
);
dto.setIp(connectionDO.getIp());
String hostName = connectionDO.getIp();
try {
InetAddress ia = InetAddress.getByAddress(getIpBytes(connectionDO.getIp()));
hostName = ia.getHostName();
} catch (Exception e) {
LOGGER.error("get hostname failed. ip:{}.", connectionDO.getIp(), e);
}
dto.setHostname(hostName.replace(KafkaConstant.BROKER_HOST_NAME_SUFFIX, ""));
return dto;
}
private byte[] getIpBytes(String ip) {
String[] ipStr=ip.split("[.]");
byte[] ipBytes=new byte[4];
for (int i = 0; i < 4; i++) {
int m=Integer.parseInt(ipStr[i]);
byte b=(byte)(m&0xff);
ipBytes[i]=b;
}
return ipBytes;
}
}

View File

@@ -0,0 +1,24 @@
package com.xiaojukeji.kafka.manager.service.service.gateway.impl;
import com.xiaojukeji.kafka.manager.dao.gateway.TopicReportDao;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.TopicReportDO;
import com.xiaojukeji.kafka.manager.service.service.gateway.TopicReportService;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import java.util.List;
/**
* @author zengqiao
* @date 20/7/29
*/
@Service("topicReportService")
public class TopicReportServiceImpl implements TopicReportService {
@Autowired
private TopicReportDao topicReportDao;
@Override
public List<TopicReportDO> getNeedReportTopic(Long clusterId) {
return topicReportDao.getNeedReportTopic(clusterId);
}
}

View File

@@ -0,0 +1,305 @@
package com.xiaojukeji.kafka.manager.service.service.impl;
import com.alibaba.fastjson.JSON;
import com.alibaba.fastjson.JSONObject;
import com.xiaojukeji.kafka.manager.common.bizenum.*;
import com.xiaojukeji.kafka.manager.common.entity.pojo.OperateRecordDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.AuthorityDO;
import com.xiaojukeji.kafka.manager.common.entity.ao.gateway.TopicQuota;
import com.xiaojukeji.kafka.manager.common.constant.Constant;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
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.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.service.*;
import com.xiaojukeji.kafka.manager.service.service.gateway.AuthorityService;
import com.xiaojukeji.kafka.manager.service.utils.KafkaZookeeperUtils;
import com.xiaojukeji.kafka.manager.service.utils.TopicCommands;
import kafka.admin.AdminOperationException;
import kafka.admin.PreferredReplicaLeaderElectionCommand;
import kafka.utils.ZkUtils;
import org.apache.kafka.common.security.JaasUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import java.util.*;
/**
* @author zengqiao
* @date 2019/11/26.
*/
@Service("adminService")
public class AdminServiceImpl implements AdminService {
private static final Logger LOGGER = LoggerFactory.getLogger(AdminServiceImpl.class);
private static final Long DEFAULT_DEAD_BROKER_LIMIT_NUM = 1L;
@Autowired
private TopicManagerService topicManagerService;
@Autowired
private TopicService topicService;
@Autowired
private RegionService regionService;
@Autowired
private AuthorityService authorityService;
@Autowired
private OperateRecordService operateRecordService;
@Override
public ResultStatus createTopic(ClusterDO clusterDO,
TopicDO topicDO,
Integer partitionNum,
Integer replicaNum,
Long regionId,
List<Integer> brokerIdList,
Properties properties,
String applicant,
String operator) {
List<Integer> fullBrokerIdList = regionService.getFullBrokerIdList(clusterDO.getId(), regionId, brokerIdList);
if (PhysicalClusterMetadataManager.getNotAliveBrokerNum(clusterDO.getId(), fullBrokerIdList) > DEFAULT_DEAD_BROKER_LIMIT_NUM) {
return ResultStatus.BROKER_NOT_EXIST;
}
// step1 创建Topic
ResultStatus rs = TopicCommands.createTopic(
clusterDO,
topicDO.getTopicName(),
partitionNum,
replicaNum,
fullBrokerIdList,
properties
);
if (!ResultStatus.SUCCESS.equals(rs)) {
// 创建失败
return rs;
}
// step2 记录操作
Map<String, Object> content = new HashMap<>(4);
content.put("clusterId", clusterDO.getId());
content.put("topicName", topicDO.getTopicName());
content.put("replicaNum", replicaNum);
content.put("partitionNum", partitionNum);
OperateRecordDO operateRecordDO = new OperateRecordDO();
operateRecordDO.setModuleId(ModuleEnum.TOPIC.getCode());
operateRecordDO.setOperateId(OperateEnum.ADD.getCode());
operateRecordDO.setResource(topicDO.getTopicName());
operateRecordDO.setContent(JSONObject.toJSONString(content));
operateRecordDO.setOperator(operator);
operateRecordService.insert(operateRecordDO);
// step3 TopicDO写DB
topicManagerService.addTopic(topicDO);
// step4 添加权限及配额
AuthorityDO authority = new AuthorityDO();
authority.setClusterId(topicDO.getClusterId());
authority.setTopicName(topicDO.getTopicName());
authority.setAppId(topicDO.getAppId());
authority.setAccess(TopicAuthorityEnum.READ_WRITE.getCode());
// authority.setApplicant(applicant);
TopicQuota topicQuotaDO = new TopicQuota();
topicQuotaDO.setClusterId(topicDO.getClusterId());
topicQuotaDO.setTopicName(topicDO.getTopicName());
topicQuotaDO.setAppId(topicDO.getAppId());
authorityService.addAuthorityAndQuota(authority, topicQuotaDO);
return ResultStatus.SUCCESS;
}
@Override
public ResultStatus deleteTopic(ClusterDO clusterDO,
String topicName,
String operator) {
// 1. 集群中删除topic
ResultStatus rs = TopicCommands.deleteTopic(clusterDO, topicName);
if (!ResultStatus.SUCCESS.equals(rs)) {
return rs;
}
// 2. 记录操作
Map<String, Object> content = new HashMap<>(2);
content.put("clusterId", clusterDO.getId());
content.put("topicName", topicName);
OperateRecordDO operateRecordDO = new OperateRecordDO();
operateRecordDO.setModuleId(ModuleEnum.TOPIC.getCode());
operateRecordDO.setOperateId(OperateEnum.DELETE.getCode());
operateRecordDO.setResource(topicName);
operateRecordDO.setContent(JSONObject.toJSONString(content));
operateRecordDO.setOperator(operator);
operateRecordService.insert(operateRecordDO);
// 3. 数据库中删除topic
topicManagerService.deleteByTopicName(clusterDO.getId(), topicName);
return rs;
}
@Override
public TaskStatusEnum preferredReplicaElectionStatus(ClusterDO clusterDO) {
ZkUtils zkUtils = null;
try {
zkUtils = ZkUtils.apply(clusterDO.getZookeeper(),
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
JaasUtils.isZkSecurityEnabled()
);
if (zkUtils.pathExists(ZkUtils.PreferredReplicaLeaderElectionPath())) {
return TaskStatusEnum.RUNNING;
}
} catch (Exception e) {
return TaskStatusEnum.UNKNOWN;
} finally {
if (null != zkUtils) {
zkUtils.close();
}
}
return TaskStatusEnum.SUCCEED;
}
@Override
public ResultStatus preferredReplicaElection(ClusterDO clusterDO, String operator) {
ZkUtils zkUtils = null;
try {
zkUtils = ZkUtils.apply(clusterDO.getZookeeper(),
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
JaasUtils.isZkSecurityEnabled()
);
PreferredReplicaLeaderElectionCommand command =
new PreferredReplicaLeaderElectionCommand(zkUtils, zkUtils.getAllPartitions());
command.moveLeaderToPreferredReplica();
} catch (AdminOperationException e) {
} catch (Throwable t) {
} finally {
if (null != zkUtils) {
zkUtils.close();
}
}
return ResultStatus.SUCCESS;
}
@Override
public ResultStatus preferredReplicaElection(ClusterDO clusterDO, Integer brokerId, String operator) {
BrokerMetadata brokerMetadata = PhysicalClusterMetadataManager.getBrokerMetadata(clusterDO.getId(), brokerId);
if (null == brokerMetadata) {
return ResultStatus.PARAM_ILLEGAL;
}
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(),
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
JaasUtils.isZkSecurityEnabled()
);
PreferredReplicaLeaderElectionCommand preferredReplicaElectionCommand =
new PreferredReplicaLeaderElectionCommand(
zkUtils,
PreferredReplicaLeaderElectionCommand.parsePreferredReplicaElectionData(
preferredReplicaElectString
)
);
preferredReplicaElectionCommand.moveLeaderToPreferredReplica();
} catch (Exception e) {
return ResultStatus.OPERATION_FAILED;
} finally {
if (zkUtils != null) {
zkUtils.close();
}
}
return ResultStatus.SUCCESS;
}
@Override
public ResultStatus expandPartitions(ClusterDO clusterDO,
String topicName,
Integer partitionNum,
Long regionId,
List<Integer> brokerIdList,
String operator) {
List<Integer> fullBrokerIdList = regionService.getFullBrokerIdList(clusterDO.getId(), regionId, brokerIdList);
if (PhysicalClusterMetadataManager.getNotAliveBrokerNum(clusterDO.getId(), fullBrokerIdList) > DEFAULT_DEAD_BROKER_LIMIT_NUM) {
return ResultStatus.BROKER_NOT_EXIST;
}
ResultStatus resultStatus = TopicCommands.expandTopic(
clusterDO,
topicName,
partitionNum,
fullBrokerIdList
);
if (!ResultStatus.SUCCESS.equals(resultStatus)) {
return resultStatus;
}
//记录操作
Map<String, Object> content = new HashMap<>(2);
content.put("clusterId", clusterDO.getId());
content.put("topicName", topicName);
content.put("partitionNum", partitionNum);
content.put("regionId", regionId);
content.put("brokerIdList", brokerIdList);
OperateRecordDO operateRecordDO = new OperateRecordDO();
operateRecordDO.setModuleId(ModuleEnum.PARTITION.getCode());
operateRecordDO.setOperateId(OperateEnum.ADD.getCode());
operateRecordDO.setResource(topicName);
operateRecordDO.setContent(JSONObject.toJSONString(content));
operateRecordDO.setOperator(operator);
operateRecordService.insert(operateRecordDO);
return resultStatus;
}
private String convert2preferredReplicaElectString(Map<String, List<Integer>> topicNamePartitionIdMap) {
List<Map<String, Object>> metaList = new ArrayList<>();
for (Map.Entry<String, List<Integer>> entry : topicNamePartitionIdMap.entrySet()) {
if (entry.getValue() == null || entry.getValue().isEmpty()) {
continue;
}
for (Integer partitionId : entry.getValue()) {
Map<String, Object> params = new HashMap<>();
params.put("topic", entry.getKey());
params.put("partition", partitionId);
metaList.add(params);
}
}
Map<String, Object> result = new HashMap<>();
result.put("partitions", metaList);
return JSON.toJSONString(result);
}
@Override
public Properties getTopicConfig(ClusterDO clusterDO, String topicName) {
ZkConfigImpl zkConfig = PhysicalClusterMetadataManager.getZKConfig(clusterDO.getId());
if (ValidateUtils.isNull(zkConfig)) {
return null;
}
return KafkaZookeeperUtils.getTopicProperties(zkConfig, topicName);
}
@Override
public ResultStatus modifyTopicConfig(ClusterDO clusterDO, String topicName, Properties properties, String operator) {
ResultStatus rs = TopicCommands.modifyTopicConfig(clusterDO, topicName, properties);
if (!ResultStatus.SUCCESS.equals(rs)) {
return rs;
}
return rs;
}
}

View File

@@ -0,0 +1,163 @@
package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.common.constant.KafkaMetricsCollections;
import com.xiaojukeji.kafka.manager.common.entity.ao.analysis.AnalysisBrokerDTO;
import com.xiaojukeji.kafka.manager.common.entity.ao.analysis.AnalysisTopicDTO;
import com.xiaojukeji.kafka.manager.common.entity.metrics.BrokerMetrics;
import com.xiaojukeji.kafka.manager.common.entity.metrics.TopicMetrics;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata;
import com.xiaojukeji.kafka.manager.service.cache.KafkaMetricsCache;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.service.AnalysisService;
import com.xiaojukeji.kafka.manager.service.service.JmxService;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import java.util.*;
/**
* Topic抖动分析
* @author huangyiminghappy@163.com, zengqaio_cn@163.com
* @date 2019-06-14
*/
@Service("analysisService")
public class AnalysisServiceImpl implements AnalysisService {
private static final Logger logger = LoggerFactory.getLogger(AnalysisServiceImpl.class);
@Autowired
private JmxService jmxService;
private static final Integer TOP_TOPIC_NUM = 5;
private static final Integer MIN_TOP_TOPIC_BYTES_IN_VALUE = 100;
private static final Integer MIN_TOP_TOPIC_QPS_VALUE = 10;
@Override
public AnalysisBrokerDTO doAnalysisBroker(Long clusterId, Integer brokerId) {
AnalysisBrokerDTO analysisBrokerDTO = new AnalysisBrokerDTO();
analysisBrokerDTO.setClusterId(clusterId);
analysisBrokerDTO.setBrokerId(brokerId);
analysisBrokerDTO.setBaseTime(System.currentTimeMillis());
analysisBrokerDTO.setTopicAnalysisVOList(new ArrayList<>());
BrokerMetrics brokerMetrics = jmxService.getBrokerMetrics(clusterId, brokerId, KafkaMetricsCollections.BROKER_ANALYSIS_METRICS);
if (ValidateUtils.isNull(brokerMetrics)) {
return analysisBrokerDTO;
}
analysisBrokerDTO.setBytesIn(brokerMetrics.getBytesInPerSecOneMinuteRate(0.0));
analysisBrokerDTO.setBytesOut(brokerMetrics.getBytesOutPerSecOneMinuteRate(0.0));
analysisBrokerDTO.setMessagesIn(brokerMetrics.getMessagesInPerSecOneMinuteRate(0.0));
analysisBrokerDTO.setTotalProduceRequests(brokerMetrics.getTotalProduceRequestsPerSecOneMinuteRate(0.0));
analysisBrokerDTO.setTotalFetchRequests(brokerMetrics.getTotalFetchRequestsPerSecOneMinuteRate(0.0));
List<TopicMetrics> topicMetricsList = new ArrayList<>();
for (String topicName: PhysicalClusterMetadataManager.getTopicNameList(clusterId)) {
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterId, topicName);
if (ValidateUtils.isNull(topicMetadata) || !topicMetadata.getBrokerIdSet().contains(brokerId)) {
continue;
}
TopicMetrics topicMetrics = KafkaMetricsCache.getTopicMetricsFromCache(clusterId, topicName);
if (ValidateUtils.isNull(topicMetrics )) {
continue;
}
if (topicMetrics.getBytesInPerSecOneMinuteRate(0.0) < MIN_TOP_TOPIC_BYTES_IN_VALUE.doubleValue()
|| topicMetrics.getTotalProduceRequestsPerSecOneMinuteRate(0.0) < MIN_TOP_TOPIC_QPS_VALUE.doubleValue()) {
continue;
}
topicMetrics = jmxService.getTopicMetrics(
clusterId,
brokerId,
topicName,
KafkaMetricsCollections.BROKER_TOPIC_ANALYSIS_METRICS,
true
);
if (ValidateUtils.isNull(topicMetrics )) {
continue;
}
topicMetricsList.add(topicMetrics);
}
Set<String> topicNameSet = new HashSet<>();
supplyAnalysisTopicDTOList(analysisBrokerDTO, topicNameSet, topicMetricsList, "BytesIn");
supplyAnalysisTopicDTOList(analysisBrokerDTO, topicNameSet, topicMetricsList, "TotalProduceRequest");
return analysisBrokerDTO;
}
private void supplyAnalysisTopicDTOList(AnalysisBrokerDTO analysisBrokerDTO,
Set<String> topicNameSet,
List<TopicMetrics> topicMetricsList,
String fieldName) {
Collections.sort(topicMetricsList, new Comparator<TopicMetrics>() {
@Override
public int compare(TopicMetrics t1, TopicMetrics t2) {
double diff = 0;
switch (fieldName) {
case "BytesIn":
diff = t1.getBytesInPerSecOneMinuteRate(0.0) - t2.getBytesInPerSecOneMinuteRate(0.0);
break;
case "TotalProduceRequest":
diff = t1.getTotalProduceRequestsPerSecOneMinuteRate(0.0) - t2.getTotalProduceRequestsPerSecOneMinuteRate(0.0);
break;
default:
diff = 0;
break;
}
if (diff > 0) {
return -1;
} else if (diff < 0) {
return 1;
}
return t1.getTopicName().compareTo(t2.getTopicName());
}
});
for (int i = 0; i < TOP_TOPIC_NUM && i < topicMetricsList.size(); ++i) {
TopicMetrics topicMetrics = topicMetricsList.get(i);
if (topicNameSet.contains(topicMetrics.getTopicName())) {
continue;
}
AnalysisTopicDTO analysisTopicDTO = new AnalysisTopicDTO();
analysisTopicDTO.setTopicName(topicMetrics.getTopicName());
analysisTopicDTO.setBytesIn(topicMetrics.getBytesInPerSecOneMinuteRate(0.0));
if (analysisBrokerDTO.getBytesIn() <= 0) {
analysisTopicDTO.setBytesInRate(0.0);
} else {
analysisTopicDTO.setBytesInRate(topicMetrics.getBytesInPerSecOneMinuteRate(0.0) / analysisBrokerDTO.getBytesIn());
}
analysisTopicDTO.setBytesOut(topicMetrics.getBytesOutPerSecOneMinuteRate(0.0));
if (analysisBrokerDTO.getBytesOut() <= 0) {
analysisTopicDTO.setBytesOutRate(0.0);
} else {
analysisTopicDTO.setBytesOutRate(topicMetrics.getBytesOutPerSecOneMinuteRate(0.0) / analysisBrokerDTO.getBytesOut());
}
analysisTopicDTO.setMessagesIn(topicMetrics.getMessagesInPerSecOneMinuteRate(0.0));
if (analysisBrokerDTO.getMessagesIn() <= 0) {
analysisTopicDTO.setMessagesInRate(0.0);
} else {
analysisTopicDTO.setMessagesInRate(topicMetrics.getMessagesInPerSecOneMinuteRate(0.0) / analysisBrokerDTO.getMessagesIn());
}
analysisTopicDTO.setTotalFetchRequests(topicMetrics.getTotalFetchRequestsPerSecOneMinuteRate(0.0));
if (analysisBrokerDTO.getTotalFetchRequests() <= 0) {
analysisTopicDTO.setTotalFetchRequestsRate(0.0);
} else {
analysisTopicDTO.setTotalFetchRequestsRate(topicMetrics.getTotalFetchRequestsPerSecOneMinuteRate(0.0) / analysisBrokerDTO.getTotalFetchRequests());
}
analysisTopicDTO.setTotalProduceRequests(topicMetrics.getTotalProduceRequestsPerSecOneMinuteRate(0.0));
if (analysisBrokerDTO.getTotalProduceRequests() <= 0) {
analysisTopicDTO.setTotalProduceRequestsRate(0.0);
} else {
analysisTopicDTO.setTotalProduceRequestsRate(topicMetrics.getTotalProduceRequestsPerSecOneMinuteRate(0.0) / analysisBrokerDTO.getTotalProduceRequests());
}
topicNameSet.add(topicMetrics.getTopicName());
analysisBrokerDTO.getTopicAnalysisVOList().add(analysisTopicDTO);
}
}
}

View File

@@ -0,0 +1,404 @@
package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.common.bizenum.DBStatusEnum;
import com.xiaojukeji.kafka.manager.common.bizenum.PeakFlowStatusEnum;
import com.xiaojukeji.kafka.manager.common.constant.ConfigConstant;
import com.xiaojukeji.kafka.manager.common.constant.KafkaMetricsCollections;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.BrokerBasicDTO;
import com.xiaojukeji.kafka.manager.common.entity.ao.TopicDiskLocation;
import com.xiaojukeji.kafka.manager.common.entity.ao.cluster.ClusterBrokerStatus;
import com.xiaojukeji.kafka.manager.common.entity.metrics.BrokerMetrics;
import com.xiaojukeji.kafka.manager.common.entity.ao.BrokerOverviewDTO;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.PartitionState;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata;
import com.xiaojukeji.kafka.manager.common.entity.pojo.BrokerDO;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.BrokerMetadata;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.dao.BrokerMetricsDao;
import com.xiaojukeji.kafka.manager.dao.BrokerDao;
import com.xiaojukeji.kafka.manager.common.entity.pojo.BrokerMetricsDO;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.cache.ThreadPool;
import com.xiaojukeji.kafka.manager.service.service.BrokerService;
import com.xiaojukeji.kafka.manager.service.service.ConfigService;
import com.xiaojukeji.kafka.manager.service.service.JmxService;
import com.xiaojukeji.kafka.manager.service.service.TopicService;
import com.xiaojukeji.kafka.manager.service.utils.MetricsConvertUtils;
import org.apache.kafka.common.TopicPartition;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import java.util.*;
import java.util.concurrent.Callable;
import java.util.concurrent.FutureTask;
/**
* @author tukun, zengqiao
* @date 2015/11/9
*/
@Service("brokerService")
public class BrokerServiceImpl implements BrokerService {
private final static Logger LOGGER = LoggerFactory.getLogger(BrokerServiceImpl.class);
@Autowired
private BrokerDao brokerDao;
@Autowired
private JmxService jmxService;
@Autowired
private TopicService topicService;
@Autowired
private ConfigService configService;
@Autowired
private BrokerMetricsDao brokerMetricDao;
@Autowired
private PhysicalClusterMetadataManager physicalClusterMetadataManager;
@Override
public ClusterBrokerStatus getClusterBrokerStatus(Long clusterId) {
// 副本同步状态
Set<Integer> brokerIdSet = new HashSet<>(PhysicalClusterMetadataManager.getBrokerIdList(clusterId));
List<BrokerMetrics> metricsList = getBrokerMetricsFromJmx(
clusterId,
brokerIdSet,
KafkaMetricsCollections.BROKER_STATUS_PAGE_METRICS
);
int underBroker = 0;
for (BrokerMetrics brokerMetrics: metricsList) {
Integer underReplicated = brokerMetrics.getSpecifiedMetrics("UnderReplicatedPartitionsValue", Integer.class);
if (!ValidateUtils.isNull(underReplicated) && underReplicated > 0) {
underBroker++;
}
}
List<Integer> brokerReplicaStatusList = Arrays.asList(
brokerIdSet.size(),
brokerIdSet.size() - underBroker,
underBroker
);
Map<Integer, Integer> peakFlowStatusMap = new HashMap<>();
// 峰值状态
List<BrokerDO> brokerDOList = brokerDao.getByClusterId(clusterId);
Long peakFlow = configService.getLongValue(
ConfigConstant.BROKER_CAPACITY_LIMIT_CONFIG_KEY, ConfigConstant.DEFAULT_BROKER_CAPACITY_LIMIT);
for (BrokerDO brokerDO : brokerDOList) {
PeakFlowStatusEnum peakFlowStatus = getPeakFlowStatus(brokerDO.getMaxAvgBytesIn(), peakFlow);
peakFlowStatusMap.put(
peakFlowStatus.code,
peakFlowStatusMap.getOrDefault(peakFlowStatus.getCode(), 0) + 1
);
}
List<Integer> brokerBytesInStatusList = Arrays.asList(
brokerDOList.size(),
peakFlowStatusMap.getOrDefault(PeakFlowStatusEnum.BETWEEN_00_60.getCode(), 0),
peakFlowStatusMap.getOrDefault(PeakFlowStatusEnum.BETWEEN_60_80.getCode(), 0),
peakFlowStatusMap.getOrDefault(PeakFlowStatusEnum.BETWEEN_80_100.getCode(), 0),
peakFlowStatusMap.getOrDefault(PeakFlowStatusEnum.BETWEEN_100_PLUS.getCode(), 0),
peakFlowStatusMap.getOrDefault(PeakFlowStatusEnum.BETWEEN_EXCEPTION.getCode(), 0)
);
ClusterBrokerStatus clusterBrokerStatus = new ClusterBrokerStatus();
clusterBrokerStatus.setBrokerReplicaStatusList(brokerReplicaStatusList);
clusterBrokerStatus.setBrokerBytesInStatusList(brokerBytesInStatusList);
return clusterBrokerStatus;
}
private PeakFlowStatusEnum getPeakFlowStatus(Double maxAvgBytesIn, Long peakFlow) {
if (ValidateUtils.isNullOrLessThanZero(maxAvgBytesIn)) {
return PeakFlowStatusEnum.BETWEEN_EXCEPTION;
}
double rate = maxAvgBytesIn / peakFlow;
if (rate <= 0.6) {
return PeakFlowStatusEnum.BETWEEN_00_60;
} else if (rate <= 0.8) {
return PeakFlowStatusEnum.BETWEEN_60_80;
} else if (rate <= 1) {
return PeakFlowStatusEnum.BETWEEN_80_100;
} else {
return PeakFlowStatusEnum.BETWEEN_100_PLUS;
}
}
@Override
public List<BrokerOverviewDTO> getBrokerOverviewList(Long clusterId, Set<Integer> brokerIdSet) {
Boolean isGetAll = brokerIdSet == null? Boolean.TRUE: Boolean.FALSE;
if (isGetAll) {
brokerIdSet = new HashSet<>(PhysicalClusterMetadataManager.getBrokerIdList(clusterId));
}
List<BrokerMetrics> metricsList = getBrokerMetricsFromJmx(
clusterId,
brokerIdSet,
KafkaMetricsCollections.BROKER_OVERVIEW_PAGE_METRICS
);
if (ValidateUtils.isNull(metricsList)) {
metricsList = new ArrayList<>();
}
Map<Integer, BrokerMetrics> brokerMap = new HashMap<>(metricsList.size());
for (BrokerMetrics metrics: metricsList) {
brokerMap.put(metrics.getBrokerId(), metrics);
}
Map<Integer, BrokerOverviewDTO> overviewDTOMap = new HashMap<>();
for (Integer brokerId: brokerIdSet) {
BrokerMetadata brokerMetadata = PhysicalClusterMetadataManager.getBrokerMetadata(clusterId, brokerId);
if (ValidateUtils.isNull(brokerMetadata)) {
continue;
}
overviewDTOMap.put(brokerId, BrokerOverviewDTO.newInstance(
brokerMetadata,
brokerMap.get(brokerId),
physicalClusterMetadataManager.getKafkaVersionFromCache(clusterId, brokerId)
));
}
Long peakFlow = configService.getLongValue(
ConfigConstant.BROKER_CAPACITY_LIMIT_CONFIG_KEY, ConfigConstant.DEFAULT_BROKER_CAPACITY_LIMIT);
List<BrokerDO> brokerDOList = brokerDao.getByClusterId(clusterId);
for (BrokerDO brokerDO : brokerDOList) {
if ((!isGetAll && !brokerIdSet.contains(brokerDO))) {
continue;
}
if (overviewDTOMap.containsKey(brokerDO.getBrokerId())) {
BrokerOverviewDTO brokerOverviewDTO = overviewDTOMap.get(brokerDO.getBrokerId());
brokerOverviewDTO.setPeakFlowStatus(getPeakFlowStatus(brokerDO.getMaxAvgBytesIn(), peakFlow).getCode());
continue;
}
BrokerOverviewDTO brokerOverviewDTO = new BrokerOverviewDTO();
brokerOverviewDTO.setBrokerId(brokerDO.getBrokerId());
brokerOverviewDTO.setHost(brokerDO.getHost());
brokerOverviewDTO.setPort(brokerDO.getPort());
brokerOverviewDTO.setJmxPort(DBStatusEnum.DEAD.getStatus());
brokerOverviewDTO.setStartTime(brokerDO.getTimestamp());
brokerOverviewDTO.setStatus(DBStatusEnum.DEAD.getStatus());
brokerOverviewDTO.setPeakFlowStatus(getPeakFlowStatus(brokerDO.getMaxAvgBytesIn(), peakFlow).getCode());
overviewDTOMap.put(brokerDO.getBrokerId(), brokerOverviewDTO);
}
return new ArrayList<>(overviewDTOMap.values());
}
@Override
public List<BrokerMetrics> getBrokerMetricsFromJmx(Long clusterId, Set<Integer> brokerIdSet, Integer metricsCode) {
if (ValidateUtils.isNull(brokerIdSet)) {
return new ArrayList<>();
}
List<Integer> brokerIdList = new ArrayList<>(brokerIdSet);
FutureTask<BrokerMetrics>[] taskList = new FutureTask[brokerIdList.size()];
for (int i = 0; i < brokerIdList.size(); i++) {
Integer brokerId = brokerIdList.get(i);
taskList[i] = new FutureTask<BrokerMetrics>(new Callable<BrokerMetrics>() {
@Override
public BrokerMetrics call() throws Exception {
return getBrokerMetricsFromJmx(clusterId, brokerId, metricsCode);
}
});
ThreadPool.submitApiCallTask(taskList[i]);
}
List<BrokerMetrics> metricsList = new ArrayList<>(brokerIdSet.size());
for (int i = 0; i < brokerIdList.size(); i++) {
try {
BrokerMetrics brokerMetrics = taskList[i].get();
if (ValidateUtils.isNull(brokerMetrics)) {
continue;
}
metricsList.add(brokerMetrics);
} catch (Exception e) {
LOGGER.error("get broker metrics from jmx error, clusterId:{}, brokerId:{}, metricsCode:{}.",
clusterId, brokerIdList.get(i), metricsCode, e);
}
}
return metricsList;
}
@Override
public BrokerMetrics getBrokerMetricsFromJmx(Long clusterId, Integer brokerId, Integer metricsCode) {
if (clusterId == null || brokerId == null || metricsCode == null) {
return null;
}
BrokerMetrics brokerMetrics = jmxService.getBrokerMetrics(clusterId, brokerId, metricsCode);
if (brokerMetrics == null) {
return null;
}
brokerMetrics.setClusterId(clusterId);
brokerMetrics.setBrokerId(brokerId);
return brokerMetrics;
}
@Override
public List<BrokerMetricsDO> getBrokerMetricsFromDB(Long clusterId,
Integer brokerId,
Date startTime,
Date endTime) {
return brokerMetricDao.getBrokerMetrics(clusterId, brokerId, startTime, endTime);
}
@Override
public List<TopicDiskLocation> getBrokerTopicLocation(Long clusterId, Integer brokerId) {
Map<TopicPartition, String> diskNameMap = jmxService.getBrokerTopicLocation(clusterId, brokerId);
Map<String, List<PartitionState>> stateMap = topicService.getTopicPartitionState(clusterId, brokerId);
// 整理数据格式<topicName-diskName, DiskTopicDTO>>
Map<String, TopicDiskLocation> locationMap = new HashMap<>();
for (Map.Entry<TopicPartition, String> entry: diskNameMap.entrySet()) {
String key = new StringBuilder().append(entry.getKey().topic()).append(entry.getValue()).toString();
TopicDiskLocation diskLocation = locationMap.get(key);
if (ValidateUtils.isNull(diskLocation)) {
diskLocation = new TopicDiskLocation();
diskLocation.setClusterId(clusterId);
diskLocation.setBrokerId(brokerId);
diskLocation.setDiskName(entry.getValue());
diskLocation.setTopicName(entry.getKey().topic());
diskLocation.setLeaderPartitions(new ArrayList<>());
diskLocation.setFollowerPartitions(new ArrayList<>());
diskLocation.setUnderReplicated(false);
diskLocation.setUnderReplicatedPartitions(new ArrayList<>());
locationMap.put(key, diskLocation);
}
diskLocation.getFollowerPartitions().add(entry.getKey().partition());
}
List<TopicDiskLocation> locationList = new ArrayList<>();
for (TopicDiskLocation diskLocation: locationMap.values()) {
locationList.add(diskLocation);
List<PartitionState> stateList = stateMap.get(diskLocation.getTopicName());
if (ValidateUtils.isNull(stateList)) {
continue;
}
for (PartitionState state: stateList) {
if (!diskLocation.getFollowerPartitions().contains(state.getPartitionId())) {
continue;
}
if (!state.isUnderReplicated()) {
diskLocation.getUnderReplicatedPartitions().add(state.getPartitionId());
diskLocation.setUnderReplicated(true);
}
if (brokerId.equals(state.getLeader())) {
diskLocation.getLeaderPartitions().add(state.getPartitionId());
}
}
}
return locationList;
}
@Override
public Double calBrokerMaxAvgBytesIn(Long clusterId,
Integer brokerId,
Integer duration,
Date startTime,
Date endTime) {
if (ValidateUtils.isNull(clusterId)
|| ValidateUtils.isNull(brokerId)
|| ValidateUtils.isNullOrLessThanZero(duration)
|| ValidateUtils.isNull(startTime)
|| ValidateUtils.isNull(endTime)) {
return -1.0;
}
List<BrokerMetricsDO> doList = this.getBrokerMetricsFromDB(clusterId, brokerId, startTime, endTime);
if (ValidateUtils.isEmptyList(doList)) {
return 0.0;
}
List<BrokerMetrics> metricsList = MetricsConvertUtils.convert2BrokerMetricsList(doList);
Double maxAvgBytesIn = 0.0;
for (int i = 0; i < duration && i < metricsList.size(); ++i) {
maxAvgBytesIn += metricsList.get(i).getBytesInPerSecOneMinuteRate(0.0);
}
Double tempMaxAvgBytesIn = maxAvgBytesIn;
for (int i = duration; i < metricsList.size(); ++i) {
tempMaxAvgBytesIn += (
metricsList.get(i).getBytesInPerSecOneMinuteRate(0.0)
- metricsList.get(i - duration).getBytesInPerSecOneMinuteRate(0.0)
);
if (tempMaxAvgBytesIn >= maxAvgBytesIn) {
maxAvgBytesIn = tempMaxAvgBytesIn;
}
}
return maxAvgBytesIn / Math.min(duration, metricsList.size());
}
@Override
public BrokerBasicDTO getBrokerBasicDTO(Long clusterId, Integer brokerId) {
if (ValidateUtils.isNull(clusterId) || ValidateUtils.isNull(brokerId)) {
return null;
}
BrokerMetadata brokerMetadata = PhysicalClusterMetadataManager.getBrokerMetadata(clusterId, brokerId);
if (ValidateUtils.isNull(brokerMetadata)) {
return null;
}
BrokerBasicDTO brokerBasicDTO = new BrokerBasicDTO();
brokerBasicDTO.setHost(brokerMetadata.getHost());
brokerBasicDTO.setPort(brokerMetadata.getPort());
brokerBasicDTO.setJmxPort(brokerMetadata.getJmxPort());
brokerBasicDTO.setStartTime(brokerMetadata.getTimestamp());
BrokerMetrics brokerMetrics = jmxService.getBrokerMetrics(
clusterId,
brokerId,
KafkaMetricsCollections.BROKER_BASIC_PAGE_METRICS
);
if (!ValidateUtils.isNull(brokerMetrics)) {
brokerBasicDTO.setPartitionCount(brokerMetrics.getSpecifiedMetrics("PartitionCountValue", Integer.class));
brokerBasicDTO.setLeaderCount(brokerMetrics.getSpecifiedMetrics("LeaderCountValue", Integer.class));
}
Integer topicNum = 0;
for (String topicName: PhysicalClusterMetadataManager.getTopicNameList(clusterId)) {
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterId, topicName);
if (ValidateUtils.isNull(topicMetadata)) {
continue;
}
if (topicMetadata.getBrokerIdSet().contains(brokerId)) {
topicNum += 1;
}
}
brokerBasicDTO.setTopicNum(topicNum);
return brokerBasicDTO;
}
@Override
public String getBrokerVersion(Long clusterId, Integer brokerId) {
return jmxService.getBrokerVersion(clusterId, brokerId);
}
@Override
public List<BrokerDO> listAll() {
try {
return brokerDao.listAll();
} catch (Exception e) {
LOGGER.error("get all broker failed.", e);
}
return new ArrayList<>();
}
@Override
public int replace(BrokerDO brokerDO) {
try {
return brokerDao.replace(brokerDO);
} catch (Exception e) {
LOGGER.error("replace broker failed, brokerDO:{}.", brokerDO, e);
}
return 0;
}
@Override
public ResultStatus delete(Long clusterId, Integer brokerId) {
try {
if (brokerDao.deleteById(clusterId, brokerId) < 1) {
return ResultStatus.OPERATION_FAILED;
}
return ResultStatus.SUCCESS;
} catch (Exception e) {
LOGGER.error("delete broker failed, clusterId:{} brokerId:{}.", clusterId, brokerId);
}
return ResultStatus.MYSQL_ERROR;
}
}

View File

@@ -0,0 +1,304 @@
package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.ClusterDetailDTO;
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.dao.ClusterDao;
import com.xiaojukeji.kafka.manager.dao.ClusterMetricsDao;
import com.xiaojukeji.kafka.manager.dao.ControllerDao;
import com.xiaojukeji.kafka.manager.service.cache.LogicalClusterMetadataManager;
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.utils.ConfigUtils;
import org.apache.zookeeper.ZooKeeper;
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 java.util.*;
/**
* ClusterServiceImpl
* @author zengqiao
* @date 19/4/3
*/
@Service("clusterService")
public class ClusterServiceImpl implements ClusterService {
private final static Logger LOGGER = LoggerFactory.getLogger(ClusterServiceImpl.class);
@Autowired
private ClusterDao clusterDao;
@Autowired
private ClusterMetricsDao clusterMetricsDao;
@Autowired
private ControllerDao controllerDao;
@Autowired
private ConsumerService consumerService;
@Autowired
private RegionService regionService;
@Autowired
private LogicalClusterMetadataManager logicalClusterMetadataManager;
@Autowired
private PhysicalClusterMetadataManager physicalClusterMetadataManager;
@Autowired
private ConfigUtils configUtils;
@Override
public ResultStatus addNew(ClusterDO clusterDO, String operator) {
if (ValidateUtils.isNull(clusterDO) || ValidateUtils.isNull(operator)) {
return ResultStatus.PARAM_ILLEGAL;
}
if (!isZookeeperLegal(clusterDO.getZookeeper())) {
return ResultStatus.CONNECT_ZOOKEEPER_FAILED;
}
try {
if (clusterDao.insert(clusterDO) <= 0) {
LOGGER.error("add new cluster failed, clusterDO:{}.", clusterDO);
return ResultStatus.MYSQL_ERROR;
}
} catch (DuplicateKeyException e) {
LOGGER.error("add new cluster failed, cluster already existed, clusterDO:{}.", clusterDO, e);
return ResultStatus.RESOURCE_ALREADY_EXISTED;
} catch (Exception e) {
LOGGER.error("add new cluster failed, operate mysql failed, clusterDO:{}.", clusterDO, e);
return ResultStatus.MYSQL_ERROR;
}
return ResultStatus.SUCCESS;
}
@Override
public ResultStatus updateById(ClusterDO clusterDO, String operator) {
if (ValidateUtils.isNull(clusterDO) || ValidateUtils.isNull(operator)) {
return ResultStatus.PARAM_ILLEGAL;
}
ClusterDO originClusterDO = this.getById(clusterDO.getId());
if (ValidateUtils.isNull(originClusterDO)) {
return ResultStatus.CLUSTER_NOT_EXIST;
}
if (!originClusterDO.getZookeeper().equals(clusterDO.getZookeeper())) {
// 不允许修改zk地址
return ResultStatus.CHANGE_ZOOKEEPER_FORBIDEN;
}
clusterDO.setStatus(originClusterDO.getStatus());
return updateById(clusterDO);
}
@Override
public ResultStatus modifyStatus(Long clusterId, Integer status, String operator) {
if (ValidateUtils.isNull(clusterId) || ValidateUtils.isNull(status)) {
return ResultStatus.PARAM_ILLEGAL;
}
ClusterDO clusterDO = this.getById(clusterId);
if (ValidateUtils.isNull(clusterDO)) {
return ResultStatus.CLUSTER_NOT_EXIST;
}
clusterDO.setStatus(status);
return updateById(clusterDO);
}
private ResultStatus updateById(ClusterDO clusterDO) {
try {
if (clusterDao.updateById(clusterDO) <= 0) {
LOGGER.error("update cluster failed, clusterDO:{}.", clusterDO);
return ResultStatus.MYSQL_ERROR;
}
} catch (Exception e) {
LOGGER.error("update cluster failed, clusterDO:{}.", clusterDO, e);
return ResultStatus.MYSQL_ERROR;
}
return ResultStatus.SUCCESS;
}
@Override
public ClusterDO getById(Long clusterId) {
if (ValidateUtils.isNull(clusterId)) {
return null;
}
try {
return clusterDao.getById(clusterId);
} catch (Exception e) {
LOGGER.error("get cluster failed, clusterId:{}.", clusterId, e);
}
return null;
}
@Override
public List<ClusterDO> list() {
try {
return clusterDao.list();
} catch (Exception e) {
LOGGER.error("list cluster failed.", e);
}
return new ArrayList<>();
}
@Override
public Map<Long, ClusterDO> listMap() {
List<ClusterDO> doList = this.list();
Map<Long, ClusterDO> doMap = new HashMap<>();
for (ClusterDO elem: doList) {
doMap.put(elem.getId(), elem);
}
return doMap;
}
@Override
public List<ClusterDO> listAll() {
try {
return clusterDao.listAll();
} catch (Exception e) {
LOGGER.error("list cluster failed.", e);
}
return new ArrayList<>();
}
@Override
public List<ClusterMetricsDO> getClusterMetricsFromDB(Long clusterId, Date startTime, Date endTime) {
return clusterMetricsDao.getClusterMetrics(clusterId, startTime, endTime);
}
@Override
public List<ControllerDO> getKafkaControllerHistory(Long clusterId) {
if (ValidateUtils.isNull(clusterId)) {
return new ArrayList<>();
}
return controllerDao.getByClusterId(clusterId);
}
private boolean isZookeeperLegal(String zookeeper) {
ZooKeeper zk = null;
try {
zk = new ZooKeeper(zookeeper, 1000, null);
} catch (Throwable t) {
return false;
} finally {
try {
if (zk != null) {
zk.close();
}
} catch (Throwable t) {
}
}
return true;
}
@Override
public ClusterDetailDTO getClusterDetailDTO(Long clusterId, Boolean needDetail) {
ClusterDO clusterDO = this.getById(clusterId);
if (ValidateUtils.isNull(clusterDO)) {
return null;
}
return getClusterDetailDTO(clusterDO, needDetail);
}
@Override
public List<ClusterDetailDTO> getClusterDetailDTOList(Boolean needDetail) {
List<ClusterDO> doList = this.listAll();
Map<Long, Integer> regionNumMap =
needDetail? regionService.getRegionNum(): new HashMap<>(0);
Map<Long, Integer> consumerGroupNumMap =
needDetail? consumerService.getConsumerGroupNumMap(doList): new HashMap<>(0);
List<ClusterDetailDTO> dtoList = new ArrayList<>();
for (ClusterDO clusterDO: doList) {
ClusterDetailDTO dto = getClusterDetailDTO(clusterDO, needDetail);
dto.setConsumerGroupNum(consumerGroupNumMap.get(clusterDO.getId()));
dto.setRegionNum(regionNumMap.get(clusterDO.getId()));
dtoList.add(dto);
}
return dtoList;
}
@Override
public ClusterNameDTO getClusterName(Long logicClusterId) {
ClusterNameDTO clusterNameDTO = new ClusterNameDTO();
LogicalClusterDO logicalClusterDO = logicalClusterMetadataManager.getLogicalCluster(logicClusterId);
if (ValidateUtils.isNull(logicalClusterDO)) {
return clusterNameDTO;
}
clusterNameDTO.setLogicalClusterId(logicalClusterDO.getId());
clusterNameDTO.setLogicalClusterName(logicalClusterDO.getName());
clusterNameDTO.setRegionIdList(ListUtils.string2LongList(logicalClusterDO.getRegionList()));
ClusterDO clusterDO = this.getById(logicalClusterDO.getClusterId());
clusterNameDTO.setPhysicalClusterId(clusterDO.getId());
clusterNameDTO.setPhysicalClusterName(clusterDO.getClusterName());
return clusterNameDTO;
}
@Override
public ResultStatus deleteById(Long clusterId) {
List<RegionDO> regionDOList = regionService.getByClusterId(clusterId);
if (!ValidateUtils.isEmptyList(regionDOList)) {
return ResultStatus.OPERATION_FORBIDDEN;
}
try {
if (clusterDao.deleteById(clusterId) <= 0) {
LOGGER.error("delete cluster failed, clusterId:{}.", clusterId);
return ResultStatus.MYSQL_ERROR;
}
} catch (Exception e) {
LOGGER.error("delete cluster failed, clusterId:{}.", clusterId, e);
return ResultStatus.MYSQL_ERROR;
}
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;
}
ClusterDetailDTO dto = new ClusterDetailDTO();
dto.setClusterId(clusterDO.getId());
dto.setClusterName(clusterDO.getClusterName());
dto.setZookeeper(clusterDO.getZookeeper());
dto.setBootstrapServers(clusterDO.getBootstrapServers());
dto.setKafkaVersion(physicalClusterMetadataManager.getKafkaVersion(clusterDO.getId()));
dto.setIdc(configUtils.getIdc());
dto.setMode(clusterDO.getMode());
dto.setSecurityProperties(clusterDO.getSecurityProperties());
dto.setStatus(clusterDO.getStatus());
dto.setGmtCreate(clusterDO.getGmtCreate());
dto.setGmtModify(clusterDO.getGmtModify());
if (ValidateUtils.isNull(needDetail) || !needDetail) {
return dto;
}
dto.setBrokerNum(PhysicalClusterMetadataManager.getBrokerIdList(clusterDO.getId()).size());
dto.setTopicNum(PhysicalClusterMetadataManager.getTopicNameList(clusterDO.getId()).size());
dto.setControllerId(PhysicalClusterMetadataManager.getControllerId(clusterDO.getId()));
return dto;
}
}

View File

@@ -0,0 +1,238 @@
package com.xiaojukeji.kafka.manager.service.service.impl;
import com.alibaba.fastjson.JSON;
import com.xiaojukeji.kafka.manager.common.constant.ConfigConstant;
import com.xiaojukeji.kafka.manager.common.constant.SystemCodeConstant;
import com.xiaojukeji.kafka.manager.common.constant.TopicCreationConstant;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.config.*;
import com.xiaojukeji.kafka.manager.common.entity.dto.config.ConfigDTO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterDO;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ConfigDO;
import com.xiaojukeji.kafka.manager.dao.ConfigDao;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.service.ConfigService;
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 java.util.ArrayList;
import java.util.List;
/**
* @author zengqiao
* @date 20/3/19
*/
@Service("configService")
public class ConfigServiceImpl implements ConfigService {
private final static Logger LOGGER = LoggerFactory.getLogger(ConfigServiceImpl.class);
@Autowired
private ConfigDao configDao;
@Override
public ResultStatus insert(ConfigDTO dto) {
try {
if (configDao.insert(convert2ConfigDO(dto)) >= 1) {
return ResultStatus.SUCCESS;
}
} catch (DuplicateKeyException e) {
return ResultStatus.RESOURCE_ALREADY_EXISTED;
} catch (Exception e) {
LOGGER.error("insert config failed, config:{}.", dto, e);
}
return ResultStatus.MYSQL_ERROR;
}
@Override
public ResultStatus deleteByKey(String configKey) {
if (ValidateUtils.isNull(configKey)) {
return ResultStatus.PARAM_ILLEGAL;
}
try {
if (configDao.deleteByKey(configKey) >= 1) {
return ResultStatus.SUCCESS;
}
return ResultStatus.CONFIG_NOT_EXIST;
} catch (Exception e) {
LOGGER.error("delete config failed, configKey:{}.", configKey, e);
}
return ResultStatus.MYSQL_ERROR;
}
@Override
public ResultStatus updateByKey(ConfigDTO dto) {
try {
if (configDao.updateByKey(convert2ConfigDO(dto)) >= 1) {
return ResultStatus.SUCCESS;
}
return ResultStatus.CONFIG_NOT_EXIST;
} catch (Exception e) {
LOGGER.error("update config failed, config:{}.", dto, e);
}
return ResultStatus.MYSQL_ERROR;
}
@Override
public ResultStatus updateByKey(String configKey, String configValue) {
try {
ConfigDO configDO = new ConfigDO();
configDO.setConfigKey(configKey);
configDO.setConfigValue(configValue);
configDO.setConfigDescription("");
if (configDao.updateByKey(configDO) >= 1) {
return ResultStatus.SUCCESS;
}
return ResultStatus.CONFIG_NOT_EXIST;
} catch (Exception e) {
LOGGER.error("update config failed, configValue:{}.", configValue, e);
}
return ResultStatus.MYSQL_ERROR;
}
@Override
public ConfigDO getByKey(String configKey) {
try {
return configDao.getByKey(configKey);
} catch (Exception e) {
LOGGER.error("get config failed, configKey:{}.", configKey, e);
}
return null;
}
@Override
public <T> T getByKey(String configKey, Class<T> clazz) {
ConfigDO configDO = this.getByKey(configKey);
if (ValidateUtils.isNull(configDO)) {
return null;
}
try {
return JSON.parseObject(configDO.getConfigValue(), clazz);
} catch (Exception e) {
LOGGER.error("get config failed, configKey:{}.", configKey, e);
}
return null;
}
@Override
public <T> List<T> getArrayByKey(String configKey, Class<T> clazz) {
ConfigDO configDO = this.getByKey(configKey);
if (ValidateUtils.isNull(configDO)) {
return null;
}
try {
return JSON.parseArray(configDO.getConfigValue(), clazz);
} catch (Exception e) {
LOGGER.error("get config failed, configKey:{}.", configKey, e);
}
return null;
}
@Override
public Long getLongValue(String configKey, Long defaultValue) {
ConfigDO configDO = this.getByKey(configKey);
if (ValidateUtils.isNull(configDO)) {
return defaultValue;
}
try {
return Long.valueOf(configDO.getConfigValue());
} catch (Exception e) {
LOGGER.error("get and convert config value failed, configKey:{}.", configKey, e);
}
return defaultValue;
}
@Override
public List<ConfigDO> listAll() {
try {
return configDao.listAll();
} catch (Exception e) {
LOGGER.error("get configs failed.", e);
}
return null;
}
private ConfigDO convert2ConfigDO(ConfigDTO dto) {
ConfigDO configDO = new ConfigDO();
configDO.setConfigKey(dto.getConfigKey());
configDO.setConfigValue(dto.getConfigValue());
configDO.setConfigDescription(dto.getConfigDescription());
return configDO;
}
@Override
public CreateTopicElemConfig getCreateTopicConfig(Long clusterId, String systemCode) {
String configKey = TopicCreationConstant.INNER_CREATE_TOPIC_CONFIG_KEY;
if (SystemCodeConstant.LOG_X.equals(systemCode)) {
configKey = TopicCreationConstant.LOG_X_CREATE_TOPIC_CONFIG_KEY_NAME;
} else if (SystemCodeConstant.CHORUS.equals(systemCode)) {
configKey = TopicCreationConstant.CHORUS_CREATE_TOPIC_CONFIG_KEY_NAME;
}
CreateTopicConfig configValue = this.getByKey(
configKey,
CreateTopicConfig.class
);
CreateTopicElemConfig config = new CreateTopicElemConfig();
config.setClusterId(clusterId);
config.setBrokerIdList(new ArrayList<>());
config.setRegionIdList(new ArrayList<>());
config.setPartitionNum(TopicCreationConstant.DEFAULT_PARTITION_NUM);
config.setReplicaNum(TopicCreationConstant.DEFAULT_REPLICA);
config.setRetentionTimeUnitHour(TopicCreationConstant.DEFAULT_RETENTION_TIME_UNIT_HOUR);
config.setAutoExecMaxPeakBytesInUnitB(TopicCreationConstant.AUTO_EXEC_MAX_BYTES_IN_UNIT_B);
if (ValidateUtils.isNull(configValue) || ValidateUtils.isEmptyList(configValue.getConfigList())) {
return config;
}
for (CreateTopicElemConfig elem: configValue.getConfigList()) {
if (!clusterId.equals(elem.getClusterId())) {
continue;
}
if (!ValidateUtils.isEmptyList(elem.getBrokerIdList())) {
config.setBrokerIdList(elem.getBrokerIdList());
}
if (!ValidateUtils.isEmptyList(elem.getRegionIdList())) {
config.setRegionIdList(elem.getRegionIdList());
}
if (!ValidateUtils.isNull(elem.getReplicaNum())) {
config.setReplicaNum(elem.getReplicaNum());
}
if (!ValidateUtils.isNull(elem.getPartitionNum())) {
config.setPartitionNum(elem.getPartitionNum());
}
if (!ValidateUtils.isNull(elem.getRetentionTimeUnitHour())) {
config.setRetentionTimeUnitHour(elem.getRetentionTimeUnitHour());
}
if (!ValidateUtils.isNull(elem.getAutoExecMaxPeakBytesInUnitB())) {
config.setAutoExecMaxPeakBytesInUnitB(elem.getAutoExecMaxPeakBytesInUnitB());
}
return config;
}
return config;
}
@Override
public ClusterDO getClusterDO(Long clusterId) {
ConfigDO configDO = this.getByKey(ConfigConstant.KAFKA_CLUSTER_DO_CONFIG_KEY);
if (ValidateUtils.isNull(configDO)) {
return null;
}
try {
List<ClusterDO> clusterDOList = JSON.parseArray(configDO.getConfigValue(), ClusterDO.class);
if (ValidateUtils.isEmptyList(clusterDOList)) {
return null;
}
for (ClusterDO clusterDO: clusterDOList) {
if (clusterId.equals(clusterDO.getId())) {
return clusterDO;
}
}
} catch (Exception e) {
LOGGER.error("get cluster do failed, clusterId:{}.", clusterId, e);
}
return null;
}
}

View File

@@ -0,0 +1,397 @@
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.pojo.ClusterDO;
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;
import com.xiaojukeji.kafka.manager.common.zookeeper.ZkConfigImpl;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.cache.ConsumerMetadataCache;
import com.xiaojukeji.kafka.manager.service.cache.KafkaClientPool;
import com.xiaojukeji.kafka.manager.service.service.ConsumerService;
import com.xiaojukeji.kafka.manager.service.service.TopicService;
import com.xiaojukeji.kafka.manager.common.zookeeper.ZkPathUtil;
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.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import scala.collection.JavaConversions;
import java.util.*;
import java.util.stream.Collectors;
/**
* @author tukun
* @date 2015/11/12
*/
@Service("consumerService")
public class ConsumerServiceImpl implements ConsumerService {
private final static Logger logger = LoggerFactory.getLogger(ConsumerServiceImpl.class);
@Autowired
private TopicService topicService;
@Override
public List<ConsumerGroupDTO> getConsumerGroupList(Long clusterId) {
List<ConsumerGroupDTO> consumerGroupDTOList = 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) {
continue;
}
for (String consumerGroup : consumerGroupSet) {
consumerGroupDTOList.add(new ConsumerGroupDTO(
clusterId,
consumerGroup,
consumerGroupAppIdMap.getOrDefault(consumerGroup, new ArrayList<>()),
location)
); }
}
return consumerGroupDTOList;
}
@Override
public List<ConsumerGroupDTO> getConsumerGroupList(Long clusterId, String topicName) {
List<ConsumerGroupDTO> consumerGroupDTOList = 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) {
continue;
}
for (String consumerGroup : consumerGroupSet) {
consumerGroupDTOList.add(new ConsumerGroupDTO(
clusterId,
consumerGroup,
consumerGroupAppIdMap.getOrDefault(consumerGroup, new ArrayList<>()),
location
)
);
}
}
return consumerGroupDTOList;
}
@Override
public List<ConsumeDetailDTO> getConsumeDetail(ClusterDO clusterDO,
String topicName,
ConsumerGroupDTO consumeGroupDTO) {
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterDO.getId(), topicName);
if (topicMetadata == null) {
return null;
}
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 (consumerGroupDetailDTOList == null) {
return null;
}
Map<TopicPartition, Long> topicPartitionLongMap = topicService.getPartitionOffset(clusterDO, topicName, OffsetPosEnum.END);
if (topicPartitionLongMap == null) {
return consumerGroupDetailDTOList;
}
for (ConsumeDetailDTO consumerGroupDetailDTO : consumerGroupDetailDTOList) {
consumerGroupDetailDTO.setOffset(topicPartitionLongMap.get(new TopicPartition(topicName, consumerGroupDetailDTO.getPartitionId())));
}
return consumerGroupDetailDTOList;
}
@Override
public List<String> getConsumerGroupConsumedTopicList(Long clusterId, String consumerGroup, String location) {
if (ValidateUtils.isNull(clusterId)
|| ValidateUtils.isNull(consumerGroup)
|| ValidateUtils.isNull(location)) {
return new ArrayList<>();
}
return ConsumerMetadataCache.getConsumerGroupConsumedTopicList(clusterId, consumerGroup, location);
}
@Override
public List<Result> resetConsumerOffset(ClusterDO clusterDO, String topicName, ConsumerGroupDTO consumerGroupDTO, 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<>();
// 创建KafkaConsumer, 修正offset值
KafkaConsumer<String, String> kafkaConsumer = null;
try {
Properties properties = KafkaClientPool.createProperties(clusterDO, false);
properties.setProperty("group.id", consumerGroupDTO.getConsumerGroup());
kafkaConsumer = new KafkaConsumer<>(properties);
checkAndCorrectPartitionOffset(kafkaConsumer, offsetMap);
return resetConsumerOffset(clusterDO, kafkaConsumer, consumerGroupDTO, offsetMap);
} catch (Exception e) {
logger.error("create kafka consumer failed, clusterId:{} topicName:{} consumerGroup:{} partition:{}.", clusterDO.getId(), topicName, consumerGroupDTO, partitionOffsetDTOList, e);
resultList.add(new Result(
ResultStatus.OPERATION_FAILED.getCode(),
"reset failed, create KafkaConsumer or check offset failed"
));
} finally {
if (kafkaConsumer != null) {
kafkaConsumer.close();
}
}
return new ArrayList<>();
}
private List<Result> resetConsumerOffset(ClusterDO cluster, KafkaConsumer<String, String> kafkaConsumer, ConsumerGroupDTO consumerGroupDTO, 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)) {
resetConsumerOffsetInBroker(kafkaConsumer, tp, offset);
}
} catch (Exception e) {
logger.error("reset failed, clusterId:{} consumerGroup:{} topic-partition:{}.", cluster.getId(), consumerGroupDTO, tp, e);
resultList.add(new Result());
}
resultList.add(new Result());
}
return resultList;
}
private void checkAndCorrectPartitionOffset(KafkaConsumer<String, String> kafkaConsumer, Map<TopicPartition, Long> offsetMap) {
List<TopicPartition> topicPartitionList = new ArrayList<>(offsetMap.keySet());
Map<TopicPartition, Long> beginningOffsets = kafkaConsumer.beginningOffsets(topicPartitionList);
Map<TopicPartition, Long> endOffsets = kafkaConsumer.endOffsets(topicPartitionList);
for (TopicPartition tp: topicPartitionList) {
Long offset = offsetMap.get(tp);
Long earliestOffset = beginningOffsets.get(tp);
Long largestOffset = endOffsets.get(tp);
if (earliestOffset != null && offset < earliestOffset) {
offsetMap.put(tp, earliestOffset);
} else if (largestOffset != null && largestOffset < offset) {
offsetMap.put(tp, largestOffset);
}
}
}
private void resetConsumerOffsetInZK(ClusterDO cluster,
String consumerGroup,
TopicPartition topicPartition,
Long offset) throws Exception {
ZkConfigImpl zkConfig = PhysicalClusterMetadataManager.getZKConfig(cluster.getId());
String offsetPath = ZkPathUtil.getConsumerGroupOffsetTopicPartitionNode(consumerGroup, topicPartition.topic(), topicPartition.partition());
zkConfig.setNodeStat(offsetPath, offset.toString());
}
private void resetConsumerOffsetInBroker(KafkaConsumer kafkaConsumer,
TopicPartition topicPartition,
Long offset) throws Exception {
kafkaConsumer.assign(Arrays.asList(topicPartition));
kafkaConsumer.seek(topicPartition, offset);
kafkaConsumer.commitSync();
}
@Override
public Map<Integer, Long> getConsumerOffset(ClusterDO clusterDO,
String topicName,
ConsumerGroupDTO consumerGroupDTO) {
if (ValidateUtils.isNull(clusterDO) || ValidateUtils.isBlank(topicName) || ValidateUtils.isNull(consumerGroupDTO)) {
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());
}
return null;
}
private Map<Integer, Long> getConsumerOffsetFromZK(Long clusterId, String topicName, String consumerGroup) {
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterId, topicName);
if (ValidateUtils.isNull(topicMetadata)) {
return new HashMap<>(0);
}
ZkConfigImpl zkConfig = PhysicalClusterMetadataManager.getZKConfig(clusterId);
Map<Integer, Long> consumerOffsetMap = new HashMap<>(topicMetadata.getPartitionNum());
for (Integer partitionId : topicMetadata.getPartitionMap().getPartitions().keySet()) {
String consumerGroupOffsetLocation =
ZkPathUtil.getConsumerGroupOffsetTopicPartitionNode(consumerGroup, topicName, partitionId);
try {
consumerOffsetMap.put(partitionId, Long.valueOf(zkConfig.get(consumerGroupOffsetLocation)));
} catch (Exception e) {
logger.error("get consumer offset from zk failed, clusterId:{} topicName:{} consumerGroup:{}.",
clusterId, topicName, consumerGroup, e);
}
}
return consumerOffsetMap;
}
private Map<Integer, Long> getConsumerOffsetFromBK(ClusterDO clusterDO,
String topicName,
String consumerGroup) {
Map<Integer, String> stringOffsetMap =
getOffsetByGroupAndTopicFromBroker(clusterDO, consumerGroup, topicName);
if (ValidateUtils.isNull(stringOffsetMap)) {
return new HashMap<>(0);
}
Map<Integer, Long> offsetMap = new HashMap<>(stringOffsetMap.size());
for (Map.Entry<Integer, String> entry: stringOffsetMap.entrySet()) {
try {
offsetMap.put(entry.getKey(), Long.valueOf(entry.getValue()));
} catch (Exception e) {
logger.error("get consumer offset from bk failed, clusterId:{} topicName:{} consumerGroup:{}.",
clusterDO.getId(), topicName, consumerGroup, e);
}
}
return offsetMap;
}
private Map<Integer, String> getConsumeIdMap(Long clusterId, String topicName, String consumerGroup) {
AdminClient.ConsumerGroupSummary consumerGroupSummary = ConsumerMetadataCache.getConsumerGroupSummary(clusterId, consumerGroup);
if (consumerGroupSummary == null) {
return new HashMap<>(0);
}
Map<Integer, String> consumerIdMap = new HashMap<>();
for (scala.collection.immutable.List<AdminClient.ConsumerSummary> scalaSubConsumerSummaryList: JavaConversions.asJavaList(consumerGroupSummary.consumers().toList())) {
List<AdminClient.ConsumerSummary> subConsumerSummaryList = JavaConversions.asJavaList(scalaSubConsumerSummaryList);
for (AdminClient.ConsumerSummary consumerSummary: subConsumerSummaryList) {
for (TopicPartition tp: JavaConversions.asJavaList(consumerSummary.assignment())) {
if (!tp.topic().equals(topicName)) {
continue;
}
consumerIdMap.put(tp.partition(), consumerSummary.host().substring(1, consumerSummary.host().length()) + ":" + consumerSummary.consumerId());
}
}
}
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());
List<ConsumeDetailDTO> consumeDetailDTOList = new ArrayList<>();
for (int partitionId : topicMetadata.getPartitionMap().getPartitions().keySet()) {
ConsumeDetailDTO consumeDetailDTO = new ConsumeDetailDTO();
consumeDetailDTO.setPartitionId(partitionId);
String consumeOffsetStr = consumeOffsetMap.get(partitionId);
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);
}
consumeDetailDTO.setConsumerId(consumerIdMap.get(partitionId));
consumeDetailDTOList.add(consumeDetailDTO);
}
return consumeDetailDTOList;
}
private List<ConsumeDetailDTO> getConsumerPartitionStateInZK(ClusterDO clusterDO,
TopicMetadata topicMetadata,
ConsumerGroupDTO consumerGroupDTO) {
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 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 consumerId = null;
try {
consumerId = zkConfig.get(consumeIdZkPath);
} catch (ConfigException e) {
// logger.error("get consumerId error for zk path:{}", consumeIdZkPath, e);
}
ConsumeDetailDTO consumeDetailDTO = new ConsumeDetailDTO();
consumeDetailDTO.setPartitionId(partitionId);
consumeDetailDTO.setConsumerId(consumerId);
consumeDetailDTO.setPartitionId(partitionId);
if (!StringUtils.isEmpty(consumeOffset)) {
consumeDetailDTO.setConsumeOffset(Long.valueOf(consumeOffset));
}
consumeDetailDTOList.add(consumeDetailDTO);
}
return consumeDetailDTOList;
}
/**
* 根据group,topic获取broker中的group中的各个消费者的offset
*/
private Map<Integer, String> getOffsetByGroupAndTopicFromBroker(ClusterDO clusterDO,
String consumerGroup,
String topicName) {
Map<Integer, String> result = new HashMap<>();
AdminClient client = KafkaClientPool.getAdminClient(clusterDO.getId());
if (null == client) {
return result;
}
Map<TopicPartition, Object> offsetMap = JavaConversions.asJavaMap(client.listGroupOffsets(consumerGroup));
for (Map.Entry<TopicPartition, Object> entry : offsetMap.entrySet()) {
TopicPartition topicPartition = entry.getKey();
if (topicPartition.topic().equals(topicName)) {
result.put(topicPartition.partition(), entry.getValue().toString());
}
}
return result;
}
@Override
public Map<Long, Integer> getConsumerGroupNumMap(List<ClusterDO> clusterDOList) {
Map<Long, Integer> consumerGroupNumMap = new HashMap<>();
for (ClusterDO clusterDO: clusterDOList) {
Integer zkConsumerGroupNum = ConsumerMetadataCache.getGroupInZkMap(clusterDO.getId()).size();
Integer brokerConsumerGroupNum = ConsumerMetadataCache.getGroupInBrokerMap(clusterDO.getId()).size();
consumerGroupNumMap.put(clusterDO.getId(), zkConsumerGroupNum + brokerConsumerGroupNum);
}
return consumerGroupNumMap;
}
@Override
public boolean checkConsumerGroupExist(OffsetLocationEnum offsetLocation, Long clusterId, String topicName, String consumerGroup) {
List<ConsumerGroupDTO> consumerGroupList = getConsumerGroupList(clusterId, topicName).stream()
.filter(group -> offsetLocation.location.equals(group.getOffsetStoreLocation()) && consumerGroup.equals(group.getConsumerGroup()))
.collect(Collectors.toList());
return !ValidateUtils.isEmptyList(consumerGroupList);
}
}

View File

@@ -0,0 +1,252 @@
package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.common.constant.ConfigConstant;
import com.xiaojukeji.kafka.manager.common.entity.ao.config.expert.RegionTopicHotConfig;
import com.xiaojukeji.kafka.manager.common.entity.ao.config.TopicAnomalyFlowConfig;
import com.xiaojukeji.kafka.manager.common.entity.ao.config.expert.TopicExpiredConfig;
import com.xiaojukeji.kafka.manager.common.entity.ao.config.expert.TopicInsufficientPartitionConfig;
import com.xiaojukeji.kafka.manager.common.entity.ao.expert.TopicAnomalyFlow;
import com.xiaojukeji.kafka.manager.common.entity.ao.expert.TopicInsufficientPartition;
import com.xiaojukeji.kafka.manager.common.entity.ao.expert.TopicRegionHot;
import com.xiaojukeji.kafka.manager.common.entity.metrics.TopicMetrics;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.TopicExpiredDO;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata;
import com.xiaojukeji.kafka.manager.service.cache.KafkaMetricsCache;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.service.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import java.util.*;
/**
* @author zengqiao
* @date 20/3/20
*/
@Service("expertService")
public class ExpertServiceImpl implements ExpertService {
private final static Logger LOGGER = LoggerFactory.getLogger(ExpertServiceImpl.class);
@Autowired
private ConfigService configService;
@Autowired
private ClusterService clusterService;
@Autowired
private RegionService regionService;
@Autowired
private TopicManagerService topicManagerService;
private static final Integer LATEST_MAX_AVG_BYTES_IN_DAY = 3;
@Override
public List<TopicRegionHot> getRegionHotTopics() {
RegionTopicHotConfig config = configService.getByKey(ConfigConstant.REGION_HOT_TOPIC_CONFIG_KEY, RegionTopicHotConfig.class);
if (ValidateUtils.isNull(config)) {
config = new RegionTopicHotConfig();
}
List<TopicRegionHot> hotTopics = new ArrayList<>();
for (ClusterDO clusterDO: clusterService.list()) {
if (config.getIgnoreClusterIdList().contains(clusterDO.getId())) {
continue;
}
hotTopics.addAll(getRegionHotTopics(clusterDO, config));
}
return hotTopics;
}
private List<TopicRegionHot> getRegionHotTopics(ClusterDO clusterDO, RegionTopicHotConfig config) {
Map<String, Set<Integer>> topicNameRegionBrokerIdMap =
regionService.getTopicNameRegionBrokerIdMap(clusterDO.getId());
List<TopicRegionHot> hotTopics = new ArrayList<>();
for (String topicName: PhysicalClusterMetadataManager.getTopicNameList(clusterDO.getId())) {
Set<Integer> regionBrokerIdSet =
topicNameRegionBrokerIdMap.get(topicName);
TopicMetadata topicMetadata =
PhysicalClusterMetadataManager.getTopicMetadata(clusterDO.getId(), topicName);
TopicMetrics metrics =
KafkaMetricsCache.getTopicMetricsFromCache(clusterDO.getId(), topicName);
if (ValidateUtils.isNull(regionBrokerIdSet)
|| ValidateUtils.isNull(topicMetadata)
|| ValidateUtils.isNull(metrics)) {
continue;
}
Double bytesIn = metrics.getBytesInPerSecOneMinuteRate(0.0);
if (topicMetadata.getPartitionNum() <= 1
|| ValidateUtils.isNull(bytesIn)
|| bytesIn <= config.getMinTopicBytesInUnitB()) {
continue;
}
TopicRegionHot hotTopic =
checkAndGetIfImBalanced(clusterDO, topicMetadata, regionBrokerIdSet, config);
if (ValidateUtils.isNull(hotTopic)) {
continue;
}
hotTopics.add(hotTopic);
}
return hotTopics;
}
private TopicRegionHot checkAndGetIfImBalanced(ClusterDO clusterDO,
TopicMetadata topicMetadata,
Set<Integer> regionBrokerIdSet,
RegionTopicHotConfig config) {
Map<Integer, Integer> brokerIdPartitionNum = new HashMap<>();
for (Integer brokerId: regionBrokerIdSet) {
brokerIdPartitionNum.put(brokerId, 0);
}
for (Map.Entry<Integer, List<Integer>> entry: topicMetadata.getPartitionMap().getPartitions().entrySet()) {
for (Integer brokerId: entry.getValue()) {
Integer partitionNum = brokerIdPartitionNum.getOrDefault(brokerId, 0);
brokerIdPartitionNum.put(brokerId, partitionNum + 1);
}
}
Integer maxPartitionNum = Integer.MIN_VALUE;
Integer minPartitionNum = Integer.MAX_VALUE;
for (Integer partitionNum: brokerIdPartitionNum.values()) {
if (maxPartitionNum < partitionNum) {
maxPartitionNum = partitionNum;
}
if (minPartitionNum > partitionNum) {
minPartitionNum = partitionNum;
}
}
if (maxPartitionNum - minPartitionNum < config.getMaxDisPartitionNum()) {
return null;
}
return new TopicRegionHot(
clusterDO,
topicMetadata.getTopic(),
PhysicalClusterMetadataManager.getTopicRetentionTime(clusterDO.getId(), topicMetadata.getTopic()),
brokerIdPartitionNum
);
}
@Override
public List<TopicInsufficientPartition> getPartitionInsufficientTopics() {
TopicInsufficientPartitionConfig config = configService.getByKey(ConfigConstant.TOPIC_INSUFFICIENT_PARTITION_CONFIG_KEY, TopicInsufficientPartitionConfig.class);
if (ValidateUtils.isNull(config)) {
config = new TopicInsufficientPartitionConfig();
}
List<TopicInsufficientPartition> dataList = new ArrayList<>();
for (ClusterDO clusterDO: clusterService.list()) {
if (config.getIgnoreClusterIdList().contains(clusterDO.getId())) {
continue;
}
dataList.addAll(getPartitionInsufficientTopics(clusterDO, config));
}
return dataList;
}
private List<TopicInsufficientPartition> getPartitionInsufficientTopics(ClusterDO clusterDO,
TopicInsufficientPartitionConfig config) {
Map<String, Set<Integer>> topicNameRegionBrokerIdMap =
regionService.getTopicNameRegionBrokerIdMap(clusterDO.getId());
Map<String, List<Double>> maxAvgBytesInMap = topicManagerService.getTopicMaxAvgBytesIn(
clusterDO.getId(),
-1 * LATEST_MAX_AVG_BYTES_IN_DAY,
config.getMinTopicBytesInUnitB().doubleValue()
);
List<TopicInsufficientPartition> dataList = new ArrayList<>();
for (String topicName: PhysicalClusterMetadataManager.getTopicNameList(clusterDO.getId())) {
if (!topicNameRegionBrokerIdMap.containsKey(topicName)) {
// Topic不属于任何Region, 直接忽略
continue;
}
// Topic不存在 or 流量不存在
TopicMetadata topicMetadata =
PhysicalClusterMetadataManager.getTopicMetadata(clusterDO.getId(), topicName);
TopicMetrics metrics = KafkaMetricsCache.getTopicMetricsFromCache(clusterDO.getId(), topicName);
if (ValidateUtils.isNull(topicMetadata) || ValidateUtils.isNull(metrics)) {
continue;
}
// 流量不存在 or 未达到阈值 or 分区数充足
Double bytesIn = metrics.getBytesInPerSecOneMinuteRate(0.0);
if (ValidateUtils.isNull(bytesIn)
|| bytesIn <= config.getMinTopicBytesInUnitB()
|| bytesIn / topicMetadata.getPartitionNum() < config.getMaxBytesInPerPartitionUnitB()) {
continue;
}
Integer suggestedPartitionNum = (int) Math.round(
bytesIn / topicMetadata.getPartitionNum() / config.getMaxBytesInPerPartitionUnitB()
);
if (suggestedPartitionNum - topicMetadata.getPartitionNum() < 1) {
continue;
}
// 分区不足的, 保存
dataList.add(new TopicInsufficientPartition(
clusterDO,
topicName,
topicMetadata.getPartitionNum(),
suggestedPartitionNum - topicMetadata.getPartitionNum(),
maxAvgBytesInMap.getOrDefault(topicName, new ArrayList<>()),
bytesIn / topicMetadata.getPartitionNum(),
new ArrayList<>(topicNameRegionBrokerIdMap.get(topicName))
));
}
return dataList;
}
@Override
public List<TopicAnomalyFlow> getAnomalyFlowTopics(Long timestamp) {
TopicAnomalyFlowConfig config = new TopicAnomalyFlowConfig();
List<TopicAnomalyFlow> anomalyFlowList = new ArrayList<>();
for (ClusterDO clusterDO: clusterService.list()) {
anomalyFlowList.addAll(getAnomalyFlowTopics(clusterDO, timestamp, config));
}
return anomalyFlowList;
}
private List<TopicAnomalyFlow> getAnomalyFlowTopics(ClusterDO clusterDO,
Long timestamp,
TopicAnomalyFlowConfig config) {
List<TopicAnomalyFlow> anomalyFlowList = new ArrayList<>();
for (String topicName: PhysicalClusterMetadataManager.getTopicNameList(clusterDO.getId())) {
TopicMetadata topicMetadata =
PhysicalClusterMetadataManager.getTopicMetadata(clusterDO.getId(), topicName);
}
return anomalyFlowList;
}
@Override
public List<TopicExpiredDO> getExpiredTopics() {
TopicExpiredConfig config = configService.getByKey(ConfigConstant.EXPIRED_TOPIC_CONFIG_KEY, TopicExpiredConfig.class);
if (ValidateUtils.isNull(config)) {
config = new TopicExpiredConfig();
}
List<TopicExpiredDO> expiredTopicList = topicManagerService.getExpiredTopics(config.getMinExpiredDay());
if (ValidateUtils.isEmptyList(expiredTopicList)) {
return new ArrayList<>();
}
List<TopicExpiredDO> filteredExpiredTopicList = new ArrayList<>();
for (TopicExpiredDO elem: expiredTopicList) {
if (config.getIgnoreClusterIdList().contains(elem.getClusterId())) {
continue;
}
filteredExpiredTopicList.add(elem);
}
return filteredExpiredTopicList;
}
}

View File

@@ -0,0 +1,490 @@
package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.common.bizenum.KafkaClientEnum;
import com.xiaojukeji.kafka.manager.common.constant.KafkaMetricsCollections;
import com.xiaojukeji.kafka.manager.common.entity.KafkaVersion;
import com.xiaojukeji.kafka.manager.common.entity.metrics.BaseMetrics;
import com.xiaojukeji.kafka.manager.common.entity.metrics.TopicMetrics;
import com.xiaojukeji.kafka.manager.common.entity.metrics.BrokerMetrics;
import com.xiaojukeji.kafka.manager.common.utils.jmx.JmxConstant;
import com.xiaojukeji.kafka.manager.common.entity.ao.PartitionAttributeDTO;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.utils.jmx.*;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.PartitionState;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.cache.ThreadPool;
import com.xiaojukeji.kafka.manager.service.service.JmxService;
import org.apache.kafka.common.TopicPartition;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import javax.management.*;
import javax.management.Attribute;
import java.util.*;
import java.util.concurrent.*;
/**
* @author tukun, zengqiao
* @date 2015/11/11.
*/
@Service("jmxService")
public class JmxServiceImpl implements JmxService {
private final static Logger LOGGER = LoggerFactory.getLogger(JmxServiceImpl.class);
@Autowired
private PhysicalClusterMetadataManager physicalClusterMetadataManager;
@Override
public BrokerMetrics getBrokerMetrics(Long clusterId, Integer brokerId, Integer metricsCode) {
if (clusterId == null || brokerId == null || metricsCode == null) {
// 参数非法
return null;
}
if (!PhysicalClusterMetadataManager.isBrokerAlive(clusterId, brokerId)) {
// Broker不存在
return null;
}
KafkaVersion kafkaVersion = physicalClusterMetadataManager.getKafkaVersion(clusterId, brokerId);
List<MbeanV2> mbeanV2List = MbeanNameUtilV2.getMbeanList(metricsCode);
if (ValidateUtils.isEmptyList(mbeanV2List)) {
return new BrokerMetrics(clusterId, brokerId);
}
JmxConnectorWrap jmxConnectorWrap = PhysicalClusterMetadataManager.getJmxConnectorWrap(clusterId, brokerId);
if (ValidateUtils.isNull(jmxConnectorWrap) || !jmxConnectorWrap.checkJmxConnectionAndInitIfNeed()) {
LOGGER.warn("get jmx connector failed, clusterId:{} brokerId:{}.", clusterId, brokerId);
return new BrokerMetrics(clusterId, brokerId);
}
BrokerMetrics metrics = new BrokerMetrics(clusterId, brokerId);
for (MbeanV2 mbeanV2 : mbeanV2List) {
try {
getAndSupplyAttributes2BaseMetrics(
metrics,
jmxConnectorWrap,
mbeanV2,
new ObjectName(mbeanV2.getObjectName(kafkaVersion.getVersionNum()))
);
} catch (Exception e) {
LOGGER.error("get broker metrics fail, clusterId:{} brokerId:{} mbean:{}.",
clusterId, brokerId, mbeanV2, e
);
}
}
return metrics;
}
@Override
public List<TopicMetrics> getTopicMetrics(Long clusterId, Integer metricsCode, Boolean byAdd) {
List<String> topicNameList = PhysicalClusterMetadataManager.getTopicNameList(clusterId);
FutureTask<TopicMetrics>[] taskList = new FutureTask[topicNameList.size()];
for (int i = 0; i < topicNameList.size(); i++) {
final String topicName = topicNameList.get(i);
taskList[i] = new FutureTask<TopicMetrics>(new Callable<TopicMetrics>() {
@Override
public TopicMetrics call() throws Exception {
return getTopicMetrics(
clusterId,
topicName,
metricsCode,
byAdd
);
}
});
ThreadPool.submitCollectMetricsTask(taskList[i]);
}
List<TopicMetrics> metricsList = new ArrayList<>();
for (int i = 0; i < taskList.length; ++i) {
try {
TopicMetrics topicMetrics = taskList[i].get();
if (ValidateUtils.isNull(topicMetrics)) {
continue;
}
metricsList.add(topicMetrics);
} catch (Exception e) {
LOGGER.error("get topic-metrics failed, clusterId:{} topicName:{}.",
clusterId, topicNameList.get(i), e
);
}
}
return metricsList;
}
@Override
public TopicMetrics getTopicMetrics(Long clusterId, String topicName, Integer metricsCode, Boolean byAdd) {
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterId, topicName);
if (topicMetadata == null) {
return null;
}
TopicMetrics metrics = null;
for (Integer brokerId : topicMetadata.getBrokerIdSet()) {
TopicMetrics subMetrics = getTopicMetrics(clusterId, brokerId, topicName, metricsCode, byAdd);
if (ValidateUtils.isNull(subMetrics)) {
continue;
}
if (ValidateUtils.isNull(metrics)) {
metrics = new TopicMetrics(clusterId, topicName);
}
if (byAdd) {
metrics.mergeByAdd(subMetrics);
} else {
metrics.mergeByMax(subMetrics);
}
}
return metrics;
}
@Override
public TopicMetrics getTopicMetrics(Long clusterId, Integer brokerId, String topicName, Integer metricsCode, Boolean byAdd) {
List<MbeanV2> mbeanV2List = MbeanNameUtilV2.getMbeanList(metricsCode);
if (ValidateUtils.isEmptyList(mbeanV2List)) {
return null;
}
JmxConnectorWrap jmxConnectorWrap = PhysicalClusterMetadataManager.getJmxConnectorWrap(clusterId, brokerId);
if (ValidateUtils.isNull(jmxConnectorWrap)|| !jmxConnectorWrap.checkJmxConnectionAndInitIfNeed()) {
return null;
}
TopicMetrics metrics = new TopicMetrics(clusterId, topicName);
for (MbeanV2 mbeanV2: mbeanV2List) {
KafkaVersion kafkaVersion = physicalClusterMetadataManager.getKafkaVersion(clusterId, brokerId);
try {
getAndSupplyAttributes2BaseMetrics(
metrics,
jmxConnectorWrap,
mbeanV2,
new ObjectName(mbeanV2.getObjectName(kafkaVersion.getVersionNum()) + ",topic=" + topicName)
);
} catch (Exception e) {
LOGGER.error("get topic metrics failed, clusterId:{} topicName:{} mbean:{}.",
clusterId, topicName, mbeanV2, e
);
}
}
return metrics;
}
private void getAndSupplyAttributes2BaseMetrics(BaseMetrics metrics,
JmxConnectorWrap jmxConnectorWrap,
MbeanV2 mbeanV2,
ObjectName objectName) {
List<Attribute> attributeList = null;
try {
attributeList = jmxConnectorWrap.getAttributes(objectName, mbeanV2.getAttributeEnum().getAttribute()).asList();
} catch (InstanceNotFoundException e) {
return;
} catch (Exception e) {
LOGGER.error("get attributes failed, metrics:{} objectName:{}.", metrics, objectName, e);
}
if (ValidateUtils.isEmptyList(attributeList)) {
return;
}
for (Attribute attribute: attributeList) {
if (JmxAttributeEnum.PERCENTILE_ATTRIBUTE.equals(mbeanV2.getAttributeEnum())) {
metrics.mergeByMax(mbeanV2.getFieldName() + attribute.getName(), attribute.getValue());
} else {
metrics.mergeByAdd(mbeanV2.getFieldName() + attribute.getName(), attribute.getValue());
}
}
}
@Override
public List<TopicMetrics> getTopicAppMetrics(Long clusterId, Integer metricsCode) {
List<MbeanV2> mbeanV2List = MbeanNameUtilV2.getMbeanList(metricsCode);
if (ValidateUtils.isEmptyList(mbeanV2List)) {
return new ArrayList<>();
}
List<Integer> brokerIdList = PhysicalClusterMetadataManager.getBrokerIdList(clusterId);
FutureTask<List<TopicMetrics>>[] taskList = new FutureTask[brokerIdList.size()];
for (int i = 0; i < brokerIdList.size(); ++i) {
final Integer brokerId = brokerIdList.get(i);
taskList[i] = new FutureTask<List<TopicMetrics>>(new Callable<List<TopicMetrics>>() {
@Override
public List<TopicMetrics> call() throws Exception {
List<TopicMetrics> metricsList = new ArrayList<>();
for (MbeanV2 mbeanV2: mbeanV2List) {
List<TopicMetrics> subMetricsList = getTopicAppMetrics(clusterId, brokerId, mbeanV2);
if (ValidateUtils.isEmptyList(subMetricsList)) {
continue;
}
metricsList.addAll(subMetricsList);
}
return metricsList;
}
});
ThreadPool.submitCollectMetricsTask(taskList[i]);
}
Map<String, TopicMetrics> metricsMap = new HashMap<>();
for (int i = 0; i < taskList.length; ++i) {
try {
List<TopicMetrics> metricsList = taskList[i].get();
if (ValidateUtils.isEmptyList(metricsList)) {
continue;
}
for (TopicMetrics elem: metricsList) {
String key = elem.getAppId() + "$" +elem.getTopicName();
TopicMetrics metrics = metricsMap.getOrDefault(
key, new TopicMetrics(elem.getAppId(), elem.getClusterId(), elem.getTopicName())
);
metrics.mergeByAdd(elem);
metricsMap.put(key, metrics);
}
} catch (Exception e) {
LOGGER.error("get app-topic-metrics failed, clusterId:{} brokerId:{}.",
clusterId, brokerIdList.get(i), e
);
}
}
return new ArrayList<>(metricsMap.values());
}
private List<TopicMetrics> getTopicAppMetrics(Long clusterId, Integer brokerId, MbeanV2 mbeanV2) {
JmxConnectorWrap jmxConnectorWrap = PhysicalClusterMetadataManager.getJmxConnectorWrap(clusterId, brokerId);
if (ValidateUtils.isNull(jmxConnectorWrap)|| !jmxConnectorWrap.checkJmxConnectionAndInitIfNeed()) {
return new ArrayList<>();
}
List<TopicMetrics> metricsList = new ArrayList<>();
Set<ObjectName> objectNameSet = queryObjectNameSet(clusterId, brokerId, mbeanV2, jmxConnectorWrap);
for (ObjectName objectName: objectNameSet) {
String topicName = objectName.getKeyProperty(JmxConstant.TOPIC);
String appId = objectName.getKeyProperty(JmxConstant.APP_ID);
if (ValidateUtils.isNull(topicName) || ValidateUtils.isNull(appId)) {
continue;
}
TopicMetrics metrics = new TopicMetrics(appId, clusterId, topicName);
getAndSupplyAttributes2BaseMetrics(metrics, jmxConnectorWrap, mbeanV2, objectName);
metricsList.add(metrics);
}
return metricsList;
}
private Set<ObjectName> queryObjectNameSet(Long clusterId,
Integer brokerId,
MbeanV2 mbeanV2,
JmxConnectorWrap jmxConnectorWrap) {
KafkaVersion kafkaVersion = physicalClusterMetadataManager.getKafkaVersion(clusterId, brokerId);
Set<ObjectName> objectNameSet = new HashSet<>();
for (String attribute: mbeanV2.getAttributeEnum().getAttribute()) {
try {
QueryExp exp = Query.gt(Query.attr(attribute), Query.value(0.0));
objectNameSet.addAll(
jmxConnectorWrap.queryNames(new ObjectName(mbeanV2.getObjectName(kafkaVersion.getVersionNum())), exp)
);
break;
} catch (Exception e) {
LOGGER.error("query objectNames failed, clusterId:{} brokerId:{} mbean:{} attribute:{}.",
clusterId, brokerId, mbeanV2, attribute, e
);
}
}
return objectNameSet;
}
@Override
public Map<TopicPartition, String> getBrokerTopicLocation(Long clusterId, Integer brokerId) {
JmxConnectorWrap jmxConnectorWrap = PhysicalClusterMetadataManager.getJmxConnectorWrap(clusterId, brokerId);
if (ValidateUtils.isNull(jmxConnectorWrap)|| !jmxConnectorWrap.checkJmxConnectionAndInitIfNeed()) {
return new HashMap<>(0);
}
String diskName = "*";
Map<TopicPartition, String> diskNameMap = new HashMap<>();
Set<ObjectName> allObjectNameSet = null;
Long now = System.currentTimeMillis();
while (System.currentTimeMillis() - now <= 5000L) {
// 在超时时间内处理完
try {
ObjectName preObjectName = new ObjectName(
"kafka.log:type=TopicPartitionMetrics,name=TopicPartitionDir,topic=*,partition=*"
);
QueryExp exp = Query.match(Query.attr("Value"), Query.value(diskName));
Set<ObjectName> subObjectNameSet = jmxConnectorWrap.queryNames(preObjectName, exp);
if (!diskName.equals("*") && subObjectNameSet != null) {
allObjectNameSet.removeAll(subObjectNameSet);
for (ObjectName objectName: subObjectNameSet) {
String subObjectName = objectName.toString()
.replaceFirst("kafka.log:type=TopicPartitionMetrics,name=TopicPartitionDir,topic=", "")
.replaceFirst(",partition=", "\t");
String[] topicNamePartitionId = subObjectName.split("\t");
diskNameMap.put(
new TopicPartition(topicNamePartitionId[0], Integer.valueOf(topicNamePartitionId[1])),
diskName
);
}
} else {
allObjectNameSet = subObjectNameSet;
}
if (ValidateUtils.isEmptySet(allObjectNameSet)) {
break;
}
ObjectName objectName = allObjectNameSet.iterator().next();
subObjectNameSet.remove(objectName);
diskName = (String) jmxConnectorWrap.getAttribute(objectName, "Value");
} catch (Exception e) {
LOGGER.error("get broker topic locations failed, clusterId:{} brokerId:{}.", clusterId, brokerId, e);
}
}
return diskNameMap;
}
@Override
public Map<Integer, PartitionAttributeDTO> getPartitionAttribute(Long clusterId,
String topicName,
List<PartitionState> partitionStateList) {
Map<Integer, PartitionAttributeDTO> partitionMap = new HashMap<>();
if (ValidateUtils.isEmptyList(partitionStateList)) {
return partitionMap;
}
Mbean logSizeMbean = MbeanNameUtil.getMbean(JmxConstant.TOPIC_PARTITION_LOG_SIZE);
for (PartitionState partitionState : partitionStateList) {
if (partitionMap.containsKey(partitionState.getPartitionId())) {
continue;
}
JmxConnectorWrap jmxConnectorWrap = PhysicalClusterMetadataManager.getJmxConnectorWrap(clusterId, partitionState.getLeader());
if (ValidateUtils.isNull(jmxConnectorWrap) || !jmxConnectorWrap.checkJmxConnectionAndInitIfNeed()) {
continue;
}
try {
PartitionAttributeDTO dto = new PartitionAttributeDTO();
dto.setLogSize((Long) jmxConnectorWrap.getAttribute(
new ObjectName(
String.format(
logSizeMbean.getObjectName(),
topicName,
partitionState.getPartitionId()
)),
logSizeMbean.getProperty()
));
partitionMap.put(partitionState.getPartitionId(), dto);
} catch (Exception e) {
LOGGER.error("", e);
}
}
return partitionMap;
}
@Override
public Set<String> getBrokerThrottleClients(Long clusterId,
Integer brokerId,
KafkaClientEnum kafkaClientEnum) {
Mbean mbean = MbeanNameUtil.getMbean(kafkaClientEnum.getName() + "ThrottleTime");
if (ValidateUtils.isNull(mbean)) {
return new HashSet<>();
}
JmxConnectorWrap jmxConnectorWrap = PhysicalClusterMetadataManager.getJmxConnectorWrap(clusterId, brokerId);
if (ValidateUtils.isNull(jmxConnectorWrap) || !jmxConnectorWrap.checkJmxConnectionAndInitIfNeed()) {
return new HashSet<>();
}
QueryExp exp = Query.gt(Query.attr(mbean.getProperty()), Query.value(0.0));
ObjectName objectName = null;
try {
objectName = new ObjectName(mbean.getObjectName());
} catch (Exception e) {
LOGGER.error("getBrokerThrottleClients@JmxServiceImpl, get failed, clusterId:{} brokerId:{} mbean:{}.",
clusterId,
brokerId,
mbean,
e);
}
if (ValidateUtils.isNull(objectName)) {
return new HashSet<>();
}
Set<String> clientSet = new HashSet<>();
try {
Set<ObjectName> objectNameSet = jmxConnectorWrap.queryNames(objectName, exp);
if (objectNameSet == null || objectNameSet.isEmpty()) {
return clientSet;
}
for (ObjectName name : objectNameSet) {
clientSet.add(name.toString().substring(mbean.getObjectName().length() - 1));
}
} catch (Exception e) {
LOGGER.error("getBrokerThrottleClients@JmxServiceImpl, get failed, clusterId:{} brokerId:{} mbean:{}.",
clusterId,
brokerId,
mbean,
e);
}
return clientSet;
}
@Override
public String getBrokerVersion(Long clusterId, Integer brokerId) {
JmxConnectorWrap jmxConnectorWrap = PhysicalClusterMetadataManager.getJmxConnectorWrap(clusterId, brokerId);
if (ValidateUtils.isNull(jmxConnectorWrap) || !jmxConnectorWrap.checkJmxConnectionAndInitIfNeed()) {
return "";
}
List<MbeanV2> mbeanV2List = MbeanNameUtilV2.getMbeanList(KafkaMetricsCollections.BROKER_VERSION);
if (mbeanV2List.isEmpty()) {
return "";
}
MbeanV2 mbeanV2 = mbeanV2List.get(0);
try {
return (String) jmxConnectorWrap.getAttribute(
new ObjectName(
mbeanV2.getObjectName(KafkaVersion.VERSION_MAX)
+ ",id="
+ String.valueOf(brokerId)
),
mbeanV2.getAttributeEnum().getAttribute()[0]
);
} catch (Exception e) {
LOGGER.error("get broker version failed, clusterId:{} brokerId:{}.", clusterId, brokerId, e);
}
return "";
}
@Override
public double getTopicAppThrottle(Long clusterId,
Integer brokerId,
String clientId,
KafkaClientEnum kafkaClientEnum) {
double throttle = 0.0;
Mbean mbean = MbeanNameUtil.getMbean(kafkaClientEnum.getName() + "ThrottleTime");
if (ValidateUtils.isNull(mbean)) {
return throttle;
}
JmxConnectorWrap jmxConnectorWrap = PhysicalClusterMetadataManager.getJmxConnectorWrap(clusterId, brokerId);
if (ValidateUtils.isNull(jmxConnectorWrap) || !jmxConnectorWrap.checkJmxConnectionAndInitIfNeed()) {
return throttle;
}
String objectName = mbean.getObjectName().substring(0, mbean.getObjectName().length() - 1) + clientId;
try {
return (Double)jmxConnectorWrap.getAttribute(new ObjectName(objectName), mbean.getProperty());
} catch (InstanceNotFoundException e) {
// object不存在, 不打日志, 避免日志太多
} catch (Exception e) {
LOGGER.error("get topic app throttle failed, clusterId:{} brokerId:{}, clientId:{}, kafkaClientEnum:{}.",
clusterId, brokerId, clientId, kafkaClientEnum, e);
}
return throttle;
}
}

View File

@@ -0,0 +1,75 @@
package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.dao.KafkaBillDao;
import com.xiaojukeji.kafka.manager.common.entity.pojo.KafkaBillDO;
import com.xiaojukeji.kafka.manager.service.service.KafkaBillService;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import java.util.ArrayList;
import java.util.Date;
import java.util.List;
/**
* @author zengqiao
* @date 20/5/12
*/
@Service("kafkaBillService")
public class KafkaBillServiceImpl implements KafkaBillService {
private final static Logger LOGGER = LoggerFactory.getLogger(KafkaBillServiceImpl.class);
@Autowired
private KafkaBillDao kafkaBillDao;
@Override
public int replace(KafkaBillDO kafkaBillDO) {
try {
return kafkaBillDao.replace(kafkaBillDO);
} catch (Exception e) {
LOGGER.error("replace kafka bill failed, kafkaBillDO:{}.", kafkaBillDO, e);
}
return 0;
}
@Override
public List<KafkaBillDO> getByTopicName(Long clusterId, String topicName, Date startTime, Date endTime) {
try {
return kafkaBillDao.getByTopicName(clusterId, topicName, startTime, endTime);
} catch (Exception e) {
LOGGER.error("get kafka bill list failed, clusterId:{}, topicName:{}.", clusterId, topicName, e);
}
return new ArrayList<>();
}
@Override
public List<KafkaBillDO> getByPrincipal(String principal, Date startTime, Date endTime) {
try {
return kafkaBillDao.getByPrincipal(principal, startTime, endTime);
} catch (Exception e) {
LOGGER.error("get kafka bill list failed, principal:{}.", principal, e);
}
return new ArrayList<>();
}
@Override
public List<KafkaBillDO> getByTimeBetween(Date startTime, Date endTime) {
try {
return kafkaBillDao.getByTimeBetween(startTime, endTime);
} catch (Exception e) {
LOGGER.error("get kafka bill list failed, startTime:{}, endTime:{}.", startTime, endTime, e);
}
return new ArrayList<>();
}
@Override
public List<KafkaBillDO> getByGmtDay(String gmtDay) {
try {
return kafkaBillDao.getByGmtDay(gmtDay);
} catch (Exception e) {
LOGGER.error("get kafka bill list failed, gmtDay:{}.", gmtDay, e);
}
return new ArrayList<>();
}
}

View File

@@ -0,0 +1,354 @@
package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.common.bizenum.ClusterModeEnum;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.cluster.LogicalCluster;
import com.xiaojukeji.kafka.manager.common.entity.ao.cluster.LogicalClusterMetrics;
import com.xiaojukeji.kafka.manager.common.entity.metrics.BrokerMetrics;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.AppDO;
import com.xiaojukeji.kafka.manager.common.utils.ListUtils;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.BrokerMetadata;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata;
import com.xiaojukeji.kafka.manager.dao.LogicalClusterDao;
import com.xiaojukeji.kafka.manager.common.entity.pojo.BrokerMetricsDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.LogicalClusterDO;
import com.xiaojukeji.kafka.manager.service.cache.LogicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.service.gateway.AppService;
import com.xiaojukeji.kafka.manager.service.service.BrokerService;
import com.xiaojukeji.kafka.manager.service.service.LogicalClusterService;
import com.xiaojukeji.kafka.manager.service.utils.MetricsConvertUtils;
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 java.util.*;
import java.util.stream.Collectors;
/**
* @author zengqiao
* @date 20/6/29
*/
@Service("logicalClusterService")
public class LogicalClusterServiceImpl implements LogicalClusterService {
private static final Logger LOGGER = LoggerFactory.getLogger(LogicalClusterServiceImpl.class);
@Autowired
private LogicalClusterDao logicalClusterDao;
@Autowired
private BrokerService brokerService;
@Autowired
private AppService appService;
@Autowired
private LogicalClusterMetadataManager logicClusterMetadataManager;
@Autowired
private PhysicalClusterMetadataManager physicalClusterMetadataManager;
@Override
public List<LogicalCluster> getAllLogicalCluster() {
return convert2LogicalClusterList(logicClusterMetadataManager.getLogicalClusterList());
}
@Override
public List<LogicalCluster> getLogicalClusterListByPrincipal(String principal) {
List<LogicalClusterDO> clusterList = logicClusterMetadataManager.getLogicalClusterList()
.stream()
.filter(elem -> ClusterModeEnum.SHARED_MODE.getCode().equals(elem.getMode()))
.collect(Collectors.toList());
if (ValidateUtils.isBlank(principal)) {
return convert2LogicalClusterList(clusterList);
}
// 获取principal名下的应用
List<AppDO> appDOList = appService.getByPrincipal(principal);
if (ValidateUtils.isEmptyList(appDOList)) {
return convert2LogicalClusterList(clusterList);
}
Set<String> appIdSet = appDOList.stream().map(elem -> elem.getAppId()).collect(Collectors.toSet());
// 获取principal名下的集群
for (LogicalClusterDO logicalClusterDO : logicClusterMetadataManager.getLogicalClusterList()) {
if (ClusterModeEnum.SHARED_MODE.getCode().equals(logicalClusterDO.getMode()) ||
!appIdSet.contains(logicalClusterDO.getAppId())) {
continue;
}
clusterList.add(logicalClusterDO);
}
return convert2LogicalClusterList(clusterList);
}
@Override
public LogicalCluster getLogicalCluster(Long logicalClusterId) {
LogicalClusterDO logicalClusterDO = logicClusterMetadataManager.getLogicalCluster(logicalClusterId);
if (ValidateUtils.isNull(logicalClusterDO)) {
return null;
}
return convert2LogicalCluster(logicalClusterDO);
}
private List<LogicalCluster> convert2LogicalClusterList(List<LogicalClusterDO> doList) {
if (ValidateUtils.isEmptyList(doList)) {
return new ArrayList<>();
}
List<LogicalCluster> dtoList = new ArrayList<>();
for (LogicalClusterDO elem: doList) {
dtoList.add(convert2LogicalCluster(elem));
}
return dtoList;
}
private LogicalCluster convert2LogicalCluster(LogicalClusterDO logicalClusterDO) {
if (ValidateUtils.isNull(logicalClusterDO)) {
return null;
}
LogicalCluster logicalCluster = new LogicalCluster();
logicalCluster.setLogicalClusterId(logicalClusterDO.getId());
logicalCluster.setLogicalClusterName(logicalClusterDO.getName());
logicalCluster.setClusterVersion(
physicalClusterMetadataManager.getKafkaVersion(
logicalClusterDO.getClusterId(),
new ArrayList<>(logicClusterMetadataManager.getBrokerIdSet(logicalClusterDO.getId()))
)
);
logicalCluster.setMode(logicalClusterDO.getMode());
logicalCluster.setTopicNum(logicClusterMetadataManager.getTopicNameSet(logicalClusterDO.getId()).size());
logicalCluster.setPhysicalClusterId(logicalClusterDO.getClusterId());
logicalCluster.setBootstrapServers("");
logicalCluster.setDescription(logicalClusterDO.getDescription());
logicalCluster.setGmtCreate(logicalClusterDO.getGmtCreate().getTime());
logicalCluster.setGmtModify(logicalClusterDO.getGmtModify().getTime());
return logicalCluster;
}
@Override
public List<TopicMetadata> getTopicMetadatas(LogicalClusterDO logicalClusterDO) {
if (ValidateUtils.isNull(logicalClusterDO)) {
return new ArrayList<>();
}
List<TopicMetadata> metadataList = new ArrayList<>();
for (String topicName: logicClusterMetadataManager.getTopicNameSet(logicalClusterDO.getId())) {
TopicMetadata topicMetadata =
PhysicalClusterMetadataManager.getTopicMetadata(logicalClusterDO.getClusterId(), topicName);
if (ValidateUtils.isNull(topicMetadata)) {
continue;
}
metadataList.add(topicMetadata);
}
return metadataList;
}
@Override
public List<BrokerMetadata> getBrokerMetadatas(LogicalClusterDO logicalClusterDO) {
if (ValidateUtils.isNull(logicalClusterDO)) {
return new ArrayList<>();
}
List<BrokerMetadata> metadataList = new ArrayList<>();
for (Integer brokerId: logicClusterMetadataManager.getBrokerIdSet(logicalClusterDO.getId())) {
BrokerMetadata brokerMetadata =
PhysicalClusterMetadataManager.getBrokerMetadata(logicalClusterDO.getClusterId(), brokerId);
if (ValidateUtils.isNull(brokerMetadata)) {
continue;
}
metadataList.add(brokerMetadata);
}
return metadataList;
}
@Override
public List<LogicalClusterMetrics> getLogicalClusterMetricsFromDB(LogicalClusterDO logicalClusterDO,
Date startTime,
Date endTime) {
Map<Long, LogicalClusterMetrics> metricsMap = new TreeMap<>();
for (Integer brokerId: logicClusterMetadataManager.getBrokerIdSet(logicalClusterDO.getId())) {
List<BrokerMetricsDO> doList =
brokerService.getBrokerMetricsFromDB(logicalClusterDO.getClusterId(), brokerId, startTime, endTime);
if (ValidateUtils.isEmptyList(doList)) {
continue;
}
for (BrokerMetricsDO brokerMetricsDO: doList) {
Long timestamp = brokerMetricsDO.getGmtCreate().getTime() / 1000;
LogicalClusterMetrics logicalClusterMetrics =
metricsMap.getOrDefault(timestamp, new LogicalClusterMetrics());
BrokerMetrics brokerMetrics = MetricsConvertUtils.convert2BrokerMetrics(brokerMetricsDO);
logicalClusterMetrics.setBytesInPerSec(
logicalClusterMetrics.getBytesInPerSec()
+ brokerMetrics.getBytesInPerSecOneMinuteRate(0.0)
);
logicalClusterMetrics.setBytesOutPerSec(
logicalClusterMetrics.getBytesOutPerSec()
+ brokerMetrics.getBytesOutPerSecOneMinuteRate(0.0));
logicalClusterMetrics.setBytesRejectedPerSec(
logicalClusterMetrics.getBytesRejectedPerSec()
+ brokerMetrics.getBytesRejectedPerSecOneMinuteRate(0.0));
logicalClusterMetrics.setMessagesInPerSec(
logicalClusterMetrics.getMessagesInPerSec()
+ brokerMetrics.getMessagesInPerSecOneMinuteRate(0.0));
logicalClusterMetrics.setTotalProduceRequestsPerSec(
logicalClusterMetrics.getTotalProduceRequestsPerSec()
+ brokerMetrics.getTotalProduceRequestsPerSecOneMinuteRate(0.0));
logicalClusterMetrics.setGmtCreate(timestamp * 1000);
metricsMap.put(timestamp, logicalClusterMetrics);
}
}
return new ArrayList<>(metricsMap.values());
}
@Override
public List<LogicalClusterDO> listAll() {
return logicalClusterDao.listAll();
}
@Override
public ResultStatus createLogicalCluster(LogicalClusterDO logicalClusterDO) {
if (ValidateUtils.isNull(logicalClusterDO)) {
return ResultStatus.PARAM_ILLEGAL;
}
try {
if (existRegionAlreadyInUse(
logicalClusterDO.getClusterId(),
null,
ListUtils.string2LongList(logicalClusterDO.getRegionList()))) {
return ResultStatus.RESOURCE_ALREADY_USED;
}
if (logicalClusterDao.insert(logicalClusterDO) > 0) {
return ResultStatus.SUCCESS;
}
} catch (DuplicateKeyException e) {
LOGGER.error("create logical cluster failed, name already existed, newLogicalClusterDO:{}.",
logicalClusterDO, e);
return ResultStatus.RESOURCE_ALREADY_EXISTED;
} catch (Exception e) {
LOGGER.error("create logical cluster failed, mysql error, newLogicalClusterDO:{}.", logicalClusterDO, e);
return ResultStatus.MYSQL_ERROR;
}
return ResultStatus.MYSQL_ERROR;
}
@Override
public List<LogicalClusterDO> getByPhysicalClusterId(Long physicalClusterId) {
try {
return logicalClusterDao.getByClusterId(physicalClusterId);
} catch (Exception e) {
LOGGER.error("get logical cluster failed, physicalClusterId:{}.", physicalClusterId, e);
}
return new ArrayList<>();
}
@Override
public LogicalClusterDO getById(Long id) {
try {
return logicalClusterDao.getById(id);
} catch (Exception e) {
LOGGER.error("get logical cluster failed, id:{}.", id, e);
}
return null;
}
@Override
public ResultStatus deleteById(Long logicalClusterId) {
if (ValidateUtils.isNull(logicalClusterId)) {
return ResultStatus.PARAM_ILLEGAL;
}
try {
if (logicalClusterDao.deleteById(logicalClusterId) > 0) {
return ResultStatus.SUCCESS;
}
return ResultStatus.RESOURCE_NOT_EXIST;
} catch (Exception e) {
return ResultStatus.MYSQL_ERROR;
}
}
@Override
public ResultStatus updateById(LogicalClusterDO logicalClusterDO) {
if (ValidateUtils.isNull(logicalClusterDO) || ValidateUtils.isNull(logicalClusterDO.getId())) {
return ResultStatus.PARAM_ILLEGAL;
}
try {
LogicalClusterDO oldLogicalClusterDO = logicalClusterDao.getById(logicalClusterDO.getId());
if (ValidateUtils.isNull(oldLogicalClusterDO)) {
return ResultStatus.RESOURCE_NOT_EXIST;
}
if (existRegionAlreadyInUse(
logicalClusterDO.getClusterId(),
logicalClusterDO.getId(),
ListUtils.string2LongList(logicalClusterDO.getRegionList()))) {
return ResultStatus.RESOURCE_ALREADY_USED;
}
if (logicalClusterDao.updateById(logicalClusterDO) > 0) {
return ResultStatus.SUCCESS;
}
} catch (Exception e) {
LOGGER.error("update logical cluster failed, newLogicalClusterDO:{}.", logicalClusterDO, e);
}
return ResultStatus.MYSQL_ERROR;
}
@Override
public ResultStatus updateById(Long logicalClusterId, List<Long> regionIdList) {
if (ValidateUtils.isNullOrLessThanZero(logicalClusterId)
|| ValidateUtils.isEmptyList(regionIdList)) {
return ResultStatus.PARAM_ILLEGAL;
}
try {
LogicalClusterDO logicalClusterDO = logicalClusterDao.getById(logicalClusterId);
if (ValidateUtils.isNull(logicalClusterDO)) {
return ResultStatus.RESOURCE_NOT_EXIST;
}
if (existRegionAlreadyInUse(
logicalClusterDO.getClusterId(),
logicalClusterId,
regionIdList)) {
return ResultStatus.RESOURCE_ALREADY_USED;
}
logicalClusterDO.setRegionList(ListUtils.longList2String(regionIdList));
if (logicalClusterDao.updateById(logicalClusterDO) > 0) {
return ResultStatus.SUCCESS;
}
} catch (Exception e) {
LOGGER.error("update logical cluster failed, logicalClusterId:{} regionIdList:{}.",
logicalClusterId, regionIdList, e);
}
return ResultStatus.MYSQL_ERROR;
}
private boolean existRegionAlreadyInUse(Long physicalClusterId,
Long logicalClusterId,
List<Long> newRegionIdList) {
if (ValidateUtils.isNull(physicalClusterId) || ValidateUtils.isEmptyList(newRegionIdList)) {
return true;
}
List<LogicalClusterDO> doList = this.getByPhysicalClusterId(physicalClusterId);
if (ValidateUtils.isEmptyList(doList)) {
return false;
}
for (LogicalClusterDO logicalClusterDO : doList) {
if (logicalClusterDO.getId().equals(logicalClusterId)) {
// 被logicalClusterId自己使用的忽略
continue;
}
List<Long> regionIdList = ListUtils.string2LongList(logicalClusterDO.getRegionList());
if (ValidateUtils.isEmptyList(regionIdList)) {
continue;
}
if (regionIdList.stream().filter(elem -> newRegionIdList.contains(elem)).count() > 0) {
// 存在被使用的情况
return true;
}
}
return false;
}
}

View File

@@ -0,0 +1,38 @@
package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.common.entity.dto.rd.OperateRecordDTO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.OperateRecordDO;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.dao.OperateRecordDao;
import com.xiaojukeji.kafka.manager.service.service.OperateRecordService;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import java.util.Date;
import java.util.List;
/**
* @author zhongyuankai
* @date 2020/09/03
*/
@Service("operateRecordService")
public class OperateRecordServiceImpl implements OperateRecordService {
@Autowired
private OperateRecordDao operateRecordDao;
@Override
public int insert(OperateRecordDO operateRecordDO) {
return operateRecordDao.insert(operateRecordDO);
}
@Override
public List<OperateRecordDO> queryByCondt(OperateRecordDTO dto) {
return operateRecordDao.queryByCondt(
dto.getModuleId(),
dto.getOperateId(),
dto.getOperator(),
ValidateUtils.isNull(dto.getStartTime()) ? null : new Date(dto.getStartTime()),
ValidateUtils.isNull(dto.getEndTime()) ? null : new Date(dto.getEndTime())
);
}
}

View File

@@ -0,0 +1,337 @@
package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.common.bizenum.TaskStatusReassignEnum;
import com.xiaojukeji.kafka.manager.common.bizenum.TopicReassignActionEnum;
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.reassign.ReassignStatus;
import com.xiaojukeji.kafka.manager.common.entity.dto.op.reassign.ReassignExecDTO;
import com.xiaojukeji.kafka.manager.common.entity.dto.op.reassign.ReassignExecSubDTO;
import com.xiaojukeji.kafka.manager.common.entity.dto.op.reassign.ReassignTopicDTO;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.service.utils.KafkaZookeeperUtils;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata;
import com.xiaojukeji.kafka.manager.dao.ReassignTaskDao;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ReassignTaskDO;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.service.ClusterService;
import com.xiaojukeji.kafka.manager.service.service.ReassignService;
import com.xiaojukeji.kafka.manager.service.service.RegionService;
import com.xiaojukeji.kafka.manager.service.utils.MetricsConvertUtils;
import com.xiaojukeji.kafka.manager.service.utils.TopicReassignUtils;
import kafka.common.TopicAndPartition;
import kafka.utils.ZkUtils;
import org.I0Itec.zkclient.exception.ZkInterruptedException;
import org.I0Itec.zkclient.exception.ZkTimeoutException;
import org.apache.kafka.common.security.JaasUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import scala.collection.JavaConversions;
import scala.collection.Seq;
import java.util.*;
/**
* Topic迁移
* @author zengqiao_cn@163.com
* @date 19/4/16
*/
@Service("reassignService")
public class ReassignServiceImpl implements ReassignService {
private static final Logger LOGGER = LoggerFactory.getLogger(ReassignServiceImpl.class);
@Autowired
private ClusterService clusterService;
@Autowired
private ReassignTaskDao reassignTaskDao;
@Autowired
private RegionService regionService;
@Override
public ResultStatus createTask(List<ReassignTopicDTO> dtoList, String operator) {
if (ValidateUtils.isEmptyList(dtoList)) {
return ResultStatus.PARAM_ILLEGAL;
}
Map<Long, ClusterDO> clusterMap = clusterService.listMap();
Long taskId = System.currentTimeMillis();
List<ReassignTaskDO> doList = new ArrayList<>();
for (ReassignTopicDTO dto: dtoList) {
// 集群是否存在
ClusterDO clusterDO = clusterMap.get(dto.getClusterId());
if (ValidateUtils.isNull(clusterDO)) {
return ResultStatus.CLUSTER_NOT_EXIST;
}
// Topic是否存在
TopicMetadata topicMetadata =
PhysicalClusterMetadataManager.getTopicMetadata(dto.getClusterId(), dto.getTopicName());
if (ValidateUtils.isNull(topicMetadata)) {
return ResultStatus.TOPIC_NOT_EXIST;
}
// 检查参数是否合法
ResultStatus rs = checkParamLegal(dto.getClusterId(), topicMetadata, dto);
if (!ResultStatus.SUCCESS.equals(rs)) {
return rs;
}
// 创建迁移脚本
String reassignmentJson = TopicReassignUtils.generateReassignmentJson(
clusterDO,
dto.getTopicName(),
dto.getPartitionIdList(),
dto.getBrokerIdList()
);
if (ValidateUtils.isBlank(reassignmentJson)) {
return ResultStatus.OPERATION_FAILED;
}
doList.add(MetricsConvertUtils.convert2ReassignTaskDO(taskId, dto.getClusterId(), topicMetadata, dto, reassignmentJson, operator));
}
// step2 任务存储到数据库
try {
if (reassignTaskDao.batchCreate(doList) >= doList.size()) {
return ResultStatus.SUCCESS;
}
} catch (Exception e) {
LOGGER.error("batch create reassign task failed, tasks:{}.", doList.toArray(), e);
}
return ResultStatus.MYSQL_ERROR;
}
/**
* 检查参数是否合法
* 1. 检查Broker是否存活以及存在
* 2. 检查分区是否存在
* 3. 旧的保存时间是否正确
*/
private ResultStatus checkParamLegal(Long clusterId,
TopicMetadata topicMetadata,
ReassignTopicDTO dto) {
// 所有的Region转成BrokerID
List<Integer> brokerIdList = regionService.getFullBrokerIdList(
clusterId,
dto.getRegionId(), dto.getBrokerIdList()
);
if (ValidateUtils.isNull(brokerIdList)) {
return ResultStatus.BROKER_NUM_NOT_ENOUGH;
}
dto.setBrokerIdList(brokerIdList);
// Broker是否存在
List<Integer> clusterBrokerIdList =
PhysicalClusterMetadataManager.getBrokerIdList(clusterId);
for (Integer brokerId : dto.getBrokerIdList()) {
if (!clusterBrokerIdList.contains(brokerId)) {
return ResultStatus.BROKER_NOT_EXIST;
}
}
if (dto.getBrokerIdList().size() < topicMetadata.getReplicaNum()) {
return ResultStatus.BROKER_NUM_NOT_ENOUGH;
}
// 旧的保存时间是否正确
Long realRetentionTime =
PhysicalClusterMetadataManager.getTopicRetentionTime(clusterId, dto.getTopicName());
if (!dto.getOriginalRetentionTime().equals(realRetentionTime)) {
return ResultStatus.PARAM_ILLEGAL;
}
// 分区是否合法
if (ValidateUtils.isEmptyList(dto.getPartitionIdList())) {
return ResultStatus.SUCCESS;
}
Set<Integer> topicPartitionIdSet = topicMetadata.getPartitionMap().getPartitions().keySet();
for (Integer partitionId: dto.getPartitionIdList()) {
if (topicPartitionIdSet.contains(partitionId)) {
continue;
}
return ResultStatus.PARTITION_NOT_EXIST;
}
return ResultStatus.SUCCESS;
}
@Override
public ResultStatus modifyTask(ReassignExecDTO dto, TopicReassignActionEnum actionEnum) {
List<ReassignTaskDO> doList = this.getTask(dto.getTaskId());
if (ValidateUtils.isNull(doList)) {
return ResultStatus.TASK_NOT_EXIST;
}
Set<Integer> statusSet = new HashSet<>();
for (ReassignTaskDO elem: doList) {
statusSet.add(elem.getStatus());
}
// 不处于新建的状态, 则不可执行启动或者取消
if (!statusSet.contains(TaskStatusReassignEnum.NEW.getCode()) || statusSet.size() != 1) {
return ResultStatus.OPERATION_FORBIDDEN;
}
for (ReassignTaskDO elem: doList) {
if (TopicReassignActionEnum.CANCEL.equals(actionEnum)) {
elem.setStatus(TaskStatusReassignEnum.CANCELED.getCode());
} else if (TopicReassignActionEnum.START.equals(actionEnum)) {
elem.setStatus(TaskStatusReassignEnum.RUNNABLE.getCode());
} else {
elem.setBeginTime(new Date(dto.getBeginTime()));
}
}
try {
reassignTaskDao.batchUpdate(doList);
} catch (Exception e) {
LOGGER.error("batch modify status failed, tasks:{}.", doList.toArray(), e);
return ResultStatus.MYSQL_ERROR;
}
return ResultStatus.SUCCESS;
}
@Override
public ResultStatus modifySubTask(ReassignExecSubDTO dto) {
ReassignTaskDO reassignTaskDO = this.getSubTask(dto.getSubTaskId());
if (ValidateUtils.isNull(reassignTaskDO)) {
return ResultStatus.TASK_NOT_EXIST;
}
reassignTaskDO.setRealThrottle(dto.getThrottle());
reassignTaskDO.setMaxThrottle(dto.getMaxThrottle());
reassignTaskDO.setMinThrottle(dto.getMinThrottle());
try {
reassignTaskDao.updateById(reassignTaskDO);
} catch (Exception e) {
LOGGER.error("modify task failed, task:{} req:{}.", reassignTaskDO, e);
return ResultStatus.MYSQL_ERROR;
}
return ResultStatus.SUCCESS;
}
@Override
public List<ReassignTaskDO> getReassignTaskList() {
try {
return reassignTaskDao.listAll();
} catch (Exception e) {
LOGGER.error("list all reassign task error.", e);
}
return new ArrayList<>();
}
@Override
public List<ReassignTaskDO> getTask(Long taskId) {
try {
return reassignTaskDao.getByTaskId(taskId);
} catch (Exception e) {
LOGGER.error("get task failed, taskId:{}.", taskId, e);
}
return null;
}
private ReassignTaskDO getSubTask(Long subTaskId) {
try {
return reassignTaskDao.getSubTask(subTaskId);
} catch (Exception e) {
LOGGER.error("get sub task failed, subTaskId:{}.", subTaskId, e);
}
return null;
}
@Override
public Result<List<ReassignStatus>> getReassignStatus(Long taskId) {
List<ReassignTaskDO> doList = this.getTask(taskId);
if (ValidateUtils.isNull(doList)) {
return Result.buildFrom(ResultStatus.TASK_NOT_EXIST);
}
Map<Long, ClusterDO> clusterMap = clusterService.listMap();
List<ReassignStatus> statusList = new ArrayList<>();
for (ReassignTaskDO elem: doList) {
try {
ReassignStatus reassignStatus = new ReassignStatus();
reassignStatus.setSubTaskId(elem.getId());
reassignStatus.setClusterId(elem.getClusterId());
reassignStatus.setStatus(elem.getStatus());
reassignStatus.setTopicName(elem.getTopicName());
reassignStatus.setRealThrottle(elem.getRealThrottle());
reassignStatus.setMaxThrottle(elem.getMaxThrottle());
reassignStatus.setMinThrottle(elem.getMinThrottle());
reassignStatus.setReassignList(KafkaZookeeperUtils.getReassignmentElemDataList(elem.getReassignmentJson()));
ClusterDO clusterDO = clusterMap.get(elem.getClusterId());
if (!ValidateUtils.isNull(clusterDO)) {
reassignStatus.setClusterName(clusterDO.getClusterName());
}
if (TaskStatusReassignEnum.CANCELED.getCode().equals(elem.getStatus())
|| TaskStatusReassignEnum.NEW.getCode().equals(elem.getStatus())
|| TaskStatusReassignEnum.RUNNABLE.getCode().equals(elem.getStatus())) {
reassignStatus.setReassignStatusMap(new HashMap<>());
statusList.add(reassignStatus);
continue;
}
Map<TopicAndPartition, TaskStatusReassignEnum> statusMap =
verifyAssignment(clusterDO.getZookeeper(), elem.getReassignmentJson());
reassignStatus.setReassignStatusMap(statusMap);
statusList.add(reassignStatus);
} catch (Exception e) {
LOGGER.error("get reassign status failed, taskId:{}.", taskId);
}
}
return new Result<>(statusList);
}
@Override
public Map<TopicAndPartition, TaskStatusReassignEnum> verifyAssignment(String zkAddr, String reassignmentJson) {
ZkUtils zkUtils = null;
try {
zkUtils = ZkUtils.apply(zkAddr,
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
JaasUtils.isZkSecurityEnabled());
return verifyAssignment(zkUtils, reassignmentJson);
} catch (ZkInterruptedException | ZkTimeoutException | IllegalStateException e) {
LOGGER.error("connect zookeeper failed, zkAddr:{}.", zkAddr, e);
} catch (Throwable t) {
LOGGER.error("verify assignment failed, reassignmentJson:{}.", reassignmentJson, t);
} finally {
if (zkUtils != null) {
zkUtils.close();
}
}
return null;
}
@Override
public Map<TopicAndPartition, TaskStatusReassignEnum> verifyAssignment(ZkUtils zkUtils,
String reassignmentJson) {
// 本地迁移Json转Map
Map<TopicAndPartition, Seq<Object>> reassignMap =
JavaConversions.asJavaMap(ZkUtils.parsePartitionReassignmentData(reassignmentJson));
// 从zk获取哪些分区正在迁移
Set<TopicAndPartition> beingReassignedMap =
JavaConversions.asJavaMap(zkUtils.getPartitionsBeingReassigned()).keySet();
// 计算迁移结果
Map<TopicAndPartition, TaskStatusReassignEnum> reassignResult = new HashMap<>(reassignMap.size());
for (TopicAndPartition tp: reassignMap.keySet()) {
if (beingReassignedMap.contains(tp)) {
reassignResult.put(tp, TaskStatusReassignEnum.RUNNING);
continue;
}
boolean status = ValidateUtils.equalList(
JavaConversions.asJavaList(reassignMap.get(tp)),
JavaConversions.asJavaList(zkUtils.getReplicasForPartition(tp.topic(), tp.partition()))
);
reassignResult.put(tp, status? TaskStatusReassignEnum.SUCCEED: TaskStatusReassignEnum.FAILED);
}
return reassignResult;
}
}

View File

@@ -0,0 +1,330 @@
package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.pojo.RegionDO;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata;
import com.xiaojukeji.kafka.manager.dao.RegionDao;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.service.RegionService;
import com.xiaojukeji.kafka.manager.common.utils.ListUtils;
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 java.util.*;
import java.util.stream.Collectors;
/**
* @author zengqiao
* @date 2017/11/13.
*/
@Service("regionService")
public class RegionServiceImpl implements RegionService {
private static final Logger LOGGER = LoggerFactory.getLogger(RegionServiceImpl.class);
@Autowired
private RegionDao regionDao;
@Override
public List<RegionDO> listAll() {
List<RegionDO> doList = null;
try {
doList = regionDao.listAll();
}catch (Exception e) {
LOGGER.error("list all region failed.", e);
}
if (ValidateUtils.isNull(doList)) {
return new ArrayList<>();
}
return doList;
}
@Override
public ResultStatus createRegion(RegionDO regionDO) {
if (ValidateUtils.isNull(regionDO)) {
return ResultStatus.PARAM_ILLEGAL;
}
try {
List<Integer> newBrokerIdList = ListUtils.string2IntList(regionDO.getBrokerList());
if (existBrokerIdAlreadyInRegion(
regionDO.getClusterId(),
newBrokerIdList,
null)) {
return ResultStatus.RESOURCE_ALREADY_USED;
}
if (PhysicalClusterMetadataManager.getNotAliveBrokerNum(regionDO.getClusterId(), newBrokerIdList) > 0) {
return ResultStatus.BROKER_NOT_EXIST;
}
if (regionDao.insert(regionDO) > 0) {
return ResultStatus.SUCCESS;
}
} catch (DuplicateKeyException e) {
LOGGER.error("create region failed, name already existed, newRegionDO:{}.", regionDO, e);
return ResultStatus.RESOURCE_ALREADY_EXISTED;
} catch (Exception e) {
LOGGER.error("create region failed, newRegionDO:{}.", regionDO, e);
return ResultStatus.MYSQL_ERROR;
}
return ResultStatus.MYSQL_ERROR;
}
@Override
public RegionDO getById(Long id) {
return regionDao.getById(id);
}
@Override
public ResultStatus deleteById(Long id) {
if (ValidateUtils.isNull(id)) {
return ResultStatus.PARAM_ILLEGAL;
}
try {
if (regionDao.deleteById(id) > 0) {
return ResultStatus.SUCCESS;
}
} catch (Exception e) {
LOGGER.error("delete region failed, regionId:{}.", id, e);
return ResultStatus.MYSQL_ERROR;
}
return ResultStatus.RESOURCE_NOT_EXIST;
}
@Override
public ResultStatus updateRegion(RegionDO newRegionDO) {
if (ValidateUtils.isNull(newRegionDO) || ValidateUtils.isNull(newRegionDO.getId())) {
return ResultStatus.PARAM_ILLEGAL;
}
try {
RegionDO oldRegionDO = regionDao.getById(newRegionDO.getId());
if (ValidateUtils.isNull(oldRegionDO)) {
return ResultStatus.RESOURCE_NOT_EXIST;
}
if (oldRegionDO.getBrokerList().equals(newRegionDO.getBrokerList())) {
// 没有改变broker列表直接更新
if (regionDao.updateById(newRegionDO) > 0) {
return ResultStatus.SUCCESS;
}
return ResultStatus.MYSQL_ERROR;
}
List<Integer> newBrokerIdList = ListUtils.string2IntList(newRegionDO.getBrokerList());
if (existBrokerIdAlreadyInRegion(
newRegionDO.getClusterId(),
newBrokerIdList,
newRegionDO.getId())) {
return ResultStatus.RESOURCE_ALREADY_USED;
}
if (PhysicalClusterMetadataManager.getNotAliveBrokerNum(newRegionDO.getClusterId(), newBrokerIdList) > 0) {
return ResultStatus.BROKER_NOT_EXIST;
}
if (regionDao.updateById(newRegionDO) > 0) {
return ResultStatus.SUCCESS;
}
} catch (Exception e) {
LOGGER.error("update region failed, newRegionDO:{}", newRegionDO, e);
}
return ResultStatus.MYSQL_ERROR;
}
@Override
public ResultStatus updateRegion(Long clusterId, String newBrokerList) {
if (ValidateUtils.isNull(clusterId) || ValidateUtils.isExistBlank(newBrokerList)) {
return ResultStatus.PARAM_ILLEGAL;
}
RegionDO regionDO = getById(clusterId);
if (ValidateUtils.isNull(regionDO)) {
return ResultStatus.CLUSTER_NOT_EXIST;
}
List<Integer> newBrokerIdList = ListUtils.string2IntList(newBrokerList);
if (ValidateUtils.isEmptyList(newBrokerIdList)) {
return ResultStatus.PARAM_ILLEGAL;
}
regionDO.setBrokerList(newBrokerList);
return updateRegion(regionDO);
}
@Override
public int updateCapacityById(RegionDO regionDO) {
return regionDao.updateCapacityById(regionDO);
}
@Override
public List<RegionDO> getByClusterId(Long clusterId) {
return regionDao.getByClusterId(clusterId);
}
@Override
public Map<Long, Integer> getRegionNum() {
List<RegionDO> regionDoList = null;
try {
regionDoList = regionDao.listAll();
}catch (Exception e) {
LOGGER.error("get region number failed.", e);
}
if (regionDoList == null) {
return new HashMap<>(0);
}
Map<Long, Integer> regionNumMap = new HashMap<>();
for (RegionDO regionDO: regionDoList) {
Integer regionNum = regionNumMap.getOrDefault(regionDO.getClusterId(), 0);
regionNumMap.put(regionDO.getClusterId(), regionNum + 1);
}
return regionNumMap;
}
@Override
public List<Integer> getFullBrokerIdList(Long clusterId, Long regionId, List<Integer> brokerIdList) {
if (ValidateUtils.isNull(regionId)) {
return ValidateUtils.isNull(brokerIdList)? new ArrayList<>(): brokerIdList;
}
RegionDO regionDO = regionDao.getById(regionId);
if (ValidateUtils.isNull(regionDO)) {
return ValidateUtils.isNull(brokerIdList)? new ArrayList<>(): brokerIdList;
}
List<Integer> regionBrokerIdList = ListUtils.string2IntList(regionDO.getBrokerList());
if (ValidateUtils.isNull(regionBrokerIdList)) {
return ValidateUtils.isNull(brokerIdList)? new ArrayList<>(): brokerIdList;
}
Set<Integer> fullBrokerIdSet = new HashSet<>(regionBrokerIdList);
if (ValidateUtils.isNull(brokerIdList)) {
return new ArrayList<>(fullBrokerIdSet);
}
fullBrokerIdSet.addAll(brokerIdList);
return new ArrayList<>(fullBrokerIdSet);
}
@Override
public Map<Integer, RegionDO> convert2BrokerIdRegionMap(List<RegionDO> regionDOList) {
if (regionDOList == null || regionDOList.isEmpty()) {
return new HashMap<>();
}
Map<Integer, RegionDO> brokerIdRegionMap = new HashMap<>();
for (RegionDO regionDO: regionDOList) {
List<Integer> brokerIdList = ListUtils.string2IntList(regionDO.getBrokerList());
if (brokerIdList == null) {
continue;
}
for (Integer brokerId: brokerIdList) {
brokerIdRegionMap.put(brokerId, regionDO);
}
}
return brokerIdRegionMap;
}
private boolean existBrokerIdAlreadyInRegion(Long clusterId, List<Integer> newBrokerIdList, Long regionId) {
if (ValidateUtils.isNull(clusterId) || ValidateUtils.isEmptyList(newBrokerIdList)) {
return true;
}
List<RegionDO> doList = getByClusterId(clusterId);
if (ValidateUtils.isEmptyList(doList)) {
return false;
}
for (RegionDO regionDO : doList) {
if (regionDO.getId().equals(regionId)) {
continue;
}
List<Integer> regionBrokerIdList = ListUtils.string2IntList(regionDO.getBrokerList());
if (ValidateUtils.isEmptyList(regionBrokerIdList)) {
continue;
}
if (regionBrokerIdList.stream().filter(brokerId -> newBrokerIdList.contains(brokerId)).count() > 0) {
return true;
}
}
return false;
}
@Override
public List<Integer> getIdleRegionBrokerList(Long physicalClusterId, List<Long> regionIdList) {
if (ValidateUtils.isNull(physicalClusterId) || ValidateUtils.isEmptyList(regionIdList)) {
return null;
}
List<RegionDO> regionDOList = getByClusterId(physicalClusterId);
if (ValidateUtils.isEmptyList(regionDOList)) {
return null;
}
RegionDO resultRegion = null;
for (RegionDO elem: regionDOList) {
if (!regionIdList.contains(elem.getId())
|| elem.getStatus().equals(1)) {
continue;
}
if (resultRegion == null) {
resultRegion = elem;
}
Long left = elem.getCapacity() - elem.getEstimateUsed();
if (left < 0 || left < (resultRegion.getCapacity() - resultRegion.getEstimateUsed())) {
continue;
}
resultRegion = elem;
}
if (ValidateUtils.isNull(resultRegion)) {
return null;
}
return ListUtils.string2IntList(resultRegion.getBrokerList());
}
@Override
public Map<String, Set<Integer>> getTopicNameRegionBrokerIdMap(Long clusterId) {
Map<Integer, List<Integer>> brokerIdRegionBrokerIdMap = new HashMap<>();
for (RegionDO regionDO: this.getByClusterId(clusterId)) {
List<Integer> brokerIdList = ListUtils.string2IntList(regionDO.getBrokerList());
if (ValidateUtils.isEmptyList(brokerIdList)) {
continue;
}
for (Integer brokerId: brokerIdList) {
brokerIdRegionBrokerIdMap.put(brokerId, brokerIdList);
}
}
Map<String, Set<Integer>> topicNameRegionBrokerIdMap = new HashMap<>();
for (String topicName: PhysicalClusterMetadataManager.getTopicNameList(clusterId)) {
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterId, topicName);
if (ValidateUtils.isNull(topicMetadata)) {
continue;
}
for (Integer brokerId: topicMetadata.getBrokerIdSet()) {
Set<Integer> brokerIdSet = topicNameRegionBrokerIdMap.getOrDefault(topicName, new HashSet<>());
if (brokerIdRegionBrokerIdMap.containsKey(brokerId)) {
// Broker属于某个Region
brokerIdSet.addAll(brokerIdRegionBrokerIdMap.get(brokerId));
} else {
// Broker不属于任何Region
brokerIdSet.add(brokerId);
}
topicNameRegionBrokerIdMap.put(topicName, brokerIdSet);
}
}
return topicNameRegionBrokerIdMap;
}
@Override
public List<RegionDO> getRegionListByTopicName(Long clusterId, String topicName) {
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterId, topicName);
if (ValidateUtils.isNull(topicMetadata)) {
return Collections.emptyList();
}
Set<Integer> brokerIdSet = topicMetadata.getBrokerIdSet();
List<RegionDO> regionDOList = regionDao.getByClusterId(clusterId);
return regionDOList.stream()
.filter(regionDO -> {
List<Integer> brokerIdList = ListUtils.string2IntList(regionDO.getBrokerList());
brokerIdList.retainAll(brokerIdSet);
if (ValidateUtils.isEmptyList(brokerIdList)) {
return false;
}
return true;
})
.collect(Collectors.toList());
}
}

View File

@@ -0,0 +1,111 @@
package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.common.bizenum.KafkaClientEnum;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.entity.metrics.TopicThrottledMetrics;
import com.xiaojukeji.kafka.manager.common.entity.pojo.TopicThrottledMetricsDO;
import com.xiaojukeji.kafka.manager.dao.TopicThrottledMetricsDao;
import com.xiaojukeji.kafka.manager.service.service.JmxService;
import com.xiaojukeji.kafka.manager.service.service.ThrottleService;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import java.util.*;
/**
* 限流信息
* @author zhongyuankai
* @date 20/4/3
*/
@Service("throttleService")
public class ThrottleServiceImpl implements ThrottleService {
private final static Logger LOGGER = LoggerFactory.getLogger(ThrottleServiceImpl.class);
@Autowired
private TopicThrottledMetricsDao topicThrottleDao;
@Autowired
private JmxService jmxService;
@Override
public int insertBatch(List<TopicThrottledMetricsDO> dataList) {
if (ValidateUtils.isEmptyList(dataList)) {
return 0;
}
return topicThrottleDao.insertBatch(dataList);
}
@Override
public List<TopicThrottledMetricsDO> getTopicThrottleFromDB(Long clusterId,
String topicName,
String appId,
Date startTime,
Date endTime) {
List<TopicThrottledMetricsDO> topicThrottleDOList = new ArrayList<>();
try {
topicThrottleDOList =
topicThrottleDao.getTopicThrottle(clusterId, topicName, appId, startTime, endTime);
if (!ValidateUtils.isNull(topicThrottleDOList)) {
return topicThrottleDOList;
}
} catch (Exception e) {
LOGGER.error("get topic throttle failed, clusterId:{} topicName:{} appId:{} startTime:{} endTime:{}.",
clusterId, topicName, appId, startTime, endTime, e);
}
return topicThrottleDOList;
}
@Override
public List<TopicThrottledMetrics> getThrottledTopicsFromJmx(Long clusterId,
Set<Integer> brokerIdSet,
List<KafkaClientEnum> kafkaClientList) {
if (ValidateUtils.isNull(brokerIdSet)
|| ValidateUtils.isNull(clusterId)
|| ValidateUtils.isEmptyList(kafkaClientList)) {
return new ArrayList<>();
}
List<TopicThrottledMetrics> metricsList = new ArrayList<>();
for (KafkaClientEnum kafkaClientEnum: kafkaClientList) {
Map<String, TopicThrottledMetrics> metricsMap = new HashMap<>();
for (Integer brokerId : brokerIdSet) {
Set<String> throttledClients =
jmxService.getBrokerThrottleClients(clusterId, brokerId, kafkaClientEnum);
for (String client: throttledClients) {
TopicThrottledMetrics metrics = metricsMap.get(client);
if (ValidateUtils.isNull(metrics)) {
metrics = buildFrom(clusterId, brokerId, client, kafkaClientEnum);
}
if (ValidateUtils.isNull(metrics)) {
continue;
}
metrics.getBrokerIdSet().add(brokerId);
metricsMap.put(client, metrics);
}
}
metricsList.addAll(metricsMap.values());
}
return metricsList;
}
private TopicThrottledMetrics buildFrom(Long clusterId,
Integer brokerId,
String client,
KafkaClientEnum kafkaClientEnum) {
TopicThrottledMetrics metrics = new TopicThrottledMetrics();
String[] splits = client.split("\\.");
if (splits.length != 2) {
return null;
}
metrics.setAppId(splits[0]);
metrics.setClusterId(clusterId);
metrics.setTopicName(splits[1]);
metrics.setClientType(kafkaClientEnum);
metrics.setBrokerIdSet(new HashSet<>());
metrics.getBrokerIdSet().add(brokerId);
return metrics;
}
}

View File

@@ -0,0 +1,78 @@
package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.topic.TopicExpiredData;
import com.xiaojukeji.kafka.manager.common.entity.pojo.TopicExpiredDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.AppDO;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.dao.TopicExpiredDao;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.service.ConfigService;
import com.xiaojukeji.kafka.manager.service.service.TopicExpiredService;
import com.xiaojukeji.kafka.manager.service.service.gateway.AppService;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import java.util.ArrayList;
import java.util.Date;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
* @author zengqiao
* @date 20/9/2
*/
@Service("topicExpiredServiceImpl")
public class TopicExpiredServiceImpl implements TopicExpiredService {
private static final Logger LOGGER = LoggerFactory.getLogger(TopicExpiredServiceImpl.class);
@Autowired
private AppService appService;
@Autowired
private ConfigService configService;
@Autowired
private TopicExpiredDao topicExpiredDao;
@Override
public List<TopicExpiredData> getExpiredTopicDataList(String username) {
List<TopicExpiredDO> expiredDOList = topicExpiredDao.getExpiredTopics(30);
if (ValidateUtils.isEmptyList(expiredDOList)) {
return new ArrayList<>();
}
List<AppDO> appDOList = appService.getByPrincipal(username);
if (ValidateUtils.isEmptyList(appDOList)) {
return new ArrayList<>();
}
Map<String, AppDO> appMap = appDOList.stream().collect(Collectors.toMap(AppDO::getAppId, elem -> elem));
return new ArrayList<>();
}
@Override
public ResultStatus retainExpiredTopic(Long physicalClusterId, String topicName, Integer retainDays) {
if (ValidateUtils.isNullOrLessThanZero(retainDays)) {
return ResultStatus.PARAM_ILLEGAL;
}
if (!PhysicalClusterMetadataManager.isTopicExist(physicalClusterId, topicName)) {
return ResultStatus.TOPIC_NOT_EXIST;
}
try {
topicExpiredDao.modifyTopicExpiredTime(
physicalClusterId,
topicName,
new Date(System.currentTimeMillis() + retainDays * 24 * 60 * 60 * 1000)
);
return ResultStatus.SUCCESS;
} catch (Exception e) {
LOGGER.error("retain expired topic failed, clusterId:{} topicName:{} retainDays:{}."
,physicalClusterId, topicName, retainDays);
}
return ResultStatus.MYSQL_ERROR;
}
}

View File

@@ -0,0 +1,546 @@
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.entity.Result;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.ao.RdTopicBasic;
import com.xiaojukeji.kafka.manager.common.entity.ao.topic.MineTopicSummary;
import com.xiaojukeji.kafka.manager.common.entity.ao.topic.TopicAppData;
import com.xiaojukeji.kafka.manager.common.entity.ao.topic.TopicBusinessInfo;
import com.xiaojukeji.kafka.manager.common.entity.ao.topic.TopicDTO;
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.NumberUtils;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.config.TopicQuotaData;
import com.xiaojukeji.kafka.manager.dao.TopicDao;
import com.xiaojukeji.kafka.manager.dao.TopicExpiredDao;
import com.xiaojukeji.kafka.manager.dao.TopicStatisticsDao;
import com.xiaojukeji.kafka.manager.common.entity.metrics.TopicThrottledMetrics;
import com.xiaojukeji.kafka.manager.common.entity.pojo.*;
import com.xiaojukeji.kafka.manager.service.cache.KafkaMetricsCache;
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.service.gateway.AuthorityService;
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.stereotype.Service;
import org.springframework.util.StringUtils;
import java.util.*;
import java.util.stream.Collectors;
/**
* @author arthur
* @date 2017/7/21.
*/
@Service("topicManagerService")
public class TopicManagerServiceImpl implements TopicManagerService {
private static final Logger LOGGER = LoggerFactory.getLogger(TopicManagerServiceImpl.class);
@Autowired
private TopicDao topicDao;
@Autowired
private TopicStatisticsDao topicStatisticsDao;
@Autowired
private TopicExpiredDao topicExpiredDao;
@Autowired
private AppService appService;
@Autowired
private AuthorityService authorityService;
@Autowired
private LogicalClusterMetadataManager logicalClusterMetadataManager;
@Autowired
private LogicalClusterService logicalClusterService;
@Autowired
private JmxService jmxService;
@Autowired
private ThrottleService throttleService;
@Autowired
private ClusterService clusterService;
@Override
public List<TopicDO> listAll() {
try {
return topicDao.listAll();
} catch (Exception e) {
}
return new ArrayList<>();
}
@Override
public List<TopicDO> getByClusterId(Long clusterId) {
if (clusterId == null) {
return new ArrayList<>();
}
return topicDao.getByClusterId(clusterId);
}
@Override
public TopicDO getByTopicName(Long clusterId, String topicName) {
if (StringUtils.isEmpty(topicName) || clusterId == null) {
return null;
}
try {
return topicDao.getByTopicName(clusterId, topicName);
} catch (Exception e) {
LOGGER.error("select failed, clusterId:{}.topicName:{}.", clusterId, topicName, e);
}
return null;
}
@Override
public int replaceTopicStatistics(TopicStatisticsDO topicStatisticsDO) {
return topicStatisticsDao.replace(topicStatisticsDO);
}
@Override
public Map<String, List<Double>> getTopicMaxAvgBytesIn(Long clusterId, Integer latestDay, Double minMaxAvgBytesIn) {
Date startTime = new Date(DateUtils.getDayStarTime(latestDay));
List<TopicStatisticsDO> doList =
topicStatisticsDao.getTopicStatisticData(clusterId, startTime, minMaxAvgBytesIn);
if (ValidateUtils.isEmptyList(doList)) {
return new HashMap<>(0);
}
Map<String, List<Double>> doMap = new HashMap<>(2);
for (TopicStatisticsDO elem: doList) {
List<Double> subDOList = doMap.getOrDefault(elem.getTopicName(), new ArrayList<>());
subDOList.add(elem.getMaxAvgBytesIn());
doMap.put(elem.getTopicName(), subDOList);
}
return doMap;
}
@Override
public Double getTopicMaxAvgBytesIn(Long clusterId,
String topicName,
Date startTime,
Date endTime,
Integer maxAvgDay) {
return topicStatisticsDao.getTopicMaxAvgBytesIn(clusterId, topicName, startTime, endTime, maxAvgDay);
}
@Override
public TopicStatisticsDO getByTopicAndDay(Long clusterId, String topicName, String gmtDay) {
return topicStatisticsDao.getByTopicAndDay(clusterId, topicName, gmtDay);
}
@Override
public List<TopicExpiredDO> getExpiredTopics(Integer expiredDay) {
return topicExpiredDao.getExpiredTopics(expiredDay);
}
@Override
public List<MineTopicSummary> getMyTopics(String username) {
List<AppDO> appDOList = appService.getByPrincipal(username);
if (ValidateUtils.isEmptyList(appDOList)) {
return new ArrayList<>();
}
// 获取app创建的topic
Set<String> appIdSet = appDOList.stream().map(appDO -> appDO.getAppId()).collect(Collectors.toSet());
Map<String, Set<String>> appTopicNameMap = new HashMap<>();
for (TopicDO topicDO : topicDao.listAll()) {
if (!appIdSet.contains(topicDO.getAppId())) {
continue;
}
Set<String> topicNameSet = appTopicNameMap.getOrDefault(topicDO.getAppId(), new HashSet<>());
topicNameSet.add(topicDO.getTopicName());
appTopicNameMap.put(topicDO.getAppId(), topicNameSet);
}
Map<String, Map<Long, Map<String, AuthorityDO>>> appMap = authorityService.getAllAuthority();
// 增加权限信息和App信息
List<MineTopicSummary> summaryList = new ArrayList<>();
for (AppDO appDO : appDOList) {
// 查权限
for (Map<String, AuthorityDO> subMap : appMap.getOrDefault(appDO.getAppId(), Collections.emptyMap()).values()) {
for (AuthorityDO authorityDO : subMap.values()) {
if (!PhysicalClusterMetadataManager.isTopicExist(authorityDO.getClusterId(), authorityDO.getTopicName())
|| TopicAuthorityEnum.DENY.getCode().equals(authorityDO.getAccess())) {
continue;
}
MineTopicSummary mineTopicSummary = convert2MineTopicSummary(
appDO,
authorityDO,
appTopicNameMap.getOrDefault(authorityDO.getAppId(), Collections.emptySet())
);
if (ValidateUtils.isNull(mineTopicSummary)) {
continue;
}
summaryList.add(mineTopicSummary);
}
}
}
// 增加流量信息
Map<Long, Map<String, TopicMetrics>> metricMap = KafkaMetricsCache.getAllTopicMetricsFromCache();
for (MineTopicSummary mineTopicSummary : summaryList) {
TopicMetrics topicMetrics = getTopicMetricsFromCacheOrJmx(
mineTopicSummary.getPhysicalClusterId(),
mineTopicSummary.getTopicName(),
metricMap);
mineTopicSummary.setBytesIn(topicMetrics.getSpecifiedMetrics("BytesInPerSecOneMinuteRate"));
mineTopicSummary.setBytesOut(topicMetrics.getSpecifiedMetrics("BytesOutPerSecOneMinuteRate"));
}
return summaryList;
}
private MineTopicSummary convert2MineTopicSummary(AppDO appDO, AuthorityDO authorityDO, Set<String> topicNameSet) {
MineTopicSummary mineTopicSummary = new MineTopicSummary();
LogicalClusterDO logicalClusterDO = logicalClusterMetadataManager.getTopicLogicalCluster(
authorityDO.getClusterId(),
authorityDO.getTopicName()
);
if (ValidateUtils.isNull(logicalClusterDO)) {
return null;
}
mineTopicSummary.setLogicalClusterId(logicalClusterDO.getId());
mineTopicSummary.setLogicalClusterName(logicalClusterDO.getName());
mineTopicSummary.setPhysicalClusterId(logicalClusterDO.getClusterId());
mineTopicSummary.setTopicName(authorityDO.getTopicName());
mineTopicSummary.setBytesIn(null);
mineTopicSummary.setBytesOut(null);
mineTopicSummary.setAppId(appDO.getAppId());
mineTopicSummary.setAppName(appDO.getName());
mineTopicSummary.setAppPrincipals(appDO.getPrincipals());
mineTopicSummary.setAccess(
topicNameSet.contains(authorityDO.getTopicName()) ?
TopicAuthorityEnum.OWNER.getCode()
: authorityDO.getAccess()
);
return mineTopicSummary;
}
private TopicMetrics getTopicMetricsFromCacheOrJmx(Long physicalClusterId,
String topicName,
Map<Long, Map<String, TopicMetrics>> metricsMap) {
Map<String, TopicMetrics> subMetricsMap = metricsMap.getOrDefault(physicalClusterId, new HashMap<>());
if (subMetricsMap.containsKey(topicName)) {
return subMetricsMap.get(topicName);
}
TopicMetrics topicMetrics = jmxService.getTopicMetrics(
physicalClusterId,
topicName,
KafkaMetricsCollections.TOPIC_METRICS_TO_DB,
true
);
if (ValidateUtils.isNull(topicMetrics)) {
topicMetrics = new TopicMetrics(physicalClusterId, topicName);
}
subMetricsMap.put(topicName, topicMetrics);
metricsMap.put(physicalClusterId, subMetricsMap);
return topicMetrics;
}
@Override
public List<TopicDTO> getTopics(String username) {
List<ClusterDO> clusterDOList = clusterService.list();
if (ValidateUtils.isEmptyList(clusterDOList)) {
return new ArrayList<>();
}
List<AppDO> appList = appService.listAll();
if (ValidateUtils.isNull(appList)) {
appList = new ArrayList<>();
}
Map<String, AppDO> appMap = new HashMap<>(appList.size());
for (AppDO appDO : appList) {
appMap.put(appDO.getAppId(), appDO);
}
List<TopicDO> topicList = this.listAll();
if (ValidateUtils.isNull(topicList)) {
return new ArrayList<>();
}
Map<Long, Map<String, TopicDO>> topicMap = new HashMap<>(appList.size());
for (TopicDO topicDO: topicList) {
Map<String, TopicDO> subTopicMap = topicMap.getOrDefault(topicDO.getClusterId(), new HashMap<>());
subTopicMap.put(topicDO.getTopicName(), topicDO);
topicMap.put(topicDO.getClusterId(), subTopicMap);
}
List<TopicDTO> dtoList = new ArrayList<>();
for (ClusterDO clusterDO: clusterDOList) {
dtoList.addAll(getTopics(clusterDO, appMap, topicMap.getOrDefault(clusterDO.getId(), new HashMap<>())));
}
return dtoList;
}
private List<TopicDTO> getTopics(ClusterDO clusterDO,
Map<String, AppDO> appMap,
Map<String, TopicDO> topicMap) {
Boolean needAuth = !ValidateUtils.isBlank(clusterDO.getSecurityProperties());
List<TopicDTO> dtoList = new ArrayList<>();
for (String topicName: PhysicalClusterMetadataManager.getTopicNameList(clusterDO.getId())) {
LogicalClusterDO logicalClusterDO = logicalClusterMetadataManager.getTopicLogicalCluster(
clusterDO.getId(),
topicName
);
if (ValidateUtils.isNull(logicalClusterDO)) {
continue;
}
TopicDTO dto = new TopicDTO();
dtoList.add(dto);
dto.setLogicalClusterId(logicalClusterDO.getId());
dto.setLogicalClusterName(logicalClusterDO.getName());
dto.setTopicName(topicName);
dto.setNeedAuth(needAuth);
TopicDO topicDO = topicMap.get(topicName);
if (ValidateUtils.isNull(topicDO)) {
continue;
}
dto.setDescription(topicDO.getDescription());
dto.setAppId(topicDO.getAppId());
AppDO appDO = appMap.get(topicDO.getAppId());
if (ValidateUtils.isNull(appDO)) {
continue;
}
dto.setAppName(appDO.getName());
dto.setAppPrincipals(appDO.getPrincipals());
}
return dtoList;
}
@Override
public ResultStatus modifyTopic(Long clusterId, String topicName, String description, String operator) {
try {
if (!PhysicalClusterMetadataManager.isTopicExist(clusterId, topicName)) {
return ResultStatus.TOPIC_NOT_EXIST;
}
TopicDO topicDO = topicDao.getByTopicName(clusterId, topicName);
if (ValidateUtils.isNull(topicDO)) {
return ResultStatus.TOPIC_NOT_EXIST;
}
topicDO.setDescription(description);
if (topicDao.updateByName(topicDO) > 0) {
return ResultStatus.SUCCESS;
}
return ResultStatus.MYSQL_ERROR;
} 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 {
return topicDao.deleteByName(clusterId, topicName);
} catch (Exception e) {
LOGGER.error("delete topic failed, clusterId:{} topicName:{}", clusterId, topicName, e);
}
return 0;
}
@Override
public int addTopic(TopicDO topicDO) {
try {
return topicDao.insert(topicDO);
} catch (Exception e) {
LOGGER.error("insert topic failed, TopicDO:{}", topicDO.toString(), e);
}
return 0;
}
@Override
public List<TopicAppData> getTopicAuthorizedApps(Long physicalClusterId, String topicName) {
TopicMetadata topicMetaData = PhysicalClusterMetadataManager.getTopicMetadata(physicalClusterId, topicName);
if (ValidateUtils.isNull(topicMetaData)) {
// Topic不存在
return new ArrayList<>();
}
List<AuthorityDO> authorityDOList = authorityService.getAuthorityByTopic(physicalClusterId, topicName);
if (ValidateUtils.isEmptyList(authorityDOList)) {
// 无任何权限
return new ArrayList<>();
}
List<TopicThrottledMetrics> throttleList = throttleService.getThrottledTopicsFromJmx(
physicalClusterId,
topicMetaData.getBrokerIdSet(),
Arrays.asList(KafkaClientEnum.values())
);
if (ValidateUtils.isNull(throttleList)) {
throttleList = new ArrayList<>();
}
// idx-0: produce, idx-1: fetch
List<Set<String>> throttledSetList = Arrays.asList(new HashSet<>(), new HashSet<>());
for (TopicThrottledMetrics metrics : throttleList) {
if (!topicName.equals(metrics.getTopicName())) {
continue;
}
throttledSetList.get(
KafkaClientEnum.PRODUCE_CLIENT.equals(metrics.getClientType())? 0: 1)
.add(metrics.getAppId());
}
List<TopicAppData> dtoList = new ArrayList<>();
for (AuthorityDO authority : authorityDOList) {
TopicAppData dto = new TopicAppData();
dto.setClusterId(physicalClusterId);
dto.setTopicName(topicName);
dto.setAppId(authority.getAppId());
dto.setAccess(authority.getAccess());
dto.setProduceThrottled(throttledSetList.get(0).contains(authority.getAppId()));
dto.setFetchThrottled(throttledSetList.get(1).contains(authority.getAppId()));
AppDO appDO = appService.getByAppId(authority.getAppId());
if (!ValidateUtils.isNull(appDO)) {
dto.setAppName(appDO.getName());
dto.setAppPrincipals(appDO.getPrincipals());
}
TopicQuotaData quota = KafkaZookeeperUtils.getTopicQuota(
PhysicalClusterMetadataManager.getZKConfig(physicalClusterId),
authority.getAppId(),
topicName
);
if (!ValidateUtils.isNull(quota)) {
dto.setConsumerQuota(NumberUtils.string2Long(quota.getConsumer_byte_rate()));
dto.setProduceQuota(NumberUtils.string2Long(quota.getProducer_byte_rate()));
}
dtoList.add(dto);
}
return dtoList;
}
@Override
public List<TopicAppData> getTopicMineApps(Long physicalClusterId, String topicName, String username) {
TopicMetadata topicMetaData = PhysicalClusterMetadataManager.getTopicMetadata(physicalClusterId, topicName);
if (ValidateUtils.isNull(topicMetaData)) {
return new ArrayList<>();
}
List<AppDO> appDOList = appService.getByPrincipal(username);
if (ValidateUtils.isEmptyList(appDOList)) {
return new ArrayList<>();
}
List<AuthorityDO> authorityDOList = authorityService.getAuthorityByTopic(physicalClusterId, topicName);
if (ValidateUtils.isNull(authorityDOList)) {
authorityDOList = new ArrayList<>();
}
Map<String, Integer> accessMap = new HashMap<>();
for (AuthorityDO authorityDO: authorityDOList) {
accessMap.put(authorityDO.getAppId(), authorityDO.getAccess());
}
List<TopicAppData> dataList = new ArrayList<>();
for (AppDO appDO : appDOList) {
TopicAppData dto = new TopicAppData();
dto.setClusterId(physicalClusterId);
dto.setTopicName(topicName);
dto.setAppId(appDO.getAppId());
dto.setAppName(appDO.getName());
dto.setAppPrincipals(appDO.getPrincipals());
dto.setAccess(accessMap.getOrDefault(appDO.getAppId(), TopicAuthorityEnum.DENY.getCode()));
TopicQuotaData quota = KafkaZookeeperUtils.getTopicQuota(
PhysicalClusterMetadataManager.getZKConfig(physicalClusterId),
appDO.getAppId(),
topicName
);
if (!ValidateUtils.isNull(quota)) {
dto.setConsumerQuota(NumberUtils.string2Long(quota.getConsumer_byte_rate()));
dto.setProduceQuota(NumberUtils.string2Long(quota.getProducer_byte_rate()));
}
dataList.add(dto);
}
return dataList;
}
@Override
public Result<RdTopicBasic> getRdTopicBasic(Long physicalClusterId, String topicName) {
ClusterDO clusterDO = clusterService.getById(physicalClusterId);
if (ValidateUtils.isNull(clusterDO)) {
return Result.buildFrom(ResultStatus.CLUSTER_NOT_EXIST);
}
if (!PhysicalClusterMetadataManager.isTopicExist(physicalClusterId, topicName)) {
return Result.buildFrom(ResultStatus.TOPIC_NOT_EXIST);
}
Properties properties = KafkaZookeeperUtils.getTopicProperties(
PhysicalClusterMetadataManager.getZKConfig(physicalClusterId),
topicName
);
TopicDO topicDO = getByTopicName(physicalClusterId, topicName);
if (ValidateUtils.isNull(topicDO)) {
return new Result<>(convert2RdTopicBasic(clusterDO, topicName, null, null, properties));
}
AppDO appDO = appService.getByAppId(topicDO.getAppId());
return new Result<>(convert2RdTopicBasic(clusterDO, topicName, topicDO, appDO, properties));
}
@Override
public List<TopicStatisticsDO> getTopicStatistic(Long clusterId, String topicName, Date startTime, Date endTime) {
return topicStatisticsDao.getTopicStatistic(clusterId, topicName, startTime, endTime);
}
@Override
public TopicBusinessInfo getTopicBusinessInfo(Long clusterId, String topicName) {
TopicDO topicDO = getByTopicName(clusterId, topicName);
if (ValidateUtils.isNull(topicDO)) {
return null;
}
TopicBusinessInfo topicBusinessInfo = new TopicBusinessInfo();
topicBusinessInfo.setClusterId(clusterId);
topicBusinessInfo.setTopicName(topicName);
AppDO appDO = appService.getByAppId(topicDO.getAppId());
if (ValidateUtils.isNull(appDO)) {
return topicBusinessInfo;
}
topicBusinessInfo.setAppId(appDO.getAppId());
topicBusinessInfo.setAppName(appDO.getName());
topicBusinessInfo.setPrincipals(appDO.getPrincipals());
return topicBusinessInfo;
}
private RdTopicBasic convert2RdTopicBasic(ClusterDO clusterDO,
String topicName,
TopicDO topicDO,
AppDO appDO,
Properties properties) {
RdTopicBasic rdTopicBasic = new RdTopicBasic();
rdTopicBasic.setClusterId(clusterDO.getId());
rdTopicBasic.setClusterName(clusterDO.getClusterName());
rdTopicBasic.setTopicName(topicName);
if (!ValidateUtils.isNull(appDO)) {
rdTopicBasic.setAppId(appDO.getAppId());
rdTopicBasic.setAppName(appDO.getName());
}
if (!ValidateUtils.isNull(topicDO)) {
rdTopicBasic.setDescription(topicDO.getDescription());
}
rdTopicBasic.setProperties(properties);
rdTopicBasic.setRetentionTime(KafkaZookeeperUtils.getTopicRetentionTime(properties));
return rdTopicBasic;
}
}

View File

@@ -0,0 +1,866 @@
package com.xiaojukeji.kafka.manager.service.service.impl;
import com.xiaojukeji.kafka.manager.common.bizenum.TopicOffsetChangedEnum;
import com.xiaojukeji.kafka.manager.common.entity.Result;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.pojo.gateway.AppDO;
import com.xiaojukeji.kafka.manager.common.bizenum.OffsetPosEnum;
import com.xiaojukeji.kafka.manager.common.constant.Constant;
import com.xiaojukeji.kafka.manager.common.constant.KafkaMetricsCollections;
import com.xiaojukeji.kafka.manager.common.constant.TopicSampleConstant;
import com.xiaojukeji.kafka.manager.common.entity.ao.PartitionAttributeDTO;
import com.xiaojukeji.kafka.manager.common.entity.ao.PartitionOffsetDTO;
import com.xiaojukeji.kafka.manager.common.entity.ao.topic.*;
import com.xiaojukeji.kafka.manager.common.entity.dto.normal.TopicDataSampleDTO;
import com.xiaojukeji.kafka.manager.common.entity.metrics.TopicMetrics;
import com.xiaojukeji.kafka.manager.common.utils.ListUtils;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.utils.jmx.JmxConstant;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.BrokerMetadata;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.PartitionMap;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.PartitionState;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata;
import com.xiaojukeji.kafka.manager.dao.TopicAppMetricsDao;
import com.xiaojukeji.kafka.manager.dao.TopicMetricsDao;
import com.xiaojukeji.kafka.manager.dao.TopicRequestMetricsDao;
import com.xiaojukeji.kafka.manager.common.entity.pojo.*;
import com.xiaojukeji.kafka.manager.service.cache.KafkaClientPool;
import com.xiaojukeji.kafka.manager.service.cache.KafkaMetricsCache;
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.utils.KafkaZookeeperUtils;
import com.xiaojukeji.kafka.manager.service.utils.MetricsConvertUtils;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
import org.apache.kafka.common.TopicPartition;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import org.springframework.util.StringUtils;
import java.util.*;
/**
* @author limeng
* @date 2018/5/4.
*/
@Service("topicService")
public class TopicServiceImpl implements TopicService {
private final static Logger LOGGER = LoggerFactory.getLogger(TopicService.class);
@Autowired
private TopicMetricsDao topicMetricsDao;
@Autowired
private TopicAppMetricsDao topicAppMetricsDao;
@Autowired
private ThrottleService topicThrottleService;
@Autowired
private JmxService jmxService;
@Autowired
private TopicRequestMetricsDao topicRequestMetricsDao;
@Autowired
private TopicManagerService topicManagerService;
@Autowired
private AppService appService;
@Autowired
private LogicalClusterMetadataManager logicalClusterMetadataManager;
@Autowired
private ClusterService clusterService;
@Override
public List<TopicMetricsDO> getTopicMetricsFromDB(Long clusterId, String topicName, Date startTime, Date endTime) {
try {
return topicMetricsDao.getTopicMetrics(clusterId, topicName, startTime, endTime);
} catch (Exception e) {
LOGGER.error("select topic metrics failed, clusterId:{} topicName:{}.", clusterId, topicName, e);
}
return null;
}
@Override
public List<TopicMetricsDTO> getTopicMetricsFromDB(String appId,
Long clusterId,
String topicName,
Date startTime,
Date endTime) {
List<TopicMetricsDO> topicMetricsDOList = this.getTopicMetricsFromDB(clusterId, topicName, startTime, endTime);
if (ValidateUtils.isNull(topicMetricsDOList)) {
topicMetricsDOList = new ArrayList<>();
}
List<TopicMetrics> topicMetricsList = MetricsConvertUtils.convert2TopicMetricsList(topicMetricsDOList);
// topic限流
List<TopicThrottledMetricsDO> topicThrottleDOList =
topicThrottleService.getTopicThrottleFromDB(clusterId, topicName, appId, startTime, endTime);
if (ValidateUtils.isNull(topicThrottleDOList)) {
topicThrottleDOList = new ArrayList<>();
}
// appId+topic维度流量信息
List<TopicMetricsDO> topicAppIdMetricsDOList =
topicAppMetricsDao.getTopicAppMetrics(clusterId, topicName, appId, startTime, endTime);
if (ValidateUtils.isNull(topicAppIdMetricsDOList)) {
topicAppIdMetricsDOList = new ArrayList<>();
}
List<TopicMetrics> topicAppIdMetricsList =
MetricsConvertUtils.convert2TopicMetricsList(topicAppIdMetricsDOList);
Map<Long, TopicMetricsDTO> dtoMap = new TreeMap<>();
for (TopicMetrics metrics: topicMetricsList) {
Long timestamp = metrics.getSpecifiedMetrics(JmxConstant.CREATE_TIME, Long.class);
if (ValidateUtils.isNull(timestamp)) {
continue;
}
timestamp = timestamp / 1000 / 60;
TopicMetricsDTO dto = dtoMap.getOrDefault(timestamp, new TopicMetricsDTO());
dto.setBytesInPerSec(metrics.getSpecifiedMetrics("BytesInPerSecOneMinuteRate"));
dto.setBytesOutPerSec(metrics.getSpecifiedMetrics("BytesOutPerSecOneMinuteRate"));
dto.setBytesRejectedPerSec(metrics.getSpecifiedMetrics("BytesRejectedPerSecOneMinuteRate"));
dto.setMessagesInPerSec(metrics.getSpecifiedMetrics("MessagesInPerSecOneMinuteRate"));
dto.setTotalProduceRequestsPerSec(metrics.getSpecifiedMetrics("TotalProduceRequestsPerSecOneMinuteRate"));
dto.setGmtCreate(timestamp * 1000 * 60);
dtoMap.put(timestamp, dto);
}
for (TopicThrottledMetricsDO data: topicThrottleDOList) {
Long timestamp = (data.getGmtCreate().getTime() / 1000 / 60);
TopicMetricsDTO dto = dtoMap.getOrDefault(timestamp, new TopicMetricsDTO());
dto.setConsumeThrottled(data.getFetchThrottled() > 0);
dto.setProduceThrottled(data.getProduceThrottled() > 0);
dto.setGmtCreate(timestamp * 1000 * 60);
dtoMap.put(timestamp, dto);
}
for (TopicMetrics metrics: topicAppIdMetricsList) {
Long timestamp = metrics.getSpecifiedMetrics(JmxConstant.CREATE_TIME, Long.class);
if (ValidateUtils.isNull(timestamp)) {
continue;
}
timestamp = timestamp / 1000 / 60;
TopicMetricsDTO dto = dtoMap.getOrDefault(timestamp, new TopicMetricsDTO());
dto.setAppIdBytesInPerSec(metrics.getSpecifiedMetrics("TopicAppIdBytesInPerSecOneMinuteRate"));
dto.setAppIdBytesOutPerSec(metrics.getSpecifiedMetrics("TopicAppIdBytesOutPerSecOneMinuteRate"));
dto.setAppIdMessagesInPerSec(metrics.getSpecifiedMetrics("TopicAppIdMessagesInPerSecOneMinuteRate"));
dto.setGmtCreate(timestamp * 1000 * 60);
dtoMap.put(timestamp, dto);
}
return new ArrayList<>(dtoMap.values());
}
@Override
public Double getMaxAvgBytesInFromDB(Long clusterId, String topicName, Date startTime, Date endTime) {
List<TopicMetricsDO> doList = getTopicMetricsFromDB(clusterId, topicName, startTime, endTime);
if (ValidateUtils.isEmptyList(doList)) {
return null;
}
List<TopicMetrics> metricsList = MetricsConvertUtils.convert2TopicMetricsList(doList);
Double bytesInSum = 0.0, bytesInSumTemp = 0.0;
for (int idx = 0; idx < metricsList.size(); ++idx) {
Double bytesIn = metricsList.get(idx).getBytesInPerSecOneMinuteRate(0.0);
bytesInSumTemp += bytesIn;
if (idx >= Constant.MAX_AVG_BYTES_DURATION) {
bytesInSumTemp -= (metricsList.get(idx - Constant.MAX_AVG_BYTES_DURATION)).getBytesInPerSecOneMinuteRate(0.0);
}
if (bytesInSumTemp > bytesInSum) {
bytesInSum = bytesInSumTemp;
}
}
return bytesInSum / Math.min(Constant.MAX_AVG_BYTES_DURATION, doList.size());
}
@Override
public Map<String, List<Integer>> getTopicPartitionIdMap(Long clusterId, Integer brokerId) {
Map<String, List<Integer>> result = new HashMap<>();
for (String topicName : PhysicalClusterMetadataManager.getTopicNameList(clusterId)) {
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterId, topicName);
if (topicMetadata == null) {
continue;
}
Set<Integer> brokerIdSet = topicMetadata.getBrokerIdSet();
if (brokerIdSet == null || !brokerIdSet.contains(brokerId)) {
continue;
}
PartitionMap partitionMap = topicMetadata.getPartitionMap();
result.put(topicName, new ArrayList<>(partitionMap.getPartitions().keySet()));
}
return result;
}
@Override
public TopicBasicDTO getTopicBasicDTO(Long clusterId, String topicName) {
TopicBasicDTO basicDTO = new TopicBasicDTO();
basicDTO.setClusterId(clusterId);
basicDTO.setTopicName(topicName);
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterId, topicName);
if (topicMetadata == null) {
return basicDTO;
}
basicDTO.setBrokerNum(topicMetadata.getBrokerIdSet().size());
basicDTO.setReplicaNum(topicMetadata.getReplicaNum());
basicDTO.setPartitionNum(topicMetadata.getPartitionNum());
basicDTO.setCreateTime(topicMetadata.getCreateTime());
basicDTO.setModifyTime(topicMetadata.getModifyTime());
basicDTO.setRetentionTime(PhysicalClusterMetadataManager.getTopicRetentionTime(clusterId, topicName));
TopicDO topicDO = topicManagerService.getByTopicName(clusterId, topicName);
if (!ValidateUtils.isNull(topicDO)) {
basicDTO.setDescription(topicDO.getDescription());
}
AppDO appDO = ValidateUtils.isNull(topicDO)? null: appService.getByAppId(topicDO.getAppId());
if (!ValidateUtils.isNull(appDO)) {
basicDTO.setAppId(appDO.getAppId());
basicDTO.setAppName(appDO.getName());
basicDTO.setPrincipals(appDO.getPrincipals());
}
LogicalClusterDO logicalClusterDO = logicalClusterMetadataManager.getTopicLogicalCluster(clusterId, topicName);
if (!ValidateUtils.isNull(logicalClusterDO)) {
basicDTO.setRegion(logicalClusterDO.getName());
}
TopicMetrics metrics = jmxService.getTopicMetrics(
clusterId,
topicName,
KafkaMetricsCollections.TOPIC_BASIC_PAGE_METRICS,
true
);
String compressionType = null;
if (!ValidateUtils.isNull(metrics)) {
compressionType = metrics.getSpecifiedMetrics("TopicCodeCValue", String.class);
}
basicDTO.setTopicCodeC(
ListUtils.strList2String(new ArrayList<>(new HashSet<>(ListUtils.string2StrList(compressionType))))
);
basicDTO.setScore(100);
return basicDTO;
}
@Override
public List<TopicPartitionDTO> getTopicPartitionDTO(ClusterDO clusterDO, String topicName, Boolean needDetail) {
if (ValidateUtils.isNull(clusterDO) || ValidateUtils.isNull(topicName)) {
return null;
}
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterDO.getId(), topicName);
if (ValidateUtils.isNull(topicMetadata)) {
return null;
}
List<PartitionState> partitionStateList = KafkaZookeeperUtils.getTopicPartitionState(
PhysicalClusterMetadataManager.getZKConfig(clusterDO.getId()),
topicName,
new ArrayList<>(topicMetadata.getPartitionMap().getPartitions().keySet())
);
Map<Integer, PartitionAttributeDTO> partitionMap = new HashMap<>();
try {
partitionMap = jmxService.getPartitionAttribute(clusterDO.getId(), topicName, partitionStateList);
} catch (Exception e) {
LOGGER.error("get topic partition failed, clusterDO:{}, topicName:{}.", clusterDO, topicName, e);
}
List<TopicPartitionDTO> dtoList = convert2TopicPartitionDTOList(
topicMetadata,
partitionStateList,
partitionMap);
if (!needDetail) {
return dtoList;
}
Map<TopicPartition, Long> endOffsetMap = getPartitionOffset(clusterDO, topicName, OffsetPosEnum.END);
Map<TopicPartition, Long> beginOffsetMap = getPartitionOffset(clusterDO, topicName, OffsetPosEnum.BEGINNING);
for (TopicPartitionDTO dto : dtoList) {
TopicPartition tp = new TopicPartition(topicName, dto.getPartitionId());
dto.setEndOffset(endOffsetMap.getOrDefault(tp, 0L));
dto.setBeginningOffset(beginOffsetMap.getOrDefault(tp, 0L));
}
return dtoList;
}
private List<TopicPartitionDTO> convert2TopicPartitionDTOList(TopicMetadata topicMetadata,
List<PartitionState> partitionStateList,
Map<Integer, PartitionAttributeDTO> partitionMap) {
List<TopicPartitionDTO> dtoList = new ArrayList<>();
for (PartitionState partitionState : partitionStateList) {
TopicPartitionDTO topicPartitionDTO = new TopicPartitionDTO();
topicPartitionDTO.setPartitionId(partitionState.getPartitionId());
topicPartitionDTO.setLeaderBrokerId(partitionState.getLeader());
topicPartitionDTO.setLeaderEpoch(partitionState.getLeaderEpoch());
topicPartitionDTO.setReplicaBrokerIdList(
topicMetadata.getPartitionMap().getPartitions().get(partitionState.getPartitionId())
);
PartitionAttributeDTO partitionAttributeDTO =
partitionMap.getOrDefault(partitionState.getPartitionId(), null);
if (!ValidateUtils.isNull(partitionAttributeDTO)) {
topicPartitionDTO.setLogSize(partitionAttributeDTO.getLogSize());
}
if (topicPartitionDTO.getReplicaBrokerIdList() != null
&& !topicPartitionDTO.getReplicaBrokerIdList().isEmpty()) {
topicPartitionDTO.setPreferredBrokerId(topicPartitionDTO.getReplicaBrokerIdList().get(0));
}
topicPartitionDTO.setIsrBrokerIdList(partitionState.getIsr());
if (topicPartitionDTO.getIsrBrokerIdList().size() < topicPartitionDTO.getReplicaBrokerIdList().size()) {
topicPartitionDTO.setUnderReplicated(false);
} else {
topicPartitionDTO.setUnderReplicated(true);
}
dtoList.add(topicPartitionDTO);
}
return dtoList;
}
@Override
public TopicMetrics getTopicMetricsFromJMX(Long clusterId, String topicName, Integer metricsCode, Boolean byAdd) {
return jmxService.getTopicMetrics(clusterId, topicName, metricsCode, byAdd);
}
@Override
public Map<TopicPartition, Long> getPartitionOffset(ClusterDO clusterDO,
String topicName,
OffsetPosEnum offsetPosEnum) {
if (clusterDO == null || StringUtils.isEmpty(topicName)) {
return new HashMap<>(0);
}
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterDO.getId(), topicName);
if (topicMetadata == null) {
return new HashMap<>();
}
List<TopicPartition> topicPartitionList = new ArrayList<>();
for (Integer partitionId = 0; partitionId < topicMetadata.getPartitionNum(); ++partitionId) {
topicPartitionList.add(new TopicPartition(topicName, partitionId));
}
Map<TopicPartition, Long> topicPartitionLongMap = new HashMap<>();
KafkaConsumer kafkaConsumer = null;
try {
kafkaConsumer = KafkaClientPool.borrowKafkaConsumerClient(clusterDO);
if ((offsetPosEnum.getCode() & OffsetPosEnum.END.getCode()) > 0) {
topicPartitionLongMap = kafkaConsumer.endOffsets(topicPartitionList);
} else if ((offsetPosEnum.getCode() & OffsetPosEnum.BEGINNING.getCode()) > 0) {
topicPartitionLongMap = kafkaConsumer.beginningOffsets(topicPartitionList);
}
} catch (Exception e) {
LOGGER.error("get topic endOffsets failed, clusterId:{} topicName:{}.",
clusterDO.getId(), topicPartitionList.get(0).topic(), e);
} finally {
KafkaClientPool.returnKafkaConsumerClient(clusterDO.getId(), kafkaConsumer);
}
return topicPartitionLongMap;
}
@Override
public List<TopicOverview> getTopicOverviewList(Long clusterId, Integer brokerId) {
if (ValidateUtils.isNull(clusterId) || ValidateUtils.isNull(brokerId)) {
return new ArrayList<>();
}
List<String> topicNameList = new ArrayList<>();
for (String topicName : PhysicalClusterMetadataManager.getTopicNameList(clusterId)) {
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterId, topicName);
if (ValidateUtils.isNull(topicMetadata)) {
continue;
}
if (!topicMetadata.getBrokerIdSet().contains(brokerId)) {
continue;
}
topicNameList.add(topicName);
}
return this.getTopicOverviewList(clusterId, topicNameList);
}
@Override
public List<TopicOverview> getTopicOverviewList(Long clusterId, List<String> topicNameList) {
if (ValidateUtils.isNull(clusterId) || ValidateUtils.isEmptyList(topicNameList)) {
return new ArrayList<>();
}
List<TopicDO> topicDOList = topicManagerService.getByClusterId(clusterId);
if (ValidateUtils.isNull(topicDOList)) {
topicDOList = new ArrayList<>();
}
Map<String, TopicDO> topicDOMap = new HashMap<>();
for (TopicDO topicDO : topicDOList) {
topicDOMap.put(topicDO.getTopicName(), topicDO);
}
List<AppDO> appDOList = appService.listAll();
if (ValidateUtils.isNull(appDOList)) {
appDOList = new ArrayList<>();
}
Map<String, AppDO> appDOMap = new HashMap<>();
for (AppDO appDO : appDOList) {
appDOMap.put(appDO.getAppId(), appDO);
}
List<TopicOverview> dtoList = new ArrayList<>();
for (String topicName : topicNameList) {
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterId, topicName);
if (ValidateUtils.isNull(topicMetadata)) {
continue;
}
TopicDO topicDO = topicDOMap.get(topicName);
AppDO appDO = topicDO == null ? null : appDOMap.get(topicDO.getAppId());
TopicOverview overview = getTopicOverview(
clusterId,
logicalClusterMetadataManager.getTopicLogicalCluster(clusterId, topicName),
topicMetadata,
topicDO,
appDO
);
if (ValidateUtils.isNull(overview)) {
continue;
}
dtoList.add(overview);
}
return dtoList;
}
private TopicOverview getTopicOverview(Long physicalClusterId,
LogicalClusterDO logicalClusterDO,
TopicMetadata topicMetadata,
TopicDO topicDO,
AppDO appDO) {
TopicOverview overview = new TopicOverview();
overview.setClusterId(physicalClusterId);
overview.setTopicName(topicMetadata.getTopic());
overview.setPartitionNum(topicMetadata.getPartitionNum());
overview.setReplicaNum(topicMetadata.getReplicaNum());
overview.setUpdateTime(topicMetadata.getModifyTime());
overview.setRetentionTime(
PhysicalClusterMetadataManager.getTopicRetentionTime(physicalClusterId, topicMetadata.getTopic())
);
if (!ValidateUtils.isNull(topicDO)) {
overview.setAppId(topicDO.getAppId());
overview.setDescription(topicDO.getDescription());
}
if (!ValidateUtils.isNull(appDO)) {
overview.setAppName(appDO.getName());
}
if (!ValidateUtils.isNull(logicalClusterDO)) {
overview.setLogicalClusterId(logicalClusterDO.getId());
}
TopicMetrics metrics = KafkaMetricsCache.getTopicMetricsFromCache(physicalClusterId, topicMetadata.getTopic());
if (ValidateUtils.isNull(metrics)) {
metrics = jmxService.getTopicMetrics(
physicalClusterId,
topicMetadata.getTopic(),
KafkaMetricsCollections.TOPIC_FLOW_OVERVIEW,
true
);
}
if (ValidateUtils.isNull(metrics)) {
return overview;
}
overview.setByteIn(metrics.getBytesInPerSecOneMinuteRate(null));
overview.setProduceRequest(metrics.getTotalProduceRequestsPerSecOneMinuteRate(null));
return overview;
}
@Override
public Map<String, List<PartitionState>> getTopicPartitionState(Long clusterId, Integer filterBrokerId) {
if (ValidateUtils.isNull(clusterId) || ValidateUtils.isNull(filterBrokerId)) {
return new HashMap<>(0);
}
List<String> topicNameList = PhysicalClusterMetadataManager.getTopicNameList(clusterId);
if (ValidateUtils.isEmptyList(topicNameList)) {
return new HashMap<>(0);
}
Map<String, List<PartitionState>> stateMap = new HashMap<>();
for (String topicName : topicNameList) {
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterId, topicName);
if (ValidateUtils.isNull(topicMetadata)) {
continue;
}
if (!(Constant.NOT_FILTER_BROKER_ID.equals(filterBrokerId)
|| topicMetadata.getBrokerIdSet().contains(filterBrokerId))) {
// 按照 filterBrokerId 进行过滤
continue;
}
List<PartitionState> partitionStateList = getTopicPartitionState(clusterId, topicMetadata);
if (ValidateUtils.isEmptyList(partitionStateList)) {
continue;
}
stateMap.put(topicName, partitionStateList);
}
return stateMap;
}
private List<PartitionState> getTopicPartitionState(Long clusterId, TopicMetadata topicMetadata) {
List<PartitionState> partitionStateList = null;
try {
partitionStateList = KafkaZookeeperUtils.getTopicPartitionState(
PhysicalClusterMetadataManager.getZKConfig(clusterId),
topicMetadata.getTopic(),
new ArrayList<>(topicMetadata.getPartitionMap().getPartitions().keySet())
);
// 判断分区副本是否在isr
for (PartitionState partitionState : partitionStateList) {
if (topicMetadata.getReplicaNum() > partitionState.getIsr().size()) {
partitionState.setUnderReplicated(false);
} else {
partitionState.setUnderReplicated(true);
}
}
} catch (Exception e) {
LOGGER.error("get partition state from zk failed, clusterId:{} topicName:{}.",
clusterId,
topicMetadata.getTopic()
);
}
return partitionStateList;
}
@Override
public List<PartitionOffsetDTO> getPartitionOffsetList(ClusterDO clusterDO, String topicName, Long timestamp) {
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterDO.getId(), topicName);
if (topicMetadata == null) {
return null;
}
Map<TopicPartition, Long> timestampsToSearch = new HashMap<>();
for (Integer partitionId : topicMetadata.getPartitionMap().getPartitions().keySet()) {
timestampsToSearch.put(new TopicPartition(topicName, partitionId), timestamp);
}
if (timestampsToSearch.isEmpty()) {
return new ArrayList<>();
}
KafkaConsumer kafkaConsumer = null;
List<PartitionOffsetDTO> partitionOffsetDTOList = new ArrayList<>();
try {
kafkaConsumer = KafkaClientPool.borrowKafkaConsumerClient(clusterDO);
Map<TopicPartition, OffsetAndTimestamp> offsetAndTimestampMap = kafkaConsumer.offsetsForTimes(timestampsToSearch);
if (offsetAndTimestampMap == null) {
return new ArrayList<>();
}
for (Map.Entry<TopicPartition, OffsetAndTimestamp> entry : offsetAndTimestampMap.entrySet()) {
TopicPartition tp = entry.getKey();
OffsetAndTimestamp offsetAndTimestamp = entry.getValue();
partitionOffsetDTOList.add(new PartitionOffsetDTO(tp.partition(), offsetAndTimestamp.offset(), offsetAndTimestamp.timestamp()));
}
} catch (Exception e) {
LOGGER.error("get offset failed, clusterId:{} topicName:{} timestamp:{}.", clusterDO.getId(), topicName, timestamp, e);
} finally {
KafkaClientPool.returnKafkaConsumerClient(clusterDO.getId(), kafkaConsumer);
}
return partitionOffsetDTOList;
}
@Override
public List<String> fetchTopicData(ClusterDO clusterDO, String topicName, TopicDataSampleDTO reqObj) {
KafkaConsumer kafkaConsumer = null;
try {
kafkaConsumer = createConsumerClient(clusterDO, reqObj.getMaxMsgNum() + 1);
return fetchTopicData(kafkaConsumer, clusterDO, topicName, reqObj);
} catch (Exception e) {
LOGGER.error("create consumer failed, clusterDO:{} req:{}.", clusterDO, reqObj, e);
} finally {
if (kafkaConsumer != null) {
kafkaConsumer.close();
}
}
return null;
}
private List<String> fetchTopicData(KafkaConsumer kafkaConsumer, ClusterDO clusterDO, String topicName, TopicDataSampleDTO reqObj) {
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterDO.getId(), topicName);
List<TopicPartition> tpList = new ArrayList<>();
for (int partitionId = 0; partitionId < topicMetadata.getPartitionNum(); ++partitionId) {
if (!ValidateUtils.isNull(reqObj.getPartitionId()) && !reqObj.getPartitionId().equals(partitionId)) {
continue;
}
tpList.add(new TopicPartition(topicName, partitionId));
}
if (ValidateUtils.isEmptyList(tpList)) {
return null;
}
kafkaConsumer.assign(tpList);
if (!ValidateUtils.isNull(reqObj.getOffset()) && tpList.size() >= 1) {
// 如若指定offset则从指定offset开始
kafkaConsumer.seek(tpList.get(0), reqObj.getOffset());
return fetchTopicData(kafkaConsumer, reqObj.getMaxMsgNum(), reqObj.getTimeout(), reqObj.getTruncate());
}
// 获取各个分区最新的数据
return fetchTopicData(kafkaConsumer, reqObj.getMaxMsgNum(), reqObj.getTimeout().longValue(), reqObj.getTruncate(), tpList);
}
@Override
public List<String> fetchTopicData(KafkaConsumer kafkaConsumer,
Integer maxMsgNum,
Long maxWaitMs,
Boolean truncated,
List<TopicPartition> tpList) {
if (maxWaitMs <= 0) {
return Collections.emptyList();
}
Map<TopicPartition, Long> endOffsetMap = kafkaConsumer.endOffsets(tpList);
Long begin = System.currentTimeMillis();
Long remainingWaitMs = maxWaitMs;
List<String> dataList = new ArrayList<>(maxMsgNum);
// 遍历所有分区最新的数据
for (Map.Entry<TopicPartition, Long> entry : endOffsetMap.entrySet()) {
if (remainingWaitMs <= 0) {
break;
}
Integer remainingMsgNum = maxMsgNum - dataList.size();
Long startOffset = Math.max(0, entry.getValue() - remainingMsgNum);
kafkaConsumer.seek(entry.getKey(), startOffset);
dataList.addAll(fetchTopicDataNotRetry(kafkaConsumer, remainingMsgNum, remainingWaitMs, truncated));
// 采样数据条数已经够了
if (dataList.size() >= maxMsgNum) {
break;
}
// 检查是否超时
long elapsed = System.currentTimeMillis() - begin;
if (elapsed >= maxWaitMs) {
break;
}
remainingWaitMs = maxWaitMs - elapsed;
}
return dataList;
}
private List<String> fetchTopicDataNotRetry(KafkaConsumer kafkaConsumer,
Integer maxMsgNum,
Long maxWaitMs,
Boolean truncated) {
if (maxWaitMs <= 0) {
return Collections.emptyList();
}
long begin = System.currentTimeMillis();
long remainingWaitMs = maxWaitMs;
List<String> dataList = new ArrayList<>();
int currentSize = dataList.size();
while (dataList.size() < maxMsgNum) {
try {
if (remainingWaitMs <= 0) {
break;
}
ConsumerRecords<String, String> records = kafkaConsumer.poll(TopicSampleConstant.POLL_TIME_OUT_UNIT_MS);
for (ConsumerRecord record : records) {
String value = (String) record.value();
dataList.add(
truncated ?
value.substring(0, Math.min(value.length(), TopicSampleConstant.MAX_DATA_LENGTH_UNIT_BYTE))
: value
);
}
// 当前批次一条数据都没拉取到,则结束拉取
if (dataList.size() - currentSize == 0) {
break;
}
currentSize = dataList.size();
// 检查是否超时
long elapsed = System.currentTimeMillis() - begin;
if (elapsed >= maxWaitMs) {
break;
}
remainingWaitMs = maxWaitMs - elapsed;
} catch (Exception e) {
LOGGER.error("fetch topic data failed, TopicPartitions:{}.", kafkaConsumer.assignment(), e);
}
}
return dataList.subList(0, Math.min(dataList.size(), maxMsgNum));
}
@Override
public List<String> fetchTopicData(KafkaConsumer kafkaConsumer,
Integer maxMsgNum,
Integer timeout,
Boolean truncated) {
List<String> dataList = new ArrayList<>();
long timestamp = System.currentTimeMillis();
while (dataList.size() < maxMsgNum) {
try {
ConsumerRecords<String, String> records = kafkaConsumer.poll(TopicSampleConstant.POLL_TIME_OUT_UNIT_MS);
for (ConsumerRecord record : records) {
String value = (String) record.value();
dataList.add(
truncated ?
value.substring(0, Math.min(value.length(), TopicSampleConstant.MAX_DATA_LENGTH_UNIT_BYTE))
: value
);
}
if (System.currentTimeMillis() - timestamp > timeout
|| dataList.size() >= maxMsgNum) {
break;
}
Thread.sleep(10);
} catch (Exception e) {
LOGGER.error("fetch topic data failed, TopicPartitions:{}.", kafkaConsumer.assignment(), e);
}
}
return dataList.subList(0, Math.min(dataList.size(), maxMsgNum));
}
@Override
public List<TopicMetricsDO> getTopicRequestMetricsFromDB(Long clusterId, String topicName, Date startTime, Date endTime) {
try {
return topicRequestMetricsDao.selectByTime(clusterId, topicName, startTime, endTime);
} catch (Exception e) {
LOGGER.error("get topic request metrics failed, clusterId:{} topicName:{}.", clusterId, topicName, e);
}
return new ArrayList<>();
}
@Override
public List<TopicBrokerDTO> getTopicBrokerList(Long clusterId, String topicName) {
List<TopicBrokerDTO> topicBrokerDOList = new ArrayList<>();
// 获取每个broker下的分区Id
Map<Integer, List<Integer>> brokerPartitionMap = new HashMap<>();
// <partitionId, brokerList>
Map<Integer, List<Integer>> partitionBrokerMap = PhysicalClusterMetadataManager.getTopicMetadata(clusterId, topicName).getPartitionMap().getPartitions();
for (Map.Entry<Integer, List<Integer>> entry : partitionBrokerMap.entrySet()) {
Integer partitionId = entry.getKey();
for (Integer brokerId : entry.getValue()) {
if (!brokerPartitionMap.containsKey(brokerId)) {
brokerPartitionMap.put(brokerId, new ArrayList<>());
}
brokerPartitionMap.get(brokerId).add(partitionId);
}
}
TopicMetadata topicMetadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterId, topicName);
// 获取每个broker下的leader分区的Id
List<PartitionState> partitionStateList = KafkaZookeeperUtils.getTopicPartitionState(
PhysicalClusterMetadataManager.getZKConfig(clusterId),
topicName,
new ArrayList<>(topicMetadata.getPartitionMap().getPartitions().keySet())
);
Map<Integer, List<Integer>> leaderPartitionIdMap = new HashMap<>();
for (PartitionState partitionState : partitionStateList) {
Integer leaderBrokerId = partitionState.getLeader();
Integer partitionId = partitionState.getPartitionId();
if (!leaderPartitionIdMap.containsKey(leaderBrokerId)) {
leaderPartitionIdMap.put(leaderBrokerId, new ArrayList<>());
}
leaderPartitionIdMap.get(leaderBrokerId).add(partitionId);
}
// 封装TopicBrokerDO
for (Map.Entry<Integer, List<Integer>> entry : brokerPartitionMap.entrySet()) {
Integer brokerId = entry.getKey();
TopicBrokerDTO topicBrokerDO = new TopicBrokerDTO();
topicBrokerDO.setBrokerId(entry.getKey());
topicBrokerDO.setLeaderPartitionIdList(leaderPartitionIdMap.getOrDefault(brokerId, new ArrayList<>()));
topicBrokerDO.setPartitionIdList(entry.getValue());
topicBrokerDO.setPartitionNum(entry.getValue().size());
topicBrokerDOList.add(topicBrokerDO);
BrokerMetadata brokerMetadata = PhysicalClusterMetadataManager.getBrokerMetadata(clusterId, brokerId);
if (!ValidateUtils.isNull(brokerMetadata)) {
topicBrokerDO.setHost(brokerMetadata.getHost());
topicBrokerDO.setAlive(true);
}
}
return topicBrokerDOList;
}
private KafkaConsumer createConsumerClient(ClusterDO clusterDO, Integer maxPollRecords) {
Properties properties = KafkaClientPool.createProperties(clusterDO, false);
properties.put("enable.auto.commit", false);
properties.put("max.poll.records", maxPollRecords);
properties.put("request.timeout.ms", 15000);
return new KafkaConsumer(properties);
}
@Override
public Result<TopicOffsetChangedEnum> checkTopicOffsetChanged(Long physicalClusterId,
String topicName,
Long latestTime) {
try {
ClusterDO clusterDO = clusterService.getById(physicalClusterId);
if (ValidateUtils.isNull(clusterDO)) {
return Result.buildFrom(ResultStatus.CLUSTER_NOT_EXIST);
}
Map<TopicPartition, Long> endOffsetMap = this.getPartitionOffset(clusterDO, topicName, OffsetPosEnum.END);
if (ValidateUtils.isEmptyMap(endOffsetMap)) {
return new Result<>(TopicOffsetChangedEnum.UNKNOWN);
}
Long timestamp = System.currentTimeMillis() - latestTime;
List<PartitionOffsetDTO> dtoList = this.getPartitionOffsetList(clusterDO, topicName, timestamp);
if (ValidateUtils.isEmptyList(dtoList)) {
return checkTopicOffsetChanged(clusterDO, topicName, endOffsetMap);
}
TopicOffsetChangedEnum offsetChangedEnum = TopicOffsetChangedEnum.NO;
for (PartitionOffsetDTO dto: dtoList) {
Long endOffset = endOffsetMap.get(new TopicPartition(topicName, dto.getPartitionId()));
if (!ValidateUtils.isNull(endOffset) && endOffset > dto.getOffset()) {
// 任意分区的数据发生变化, 则表示有数据写入
return new Result<>(TopicOffsetChangedEnum.YES);
}
if (ValidateUtils.isNull(endOffset)) {
offsetChangedEnum = TopicOffsetChangedEnum.UNKNOWN;
}
}
if (!TopicOffsetChangedEnum.NO.equals(offsetChangedEnum) || endOffsetMap.size() != dtoList.size()) {
return new Result<>(TopicOffsetChangedEnum.UNKNOWN);
}
return new Result<>(TopicOffsetChangedEnum.NO);
} catch (Exception e) {
LOGGER.error("check topic expired failed, clusterId:{} topicName:{} latestTime:{}."
,physicalClusterId, topicName, latestTime);
}
return new Result<>(TopicOffsetChangedEnum.UNKNOWN);
}
private Result<TopicOffsetChangedEnum> checkTopicOffsetChanged(ClusterDO clusterDO,
String topicName,
Map<TopicPartition, Long> endOffsetMap) {
if (ValidateUtils.isNull(clusterDO)
|| ValidateUtils.isNull(topicName)
|| ValidateUtils.isEmptyMap(endOffsetMap)) {
return new Result<>(TopicOffsetChangedEnum.UNKNOWN);
}
Map<TopicPartition, Long> beginningOffsetMap =
this.getPartitionOffset(clusterDO, topicName, OffsetPosEnum.BEGINNING);
if (ValidateUtils.isEmptyMap(beginningOffsetMap)) {
return new Result<>(TopicOffsetChangedEnum.UNKNOWN);
}
TopicOffsetChangedEnum changedEnum = TopicOffsetChangedEnum.NO;
for (Map.Entry<TopicPartition, Long> entry: endOffsetMap.entrySet()) {
Long beginningOffset = beginningOffsetMap.get(entry.getKey());
if (!ValidateUtils.isNull(beginningOffset) && beginningOffset < entry.getValue()) {
return new Result<>(TopicOffsetChangedEnum.YES);
}
if (ValidateUtils.isNull(beginningOffset)) {
changedEnum = TopicOffsetChangedEnum.UNKNOWN;
}
}
if (TopicOffsetChangedEnum.UNKNOWN.equals(changedEnum)
|| endOffsetMap.size() != beginningOffsetMap.size()) {
return new Result<>(TopicOffsetChangedEnum.UNKNOWN);
}
return new Result<>(TopicOffsetChangedEnum.NO);
}
}

View File

@@ -0,0 +1,43 @@
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.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.zookeeper.ZkConfigImpl;
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;
import com.xiaojukeji.kafka.manager.service.service.ZookeeperService;
import com.xiaojukeji.kafka.manager.service.utils.KafkaZookeeperUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.stereotype.Service;
/**
* @author zengqiao
* @date 20/8/27
*/
@Service("zookeeperService")
public class ZookeeperServiceImpl implements ZookeeperService {
private final static Logger LOGGER = LoggerFactory.getLogger(ZookeeperServiceImpl.class);
@Override
public Result openTopicJmx(Long clusterId, String topicName, TopicJmxSwitch jmxSwitch) {
if (ValidateUtils.isNull(clusterId) || ValidateUtils.isNull(topicName)) {
return Result.buildFrom(ResultStatus.PARAM_ILLEGAL);
}
TopicMetadata metadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterId, topicName);
if (ValidateUtils.isNull(metadata)) {
return Result.buildFrom(ResultStatus.TOPIC_NOT_EXIST);
}
ZkConfigImpl zkConfig = PhysicalClusterMetadataManager.getZKConfig(clusterId);
for (Integer brokerId: metadata.getBrokerIdSet()) {
if (!KafkaZookeeperUtils.openBrokerTopicJmx(zkConfig, brokerId, topicName, jmxSwitch)) {
return Result.buildFrom(ResultStatus.OPERATION_FAILED);
}
}
return new Result();
}
}

View File

@@ -0,0 +1,11 @@
package com.xiaojukeji.kafka.manager.service.strategy;
import com.xiaojukeji.kafka.manager.common.entity.ao.gateway.TopicQuota;
/**
* @author zengqiao
* @date 20/8/20
*/
public abstract class AbstractAllocateQuotaStrategy {
public abstract TopicQuota getNewTopicQuota(TopicQuota originTopicQuota, Integer access, Long peakBytesIn);
}

View File

@@ -0,0 +1,16 @@
package com.xiaojukeji.kafka.manager.service.strategy;
import com.xiaojukeji.kafka.manager.common.entity.metrics.BrokerMetrics;
/**
* @author zengqiao
* @date 20/9/23
*/
public abstract class AbstractHealthScoreStrategy {
public abstract Integer calBrokerHealthScore(Long clusterId, Integer brokerId, BrokerMetrics brokerMetrics);
public abstract Integer calBrokerHealthScore(Long clusterId, Integer brokerId);
public abstract Integer calTopicHealthScore(Long clusterId, String topicName);
}

View File

@@ -0,0 +1,149 @@
package com.xiaojukeji.kafka.manager.service.strategy.healthscore;
import com.xiaojukeji.kafka.manager.common.constant.Constant;
import com.xiaojukeji.kafka.manager.common.constant.KafkaMetricsCollections;
import com.xiaojukeji.kafka.manager.common.entity.metrics.BrokerMetrics;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.BrokerMetadata;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.cache.ThreadPool;
import com.xiaojukeji.kafka.manager.service.service.JmxService;
import com.xiaojukeji.kafka.manager.service.strategy.AbstractHealthScoreStrategy;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.FutureTask;
/**
* @author zengqiao
* @date 20/9/23
*/
@Service("healthScoreStrategy")
public class DidiHealthScoreStrategy extends AbstractHealthScoreStrategy {
private static final Logger LOGGER = LoggerFactory.getLogger(DidiHealthScoreStrategy.class);
private static final Integer QUEUED_MAX_REQUESTS = 500;
private static final Integer KAFKA_REQUEST_HANDLER_POOL_SIZE = 3;
private static final Double MIN_IDLE = 0.8;
private static final Integer HEALTH_SCORE_HEALTHY = 100;
private static final Integer HEALTH_SCORE_NORMAL = 90;
private static final Integer HEALTH_SCORE_BAD = 60;
private static final Integer HEALTH_SCORE_VERY_BAD = 30;
@Autowired
private JmxService jmxService;
@Override
public Integer calBrokerHealthScore(Long clusterId, Integer brokerId) {
BrokerMetadata brokerMetadata = PhysicalClusterMetadataManager.getBrokerMetadata(clusterId, brokerId);
if (ValidateUtils.isNull(brokerMetadata)) {
return HEALTH_SCORE_HEALTHY;
}
BrokerMetrics metrics = jmxService.getBrokerMetrics(clusterId, brokerId, KafkaMetricsCollections.BROKER_HEALTH_SCORE_METRICS);
if (ValidateUtils.isNull(metrics)) {
return Constant.INVALID_CODE;
}
return calBrokerHealthScore(clusterId, brokerId, metrics);
}
@Override
public Integer calBrokerHealthScore(Long clusterId, Integer brokerId, BrokerMetrics metrics) {
try {
if (ValidateUtils.isNull(metrics)) {
return Constant.INVALID_CODE;
}
Object failedFetchRequestsPerSecOneMinuteRate = metrics.getSpecifiedMetrics("FailedFetchRequestsPerSecOneMinuteRate");
Object failedProduceRequestsPerSecOneMinuteRate = metrics.getSpecifiedMetrics("FailedProduceRequestsPerSecOneMinuteRate");
if (ValidateUtils.isNull(failedFetchRequestsPerSecOneMinuteRate) || ValidateUtils.isNull(failedProduceRequestsPerSecOneMinuteRate)) {
// 数据获取失败
return Constant.INVALID_CODE;
}
if (((Double) failedFetchRequestsPerSecOneMinuteRate) > 0
|| ((Double) failedProduceRequestsPerSecOneMinuteRate) > 0) {
return HEALTH_SCORE_VERY_BAD;
}
Object requestQueueSizeValue = metrics.getMetricsMap().get("RequestQueueSizeValue");
Object responseQueueSizeValue = metrics.getMetricsMap().get("ResponseQueueSizeValue");
if (ValidateUtils.isNull(requestQueueSizeValue)
|| ValidateUtils.isNull(responseQueueSizeValue)) {
// 数据获取失败
return Constant.INVALID_CODE;
}
if (((Integer) requestQueueSizeValue) >= QUEUED_MAX_REQUESTS
|| ((Integer) responseQueueSizeValue) >= QUEUED_MAX_REQUESTS) {
return HEALTH_SCORE_BAD;
}
Object RequestHandlerAvgIdlePercentOneMinuteRate = metrics.getMetricsMap().get("RequestHandlerAvgIdlePercentOneMinuteRate");
Object NetworkProcessorAvgIdlePercentValue = metrics.getMetricsMap().get("NetworkProcessorAvgIdlePercentValue");
if (ValidateUtils.isNull(RequestHandlerAvgIdlePercentOneMinuteRate)
|| ValidateUtils.isNull(NetworkProcessorAvgIdlePercentValue)) {
// 数据获取失败
return Constant.INVALID_CODE;
}
if (((Double) RequestHandlerAvgIdlePercentOneMinuteRate) < MIN_IDLE * KAFKA_REQUEST_HANDLER_POOL_SIZE
|| ((Double) NetworkProcessorAvgIdlePercentValue) < MIN_IDLE) {
return HEALTH_SCORE_NORMAL;
}
return HEALTH_SCORE_HEALTHY;
} catch (Exception e) {
LOGGER.error("cal broker health score failed, clusterId:{} brokerId:{}.", clusterId, brokerId, e);
}
return Constant.INVALID_CODE;
}
@Override
public Integer calTopicHealthScore(Long clusterId, String topicName) {
TopicMetadata metadata = PhysicalClusterMetadataManager.getTopicMetadata(clusterId, topicName);
if (ValidateUtils.isNull(metadata)) {
return Constant.INVALID_CODE;
}
List<Integer> brokerIdList = new ArrayList<>(metadata.getBrokerIdSet().size());
FutureTask<Integer>[] taskList = new FutureTask[brokerIdList.size()];
for (int i = 0; i < brokerIdList.size(); ++i) {
final Integer brokerId = brokerIdList.get(i);
taskList[i] = new FutureTask<Integer>(new Callable<Integer>() {
@Override
public Integer call() throws Exception {
return calBrokerHealthScore(clusterId, brokerId);
}
});
ThreadPool.submitApiCallTask(taskList[i]);
}
Integer topicHealthScore = HEALTH_SCORE_HEALTHY;
for (int i = 0; i < taskList.length; ++i) {
try {
Integer brokerHealthScore = taskList[i].get();
if (ValidateUtils.isNull(brokerHealthScore)) {
// 如果某台broker健康分计算失败, 则直接返回计算失败
return Constant.INVALID_CODE;
}
topicHealthScore = Math.min(topicHealthScore, brokerHealthScore);
} catch (Exception e) {
LOGGER.error("cal broker health failed, clusterId:{} brokerId:{}.",
clusterId, brokerIdList.get(i), e
);
return Constant.INVALID_CODE;
}
}
return topicHealthScore;
}
}

View File

@@ -0,0 +1,19 @@
package com.xiaojukeji.kafka.manager.service.strategy.quota;
import com.xiaojukeji.kafka.manager.common.entity.ao.gateway.TopicQuota;
import com.xiaojukeji.kafka.manager.service.strategy.AbstractAllocateQuotaStrategy;
import org.springframework.stereotype.Service;
/**
* 基本的Quota分配策略即用户申请多少则分配多少
* @author zengqiao
* @date 20/8/20
*/
@Service("allocateQuotaStrategy")
public class BaseAllocateQuotaStrategy extends AbstractAllocateQuotaStrategy {
@Override
public TopicQuota getNewTopicQuota(TopicQuota originTopicQuota, Integer access, Long peakBytesIn) {
return originTopicQuota;
}
}

View File

@@ -0,0 +1,34 @@
package com.xiaojukeji.kafka.manager.service.utils;
import org.springframework.beans.factory.annotation.Value;
import org.springframework.stereotype.Service;
/**
* @author zengqiao
* @date 20/4/26
*/
@Service("configUtils")
public class ConfigUtils {
@Value(value = "${custom.idc}")
private String idc;
@Value(value = "${spring.profiles.active}")
private String kafkaManagerEnv;
public String getIdc() {
return idc;
}
public void setIdc(String idc) {
this.idc = idc;
}
public String getKafkaManagerEnv() {
return kafkaManagerEnv;
}
public void setKafkaManagerEnv(String kafkaManagerEnv) {
this.kafkaManagerEnv = kafkaManagerEnv;
}
}

View File

@@ -0,0 +1,201 @@
package com.xiaojukeji.kafka.manager.service.utils;
import com.alibaba.fastjson.JSON;
import com.alibaba.fastjson.JSONObject;
import com.xiaojukeji.kafka.manager.common.constant.TopicCreationConstant;
import com.xiaojukeji.kafka.manager.common.entity.ao.gateway.TopicQuota;
import com.xiaojukeji.kafka.manager.common.utils.NumberUtils;
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.ReassignmentElemData;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.ReassignmentJsonData;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.PartitionState;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.config.ChangeData;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.config.TopicQuotaData;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.config.ConfigNodeData;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.didi.JmxSwitchDataConstant;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.didi.TopicJmxSwitch;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.*;
/**
* Kafka ZK 工具类
* @author zengqiao
* @date 20/4/8
*/
public class KafkaZookeeperUtils {
private final static Logger LOGGER = LoggerFactory.getLogger(KafkaZookeeperUtils.class);
private static final Integer DATA_VERSION_ONE = 1;
public static List<ReassignmentElemData> getReassignmentElemDataList(String reassignmentJson) {
try {
ReassignmentJsonData data = JSONObject.parseObject(reassignmentJson, ReassignmentJsonData.class);
return data.getPartitions();
} catch (Exception e) {
}
return new ArrayList<>();
}
public static Long getTopicRetentionTime(Properties properties) {
if (ValidateUtils.isNull(properties)
|| !properties.containsKey(TopicCreationConstant.TOPIC_RETENTION_TIME_KEY_NAME)) {
return null;
}
return NumberUtils.string2Long(properties.getProperty(TopicCreationConstant.TOPIC_RETENTION_TIME_KEY_NAME));
}
public static Long getTopicRetentionTime(ZkConfigImpl zkConfig, String topicName) {
Properties properties = getTopicProperties(zkConfig, topicName);
if (ValidateUtils.isNull(properties)
|| !properties.containsKey(TopicCreationConstant.TOPIC_RETENTION_TIME_KEY_NAME)) {
return null;
}
return NumberUtils.string2Long(properties.getProperty(TopicCreationConstant.TOPIC_RETENTION_TIME_KEY_NAME));
}
public static Properties getTopicProperties(ZkConfigImpl zkConfig, String topicName) {
return getConfigData(zkConfig, ZkPathUtil.getConfigTopicNode(topicName));
}
public static List<PartitionState> getTopicPartitionState(ZkConfigImpl zkConfig,
String topicName,
List<Integer> partitionIdList) {
List<PartitionState> partitionStateList = new ArrayList<>();
for (Integer partitionId: partitionIdList) {
try {
PartitionState partitionState = zkConfig.get(
ZkPathUtil.getBrokerTopicPartitionStatePath(topicName, partitionId),
PartitionState.class
);
partitionState.setPartitionId(partitionId);
partitionStateList.add(partitionState);
} catch (Exception e) {
LOGGER.error("get partition state failed, topicName:{} partitionId:{}."
,topicName, partitionId, e);
}
}
return partitionStateList;
}
public static boolean openBrokerTopicJmx(ZkConfigImpl zkConfig,
Integer brokerId,
String topicName,
TopicJmxSwitch jmxSwitch) {
String dataPath = ZkPathUtil.getKafkaExtraMetricsPath(brokerId);
// 获取旧的配置
Properties properties = getConfigData(zkConfig, dataPath);
if (ValidateUtils.isNull(properties)) {
properties = new Properties();
}
// 合并新旧的配置
if (jmxSwitch.getOpenTopicRequestMetrics()) {
properties.put(JmxSwitchDataConstant.TOPIC_REQUEST_METRICS + topicName, Boolean.TRUE);
}
if (jmxSwitch.getOpenAppIdTopicMetrics()) {
properties.put(JmxSwitchDataConstant.APP_ID_TOPIC_METRICS + topicName, Boolean.TRUE);
}
if (jmxSwitch.getOpenClientRequestMetrics()) {
properties.put(JmxSwitchDataConstant.CLIENT_REQUEST_METRICS + topicName, Boolean.TRUE);
}
// 新配置写ZK
ConfigNodeData<Properties> configNodeData = new ConfigNodeData<>();
configNodeData.setVersion(ConfigNodeData.CONFIGDATA_VERSION);
configNodeData.setConfig(properties);
return setConfigNodeData(zkConfig, configNodeData, dataPath);
}
public static boolean closeBrokerTopicJmx(ZkConfigImpl zkConfig,
Integer brokerId,
String topicName) {
String dataPath = ZkPathUtil.getKafkaExtraMetricsPath(brokerId);
// 获取旧的配置
Properties properties = getConfigData(zkConfig, dataPath);
if (ValidateUtils.isNull(properties) || properties.isEmpty()) {
return true;
}
// 移除该Topic的配置
properties.remove(JmxSwitchDataConstant.TOPIC_REQUEST_METRICS + topicName);
properties.remove(JmxSwitchDataConstant.APP_ID_TOPIC_METRICS + topicName);
properties.remove(JmxSwitchDataConstant.CLIENT_REQUEST_METRICS + topicName);
// 新配置写ZK
ConfigNodeData<Properties> configNodeData = new ConfigNodeData<>();
configNodeData.setVersion(ConfigNodeData.CONFIGDATA_VERSION);
configNodeData.setConfig(properties);
return setConfigNodeData(zkConfig, configNodeData, dataPath);
}
public static TopicQuotaData getTopicQuota(ZkConfigImpl zkConfig, String appId, String topicName) {
ConfigNodeData configNodeData =
getConfigNodeData(zkConfig, ZkPathUtil.getConfigClientNodePath(appId, topicName));
if (ValidateUtils.isNull(configNodeData) || ValidateUtils.isNull(configNodeData.getConfig())) {
return null;
}
return JSONObject.parseObject(JSON.toJSONString(configNodeData.getConfig()), TopicQuotaData.class);
}
public static boolean setTopicQuota(ZkConfigImpl zkConfig, TopicQuota topicQuotaDO) {
ConfigNodeData<TopicQuotaData> configNodeData = new ConfigNodeData<>();
TopicQuotaData clientData =
TopicQuotaData.getClientData(topicQuotaDO.getProduceQuota(), topicQuotaDO.getConsumeQuota());
configNodeData.setVersion(ConfigNodeData.CONFIGDATA_VERSION);
configNodeData.setConfig(clientData);
return setConfigNodeData(zkConfig,
configNodeData,
ZkPathUtil.getConfigClientNodePath(topicQuotaDO.getAppId(), topicQuotaDO.getTopicName())
);
}
private static Properties getConfigData(ZkConfigImpl zkConfig, String path) {
ConfigNodeData configNodeData = getConfigNodeData(zkConfig, path);
if (ValidateUtils.isNull(configNodeData) || ValidateUtils.isNull(configNodeData.getConfig())) {
return null;
}
return JSON.parseObject(JSON.toJSONString(configNodeData.getConfig()), Properties.class);
}
/**
* 获取config node节点的数据
*/
private static ConfigNodeData getConfigNodeData(ZkConfigImpl zkConfig, String path) {
try {
if (!zkConfig.checkPathExists(path)) {
return null;
}
return zkConfig.get(path, ConfigNodeData.class);
} catch (Exception e) {
LOGGER.error("get config data failed, path:{}.", path, e);
}
return null;
}
/**
* 设置config node节点的数据
*/
private static boolean setConfigNodeData(ZkConfigImpl zkConfig,
ConfigNodeData configNodeData,
String dataPath) {
try {
String entityPath = dataPath.substring(ZkPathUtil.CONFIG_ROOT_NODE.length() + 1);
zkConfig.setOrCreatePersistentNodeStat(dataPath, JSON.toJSONString(configNodeData));
zkConfig.createPersistentSequential(
ZkPathUtil.CONFIG_ENTITY_CHANGES_ROOT_NODE,
JSON.toJSONString(ChangeData.getChangeData(entityPath))
);
return true;
} catch (Exception e) {
LOGGER.error("set config data failed, dataPath:{} configNodeData:{}."
, dataPath, configNodeData, e);
}
return false;
}
}

View File

@@ -0,0 +1,203 @@
package com.xiaojukeji.kafka.manager.service.utils;
import com.alibaba.fastjson.JSON;
import com.xiaojukeji.kafka.manager.common.entity.dto.op.reassign.ReassignTopicDTO;
import com.xiaojukeji.kafka.manager.common.entity.metrics.BaseMetrics;
import com.xiaojukeji.kafka.manager.common.entity.metrics.BrokerMetrics;
import com.xiaojukeji.kafka.manager.common.entity.metrics.ClusterMetrics;
import com.xiaojukeji.kafka.manager.common.entity.metrics.TopicMetrics;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ReassignTaskDO;
import com.xiaojukeji.kafka.manager.common.utils.ListUtils;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.entity.pojo.BrokerMetricsDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterMetricsDO;
import com.xiaojukeji.kafka.manager.common.entity.pojo.TopicMetricsDO;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.TopicMetadata;
import java.util.ArrayList;
import java.util.Date;
import java.util.List;
/**
* @author zengqiao
* @date 20/6/17
*/
public class MetricsConvertUtils {
public static List<ClusterMetrics> convert2ClusterMetricsList(List<ClusterMetricsDO> doList) {
if (ValidateUtils.isEmptyList(doList)) {
return new ArrayList<>();
}
List<ClusterMetrics> metricsList = new ArrayList<>();
for (ClusterMetricsDO metricsDO: doList) {
metricsList.add(convert2ClusterMetrics(metricsDO));
}
return metricsList;
}
public static ClusterMetrics convert2ClusterMetrics(ClusterMetricsDO metricsDO) {
if (ValidateUtils.isNull(metricsDO)) {
return null;
}
ClusterMetrics metrics = new ClusterMetrics(metricsDO.getClusterId());
metrics.setMetricsMap(JSON.parseObject(metricsDO.getMetrics()));
return metrics;
}
public static List<BrokerMetrics> convert2BrokerMetricsList(List<BrokerMetricsDO> doList) {
if (ValidateUtils.isEmptyList(doList)) {
return new ArrayList<>();
}
List<BrokerMetrics> metricsList = new ArrayList<>();
for (BrokerMetricsDO metricsDO: doList) {
metricsList.add(convert2BrokerMetrics(metricsDO));
}
return metricsList;
}
public static BrokerMetrics convert2BrokerMetrics(BrokerMetricsDO metricsDO) {
if (ValidateUtils.isNull(metricsDO)) {
return null;
}
BrokerMetrics metrics = new BrokerMetrics(metricsDO.getClusterId(), metricsDO.getBrokerId());
metrics.setMetricsMap(JSON.parseObject(metricsDO.getMetrics()));
return metrics;
}
public static List<TopicMetrics> convert2TopicMetricsList(List<TopicMetricsDO> doList) {
if (ValidateUtils.isEmptyList(doList)) {
return new ArrayList<>();
}
List<TopicMetrics> metricsList = new ArrayList<>();
for (TopicMetricsDO metricsDO: doList) {
metricsList.add(convert2TopicMetrics(metricsDO));
}
return metricsList;
}
public static TopicMetrics convert2TopicMetrics(TopicMetricsDO metricsDO) {
if (ValidateUtils.isNull(metricsDO)) {
return null;
}
TopicMetrics metrics = new TopicMetrics(
metricsDO.getAppId(),
metricsDO.getClusterId(),
metricsDO.getTopicName()
);
metrics.setMetricsMap(JSON.parseObject(metricsDO.getMetrics()));
return metrics;
}
public static List<TopicMetricsDO> convertAndUpdateCreateTime2TopicMetricsDOList(
Long timestamp,
List<TopicMetrics> metricsList) {
if (ValidateUtils.isEmptyList(metricsList)) {
return new ArrayList<>();
}
List<TopicMetricsDO> doList = new ArrayList<>();
for (TopicMetrics elem: metricsList) {
elem.updateCreateTime(timestamp);
if (elem.getMetricsMap().size() == 1) {
// 没有指标数据, 直接过滤掉
continue;
}
TopicMetricsDO metricsDO = new TopicMetricsDO();
metricsDO.setAppId(elem.getAppId());
metricsDO.setClusterId(elem.getClusterId());
metricsDO.setTopicName(elem.getTopicName());
metricsDO.setMetrics(JSON.toJSONString(elem.getMetricsMap()));
doList.add(metricsDO);
}
return doList;
}
public static List<BrokerMetricsDO> convertAndUpdateCreateTime2BrokerMetricsDOList(
Long timestamp,
List<BrokerMetrics> metricsList) {
if (ValidateUtils.isEmptyList(metricsList)) {
return new ArrayList<>();
}
List<BrokerMetricsDO> doList = new ArrayList<>();
for (BrokerMetrics elem: metricsList) {
elem.updateCreateTime(timestamp);
if (elem.getMetricsMap().size() == 1) {
// 没有指标数据, 直接过滤掉
continue;
}
BrokerMetricsDO metricsDO = new BrokerMetricsDO();
metricsDO.setClusterId(elem.getClusterId());
metricsDO.setBrokerId(elem.getBrokerId());
metricsDO.setMetrics(JSON.toJSONString(elem.getMetricsMap()));
doList.add(metricsDO);
}
return doList;
}
public static List<ClusterMetricsDO> convertAndUpdateCreateTime2ClusterMetricsDOList(
Long timestamp,
List<ClusterMetrics> metricsList) {
if (ValidateUtils.isEmptyList(metricsList)) {
return new ArrayList<>();
}
List<ClusterMetricsDO> doList = new ArrayList<>();
for (ClusterMetrics elem: metricsList) {
elem.updateCreateTime(timestamp);
if (elem.getMetricsMap().size() == 1) {
// 没有指标数据, 直接过滤掉
continue;
}
ClusterMetricsDO metricsDO = new ClusterMetricsDO();
metricsDO.setClusterId(elem.getClusterId());
metricsDO.setMetrics(JSON.toJSONString(elem.getMetricsMap()));
doList.add(metricsDO);
}
return doList;
}
public static BaseMetrics merge2BaseMetricsByAdd(List<? extends BaseMetrics> metricsList) {
if (ValidateUtils.isEmptyList(metricsList)) {
return new BaseMetrics();
}
BaseMetrics metrics = new BaseMetrics();
for (BaseMetrics elem: metricsList) {
metrics.mergeByAdd(elem);
}
return metrics;
}
public static BaseMetrics merge2BaseMetricsByMax(List<? extends BaseMetrics> metricsList) {
if (ValidateUtils.isEmptyList(metricsList)) {
return new BaseMetrics();
}
BaseMetrics metrics = new BaseMetrics();
for (BaseMetrics elem: metricsList) {
metrics.mergeByMax(elem);
}
return metrics;
}
public static ReassignTaskDO convert2ReassignTaskDO(Long taskId,
Long clusterId,
TopicMetadata topicMetadata,
ReassignTopicDTO dto,
String reassignmentJson,
String operator) {
ReassignTaskDO reassignTaskDO = new ReassignTaskDO();
reassignTaskDO.setTaskId(taskId);
reassignTaskDO.setClusterId(clusterId);
reassignTaskDO.setTopicName(topicMetadata.getTopic());
reassignTaskDO.setPartitions(ListUtils.intList2String(dto.getPartitionIdList()));
reassignTaskDO.setReassignmentJson(reassignmentJson);
reassignTaskDO.setRealThrottle(dto.getThrottle());
reassignTaskDO.setMinThrottle(dto.getMinThrottle());
reassignTaskDO.setMaxThrottle(dto.getMaxThrottle());
reassignTaskDO.setBeginTime(new Date(dto.getBeginTime()));
reassignTaskDO.setSrcBrokers(ListUtils.intList2String(new ArrayList<>(topicMetadata.getBrokerIdSet())));
reassignTaskDO.setDestBrokers(ListUtils.intList2String(dto.getBrokerIdList()));
reassignTaskDO.setOriginalRetentionTime(dto.getOriginalRetentionTime());
reassignTaskDO.setReassignRetentionTime(dto.getReassignRetentionTime());
reassignTaskDO.setDescription(dto.getDescription());
reassignTaskDO.setOperator(operator);
return reassignTaskDO;
}
}

View File

@@ -0,0 +1,201 @@
package com.xiaojukeji.kafka.manager.service.utils;
import com.xiaojukeji.kafka.manager.common.constant.Constant;
import com.xiaojukeji.kafka.manager.common.entity.ResultStatus;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterDO;
import kafka.admin.AdminOperationException;
import kafka.admin.AdminUtils;
import kafka.admin.BrokerMetadata;
import kafka.common.TopicAndPartition;
import kafka.utils.ZkUtils;
import org.I0Itec.zkclient.exception.ZkNodeExistsException;
import org.apache.kafka.common.errors.*;
import org.apache.kafka.common.security.JaasUtils;
import scala.Option;
import scala.collection.JavaConversions;
import scala.collection.Seq;
import java.util.*;
/**
* @author zengqiao
* @date 20/4/22
*/
public class TopicCommands {
public static ResultStatus createTopic(ClusterDO clusterDO,
String topicName,
Integer partitionNum,
Integer replicaNum,
List<Integer> brokerIdList,
Properties config) {
ZkUtils zkUtils = null;
try {
zkUtils = ZkUtils.apply(
clusterDO.getZookeeper(),
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
JaasUtils.isZkSecurityEnabled()
);
// 生成分配策略
scala.collection.Map<Object, scala.collection.Seq<Object>> replicaAssignment =
AdminUtils.assignReplicasToBrokers(
convert2BrokerMetadataSeq(brokerIdList),
partitionNum,
replicaNum,
randomFixedStartIndex(),
-1
);
// 写ZK
AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(
zkUtils,
topicName,
replicaAssignment,
config,
false
);
} catch (NullPointerException e) {
return ResultStatus.TOPIC_OPERATION_PARAM_NULL_POINTER;
} catch (InvalidPartitionsException e) {
return ResultStatus.TOPIC_OPERATION_PARTITION_NUM_ILLEGAL;
} catch (InvalidReplicationFactorException e) {
return ResultStatus.BROKER_NUM_NOT_ENOUGH;
} catch (TopicExistsException | ZkNodeExistsException e) {
return ResultStatus.TOPIC_OPERATION_TOPIC_EXISTED;
} catch (InvalidTopicException e) {
return ResultStatus.TOPIC_OPERATION_TOPIC_NAME_ILLEGAL;
} catch (Throwable t) {
return ResultStatus.TOPIC_OPERATION_UNKNOWN_ERROR;
} finally {
if (zkUtils != null) {
zkUtils.close();
}
}
return ResultStatus.SUCCESS;
}
public static ResultStatus deleteTopic(ClusterDO clusterDO, String topicName) {
ZkUtils zkUtils = null;
try {
zkUtils = ZkUtils.apply(
clusterDO.getZookeeper(),
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
JaasUtils.isZkSecurityEnabled()
);
AdminUtils.deleteTopic(zkUtils, topicName);
} catch (UnknownTopicOrPartitionException e) {
return ResultStatus.TOPIC_OPERATION_UNKNOWN_TOPIC_PARTITION;
} catch (ZkNodeExistsException e) {
return ResultStatus.TOPIC_OPERATION_TOPIC_IN_DELETING;
} catch (Throwable t) {
return ResultStatus.TOPIC_OPERATION_UNKNOWN_ERROR;
} finally {
if (zkUtils != null) {
zkUtils.close();
}
}
return ResultStatus.SUCCESS;
}
public static ResultStatus modifyTopicConfig(ClusterDO clusterDO, String topicName, Properties config) {
ZkUtils zkUtils = null;
try {
zkUtils = ZkUtils.apply(
clusterDO.getZookeeper(),
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
JaasUtils.isZkSecurityEnabled()
);
AdminUtils.changeTopicConfig(zkUtils, topicName, config);
} catch (AdminOperationException e) {
return ResultStatus.TOPIC_OPERATION_UNKNOWN_TOPIC_PARTITION;
} catch (InvalidConfigurationException e) {
return ResultStatus.TOPIC_OPERATION_TOPIC_CONFIG_ILLEGAL;
} catch (Throwable t) {
return ResultStatus.TOPIC_OPERATION_UNKNOWN_ERROR;
} finally {
if (zkUtils != null) {
zkUtils.close();
}
}
return ResultStatus.SUCCESS;
}
public static ResultStatus expandTopic(ClusterDO clusterDO,
String topicName,
Integer partitionNum,
List<Integer> brokerIdList) {
ZkUtils zkUtils = null;
try {
zkUtils = ZkUtils.apply(
clusterDO.getZookeeper(),
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
JaasUtils.isZkSecurityEnabled()
);
// 已有分区的分配策略
scala.collection.mutable.Map<TopicAndPartition, Seq<Object>> existingAssignScalaMap =
zkUtils.getReplicaAssignmentForTopics(JavaConversions.asScalaBuffer(Arrays.asList(topicName)));
// 新增分区的分配策略
Map<Object, Seq<Object>> newAssignMap = JavaConversions.asJavaMap(
AdminUtils.assignReplicasToBrokers(
convert2BrokerMetadataSeq(brokerIdList),
partitionNum,
existingAssignScalaMap.head()._2().size(),
randomFixedStartIndex(),
existingAssignScalaMap.size()
)
);
Map<TopicAndPartition, scala.collection.Seq<Object>> existingAssignJavaMap =
JavaConversions.asJavaMap(existingAssignScalaMap);
// 新增分区的分配策略和旧的分配策略合并
Map<Object, Seq<Object>> targetMap = new HashMap<>();
for (Map.Entry<TopicAndPartition, Seq<Object>> entry : existingAssignJavaMap.entrySet()) {
targetMap.put(entry.getKey().partition(), entry.getValue());
}
for (Map.Entry<Object, Seq<Object>> entry : newAssignMap.entrySet()) {
targetMap.put(entry.getKey(), entry.getValue());
}
// 更新ZK上的assign
AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(
zkUtils,
topicName,
JavaConversions.asScalaMap(targetMap),
null,
true
);
} catch (Throwable t) {
return ResultStatus.TOPIC_OPERATION_UNKNOWN_ERROR;
} finally {
if (zkUtils != null) {
zkUtils.close();
}
}
return ResultStatus.SUCCESS;
}
private static Seq<BrokerMetadata> convert2BrokerMetadataSeq(List<Integer> brokerIdList) {
List<BrokerMetadata> brokerMetadataList = new ArrayList<>();
for (Integer brokerId: brokerIdList) {
brokerMetadataList.add(new BrokerMetadata(brokerId, Option.<String>empty()));
}
return JavaConversions.asScalaBuffer(brokerMetadataList).toSeq();
}
/**
* 生成一个伪随机数, 即随机选择一个起始位置的Broker
*/
private static int randomFixedStartIndex() {
return (int) System.currentTimeMillis() % 1013;
}
}

View File

@@ -0,0 +1,105 @@
package com.xiaojukeji.kafka.manager.service.utils;
import com.alibaba.fastjson.JSON;
import com.xiaojukeji.kafka.manager.common.constant.Constant;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ClusterDO;
import com.xiaojukeji.kafka.manager.common.utils.ValidateUtils;
import com.xiaojukeji.kafka.manager.common.zookeeper.ZkPathUtil;
import kafka.admin.ReassignPartitionsCommand;
import kafka.common.TopicAndPartition;
import kafka.utils.ZkUtils;
import org.apache.kafka.common.security.JaasUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.collection.JavaConverters;
import scala.collection.Seq;
import java.util.*;
/**
* @author zengqiao
* @date 20/9/21
*/
public class TopicReassignUtils {
private final static Logger LOGGER = LoggerFactory.getLogger(TopicReassignUtils.class);
private static final Integer DATA_VERSION_ONE = 1;
public static String generateReassignmentJson(ClusterDO clusterDO,
String topicName,
List<Integer> partitionIdList,
List<Integer> brokerIdList) {
ZkUtils zkUtils = null;
try {
zkUtils = ZkUtils.apply(clusterDO.getZookeeper(),
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
Constant.DEFAULT_SESSION_TIMEOUT_UNIT_MS,
JaasUtils.isZkSecurityEnabled());
if (zkUtils.pathExists(ZkPathUtil.REASSIGN_PARTITIONS_ROOT_NODE)) {
// 任务已经存在, 不知道是谁弄的
return null;
}
// 生成迁移JSON
return generateReassignmentJson(zkUtils, topicName, partitionIdList, brokerIdList);
} catch (Throwable t) {
LOGGER.error("generate assignment json failed, clusterId:{} topicName:{} partitions:{} brokers:{}."
, clusterDO.getId(), topicName, partitionIdList, brokerIdList, t);
} finally {
if (zkUtils != null) {
zkUtils.close();
}
zkUtils = null;
}
return null;
}
private static String generateReassignmentJson(ZkUtils zkUtils,
String topicName,
List<Integer> partitionIdList,
List<Integer> brokerIdList) {
Map<TopicAndPartition, Seq<Object>> reassignMap = createReassignmentMap(
zkUtils,
topicName,
new ArrayList<>(brokerIdList)
);
if (!ValidateUtils.isEmptyList(partitionIdList)) {
Iterator<Map.Entry<TopicAndPartition, Seq<Object>>> it = reassignMap.entrySet().iterator();
while (it.hasNext()) {
Map.Entry<TopicAndPartition, Seq<Object>> entry = it.next();
if (partitionIdList.contains(entry.getKey().partition())) {
continue;
}
// 移除不在迁移中的对象
it.remove();
}
}
return ZkUtils.formatAsReassignmentJson(JavaConverters.mapAsScalaMapConverter(reassignMap).asScala());
}
private static Map<TopicAndPartition, Seq<Object>> createReassignmentMap(ZkUtils zkUtils,
String topicName,
List<Object> brokerIdList) {
scala.collection.Map<TopicAndPartition, Seq<Object>> scalaReassignmentMap =
ReassignPartitionsCommand.generateAssignment(
zkUtils,
JavaConverters.asScalaIteratorConverter(brokerIdList.iterator()).asScala().toSeq(),
JSON.toJSONString(generateTopicMoveProperties(topicName)),
false)
._1();
return JavaConverters.mapAsJavaMapConverter(scalaReassignmentMap).asJava();
}
private static Properties generateTopicMoveProperties (String topicName) {
Map<String, Object> topicNameMap = new HashMap<>(1);
topicNameMap.put("topic", topicName);
Properties properties = new Properties();
properties.put("topics", Arrays.asList(
topicNameMap
));
properties.put("version", DATA_VERSION_ONE);
return properties;
}
}

View File

@@ -0,0 +1,89 @@
package com.xiaojukeji.kafka.manager.service.zookeeper;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.BrokerMetadata;
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.service.cache.PhysicalClusterMetadataManager;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.List;
/**
* ZK Broker节点变更
* @author zengqiao
* @date 20/5/14
*/
public class BrokerStateListener implements StateChangeListener {
private final static Logger LOGGER = LoggerFactory.getLogger(BrokerStateListener.class);
private Long clusterId;
private ZkConfigImpl zkConfig;
public BrokerStateListener(Long clusterId, ZkConfigImpl zkConfig) {
this.clusterId = clusterId;
this.zkConfig = zkConfig;
}
@Override
public void init() {
try {
List<String> brokerIdList = zkConfig.getChildren(ZkPathUtil.BROKER_IDS_ROOT);
for (String brokerId: brokerIdList) {
processBrokerAdded(Integer.valueOf(brokerId));
}
} catch (Exception e) {
LOGGER.error("init brokers metadata failed, clusterId:{}.", clusterId, e);
}
}
@Override
public void onChange(State state, String path) {
try {
String brokerId = ZkPathUtil.parseLastPartFromZkPath(path);
switch (state) {
case CHILD_ADDED:
case CHILD_UPDATED:
processBrokerAdded(Integer.valueOf(brokerId));
break;
case CHILD_DELETED:
processBrokerDelete(Integer.valueOf(brokerId));
break;
default:
break;
}
} catch (Exception e) {
LOGGER.error("process broker state change failed, clusterId:{} state:{} path:{}.",
clusterId, state, path, e);
}
}
private void processBrokerDelete(Integer brokerId) {
LOGGER.warn("delete broker, clusterId:{} brokerId:{}.", clusterId, brokerId);
PhysicalClusterMetadataManager.removeBrokerMetadata(clusterId, brokerId);
}
private void processBrokerAdded(Integer brokerId) {
LOGGER.warn("add broker, clusterId:{} brokerId:{}.", clusterId, brokerId);
BrokerMetadata brokerMetadata = null;
try {
brokerMetadata = zkConfig.get(ZkPathUtil.getBrokerIdNodePath(brokerId), BrokerMetadata.class);
if (!brokerMetadata.getEndpoints().isEmpty()) {
String endpoint = brokerMetadata.getEndpoints().get(0);
int idx = endpoint.indexOf("://");
endpoint = endpoint.substring(idx + "://".length());
idx = endpoint.indexOf(":");
brokerMetadata.setHost(endpoint.substring(0, idx));
brokerMetadata.setPort(Integer.parseInt(endpoint.substring(idx + 1)));
}
brokerMetadata.setClusterId(clusterId);
brokerMetadata.setBrokerId(brokerId);
PhysicalClusterMetadataManager.putBrokerMetadata(clusterId, brokerId, brokerMetadata);
} catch (Exception e) {
LOGGER.error("add broker failed, clusterId:{} brokerMetadata:{}.", clusterId, brokerMetadata, e);
}
}
}

View File

@@ -0,0 +1,83 @@
package com.xiaojukeji.kafka.manager.service.zookeeper;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.BrokerMetadata;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.ControllerData;
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.ControllerDao;
import com.xiaojukeji.kafka.manager.common.entity.pojo.ControllerDO;
import com.xiaojukeji.kafka.manager.service.cache.PhysicalClusterMetadataManager;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.dao.DuplicateKeyException;
/**
* Controller 变更监听
* @author zengqiao
* @date 20/5/14
*/
public class ControllerStateListener implements StateChangeListener {
private final static Logger LOGGER = LoggerFactory.getLogger(ControllerStateListener.class);
private Long clusterId;
private ZkConfigImpl zkConfig;
private ControllerDao controllerDao;
public ControllerStateListener(Long clusterId, ZkConfigImpl zkConfig, ControllerDao controllerDao) {
this.clusterId = clusterId;
this.zkConfig = zkConfig;
this.controllerDao = controllerDao;
}
@Override
public void init() {
processControllerChange();
return;
}
@Override
public void onChange(State state, String path) {
try {
switch (state) {
case NODE_DATA_CHANGED:
processControllerChange();
default:
break;
}
} catch (Exception e) {
LOGGER.error("process controller state change failed, clusterId:{} state:{} path:{}.",
clusterId, state, path, e);
}
}
private void processControllerChange(){
LOGGER.warn("init controllerData or controller change, clusterId:{}.", clusterId);
ControllerData controllerData = null;
try {
controllerData = zkConfig.get(ZkPathUtil.CONTROLLER_ROOT_NODE, ControllerData.class);
if (controllerData == null) {
PhysicalClusterMetadataManager.removeControllerData(clusterId);
}
PhysicalClusterMetadataManager.putControllerData(clusterId, controllerData);
BrokerMetadata brokerMetadata =
PhysicalClusterMetadataManager.getBrokerMetadata(clusterId, controllerData.getBrokerid());
ControllerDO controllerDO = ControllerDO.newInstance(
clusterId,
controllerData.getBrokerid(),
brokerMetadata != null? brokerMetadata.getHost(): "",
controllerData.getTimestamp(),
controllerData.getVersion()
);
controllerDao.insert(controllerDO);
} catch (DuplicateKeyException e) {
//ignore
} catch (Exception e) {
LOGGER.error("add controller failed, clusterId:{} controllerData:{}.", clusterId, controllerData, e);
}
}
}

View File

@@ -0,0 +1,110 @@
package com.xiaojukeji.kafka.manager.service.zookeeper;
import com.xiaojukeji.kafka.manager.common.zookeeper.znode.brokers.PartitionMap;
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.service.cache.PhysicalClusterMetadataManager;
import com.xiaojukeji.kafka.manager.service.cache.ThreadPool;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.*;
/**
* @author zengqiao
* @date 20/5/14
*/
public class TopicStateListener implements StateChangeListener {
private final static Logger LOGGER = LoggerFactory.getLogger(TopicStateListener.class);
private Long clusterId;
private ZkConfigImpl zkConfig;
public TopicStateListener(Long clusterId, ZkConfigImpl zkConfig) {
this.clusterId = clusterId;
this.zkConfig = zkConfig;
}
@Override
public void init() {
try {
List<String> topicNameList = zkConfig.getChildren(ZkPathUtil.BROKER_TOPICS_ROOT);
FutureTask[] taskList = new FutureTask[topicNameList.size()];
for (int i = 0; i < topicNameList.size(); i++) {
String topicName = topicNameList.get(i);
taskList[i] = new FutureTask(new Callable() {
@Override
public Object call() throws Exception {
processTopicAdded(topicName);
return null;
}
});
ThreadPool.submitCollectMetricsTask(taskList[i]);
}
} catch (Exception e) {
LOGGER.error("init topics metadata failed, clusterId:{}.", clusterId, e);
}
return;
}
@Override
public void onChange(State state, String path) {
try {
String topicName = ZkPathUtil.parseLastPartFromZkPath(path);
switch (state) {
case CHILD_ADDED:
case CHILD_UPDATED:
processTopicAdded(topicName);
break;
case CHILD_DELETED:
processTopicDelete(topicName);
break;
default:
break;
}
} catch (Exception e) {
LOGGER.error("process topic state change failed, clusterId:{} state:{} path:{}.",
clusterId, state, path, e);
}
}
private void processTopicDelete(String topicName) {
LOGGER.warn("delete topic, clusterId:{} topicName:{}.", clusterId, topicName);
PhysicalClusterMetadataManager.removeTopicMetadata(clusterId, topicName);
}
private void processTopicAdded(String topicName) {
LOGGER.info("add topic, clusterId:{} topicName:{}.", clusterId, topicName);
TopicMetadata topicMetadata = new TopicMetadata();
try {
topicMetadata.setTopic(topicName);
Stat stat = zkConfig.getNodeStat(ZkPathUtil.getBrokerTopicRoot(topicName));
topicMetadata.setCreateTime(stat.getCtime());
topicMetadata.setModifyTime(stat.getMtime());
PartitionMap partitionMap = zkConfig.get(ZkPathUtil.getBrokerTopicRoot(topicName), PartitionMap.class);
topicMetadata.setPartitionMap(partitionMap);
topicMetadata.setReplicaNum(partitionMap.getPartitions().values().iterator().next().size());
topicMetadata.setPartitionNum(partitionMap.getPartitions().size());
Set<Integer> brokerIdSet = new HashSet<>();
Map<Integer, List<Integer>> topicBrokers = partitionMap.getPartitions();
for (Map.Entry<Integer, List<Integer>> entry : topicBrokers.entrySet()) {
brokerIdSet.addAll(entry.getValue());
}
topicMetadata.setBrokerIdSet(brokerIdSet);
PhysicalClusterMetadataManager.putTopicMetadata(clusterId, topicName, topicMetadata);
} catch (Exception e) {
LOGGER.error("add topic failed, clusterId:{} topicMetadata:{}.", clusterId, topicMetadata, e);
}
}
}

View File

@@ -0,0 +1,3 @@
Manifest-Version: 1.0
Class-Path:

View File

@@ -0,0 +1,47 @@
package com.xiaojukeji.kafka.manager.service;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.FutureTask;
import org.junit.Test;
public class FutureTest {
@Test
public void test() throws InterruptedException, ExecutionException {
FutureTask<Integer> f1 = new FutureTask<Integer>(new Callable<Integer>() {
@Override
public Integer call() throws InterruptedException {
Thread.sleep(1000L);
return 1;
}
});
FutureTask<Integer> f2 = new FutureTask<Integer>(new Callable<Integer>() {
@Override
public Integer call() throws InterruptedException {
Thread.sleep(1000L);
return 2;
}
});
ExecutorService threadPool = Executors.newCachedThreadPool();
long ct = System.currentTimeMillis();
threadPool.submit(f1);
threadPool.submit(f2);
threadPool.shutdown();
System.out.println(f1.get() + " : " + f2.get() + " use:"
+ (System.currentTimeMillis() - ct));
}
}

View File

@@ -0,0 +1,13 @@
package com.xiaojukeji.kafka.manager.service.utils;
import org.junit.runner.RunWith;
import org.springframework.test.context.ContextConfiguration;
import org.springframework.test.context.junit4.SpringJUnit4ClassRunner;
/**
* Created by arthur on 2017/5/31.
*/
@RunWith(SpringJUnit4ClassRunner.class)
@ContextConfiguration(locations = { "classpath:biz-test.xml" })
public class SpringTestBase {
}