forked from kafbat/kafka-ui
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathTopicsService.java
475 lines (437 loc) · 19.9 KB
/
TopicsService.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
package io.kafbat.ui.service;
import static java.util.stream.Collectors.toList;
import static java.util.stream.Collectors.toMap;
import com.google.common.collect.Sets;
import io.kafbat.ui.config.ClustersProperties;
import io.kafbat.ui.exception.TopicMetadataException;
import io.kafbat.ui.exception.TopicNotFoundException;
import io.kafbat.ui.exception.TopicRecreationException;
import io.kafbat.ui.exception.ValidationException;
import io.kafbat.ui.model.ClusterFeature;
import io.kafbat.ui.model.InternalLogDirStats;
import io.kafbat.ui.model.InternalPartition;
import io.kafbat.ui.model.InternalPartitionsOffsets;
import io.kafbat.ui.model.InternalReplica;
import io.kafbat.ui.model.InternalTopic;
import io.kafbat.ui.model.InternalTopicConfig;
import io.kafbat.ui.model.KafkaCluster;
import io.kafbat.ui.model.Metrics;
import io.kafbat.ui.model.PartitionsIncreaseDTO;
import io.kafbat.ui.model.PartitionsIncreaseResponseDTO;
import io.kafbat.ui.model.ReplicationFactorChangeDTO;
import io.kafbat.ui.model.ReplicationFactorChangeResponseDTO;
import io.kafbat.ui.model.Statistics;
import io.kafbat.ui.model.TopicCreationDTO;
import io.kafbat.ui.model.TopicUpdateDTO;
import java.time.Duration;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Collectors;
import lombok.RequiredArgsConstructor;
import org.apache.kafka.clients.admin.ConfigEntry;
import org.apache.kafka.clients.admin.NewPartitionReassignment;
import org.apache.kafka.clients.admin.NewPartitions;
import org.apache.kafka.clients.admin.OffsetSpec;
import org.apache.kafka.clients.admin.ProducerState;
import org.apache.kafka.clients.admin.TopicDescription;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.TopicExistsException;
import org.springframework.beans.factory.annotation.Value;
import org.springframework.stereotype.Service;
import reactor.core.publisher.Mono;
import reactor.util.retry.Retry;
@Service
@RequiredArgsConstructor
public class TopicsService {
private final AdminClientService adminClientService;
private final StatisticsCache statisticsCache;
private final ClustersProperties clustersProperties;
@Value("${topic.recreate.maxRetries:15}")
private int recreateMaxRetries;
@Value("${topic.recreate.delay.seconds:1}")
private int recreateDelayInSeconds;
@Value("${topic.load.after.create.maxRetries:10}")
private int loadTopicAfterCreateRetries;
@Value("${topic.load.after.create.delay.ms:500}")
private int loadTopicAfterCreateDelayInMs;
public Mono<List<InternalTopic>> loadTopics(KafkaCluster c, List<String> topics) {
if (topics.isEmpty()) {
return Mono.just(List.of());
}
return adminClientService.get(c)
.flatMap(ac ->
ac.describeTopics(topics).zipWith(ac.getTopicsConfig(topics, false),
(descriptions, configs) -> {
statisticsCache.update(c, descriptions, configs);
return getPartitionOffsets(descriptions, ac).map(offsets -> {
var metrics = statisticsCache.get(c);
return createList(
topics,
descriptions,
configs,
offsets,
metrics.getMetrics(),
metrics.getLogDirInfo()
);
});
})).flatMap(Function.identity());
}
private Mono<InternalTopic> loadTopic(KafkaCluster c, String topicName) {
return loadTopics(c, List.of(topicName))
.flatMap(lst -> lst.stream().findFirst()
.map(Mono::just)
.orElse(Mono.error(TopicNotFoundException::new)));
}
/**
* After creation topic can be invisible via API for some time.
* To workaround this, we're retrying topic loading until it becomes visible.
*/
private Mono<InternalTopic> loadTopicAfterCreation(KafkaCluster c, String topicName) {
return loadTopic(c, topicName)
.retryWhen(
Retry
.fixedDelay(
loadTopicAfterCreateRetries,
Duration.ofMillis(loadTopicAfterCreateDelayInMs)
)
.filter(TopicNotFoundException.class::isInstance)
.onRetryExhaustedThrow((spec, sig) ->
new TopicMetadataException(
String.format(
"Error while loading created topic '%s' - topic is not visible via API "
+ "after waiting for %d ms.",
topicName,
loadTopicAfterCreateDelayInMs * loadTopicAfterCreateRetries)))
);
}
private List<InternalTopic> createList(List<String> orderedNames,
Map<String, TopicDescription> descriptions,
Map<String, List<ConfigEntry>> configs,
InternalPartitionsOffsets partitionsOffsets,
Metrics metrics,
InternalLogDirStats logDirInfo) {
return orderedNames.stream()
.filter(descriptions::containsKey)
.map(t -> InternalTopic.from(
descriptions.get(t),
configs.getOrDefault(t, List.of()),
partitionsOffsets,
metrics,
logDirInfo,
clustersProperties.getInternalTopicPrefix()
))
.collect(toList());
}
private Mono<InternalPartitionsOffsets> getPartitionOffsets(Map<String, TopicDescription> descriptionsMap,
ReactiveAdminClient ac) {
var descriptions = descriptionsMap.values();
return ac.listOffsets(descriptions, OffsetSpec.earliest())
.zipWith(ac.listOffsets(descriptions, OffsetSpec.latest()),
(earliest, latest) ->
Sets.intersection(earliest.keySet(), latest.keySet())
.stream()
.map(tp ->
Map.entry(tp,
new InternalPartitionsOffsets.Offsets(
earliest.get(tp), latest.get(tp))))
.collect(toMap(Map.Entry::getKey, Map.Entry::getValue)))
.map(InternalPartitionsOffsets::new);
}
public Mono<InternalTopic> getTopicDetails(KafkaCluster cluster, String topicName) {
return loadTopic(cluster, topicName);
}
public Mono<List<ConfigEntry>> getTopicConfigs(KafkaCluster cluster, String topicName) {
// there 2 case that we cover here:
// 1. topic not found/visible - describeTopic() will be empty and we will throw TopicNotFoundException
// 2. topic is visible, but we don't have DESCRIBE_CONFIG permission - we should return empty list
return adminClientService.get(cluster)
.flatMap(ac -> ac.describeTopic(topicName)
.switchIfEmpty(Mono.error(new TopicNotFoundException()))
.then(ac.getTopicsConfig(List.of(topicName), true))
.map(m -> m.values().stream().findFirst().orElse(List.of())));
}
private Mono<InternalTopic> createTopic(KafkaCluster c, ReactiveAdminClient adminClient, TopicCreationDTO topicData) {
return adminClient.createTopic(
topicData.getName(),
topicData.getPartitions(),
topicData.getReplicationFactor(),
topicData.getConfigs())
.thenReturn(topicData)
.onErrorMap(t -> new TopicMetadataException(t.getMessage(), t))
.then(loadTopicAfterCreation(c, topicData.getName()));
}
public Mono<InternalTopic> createTopic(KafkaCluster cluster, TopicCreationDTO topicCreation) {
return adminClientService.get(cluster)
.flatMap(ac -> createTopic(cluster, ac, topicCreation));
}
public Mono<InternalTopic> recreateTopic(KafkaCluster cluster, String topicName) {
return loadTopic(cluster, topicName)
.flatMap(t -> deleteTopic(cluster, topicName)
.thenReturn(t)
.delayElement(Duration.ofSeconds(recreateDelayInSeconds))
.flatMap(topic ->
adminClientService.get(cluster)
.flatMap(ac ->
ac.createTopic(
topic.getName(),
topic.getPartitionCount(),
topic.getReplicationFactor(),
topic.getTopicConfigs()
.stream()
.collect(Collectors.toMap(InternalTopicConfig::getName,
InternalTopicConfig::getValue))
)
.thenReturn(topicName)
)
.retryWhen(
Retry.fixedDelay(recreateMaxRetries, Duration.ofSeconds(recreateDelayInSeconds))
.filter(TopicExistsException.class::isInstance)
.onRetryExhaustedThrow((a, b) ->
new TopicRecreationException(topicName,
recreateMaxRetries * recreateDelayInSeconds))
)
.flatMap(a -> loadTopicAfterCreation(cluster, topicName))
)
);
}
private Mono<InternalTopic> updateTopic(KafkaCluster cluster,
String topicName,
TopicUpdateDTO topicUpdate) {
return adminClientService.get(cluster)
.flatMap(ac ->
ac.updateTopicConfig(topicName, topicUpdate.getConfigs())
.then(loadTopic(cluster, topicName)));
}
public Mono<InternalTopic> updateTopic(KafkaCluster cl, String topicName, Mono<TopicUpdateDTO> topicUpdate) {
return topicUpdate
.flatMap(t -> updateTopic(cl, topicName, t));
}
private Mono<InternalTopic> changeReplicationFactor(
KafkaCluster cluster,
ReactiveAdminClient adminClient,
String topicName,
Map<TopicPartition, Optional<NewPartitionReassignment>> reassignments
) {
return adminClient.alterPartitionReassignments(reassignments)
.then(loadTopic(cluster, topicName));
}
/**
* Change topic replication factor, works on brokers versions 5.4.x and higher
*/
public Mono<ReplicationFactorChangeResponseDTO> changeReplicationFactor(
KafkaCluster cluster,
String topicName,
ReplicationFactorChangeDTO replicationFactorChange) {
return loadTopic(cluster, topicName).flatMap(topic -> adminClientService.get(cluster)
.flatMap(ac -> {
Integer actual = topic.getReplicationFactor();
Integer requested = replicationFactorChange.getTotalReplicationFactor();
Integer brokersCount = statisticsCache.get(cluster).getClusterDescription()
.getNodes().size();
if (requested.equals(actual)) {
return Mono.error(
new ValidationException(
String.format("Topic already has replicationFactor %s.", actual)));
}
if (requested <= 0) {
return Mono.error(
new ValidationException(
String.format("Requested replication factor (%s) should be greater or equal to 1.", requested)));
}
if (requested > brokersCount) {
return Mono.error(
new ValidationException(
String.format("Requested replication factor %s more than brokers count %s.",
requested, brokersCount)));
}
return changeReplicationFactor(cluster, ac, topicName,
getPartitionsReassignments(cluster, topic,
replicationFactorChange));
})
.map(t -> new ReplicationFactorChangeResponseDTO()
.topicName(t.getName())
.totalReplicationFactor(t.getReplicationFactor())));
}
private Map<TopicPartition, Optional<NewPartitionReassignment>> getPartitionsReassignments(
KafkaCluster cluster,
InternalTopic topic,
ReplicationFactorChangeDTO replicationFactorChange) {
// Current assignment map (Partition number -> List of brokers)
Map<Integer, List<Integer>> currentAssignment = getCurrentAssignment(topic);
// Brokers map (Broker id -> count)
Map<Integer, Integer> brokersUsage = getBrokersMap(cluster, currentAssignment);
int currentReplicationFactor = topic.getReplicationFactor();
// If we should to increase Replication factor
if (replicationFactorChange.getTotalReplicationFactor() > currentReplicationFactor) {
// For each partition
for (var assignmentList : currentAssignment.values()) {
// Get brokers list sorted by usage
var brokers = brokersUsage.entrySet().stream()
.sorted(Map.Entry.comparingByValue())
.map(Map.Entry::getKey)
.toList();
// Iterate brokers and try to add them in assignment
// while partition replicas count != requested replication factor
for (Integer broker : brokers) {
if (!assignmentList.contains(broker)) {
assignmentList.add(broker);
brokersUsage.merge(broker, 1, Integer::sum);
}
if (assignmentList.size() == replicationFactorChange.getTotalReplicationFactor()) {
break;
}
}
if (assignmentList.size() != replicationFactorChange.getTotalReplicationFactor()) {
throw new ValidationException("Something went wrong during adding replicas");
}
}
// If we should to decrease Replication factor
} else if (replicationFactorChange.getTotalReplicationFactor() < currentReplicationFactor) {
for (Map.Entry<Integer, List<Integer>> assignmentEntry : currentAssignment.entrySet()) {
var partition = assignmentEntry.getKey();
var brokers = assignmentEntry.getValue();
// Get brokers list sorted by usage in reverse order
var brokersUsageList = brokersUsage.entrySet().stream()
.sorted(Map.Entry.comparingByValue(Comparator.reverseOrder()))
.map(Map.Entry::getKey)
.toList();
// Iterate brokers and try to remove them from assignment
// while partition replicas count != requested replication factor
for (Integer broker : brokersUsageList) {
// Check is the broker the leader of partition
if (!topic.getPartitions().get(partition).getLeader()
.equals(broker)) {
brokers.remove(broker);
brokersUsage.merge(broker, -1, Integer::sum);
}
if (brokers.size() == replicationFactorChange.getTotalReplicationFactor()) {
break;
}
}
if (brokers.size() != replicationFactorChange.getTotalReplicationFactor()) {
throw new ValidationException("Something went wrong during removing replicas");
}
}
} else {
throw new ValidationException("Replication factor already equals requested");
}
// Return result map
return currentAssignment.entrySet().stream().collect(toMap(
e -> new TopicPartition(topic.getName(), e.getKey()),
e -> Optional.of(new NewPartitionReassignment(e.getValue()))
));
}
private Map<Integer, List<Integer>> getCurrentAssignment(InternalTopic topic) {
return topic.getPartitions().values().stream()
.collect(toMap(
InternalPartition::getPartition,
p -> p.getReplicas().stream()
.map(InternalReplica::getBroker)
.collect(toList())
));
}
private Map<Integer, Integer> getBrokersMap(KafkaCluster cluster,
Map<Integer, List<Integer>> currentAssignment) {
Map<Integer, Integer> result = statisticsCache.get(cluster).getClusterDescription().getNodes()
.stream()
.map(Node::id)
.collect(toMap(
c -> c,
c -> 0
));
currentAssignment.values().forEach(brokers -> brokers
.forEach(broker -> result.put(broker, result.get(broker) + 1)));
return result;
}
public Mono<PartitionsIncreaseResponseDTO> increaseTopicPartitions(
KafkaCluster cluster,
String topicName,
PartitionsIncreaseDTO partitionsIncrease) {
return loadTopic(cluster, topicName).flatMap(topic ->
adminClientService.get(cluster).flatMap(ac -> {
Integer actualCount = topic.getPartitionCount();
Integer requestedCount = partitionsIncrease.getTotalPartitionsCount();
if (requestedCount < actualCount) {
return Mono.error(
new ValidationException(String.format(
"Topic currently has %s partitions, which is higher than the requested %s.",
actualCount, requestedCount)));
}
if (requestedCount.equals(actualCount)) {
return Mono.error(
new ValidationException(
String.format("Topic already has %s partitions.", actualCount)));
}
Map<String, NewPartitions> newPartitionsMap = Collections.singletonMap(
topicName,
NewPartitions.increaseTo(partitionsIncrease.getTotalPartitionsCount())
);
return ac.createPartitions(newPartitionsMap)
.then(loadTopic(cluster, topicName));
}).map(t -> new PartitionsIncreaseResponseDTO()
.topicName(t.getName())
.totalPartitionsCount(t.getPartitionCount())
)
);
}
public Mono<Void> deleteTopic(KafkaCluster cluster, String topicName) {
if (statisticsCache.get(cluster).getFeatures().contains(ClusterFeature.TOPIC_DELETION)) {
return adminClientService.get(cluster).flatMap(c -> c.deleteTopic(topicName))
.doOnSuccess(t -> statisticsCache.onTopicDelete(cluster, topicName));
} else {
return Mono.error(new ValidationException("Topic deletion restricted"));
}
}
public Mono<InternalTopic> cloneTopic(
KafkaCluster cluster, String topicName, String newTopicName) {
return loadTopic(cluster, topicName).flatMap(topic ->
adminClientService.get(cluster)
.flatMap(ac ->
ac.createTopic(
newTopicName,
topic.getPartitionCount(),
topic.getReplicationFactor(),
topic.getTopicConfigs()
.stream()
.collect(Collectors
.toMap(InternalTopicConfig::getName, InternalTopicConfig::getValue))
)
).thenReturn(newTopicName)
.flatMap(a -> loadTopicAfterCreation(cluster, newTopicName))
);
}
public Mono<List<InternalTopic>> getTopicsForPagination(KafkaCluster cluster) {
Statistics stats = statisticsCache.get(cluster);
return filterExisting(cluster, stats.getTopicDescriptions().keySet())
.map(lst -> lst.stream()
.map(topicName ->
InternalTopic.from(
stats.getTopicDescriptions().get(topicName),
stats.getTopicConfigs().getOrDefault(topicName, List.of()),
InternalPartitionsOffsets.empty(),
stats.getMetrics(),
stats.getLogDirInfo(),
clustersProperties.getInternalTopicPrefix()
))
.collect(toList())
);
}
public Mono<Map<TopicPartition, List<ProducerState>>> getActiveProducersState(KafkaCluster cluster, String topic) {
return adminClientService.get(cluster)
.flatMap(ac -> ac.getActiveProducersState(topic));
}
private Mono<List<String>> filterExisting(KafkaCluster cluster, Collection<String> topics) {
return adminClientService.get(cluster)
.flatMap(ac -> ac.listTopics(true))
.map(existing -> existing
.stream()
.filter(topics::contains)
.collect(toList()));
}
}