瀏覽代碼

fix review issues //
save errors //
save connections and update connection logic //
save jmx, zookeeper, kafka statuses //
error with getting one topic doesn't fail others //
async metrics processing //
cluster data storage refactoring

Zhenya Taran 5 年之前
父節點
當前提交
633fb5686e

+ 2 - 0
kafka-ui-api/src/main/java/com/provectus/kafka/ui/KafkaMetricsApplication.java

@@ -2,10 +2,12 @@ package com.provectus.kafka.ui;
 
 import org.springframework.boot.SpringApplication;
 import org.springframework.boot.autoconfigure.SpringBootApplication;
+import org.springframework.scheduling.annotation.EnableAsync;
 import org.springframework.scheduling.annotation.EnableScheduling;
 
 @SpringBootApplication
 @EnableScheduling
+@EnableAsync
 public class KafkaMetricsApplication {
 
 	public static void main(String[] args) {

+ 0 - 36
kafka-ui-api/src/main/java/com/provectus/kafka/ui/cluster/ClustersMetricUpdateScheduler.java

@@ -1,36 +0,0 @@
-package com.provectus.kafka.ui.cluster;
-
-import com.provectus.kafka.ui.cluster.model.KafkaCluster;
-import com.provectus.kafka.ui.cluster.service.ClusterService;
-import com.provectus.kafka.ui.jmx.JmxService;
-import com.provectus.kafka.ui.kafka.KafkaService;
-import com.provectus.kafka.ui.model.ClusterStatus;
-import lombok.RequiredArgsConstructor;
-import lombok.extern.log4j.Log4j2;
-import org.springframework.scheduling.annotation.Scheduled;
-import org.springframework.stereotype.Component;
-
-@Component
-@RequiredArgsConstructor
-@Log4j2
-public class ClustersMetricUpdateScheduler {
-
-    private final JmxService jmxService;
-    private final KafkaService kafkaService;
-
-    private final ClusterService clusterService;
-
-    @Scheduled(fixedRate = 30000)
-    public void updateMetrics() {
-        for (KafkaCluster cluster : clusterService.getKafkaClusters()) {
-            try {
-                kafkaService.loadClusterMetrics(cluster);
-                jmxService.loadClusterMetrics(cluster);
-                cluster.setStatus(ClusterStatus.ONLINE);
-            } catch (Exception e) {
-                log.error(e);
-                cluster.setStatus(ClusterStatus.OFFLINE);
-            }
-        }
-    }
-}

+ 26 - 0
kafka-ui-api/src/main/java/com/provectus/kafka/ui/cluster/ClustersMetricsScheduler.java

@@ -0,0 +1,26 @@
+package com.provectus.kafka.ui.cluster;
+
+import com.provectus.kafka.ui.cluster.model.ClustersStorage;
+import com.provectus.kafka.ui.cluster.model.KafkaCluster;
+import com.provectus.kafka.ui.cluster.service.MetricsUpdateService;
+import lombok.RequiredArgsConstructor;
+import lombok.extern.log4j.Log4j2;
+import org.springframework.scheduling.annotation.Scheduled;
+import org.springframework.stereotype.Component;
+
+@Component
+@RequiredArgsConstructor
+@Log4j2
+public class ClustersMetricsScheduler {
+
+    private final ClustersStorage clustersStorage;
+
+    private final MetricsUpdateService metricsUpdateService;
+
+    @Scheduled(fixedRate = 30000)
+    public void updateMetrics() {
+        for (KafkaCluster kafkaCluster : clustersStorage.getKafkaClusters()) {
+            metricsUpdateService.updateMetrics(kafkaCluster);
+        }
+    }
+}

+ 33 - 0
kafka-ui-api/src/main/java/com/provectus/kafka/ui/cluster/model/ClustersStorage.java

@@ -0,0 +1,33 @@
+package com.provectus.kafka.ui.cluster.model;
+
+import com.provectus.kafka.ui.cluster.config.ClustersProperties;
+import com.provectus.kafka.ui.cluster.mapper.ClusterMapper;
+import lombok.RequiredArgsConstructor;
+import org.mapstruct.factory.Mappers;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.PostConstruct;
+import java.util.ArrayList;
+import java.util.List;
+
+@Component
+@RequiredArgsConstructor
+public class ClustersStorage {
+
+    private final List<KafkaCluster> kafkaClusters = new ArrayList<>();
+
+    private final ClustersProperties clusterProperties;
+
+    private final ClusterMapper clusterMapper = Mappers.getMapper(ClusterMapper.class);
+
+    @PostConstruct
+    public void init() {
+        for (ClustersProperties.Cluster clusterProperties : clusterProperties.getClusters()) {
+            kafkaClusters.add(clusterMapper.toKafkaCluster(clusterProperties));
+        }
+    }
+
+    public List<KafkaCluster> getKafkaClusters() {
+        return kafkaClusters;
+    }
+}

+ 8 - 5
kafka-ui-api/src/main/java/com/provectus/kafka/ui/cluster/model/KafkaCluster.java

@@ -1,6 +1,6 @@
 package com.provectus.kafka.ui.cluster.model;
 
-import com.provectus.kafka.ui.model.ClusterStatus;
+import com.provectus.kafka.ui.model.ServerStatus;
 import com.provectus.kafka.ui.model.Topic;
 import com.provectus.kafka.ui.model.TopicDetails;
 import lombok.AccessLevel;
@@ -25,7 +25,6 @@ public class KafkaCluster {
     String jmxPort;
     String bootstrapServers;
     String zookeeper;
-    ClusterStatus status = ClusterStatus.OFFLINE;
 
     Map<String, String> metricsMap = new ConcurrentHashMap<>();
     List<Topic> topics = new ArrayList<>();
@@ -35,9 +34,13 @@ public class KafkaCluster {
     ZkClient zkClient;
     AdminClient adminClient;
 
-    Exception kafkaException;
-    Exception jmxException;
-    Exception zookeeperException;
+    ServerStatus status = ServerStatus.OFFLINE;
+    ServerStatus jmxStatus = ServerStatus.OFFLINE;
+    ServerStatus zookeeperStatus = ServerStatus.OFFLINE;
+
+    Exception lastKafkaException;
+    Exception lastJmxException;
+    Exception lastZookeeperException;
 
     public void putMetric(String metricKey, String metricValue) {
         metricsMap.put(metricKey, metricValue);

+ 1 - 2
kafka-ui-api/src/main/java/com/provectus/kafka/ui/cluster/model/MetricsConstants.java

@@ -2,7 +2,6 @@ package com.provectus.kafka.ui.cluster.model;
 
 public final class MetricsConstants {
 
-
     private MetricsConstants() {}
 
     public static final String CLUSTER_ID = "ClusterId";
@@ -11,7 +10,7 @@ public final class MetricsConstants {
     public static final String BROKERS_COUNT = "BrokersCount";
     public static final String TOPIC_COUNT = "TopicCount";
     public static final String PARTITIONS_COUNT = "PartitionsCount";
-    public static final String ZOOKEEPER_STATUS = "ZookeeperStatus";
+    public static final String ZOOKEEPER_STATUS = "ZooKeeperStatus";
     public static final String ACTIVE_CONTROLLER_COUNT = "ActiveControllerCount";
     public static final String ONLINE_PARTITION_COUNT = "OnlinePartitionCount";
     public static final String OFFLINE_PARTITION_COUNT = "OfflinePartitionCount";

+ 6 - 20
kafka-ui-api/src/main/java/com/provectus/kafka/ui/cluster/service/ClusterService.java

@@ -1,7 +1,7 @@
 package com.provectus.kafka.ui.cluster.service;
 
-import com.provectus.kafka.ui.cluster.config.ClustersProperties;
 import com.provectus.kafka.ui.cluster.mapper.ClusterMapper;
+import com.provectus.kafka.ui.cluster.model.ClustersStorage;
 import com.provectus.kafka.ui.cluster.model.KafkaCluster;
 import com.provectus.kafka.ui.cluster.model.MetricsConstants;
 import com.provectus.kafka.ui.model.BrokerMetrics;
@@ -15,8 +15,6 @@ import org.springframework.stereotype.Service;
 import reactor.core.publisher.Flux;
 import reactor.core.publisher.Mono;
 
-import javax.annotation.PostConstruct;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.stream.Collectors;
 
@@ -26,21 +24,13 @@ import static com.provectus.kafka.ui.cluster.model.MetricsConstants.CLUSTER_ID;
 @RequiredArgsConstructor
 public class ClusterService {
 
-    private final List<KafkaCluster> kafkaClusters = new ArrayList<>();
-
-    private final ClustersProperties clusterProperties;
+    private final ClustersStorage clustersStorage;
 
     private final ClusterMapper clusterMapper = Mappers.getMapper(ClusterMapper.class);
 
-    @PostConstruct
-    public void init() {
-        for (ClustersProperties.Cluster clusterProperties : clusterProperties.getClusters()) {
-            kafkaClusters.add(clusterMapper.toKafkaCluster(clusterProperties));
-        }
-    }
 
     public Mono<ResponseEntity<Flux<Cluster>>> getClusters() {
-        List<Cluster> clusters = kafkaClusters
+        List<Cluster> clusters = clustersStorage.getKafkaClusters()
                 .stream()
                 .map(kafkaCluster -> {
                     Cluster cluster = clusterMapper.toOpenApiCluster(kafkaCluster);
@@ -58,7 +48,7 @@ public class ClusterService {
     }
 
     public Mono<ResponseEntity<BrokerMetrics>> getBrokerMetrics(String clusterId) {
-        KafkaCluster cluster = kafkaClusters.stream()
+        KafkaCluster cluster = clustersStorage.getKafkaClusters().stream()
                 .filter(cltr -> cltr.getMetricsMap().get(CLUSTER_ID).equals(clusterId))
                 .findFirst()
                 .orElseThrow();
@@ -67,7 +57,7 @@ public class ClusterService {
         brokerMetrics.setClusterId(cluster.getMetricsMap().get(CLUSTER_ID));
         brokerMetrics.setBrokerCount(intValueOfOrNull(cluster.getMetric(MetricsConstants.BROKERS_COUNT)));
         brokerMetrics.setBytesInPerSec(intValueOfOrNull(cluster.getMetric(MetricsConstants.BYTES_IN_PER_SEC)));
-        brokerMetrics.setZookeeperStatus(intValueOfOrNull(cluster.getMetric(MetricsConstants.ZOOKEEPER_STATUS)));
+        brokerMetrics.setZooKeeperStatus(intValueOfOrNull(cluster.getMetric(MetricsConstants.ZOOKEEPER_STATUS)));
         brokerMetrics.setActiveControllers(intValueOfOrNull(cluster.getMetric(MetricsConstants.ACTIVE_CONTROLLER_COUNT)));
         brokerMetrics.setOnlinePartitionCount(intValueOfOrNull(cluster.getMetric(MetricsConstants.ONLINE_PARTITION_COUNT)));
         brokerMetrics.setOfflinePartitionCount(intValueOfOrNull(cluster.getMetric(MetricsConstants.OFFLINE_PARTITION_COUNT)));
@@ -77,7 +67,7 @@ public class ClusterService {
     }
 
     public Mono<ResponseEntity<Flux<Topic>>> getTopics(String clusterId) {
-        KafkaCluster cluster = kafkaClusters.stream()
+        KafkaCluster cluster = clustersStorage.getKafkaClusters().stream()
                 .filter(cltr -> cltr.getMetricsMap().get(CLUSTER_ID).equals(clusterId))
                 .findFirst()
                 .orElseThrow();
@@ -85,10 +75,6 @@ public class ClusterService {
         return Mono.just(ResponseEntity.ok(Flux.fromIterable(cluster.getTopics())));
     }
 
-    public List<KafkaCluster> getKafkaClusters() {
-        return kafkaClusters;
-    }
-
     private Integer intValueOfOrNull(String s) {
         try {
             return Integer.valueOf(s);

+ 28 - 0
kafka-ui-api/src/main/java/com/provectus/kafka/ui/cluster/service/MetricsUpdateService.java

@@ -0,0 +1,28 @@
+package com.provectus.kafka.ui.cluster.service;
+
+import com.provectus.kafka.ui.cluster.model.KafkaCluster;
+import com.provectus.kafka.ui.jmx.JmxService;
+import com.provectus.kafka.ui.kafka.KafkaService;
+import com.provectus.kafka.ui.zookeeper.ZookeeperService;
+import lombok.RequiredArgsConstructor;
+import lombok.extern.log4j.Log4j2;
+import org.springframework.scheduling.annotation.Async;
+import org.springframework.stereotype.Service;
+
+@Service
+@RequiredArgsConstructor
+@Log4j2
+public class MetricsUpdateService {
+
+    private final JmxService jmxService;
+    private final KafkaService kafkaService;
+    private final ZookeeperService zookeeperService;
+
+    @Async
+    public void updateMetrics(KafkaCluster kafkaCluster) {
+        log.debug("Start getting metrics for kafkaCluster: " + kafkaCluster.getName());
+        kafkaService.loadClusterMetrics(kafkaCluster);
+        jmxService.loadClusterMetrics(kafkaCluster);
+        zookeeperService.checkZookeeperStatus(kafkaCluster);
+    }
+}

+ 54 - 14
kafka-ui-api/src/main/java/com/provectus/kafka/ui/jmx/JmxService.java

@@ -1,16 +1,18 @@
 package com.provectus.kafka.ui.jmx;
 
 import com.provectus.kafka.ui.cluster.model.KafkaCluster;
+import com.provectus.kafka.ui.model.ServerStatus;
 import lombok.RequiredArgsConstructor;
 import lombok.SneakyThrows;
 import lombok.extern.log4j.Log4j2;
+import org.springframework.scheduling.annotation.Async;
 import org.springframework.stereotype.Service;
 
-import javax.management.MBeanServerConnection;
 import javax.management.ObjectName;
 import javax.management.remote.JMXConnector;
 import javax.management.remote.JMXConnectorFactory;
 import javax.management.remote.JMXServiceURL;
+import java.io.IOException;
 import java.util.Map;
 
 @Service
@@ -19,24 +21,62 @@ import java.util.Map;
 public class JmxService {
 
     @SneakyThrows
+    @Async
     public void loadClusterMetrics(KafkaCluster kafkaCluster) {
-        // check before getting something
+        log.debug("Start getting JMX metrics for kafkaCluster: " + kafkaCluster.getName());
+        boolean isConnected = false;
+        if (kafkaCluster.getMBeanServerConnection() != null) {
+            isConnected = isJmxConnected(kafkaCluster);
+        }
+        if (kafkaCluster.getMBeanServerConnection() == null || !isConnected) {
+            isConnected = createJmxConnection(kafkaCluster);
+        }
+
+        if (!isConnected) {
+            kafkaCluster.setJmxStatus(ServerStatus.OFFLINE);
+            return;
+        }
+
+        kafkaCluster.setJmxStatus(ServerStatus.ONLINE);
+        loadJmxMetrics(kafkaCluster);
+    }
+
+    @SneakyThrows
+    private void loadJmxMetrics(KafkaCluster kafkaCluster) {
+        for (Map.Entry<MBeanInfo, String> mbeanToMetric : JmxConstants.mbeanToAttributeMap.entrySet()) {
+            MBeanInfo mBeanInfo = mbeanToMetric.getKey();
+            Object attributeValue = kafkaCluster.getMBeanServerConnection().getAttribute(new ObjectName(mBeanInfo.getName()), mBeanInfo.getAttribute());
+            kafkaCluster.putMetric(mbeanToMetric.getValue(), attributeValue.toString());
+        }
+
+    }
+
+    private boolean createJmxConnection(KafkaCluster kafkaCluster) {
         try {
-            if (kafkaCluster.getMBeanServerConnection() == null) {
-                String url = "service:jmx:rmi:///jndi/rmi://" + kafkaCluster.getJmxHost() + ":" + kafkaCluster.getJmxPort() + "/jmxrmi";
-                JMXServiceURL serviceUrl = new JMXServiceURL(url);
-                JMXConnector jmxConnector = JMXConnectorFactory.connect(serviceUrl, null);
-                kafkaCluster.setMBeanServerConnection(jmxConnector.getMBeanServerConnection());
-            }
-            for (Map.Entry<MBeanInfo, String> mbeanToMetric : JmxConstants.mbeanToAttributeMap.entrySet()) {
-                MBeanInfo mBeanInfo = mbeanToMetric.getKey();
-                Object attributeValue = kafkaCluster.getMBeanServerConnection().getAttribute(new ObjectName(mBeanInfo.getName()), mBeanInfo.getAttribute());
-                kafkaCluster.putMetric(mbeanToMetric.getValue(), attributeValue.toString());
-            }
+            String url = "service:jmx:rmi:///jndi/rmi://" + kafkaCluster.getJmxHost() + ":" + kafkaCluster.getJmxPort() + "/jmxrmi";
+            JMXServiceURL serviceUrl = new JMXServiceURL(url);
+            JMXConnector jmxConnector = JMXConnectorFactory.connect(serviceUrl, null);
+            kafkaCluster.setMBeanServerConnection(jmxConnector.getMBeanServerConnection());
+
+            return true;
         } catch (Exception e) {
             log.error(e);
-            kafkaCluster.setMBeanServerConnection(null);
+            kafkaCluster.setLastJmxException(e);
+
+            return false;
         }
+    }
 
+    private boolean isJmxConnected(KafkaCluster kafkaCluster) {
+        try {
+            kafkaCluster.getMBeanServerConnection().getMBeanCount();
+
+            return true;
+        } catch (IOException e) {
+            log.error(e);
+            kafkaCluster.setLastJmxException(e);
+
+            return false;
+        }
     }
 }

+ 75 - 33
kafka-ui-api/src/main/java/com/provectus/kafka/ui/kafka/KafkaService.java

@@ -4,55 +4,100 @@ import com.provectus.kafka.ui.cluster.model.KafkaCluster;
 import com.provectus.kafka.ui.cluster.model.MetricsConstants;
 import com.provectus.kafka.ui.model.Partition;
 import com.provectus.kafka.ui.model.Replica;
+import com.provectus.kafka.ui.model.ServerStatus;
 import com.provectus.kafka.ui.model.Topic;
 import lombok.RequiredArgsConstructor;
 import lombok.SneakyThrows;
-import org.I0Itec.zkclient.ZkClient;
+import lombok.extern.log4j.Log4j2;
 import org.apache.kafka.clients.admin.*;
 import org.apache.kafka.common.KafkaFuture;
 import org.apache.kafka.common.Node;
 import org.apache.kafka.common.TopicPartitionInfo;
+import org.springframework.scheduling.annotation.Async;
 import org.springframework.stereotype.Service;
 
 import java.util.*;
 
 import static com.provectus.kafka.ui.cluster.model.MetricsConstants.CLUSTER_ID;
-import static com.provectus.kafka.ui.cluster.model.MetricsConstants.ZOOKEEPER_STATUS;
 
 @Service
 @RequiredArgsConstructor
+@Log4j2
 public class KafkaService {
 
     @SneakyThrows
+    @Async
     public void loadClusterMetrics(KafkaCluster kafkaCluster) {
-        checkZookeperConnection(kafkaCluster);
+        log.debug("Start getting Kafka metrics for cluster: " + kafkaCluster.getName());
+        boolean isConnected = false;
+        if (kafkaCluster.getAdminClient() != null) {
+            isConnected = isAdminClientConnected(kafkaCluster);
+        }
+        if (kafkaCluster.getAdminClient() == null || !isConnected) {
+            isConnected = createAdminClient(kafkaCluster);
+        }
+
+        if (!isConnected) {
+            kafkaCluster.setStatus(ServerStatus.OFFLINE);
+
+            return;
+        }
+
+        kafkaCluster.setStatus(ServerStatus.ONLINE);
+        loadMetrics(kafkaCluster);
+        loadTopics(kafkaCluster);
+        loadTopicsDetails(kafkaCluster);
+    }
+
+    private boolean createAdminClient(KafkaCluster kafkaCluster) {
+        try {
+            Properties properties = new Properties();
+            properties.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaCluster.getBootstrapServers());
+            properties.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, 5000);
+            kafkaCluster.setAdminClient(AdminClient.create(properties));
 
-        Properties properties = new Properties();
-        properties.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaCluster.getBootstrapServers());
-        properties.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, 5000);
-        AdminClient adminClient = AdminClient.create(properties);
+            return true;
+        } catch (Exception e) {
+            log.error(e);
+            kafkaCluster.setLastKafkaException(e);
 
-        loadMetrics(kafkaCluster, adminClient);
-        loadTopics(kafkaCluster, adminClient);
-        loadTopicsDetails(kafkaCluster, adminClient);
+            return false;
+        }
+    }
+
+    private boolean isAdminClientConnected(KafkaCluster kafkaCluster) {
+        try {
+            kafkaCluster.getAdminClient().listTopics();
+
+            return true;
+        } catch (Exception e) {
+            log.error(e);
+            kafkaCluster.setLastKafkaException(e);
+
+            return false;
+        }
     }
 
-    private void loadTopicsDetails(KafkaCluster kafkaCluster, AdminClient adminClient) {
+    private void loadTopicsDetails(KafkaCluster kafkaCluster) {
 
     }
 
     @SneakyThrows
-    private void loadTopics(KafkaCluster kafkaCluster, AdminClient adminClient) {
+    private void loadTopics(KafkaCluster kafkaCluster) {
+        AdminClient adminClient = kafkaCluster.getAdminClient();
         ListTopicsOptions listTopicsOptions = new ListTopicsOptions();
         listTopicsOptions.listInternal(true);
-        Collection<String> topicListings = adminClient.listTopics(listTopicsOptions).names().get();
+        var topicListings = adminClient.listTopics(listTopicsOptions).names().get();
 
-        DescribeTopicsResult topicDescriptionsWrapper= adminClient.describeTopics(topicListings);
+        DescribeTopicsResult topicDescriptionsWrapper = adminClient.describeTopics(topicListings);
         Map<String, KafkaFuture<TopicDescription>> topicDescriptionFuturesMap = topicDescriptionsWrapper.values();
         List<Topic> foundTopics = new ArrayList<>();
         String clusterId = kafkaCluster.getMetricsMap().get(CLUSTER_ID);
-        for (KafkaFuture<TopicDescription> value : topicDescriptionFuturesMap.values()) {
-            TopicDescription topicDescription = value.get();
+
+        for (var entry : topicDescriptionFuturesMap.entrySet()) {
+            var topicDescription = getTopicDescription(entry);
+            if (topicDescription == null) continue;
+
             var topic = new Topic();
             topic.setClusterId(clusterId);
             topic.setInternal(topicDescription.isInternal());
@@ -79,9 +124,22 @@ public class KafkaService {
             foundTopics.add(topic);
         }
         kafkaCluster.setTopics(foundTopics);
+
+
     }
 
-    private void loadMetrics(KafkaCluster kafkaCluster, AdminClient adminClient) throws InterruptedException, java.util.concurrent.ExecutionException {
+    private TopicDescription getTopicDescription(Map.Entry<String, KafkaFuture<TopicDescription>> entry) {
+        try {
+            return entry.getValue().get();
+        } catch (Exception e) {
+            log.error("Can't get topic with name: " + entry.getKey(), e);
+
+            return null;
+        }
+    }
+
+    private void loadMetrics(KafkaCluster kafkaCluster) throws InterruptedException, java.util.concurrent.ExecutionException {
+        AdminClient adminClient = kafkaCluster.getAdminClient();
         kafkaCluster.putMetric(MetricsConstants.BROKERS_COUNT, String.valueOf(adminClient.describeCluster().nodes().get().size()));
 
         ListTopicsOptions listTopicsOptions = new ListTopicsOptions();
@@ -96,20 +154,4 @@ public class KafkaService {
         }
         kafkaCluster.putMetric(MetricsConstants.PARTITIONS_COUNT, String.valueOf(partitionsNum));
     }
-
-    public static void checkZookeperConnection(KafkaCluster kafkaCluster){
-        try {
-            if (kafkaCluster.getZkClient() == null) {
-
-            } else {
-            }
-            kafkaCluster.setZkClient(new ZkClient(kafkaCluster.getZookeeper(), 1000));
-            kafkaCluster.putMetric(ZOOKEEPER_STATUS, "1");
-        } catch (Exception e) {
-            kafkaCluster.setZkClient(null);
-            kafkaCluster.putMetric(ZOOKEEPER_STATUS, "0");
-            kafkaCluster.setZookeeperException(e);
-            throw e;
-        }
-    }
 }

+ 4 - 25
kafka-ui-api/src/main/java/com/provectus/kafka/ui/rest/MetricsRestController.java

@@ -8,55 +8,34 @@ import com.provectus.kafka.ui.model.Topic;
 import com.provectus.kafka.ui.model.TopicDetails;
 import lombok.RequiredArgsConstructor;
 import org.springframework.http.ResponseEntity;
-import org.springframework.web.bind.annotation.*;
+import org.springframework.web.bind.annotation.RestController;
 import org.springframework.web.server.ServerWebExchange;
 import reactor.core.publisher.Flux;
 import reactor.core.publisher.Mono;
 
 @RestController
 @RequiredArgsConstructor
-@RequestMapping("/clusters")
 public class MetricsRestController implements ClustersApi {
 
     private final ClusterService clusterService;
 
-    @GetMapping("/{clusterId}/brokers")
-    public void getBrokers(@PathVariable("clusterId") String clusterId) {
-        //TODO: currently isn't displayed on UI, to do later
-    }
-
     @Override
-    @GetMapping
     public Mono<ResponseEntity<Flux<Cluster>>> getClusters(ServerWebExchange exchange) {
         return clusterService.getClusters();
     }
 
     @Override
-    @GetMapping("/{clusterId}/metrics/broker")
-    public Mono<ResponseEntity<BrokerMetrics>> getBrokersMetrics(@PathVariable String clusterId, ServerWebExchange exchange) {
+    public Mono<ResponseEntity<BrokerMetrics>> getBrokersMetrics(String clusterId, ServerWebExchange exchange) {
         return clusterService.getBrokerMetrics(clusterId);
     }
 
     @Override
-    @GetMapping("/{clusterId}/topics")
-    public Mono<ResponseEntity<Flux<Topic>>> getTopics(@PathVariable String clusterId, ServerWebExchange exchange) {
+    public Mono<ResponseEntity<Flux<Topic>>> getTopics(String clusterId, ServerWebExchange exchange) {
         return clusterService.getTopics(clusterId);
     }
 
     @Override
-    @GetMapping("/{clusterId}/topics/{topicName}")
-    public Mono<ResponseEntity<TopicDetails>> getTopicDetails(@PathVariable("clusterId") String clusterId,
-                                                              @PathVariable("topicName") String topicName, ServerWebExchange exchange) {
+    public Mono<ResponseEntity<TopicDetails>> getTopicDetails(String clusterId, String topicName, ServerWebExchange exchange) {
         return clusterService.getTopicDetails(clusterId, topicName);
     }
-
-    @PostMapping("/{clusterId}/topics")
-    public void createTopic(@PathVariable("clusterId") String clusterId) {
-
-    }
-
-    @PutMapping("/{clusterId}/topics/{topicId}")
-    public void putTopic(@PathVariable("clusterId") String clusterId,
-                         @PathVariable("topicId") String topicId) {
-    }
 }

+ 64 - 0
kafka-ui-api/src/main/java/com/provectus/kafka/ui/zookeeper/ZookeeperService.java

@@ -0,0 +1,64 @@
+package com.provectus.kafka.ui.zookeeper;
+
+import com.provectus.kafka.ui.cluster.model.KafkaCluster;
+import com.provectus.kafka.ui.model.ServerStatus;
+import lombok.RequiredArgsConstructor;
+import lombok.extern.log4j.Log4j2;
+import org.I0Itec.zkclient.ZkClient;
+import org.springframework.scheduling.annotation.Async;
+import org.springframework.stereotype.Service;
+
+import static com.provectus.kafka.ui.cluster.model.MetricsConstants.ZOOKEEPER_STATUS;
+
+@Service
+@RequiredArgsConstructor
+@Log4j2
+public class ZookeeperService {
+
+    @Async
+    public void checkZookeeperStatus(KafkaCluster kafkaCluster) {
+        log.debug("Start getting Zookeeper metrics for kafkaCluster: " + kafkaCluster.getName());
+        boolean isConnected = false;
+        if (kafkaCluster.getZkClient() != null) {
+            isConnected = isZkClientConnected(kafkaCluster);
+        }
+        if (kafkaCluster.getZkClient() == null || !isConnected) {
+            isConnected = createZookeeperConnection(kafkaCluster);
+        }
+
+        if (!isConnected) {
+            kafkaCluster.putMetric(ZOOKEEPER_STATUS, "0");
+            kafkaCluster.setZookeeperStatus(ServerStatus.OFFLINE);
+            return;
+        }
+
+        kafkaCluster.putMetric(ZOOKEEPER_STATUS, "1");
+        kafkaCluster.setZookeeperStatus(ServerStatus.ONLINE);
+    }
+
+    private boolean createZookeeperConnection(KafkaCluster kafkaCluster) {
+        try {
+            kafkaCluster.setZkClient(new ZkClient(kafkaCluster.getZookeeper(), 1000));
+
+            return true;
+        } catch (Exception e) {
+            log.error(e);
+            kafkaCluster.setLastZookeeperException(e);
+
+            return false;
+        }
+    }
+
+    private boolean isZkClientConnected(KafkaCluster kafkaCluster) {
+        try {
+            kafkaCluster.getZkClient().getChildren("/brokers/ids");
+
+            return true;
+        } catch (Exception e) {
+            log.error(e);
+            kafkaCluster.setLastZookeeperException(e);
+
+            return false;
+        }
+    }
+}

+ 3 - 3
kafka-ui-contract/src/main/resources/swagger/kafka-ui-api.yaml

@@ -109,7 +109,7 @@ components:
         defaultCluster:
           type: boolean
         status:
-          $ref: '#/components/schemas/ClusterStatus'
+          $ref: '#/components/schemas/ServerStatus'
         brokerCount:
           type: integer
         onlinePartitionCount:
@@ -125,7 +125,7 @@ components:
         - name
         - status
 
-    ClusterStatus:
+    ServerStatus:
       type: string
       enum:
         - online
@@ -140,7 +140,7 @@ components:
           type: integer
         brokerCount:
           type: integer
-        zookeeperStatus:
+        zooKeeperStatus:
           type: integer
         activeControllers:
           type: integer

+ 1 - 0
kafka-ui-react-app/.gitignore

@@ -4,6 +4,7 @@
 node_modules
 .pnp
 .pnp.js
+node
 
 # testing
 coverage