Backend for updating topics, fixed backend from another pr branches (#52)

* Backend for updating topics, fixed backend from another pr branches

* Changed caching of extendedAdminClient instances

Co-authored-by: Roman Nedzvetskiy <roman@Romans-MacBook-Pro.local>
This commit is contained in:
Roman Nedzvetskiy 2020-05-28 14:18:22 +03:00 committed by GitHub
parent 9a5ffc9eb5
commit f3f6e74b1e
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
8 changed files with 279 additions and 59 deletions

View file

@ -0,0 +1,27 @@
package com.provectus.kafka.ui.cluster.model;
import com.provectus.kafka.ui.cluster.util.ClusterUtil;
import lombok.Data;
import lombok.RequiredArgsConstructor;
import org.apache.kafka.clients.admin.AdminClient;
import reactor.core.publisher.Mono;
import java.util.Set;
@Data
@RequiredArgsConstructor
public class ExtendedAdminClient {
private final AdminClient adminClient;
private final Set<SupportedFeature> supportedFeatures;
public enum SupportedFeature {
INCREMENTAL_ALTER_CONFIGS,
ALTER_CONFIGS
}
public static Mono<ExtendedAdminClient> extendedAdminClient(AdminClient adminClient) {
return ClusterUtil.getSupportedFeatures(adminClient)
.map(s -> new ExtendedAdminClient(adminClient, s));
}
}

View file

@ -15,7 +15,6 @@ import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringDeserializer;
import org.springframework.http.ResponseEntity;
import org.springframework.stereotype.Service; import org.springframework.stereotype.Service;
import reactor.core.publisher.Flux; import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono; import reactor.core.publisher.Mono;
@ -81,7 +80,7 @@ public class ClusterService {
var cluster = clustersStorage.getClusterByName(clusterName).orElseThrow(Throwable::new); var cluster = clustersStorage.getClusterByName(clusterName).orElseThrow(Throwable::new);
return kafkaService.getOrCreateAdminClient(cluster).map(ac -> return kafkaService.getOrCreateAdminClient(cluster).map(ac ->
ac.describeConsumerGroups(Collections.singletonList(consumerGroupId)).all() ac.getAdminClient().describeConsumerGroups(Collections.singletonList(consumerGroupId)).all()
).flatMap(groups -> ).flatMap(groups ->
groupMetadata(cluster, consumerGroupId) groupMetadata(cluster, consumerGroupId)
.flatMap(offsets -> { .flatMap(offsets -> {
@ -98,7 +97,7 @@ public class ClusterService {
public Mono<Map<TopicPartition, OffsetAndMetadata>> groupMetadata(KafkaCluster cluster, String consumerGroupId) { public Mono<Map<TopicPartition, OffsetAndMetadata>> groupMetadata(KafkaCluster cluster, String consumerGroupId) {
return return
kafkaService.getOrCreateAdminClient(cluster) kafkaService.getOrCreateAdminClient(cluster)
.map(ac -> ac.listConsumerGroupOffsets(consumerGroupId).partitionsToOffsetAndMetadata()) .map(ac -> ac.getAdminClient().listConsumerGroupOffsets(consumerGroupId).partitionsToOffsetAndMetadata())
.flatMap(ClusterUtil::toMono); .flatMap(ClusterUtil::toMono);
} }
@ -119,20 +118,11 @@ public class ClusterService {
return clustersStorage.getClusterByName(clusterName) return clustersStorage.getClusterByName(clusterName)
.map(kafkaService::getConsumerGroups) .map(kafkaService::getConsumerGroups)
.orElse(Mono.empty()); .orElse(Mono.empty());
// var cluster = clustersStorage.getClusterByName(clusterName).orElseThrow(Throwable::new);
// return kafkaService.getOrCreateAdminClient(cluster).map(ac -> ac.listConsumerGroups().all())
// .flatMap(s ->
// kafkaService.getOrCreateAdminClient(cluster).flatMap(ac ->
// ClusterUtil.toMono(s).map(s1 -> s1.stream().map(ConsumerGroupListing::groupId).collect(Collectors.toList())).map(ac::describeConsumerGroups)
// ))
// .flatMap(s -> ClusterUtil.toMono(s.all()).map(details -> details.values().stream()
// .map(c -> ClusterUtil.convertToConsumerGroup(c, cluster)).collect(Collectors.toList())));
} }
public Flux<Broker> getBrokers (String clusterName) { public Flux<Broker> getBrokers (String clusterName) {
return kafkaService.getOrCreateAdminClient(clustersStorage.getClusterByName(clusterName).orElseThrow()) return kafkaService.getOrCreateAdminClient(clustersStorage.getClusterByName(clusterName).orElseThrow())
.flatMap(client -> ClusterUtil.toMono(client.describeCluster().nodes()) .flatMap(client -> ClusterUtil.toMono(client.getAdminClient().describeCluster().nodes())
.map(n -> n.stream().map(node -> { .map(n -> n.stream().map(node -> {
Broker broker = new Broker(); Broker broker = new Broker();
broker.setId(node.idString()); broker.setId(node.idString());
@ -141,6 +131,24 @@ public class ClusterService {
.flatMapMany(Flux::fromIterable); .flatMapMany(Flux::fromIterable);
} }
@SneakyThrows
public Mono<Topic> updateTopic(String clusterName, String topicName, Mono<TopicFormData> topicFormData) {
return clustersStorage.getClusterByName(clusterName).map(cl ->
topicFormData
.flatMap(t -> kafkaService.updateTopic(cl, topicName, t))
.flatMap(t -> updateCluster(t, clusterName, cl))
)
.orElse(Mono.empty());
}
private <T> Mono<T> updateCluster(T topic, String clusterName, KafkaCluster cluster) {
return kafkaService.getUpdatedCluster(cluster)
.map(c -> {
clustersStorage.setKafkaCluster(clusterName, c);
return topic;
});
}
public Flux<TopicMessage> getMessages(String clusterName, String topicName, Integer partition, Long offset, OffsetDateTime timestamp) { public Flux<TopicMessage> getMessages(String clusterName, String topicName, Integer partition, Long offset, OffsetDateTime timestamp) {
return clustersStorage.getClusterByName(clusterName) return clustersStorage.getClusterByName(clusterName)
.map(c -> consumingService.loadMessages(c, topicName)) .map(c -> consumingService.loadMessages(c, topicName))

View file

@ -1,19 +1,17 @@
package com.provectus.kafka.ui.cluster.util; package com.provectus.kafka.ui.cluster.util;
import com.provectus.kafka.ui.cluster.model.*; import com.provectus.kafka.ui.cluster.model.*;
import com.provectus.kafka.ui.model.ConsumerGroup; import com.provectus.kafka.ui.model.*;
import com.provectus.kafka.ui.model.ConsumerTopicPartitionDetail; import lombok.extern.slf4j.Slf4j;
import com.provectus.kafka.ui.model.ServerStatus;
import com.provectus.kafka.ui.model.TopicMessage; import com.provectus.kafka.ui.model.TopicMessage;
import org.apache.kafka.clients.admin.ConfigEntry; import org.apache.kafka.clients.admin.*;
import org.apache.kafka.clients.admin.ConsumerGroupDescription;
import org.apache.kafka.clients.admin.MemberDescription;
import org.apache.kafka.clients.admin.TopicDescription;
import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.config.ConfigResource;
import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Bytes;
@ -23,16 +21,18 @@ import java.time.Instant;
import java.time.OffsetDateTime; import java.time.OffsetDateTime;
import java.time.ZoneId; import java.time.ZoneId;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.*;
import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.Stream; import java.util.stream.Stream;
import static com.provectus.kafka.ui.kafka.KafkaConstants.TOPIC_DEFAULT_CONFIGS; import static com.provectus.kafka.ui.kafka.KafkaConstants.TOPIC_DEFAULT_CONFIGS;
import static org.apache.kafka.common.config.TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG; import static org.apache.kafka.common.config.TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG;
@Slf4j
public class ClusterUtil { public class ClusterUtil {
private static final String CLUSTER_VERSION_PARAM_KEY = "inter.broker.protocol.version";
private static final ZoneId UTC_ZONE_ID = ZoneId.of("UTC"); private static final ZoneId UTC_ZONE_ID = ZoneId.of("UTC");
public static <T> Mono<T> toMono(KafkaFuture<T> future){ public static <T> Mono<T> toMono(KafkaFuture<T> future){
@ -45,6 +45,16 @@ public class ClusterUtil {
})); }));
} }
public static Mono<String> toMono(KafkaFuture<Void> future, String topicName){
return Mono.create(sink -> future.whenComplete((res, ex)->{
if (ex!=null) {
sink.error(ex);
} else {
sink.success(topicName);
}
}));
}
public static ConsumerGroup convertToConsumerGroup(ConsumerGroupDescription c, KafkaCluster cluster) { public static ConsumerGroup convertToConsumerGroup(ConsumerGroupDescription c, KafkaCluster cluster) {
ConsumerGroup consumerGroup = new ConsumerGroup(); ConsumerGroup consumerGroup = new ConsumerGroup();
consumerGroup.setConsumerGroupId(c.groupId()); consumerGroup.setConsumerGroupId(c.groupId());
@ -173,6 +183,49 @@ public class ClusterUtil {
throw new IllegalArgumentException("Unknown timestampType: " + timestampType); throw new IllegalArgumentException("Unknown timestampType: " + timestampType);
} }
} }
public static Mono<Set<ExtendedAdminClient.SupportedFeature>> getSupportedFeatures(AdminClient adminClient) {
return ClusterUtil.toMono(adminClient.describeCluster().controller())
.map(Node::id)
.map(id -> Collections.singletonList(new ConfigResource(ConfigResource.Type.BROKER, id.toString())))
.map(brokerCR -> adminClient.describeConfigs(brokerCR).all())
.flatMap(ClusterUtil::toMono)
.map(ClusterUtil::getSupportedUpdateFeature)
.map(Collections::singleton);
}
private static ExtendedAdminClient.SupportedFeature getSupportedUpdateFeature(Map<ConfigResource, Config> configs) {
String version = configs.values().stream()
.map(Config::entries)
.flatMap(Collection::stream)
.filter(entry -> entry.name().contains(CLUSTER_VERSION_PARAM_KEY))
.findFirst().orElseThrow().value();
try {
return Float.parseFloat(version.split("-")[0]) <= 2.3f
? ExtendedAdminClient.SupportedFeature.ALTER_CONFIGS : ExtendedAdminClient.SupportedFeature.INCREMENTAL_ALTER_CONFIGS;
} catch (Exception e) {
log.error("Conversion clusterVersion {} to float value failed", version);
throw e;
}
}
public static Topic convertToTopic (InternalTopic internalTopic) {
Topic topic = new Topic();
topic.setName(internalTopic.getName());
List<Partition> partitions = internalTopic.getPartitions().stream().flatMap(s -> {
Partition partition = new Partition();
partition.setPartition(s.getPartition());
partition.setLeader(s.getLeader());
partition.setReplicas(s.getReplicas().stream().flatMap(r -> {
Replica replica = new Replica();
replica.setBroker(r.getBroker());
return Stream.of(replica);
}).collect(Collectors.toList()));
return Stream.of(partition);
}).collect(Collectors.toList());
topic.setPartitions(partitions);
return topic;
}
public static <T, R> Map<T, R> toSingleMap (Stream<Map<T, R>> streamOfMaps) { public static <T, R> Map<T, R> toSingleMap (Stream<Map<T, R>> streamOfMaps) {
return streamOfMaps.reduce((map1, map2) -> Stream.concat(map1.entrySet().stream(), map2.entrySet().stream()) return streamOfMaps.reduce((map1, map2) -> Stream.concat(map1.entrySet().stream(), map2.entrySet().stream())
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))).orElseThrow(); .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))).orElseThrow();

View file

@ -4,6 +4,7 @@ import com.provectus.kafka.ui.cluster.model.*;
import com.provectus.kafka.ui.cluster.util.ClusterUtil; import com.provectus.kafka.ui.cluster.util.ClusterUtil;
import com.provectus.kafka.ui.model.ConsumerGroup; import com.provectus.kafka.ui.model.ConsumerGroup;
import com.provectus.kafka.ui.model.ServerStatus; import com.provectus.kafka.ui.model.ServerStatus;
import com.provectus.kafka.ui.model.Topic;
import com.provectus.kafka.ui.model.TopicFormData; import com.provectus.kafka.ui.model.TopicFormData;
import com.provectus.kafka.ui.zookeeper.ZookeeperService; import com.provectus.kafka.ui.zookeeper.ZookeeperService;
import lombok.RequiredArgsConstructor; import lombok.RequiredArgsConstructor;
@ -16,6 +17,7 @@ import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.Node; import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.config.ConfigResource;
import org.springframework.beans.factory.annotation.Value;
import org.apache.kafka.common.serialization.BytesDeserializer; import org.apache.kafka.common.serialization.BytesDeserializer;
import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Bytes;
import org.springframework.stereotype.Service; import org.springframework.stereotype.Service;
@ -33,20 +35,23 @@ import java.util.stream.Stream;
@Log4j2 @Log4j2
public class KafkaService { public class KafkaService {
@Value("${kafka.admin-client-timeout}")
private int clientTimeout;
private static final ListTopicsOptions LIST_TOPICS_OPTIONS = new ListTopicsOptions().listInternal(true); private static final ListTopicsOptions LIST_TOPICS_OPTIONS = new ListTopicsOptions().listInternal(true);
private final ZookeeperService zookeeperService; private final ZookeeperService zookeeperService;
private final Map<String, AdminClient> adminClientCache = new ConcurrentHashMap<>(); private final Map<String, ExtendedAdminClient> adminClientCache = new ConcurrentHashMap<>();
private final Map<AdminClient, Map<TopicPartition, Integer>> leadersCache = new ConcurrentHashMap<>(); private final Map<AdminClient, Map<TopicPartition, Integer>> leadersCache = new ConcurrentHashMap<>();
@SneakyThrows @SneakyThrows
public Mono<KafkaCluster> getUpdatedCluster(KafkaCluster cluster) { public Mono<KafkaCluster> getUpdatedCluster(KafkaCluster cluster) {
return getOrCreateAdminClient(cluster).flatMap( return getOrCreateAdminClient(cluster).flatMap(
ac -> getClusterMetrics(ac) ac -> getClusterMetrics(ac.getAdminClient())
.flatMap( clusterMetrics -> .flatMap( clusterMetrics ->
getTopicsData(ac).flatMap( topics -> getTopicsData(ac.getAdminClient()).flatMap( topics ->
loadTopicsConfig(ac, topics.stream().map(InternalTopic::getName).collect(Collectors.toList())) loadTopicsConfig(ac.getAdminClient(), topics.stream().map(InternalTopic::getName).collect(Collectors.toList()))
.map( configs -> mergeWithConfigs(topics, configs)) .map( configs -> mergeWithConfigs(topics, configs))
.flatMap(it -> updateSegmentMetrics(ac, clusterMetrics, it)) .flatMap(it -> updateSegmentMetrics(ac, clusterMetrics, it))
).map( segmentSizeDto -> buildFromData(cluster, segmentSizeDto)) ).map( segmentSizeDto -> buildFromData(cluster, segmentSizeDto))
@ -170,8 +175,7 @@ public class KafkaService {
public Mono<InternalTopic> createTopic(KafkaCluster cluster, Mono<TopicFormData> topicFormData) { public Mono<InternalTopic> createTopic(KafkaCluster cluster, Mono<TopicFormData> topicFormData) {
AdminClient adminClient = this.createAdminClient(cluster); return getOrCreateAdminClient(cluster).flatMap(ac -> createTopic(ac.getAdminClient(), topicFormData));
return this.createTopic(adminClient, topicFormData);
} }
@SneakyThrows @SneakyThrows
@ -200,24 +204,18 @@ public class KafkaService {
} }
public Mono<AdminClient> getOrCreateAdminClient(KafkaCluster cluster) { public Mono<ExtendedAdminClient> getOrCreateAdminClient(KafkaCluster cluster) {
AdminClient adminClient = adminClientCache.computeIfAbsent( return Mono.justOrEmpty(adminClientCache.get(cluster.getName()))
cluster.getName(), .switchIfEmpty(createAdminClient(cluster))
(id) -> createAdminClient(cluster) .map(e -> adminClientCache.computeIfAbsent(cluster.getName(), key -> e));
);
return isAdminClientConnected(adminClient);
} }
public AdminClient createAdminClient(KafkaCluster kafkaCluster) { public Mono<ExtendedAdminClient> createAdminClient(KafkaCluster kafkaCluster) {
Properties properties = new Properties(); Properties properties = new Properties();
properties.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaCluster.getBootstrapServers()); properties.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaCluster.getBootstrapServers());
properties.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, 5000); properties.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, clientTimeout);
return AdminClient.create(properties); AdminClient adminClient = AdminClient.create(properties);
} return ExtendedAdminClient.extendedAdminClient(adminClient);
private Mono<AdminClient> isAdminClientConnected(AdminClient adminClient) {
return getClusterId(adminClient).map( r -> adminClient);
} }
@ -251,12 +249,11 @@ public class KafkaService {
} }
public Mono<List<ConsumerGroup>> getConsumerGroups(KafkaCluster cluster) { public Mono<List<ConsumerGroup>> getConsumerGroups(KafkaCluster cluster) {
var adminClient = this.createAdminClient(cluster); return getOrCreateAdminClient(cluster).flatMap(ac -> ClusterUtil.toMono(ac.getAdminClient().listConsumerGroups().all())
return ClusterUtil.toMono(adminClient.listConsumerGroups().all()) .flatMap(s -> ClusterUtil.toMono(ac.getAdminClient()
.flatMap(s -> ClusterUtil.toMono(adminClient
.describeConsumerGroups(s.stream().map(ConsumerGroupListing::groupId).collect(Collectors.toList())).all())) .describeConsumerGroups(s.stream().map(ConsumerGroupListing::groupId).collect(Collectors.toList())).all()))
.map(s -> s.values().stream() .map(s -> s.values().stream()
.map(c -> ClusterUtil.convertToConsumerGroup(c, cluster)).collect(Collectors.toList())); .map(c -> ClusterUtil.convertToConsumerGroup(c, cluster)).collect(Collectors.toList())));
} }
public KafkaConsumer<Bytes, Bytes> createConsumer(KafkaCluster cluster) { public KafkaConsumer<Bytes, Bytes> createConsumer(KafkaCluster cluster) {
@ -271,12 +268,47 @@ public class KafkaService {
@SneakyThrows @SneakyThrows
private Mono<Void> createTopic(AdminClient adminClient, NewTopic newTopic) { private Mono<String> createTopic(AdminClient adminClient, NewTopic newTopic) {
return ClusterUtil.toMono(adminClient.createTopics(Collections.singletonList(newTopic)) return ClusterUtil.toMono(adminClient.createTopics(Collections.singletonList(newTopic)).all(), newTopic.name());
.values() }
.values()
.iterator() @SneakyThrows
.next()); public Mono<Topic> updateTopic(KafkaCluster cluster, String topicName, TopicFormData topicFormData) {
ConfigResource topicCR = new ConfigResource(ConfigResource.Type.TOPIC, topicName);
return getOrCreateAdminClient(cluster)
.flatMap(ac -> {
if (ac.getSupportedFeatures().contains(ExtendedAdminClient.SupportedFeature.INCREMENTAL_ALTER_CONFIGS)) {
return incrementalAlterConfig(topicFormData, topicCR, ac)
.flatMap(c -> getUpdatedTopic(ac, topicName));
} else {
return alterConfig(topicFormData, topicCR, ac)
.flatMap(c -> getUpdatedTopic(ac, topicName));
}
});
}
private Mono<Topic> getUpdatedTopic (ExtendedAdminClient ac, String topicName) {
return getTopicsData(ac.getAdminClient())
.map(s -> s.stream()
.filter(t -> t.getName().equals(topicName)).findFirst().orElseThrow())
.map(ClusterUtil::convertToTopic);
}
private Mono<String> incrementalAlterConfig(TopicFormData topicFormData, ConfigResource topicCR, ExtendedAdminClient ac) {
List<AlterConfigOp> listOp = topicFormData.getConfigs().entrySet().stream()
.flatMap(cfg -> Stream.of(new AlterConfigOp(new ConfigEntry(cfg.getKey(), cfg.getValue()), AlterConfigOp.OpType.SET))).collect(Collectors.toList());
return ClusterUtil.toMono(ac.getAdminClient().incrementalAlterConfigs(Collections.singletonMap(topicCR, listOp)).all(), topicCR.name());
}
private Mono<String> alterConfig(TopicFormData topicFormData, ConfigResource topicCR, ExtendedAdminClient ac) {
List<ConfigEntry> configEntries = topicFormData.getConfigs().entrySet().stream()
.flatMap(cfg -> Stream.of(new ConfigEntry(cfg.getKey(), cfg.getValue()))).collect(Collectors.toList());
Config config = new Config(configEntries);
Map<ConfigResource, Config> map = Collections.singletonMap(topicCR, config);
return ClusterUtil.toMono(ac.getAdminClient().alterConfigs(map).all(), topicCR.name());
} }
private Mono<InternalSegmentSizeDto> updateSegmentMetrics(AdminClient ac, InternalClusterMetrics clusterMetrics, Map<String, InternalTopic> internalTopic) { private Mono<InternalSegmentSizeDto> updateSegmentMetrics(AdminClient ac, InternalClusterMetrics clusterMetrics, Map<String, InternalTopic> internalTopic) {

View file

@ -11,9 +11,8 @@ import org.springframework.web.server.ServerWebExchange;
import reactor.core.publisher.Flux; import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono; import reactor.core.publisher.Mono;
import java.time.OffsetDateTime;
import javax.validation.Valid; import javax.validation.Valid;
import java.time.OffsetDateTime;
@RestController @RestController
@RequiredArgsConstructor @RequiredArgsConstructor
@ -90,4 +89,9 @@ public class MetricsRestController implements ApiClustersApi {
public Mono<ResponseEntity<ConsumerGroupDetails>> getConsumerGroup(String clusterName, String consumerGroupId, ServerWebExchange exchange) { public Mono<ResponseEntity<ConsumerGroupDetails>> getConsumerGroup(String clusterName, String consumerGroupId, ServerWebExchange exchange) {
return clusterService.getConsumerGroupDetail(clusterName, consumerGroupId).map(ResponseEntity::ok); return clusterService.getConsumerGroupDetail(clusterName, consumerGroupId).map(ResponseEntity::ok);
} }
@Override
public Mono<ResponseEntity<Topic>> updateTopic(String clusterId, String topicName, @Valid Mono<TopicFormData> topicFormData, ServerWebExchange exchange) {
return clusterService.updateTopic(clusterId, topicName, topicFormData).map(ResponseEntity::ok);
}
} }

View file

@ -12,3 +12,6 @@ kafka:
name: localReplica name: localReplica
bootstrapServers: localhost:29093 bootstrapServers: localhost:29093
zookeeper: localhost:2183 zookeeper: localhost:2183
admin-client-timeout: 5000
zookeeper:
connection-timeout: 1000

View file

@ -8,3 +8,6 @@ kafka:
name: secondLocal name: secondLocal
zookeeper: zookeeper1:2181 zookeeper: zookeeper1:2181
bootstrapServers: kafka1:29092 bootstrapServers: kafka1:29092
admin-client-timeout: 5000
zookeeper:
connection-timeout: 1000

View file

@ -141,6 +141,34 @@ paths:
application/json: application/json:
schema: schema:
$ref: '#/components/schemas/TopicDetails' $ref: '#/components/schemas/TopicDetails'
patch:
tags:
- /api/clusters
summary: updateTopic
operationId: updateTopic
parameters:
- name: clusterName
in: path
required: true
schema:
type: string
- name: topicName
in: path
required: true
schema:
type: string
requestBody:
content:
application/json:
schema:
$ref: '#/components/schemas/TopicFormData'
responses:
200:
description: Updated
content:
application/json:
schema:
$ref: '#/components/schemas/Topic'
/api/clusters/{clusterName}/topics/{topicName}/config: /api/clusters/{clusterName}/topics/{topicName}/config:
get: get:
@ -210,12 +238,37 @@ paths:
items: items:
$ref: '#/components/schemas/TopicMessage' $ref: '#/components/schemas/TopicMessage'
/api/clusters/{clusterName}/consumer-groups/{id}:
get:
tags:
- /api/clusters
summary: get Consumer Group By Id
operationId: getConsumerGroup
parameters:
- name: clusterName
in: path
required: true
schema:
type: string
- name: id
in: path
required: true
schema:
type: string
responses:
200:
description: OK
content:
application/json:
schema:
$ref: '#/components/schemas/ConsumerGroupDetails'
/api/clusters/{clusterName}/consumerGroups: /api/clusters/{clusterName}/consumerGroups:
get: get:
tags: tags:
- /api/clusters - /api/clusters
summary: getConsumerGroup summary: get all ConsumerGroups
operationId: getConsumerGroup operationId: getConsumerGroups
parameters: parameters:
- name: clusterName - name: clusterName
in: path in: path
@ -409,3 +462,40 @@ components:
- partition - partition
- offset - offset
- timestamp - timestamp
TopicPartitionDto:
type: object
properties:
topic:
type: string
partition:
type: integer
required:
- topic
- partition
ConsumerTopicPartitionDetail:
type: object
properties:
consumerId:
type: string
topic:
type: string
partition:
type: integer
currentOffset:
type: long
endOffset:
type: long
messagesBehind:
type: long
ConsumerGroupDetails:
type: object
properties:
consumerGroupId:
type: string
consumers:
type: array
items:
$ref: '#/components/schemas/ConsumerTopicPartitionDetail'