ISSUE-2103: Fixing AdminClient timeout on listOffsets (#2781)
ISSUE-2103: Fixing AdminClient timeout on listOffsets when partition's leader in not available: 1. ReactiveAdminClient.listOffsets methods now check if partitions have leader before executing 2. In all places where ReactiveAdminClient.listOffsets methods are used, we specify what to do if partition leader is not available - fail or skip such partitions 3. AdminClient.listOffsets calls splitted with partitionCalls
This commit is contained in:
parent
cb8049e769
commit
aa6c3083c2
6 changed files with 96 additions and 37 deletions
|
@ -13,12 +13,12 @@ public class InternalPartition {
|
|||
private final int inSyncReplicasCount;
|
||||
private final int replicasCount;
|
||||
|
||||
private final long offsetMin;
|
||||
private final long offsetMax;
|
||||
private final Long offsetMin;
|
||||
private final Long offsetMax;
|
||||
|
||||
// from log dir
|
||||
private final long segmentSize;
|
||||
private final long segmentCount;
|
||||
private final Long segmentSize;
|
||||
private final Integer segmentCount;
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -49,8 +49,8 @@ public class ConsumerGroupService {
|
|||
var tpsFromGroupOffsets = groupOffsetsMap.values().stream()
|
||||
.flatMap(v -> v.keySet().stream())
|
||||
.collect(Collectors.toSet());
|
||||
// 2. getting end offsets for partitions with in committed offsets
|
||||
return ac.listOffsets(tpsFromGroupOffsets, OffsetSpec.latest())
|
||||
// 2. getting end offsets for partitions with committed offsets
|
||||
return ac.listOffsets(tpsFromGroupOffsets, OffsetSpec.latest(), false)
|
||||
.map(endOffsets ->
|
||||
descriptions.stream()
|
||||
.map(desc -> {
|
||||
|
@ -68,7 +68,7 @@ public class ConsumerGroupService {
|
|||
String topic) {
|
||||
return adminClientService.get(cluster)
|
||||
// 1. getting topic's end offsets
|
||||
.flatMap(ac -> ac.listOffsets(topic, OffsetSpec.latest())
|
||||
.flatMap(ac -> ac.listTopicOffsets(topic, OffsetSpec.latest(), false)
|
||||
.flatMap(endOffsets -> {
|
||||
var tps = new ArrayList<>(endOffsets.keySet());
|
||||
// 2. getting all consumer groups
|
||||
|
|
|
@ -65,8 +65,8 @@ public class MessagesService {
|
|||
private Mono<Map<TopicPartition, Long>> offsetsForDeletion(KafkaCluster cluster, String topicName,
|
||||
List<Integer> partitionsToInclude) {
|
||||
return adminClientService.get(cluster).flatMap(ac ->
|
||||
ac.listOffsets(topicName, OffsetSpec.earliest())
|
||||
.zipWith(ac.listOffsets(topicName, OffsetSpec.latest()),
|
||||
ac.listTopicOffsets(topicName, OffsetSpec.earliest(), true)
|
||||
.zipWith(ac.listTopicOffsets(topicName, OffsetSpec.latest(), true),
|
||||
(start, end) ->
|
||||
end.entrySet().stream()
|
||||
.filter(e -> partitionsToInclude.isEmpty()
|
||||
|
|
|
@ -47,11 +47,12 @@ public class OffsetsResetService {
|
|||
@Nullable Collection<Integer> partitions,
|
||||
OffsetSpec spec) {
|
||||
if (partitions == null) {
|
||||
return client.listOffsets(topic, spec);
|
||||
return client.listTopicOffsets(topic, spec, true);
|
||||
}
|
||||
return client.listOffsets(
|
||||
partitions.stream().map(idx -> new TopicPartition(topic, idx)).collect(toSet()),
|
||||
spec
|
||||
spec,
|
||||
true
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -84,9 +85,9 @@ public class OffsetsResetService {
|
|||
.collect(toMap(e -> new TopicPartition(topic, e.getKey()), Map.Entry::getValue));
|
||||
return checkGroupCondition(cluster, group).flatMap(
|
||||
ac ->
|
||||
ac.listOffsets(partitionOffsets.keySet(), OffsetSpec.earliest())
|
||||
ac.listOffsets(partitionOffsets.keySet(), OffsetSpec.earliest(), true)
|
||||
.flatMap(earliest ->
|
||||
ac.listOffsets(partitionOffsets.keySet(), OffsetSpec.latest())
|
||||
ac.listOffsets(partitionOffsets.keySet(), OffsetSpec.latest(), true)
|
||||
.map(latest -> editOffsetsBounds(partitionOffsets, earliest, latest))
|
||||
.flatMap(offsetsToCommit -> resetOffsets(ac, group, offsetsToCommit)))
|
||||
);
|
||||
|
|
|
@ -8,12 +8,15 @@ import com.google.common.collect.ImmutableMap;
|
|||
import com.google.common.collect.Iterators;
|
||||
import com.provectus.kafka.ui.exception.IllegalEntityStateException;
|
||||
import com.provectus.kafka.ui.exception.NotFoundException;
|
||||
import com.provectus.kafka.ui.exception.ValidationException;
|
||||
import com.provectus.kafka.ui.util.MapUtil;
|
||||
import com.provectus.kafka.ui.util.NumberUtil;
|
||||
import com.provectus.kafka.ui.util.annotations.KafkaClientInternalsDependant;
|
||||
import java.io.Closeable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -25,6 +28,7 @@ import java.util.concurrent.ExecutionException;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.function.BiFunction;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -51,6 +55,7 @@ import org.apache.kafka.common.KafkaException;
|
|||
import org.apache.kafka.common.KafkaFuture;
|
||||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.TopicPartitionInfo;
|
||||
import org.apache.kafka.common.TopicPartitionReplica;
|
||||
import org.apache.kafka.common.acl.AclOperation;
|
||||
import org.apache.kafka.common.config.ConfigResource;
|
||||
|
@ -400,31 +405,80 @@ public class ReactiveAdminClient implements Closeable {
|
|||
.all());
|
||||
}
|
||||
|
||||
public Mono<Map<TopicPartition, Long>> listOffsets(String topic,
|
||||
OffsetSpec offsetSpec) {
|
||||
return topicPartitions(topic).flatMap(tps -> listOffsets(tps, offsetSpec));
|
||||
/**
|
||||
* List offset for the topic's partitions and OffsetSpec.
|
||||
* @param failOnUnknownLeader true - throw exception in case of no-leader partitions,
|
||||
* false - skip partitions with no leader
|
||||
*/
|
||||
public Mono<Map<TopicPartition, Long>> listTopicOffsets(String topic,
|
||||
OffsetSpec offsetSpec,
|
||||
boolean failOnUnknownLeader) {
|
||||
return describeTopic(topic)
|
||||
.map(td -> filterPartitionsWithLeaderCheck(List.of(td), p -> true, failOnUnknownLeader))
|
||||
.flatMap(partitions -> listOffsetsUnsafe(partitions, offsetSpec));
|
||||
}
|
||||
|
||||
/**
|
||||
* List offset for the specified partitions and OffsetSpec.
|
||||
* @param failOnUnknownLeader true - throw exception in case of no-leader partitions,
|
||||
* false - skip partitions with no leader
|
||||
*/
|
||||
public Mono<Map<TopicPartition, Long>> listOffsets(Collection<TopicPartition> partitions,
|
||||
OffsetSpec offsetSpec,
|
||||
boolean failOnUnknownLeader) {
|
||||
return filterPartitionsWithLeaderCheck(partitions, failOnUnknownLeader)
|
||||
.flatMap(parts -> listOffsetsUnsafe(parts, offsetSpec));
|
||||
}
|
||||
|
||||
private Mono<Collection<TopicPartition>> filterPartitionsWithLeaderCheck(Collection<TopicPartition> partitions,
|
||||
boolean failOnUnknownLeader) {
|
||||
var targetTopics = partitions.stream().map(TopicPartition::topic).collect(Collectors.toSet());
|
||||
return describeTopicsImpl(targetTopics)
|
||||
.map(descriptions ->
|
||||
filterPartitionsWithLeaderCheck(
|
||||
descriptions.values(), partitions::contains, failOnUnknownLeader));
|
||||
}
|
||||
|
||||
private Set<TopicPartition> filterPartitionsWithLeaderCheck(Collection<TopicDescription> topicDescriptions,
|
||||
Predicate<TopicPartition> partitionPredicate,
|
||||
boolean failOnUnknownLeader) {
|
||||
var goodPartitions = new HashSet<TopicPartition>();
|
||||
for (TopicDescription description : topicDescriptions) {
|
||||
for (TopicPartitionInfo partitionInfo : description.partitions()) {
|
||||
TopicPartition topicPartition = new TopicPartition(description.name(), partitionInfo.partition());
|
||||
if (!partitionPredicate.test(topicPartition)) {
|
||||
continue;
|
||||
}
|
||||
if (partitionInfo.leader() != null) {
|
||||
goodPartitions.add(topicPartition);
|
||||
} else if (failOnUnknownLeader) {
|
||||
throw new ValidationException(String.format("Topic partition %s has no leader", topicPartition));
|
||||
}
|
||||
}
|
||||
}
|
||||
return goodPartitions;
|
||||
}
|
||||
|
||||
// 1. NOTE(!): should only apply for partitions with existing leader,
|
||||
// otherwise AdminClient will try to fetch topic metadata, fail and retry infinitely (until timeout)
|
||||
// 2. TODO: check if it is a bug that AdminClient never throws LeaderNotAvailableException and just retrying instead
|
||||
@KafkaClientInternalsDependant
|
||||
public Mono<Map<TopicPartition, Long>> listOffsetsUnsafe(Collection<TopicPartition> partitions,
|
||||
OffsetSpec offsetSpec) {
|
||||
//TODO: need to split this into multiple calls if number of target partitions is big
|
||||
return toMono(
|
||||
client.listOffsets(partitions.stream().collect(toMap(tp -> tp, tp -> offsetSpec))).all())
|
||||
.map(offsets -> offsets.entrySet()
|
||||
.stream()
|
||||
|
||||
Function<Collection<TopicPartition>, Mono<Map<TopicPartition, Long>>> call =
|
||||
parts -> toMono(
|
||||
client.listOffsets(parts.stream().collect(toMap(tp -> tp, tp -> offsetSpec))).all())
|
||||
.map(offsets -> offsets.entrySet().stream()
|
||||
// filtering partitions for which offsets were not found
|
||||
.filter(e -> e.getValue().offset() >= 0)
|
||||
.collect(toMap(Map.Entry::getKey, e -> e.getValue().offset())));
|
||||
}
|
||||
|
||||
private Mono<Set<TopicPartition>> topicPartitions(String topic) {
|
||||
return toMono(client.describeTopics(List.of(topic)).all())
|
||||
.map(r -> r.values().stream()
|
||||
.findFirst()
|
||||
.stream()
|
||||
.flatMap(d -> d.partitions().stream())
|
||||
.map(p -> new TopicPartition(topic, p.partition()))
|
||||
.collect(Collectors.toSet())
|
||||
return partitionCalls(
|
||||
partitions,
|
||||
200,
|
||||
call,
|
||||
(m1, m2) -> ImmutableMap.<TopicPartition, Long>builder().putAll(m1).putAll(m2).build()
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -138,11 +138,15 @@ public class TopicsService {
|
|||
ReactiveAdminClient ac) {
|
||||
var topicPartitions = descriptions.values().stream()
|
||||
.flatMap(desc ->
|
||||
desc.partitions().stream().map(p -> new TopicPartition(desc.name(), p.partition())))
|
||||
desc.partitions().stream()
|
||||
// list offsets should only be applied to partitions with existing leader
|
||||
// (see ReactiveAdminClient.listOffsetsUnsafe(..) docs)
|
||||
.filter(tp -> tp.leader() != null)
|
||||
.map(p -> new TopicPartition(desc.name(), p.partition())))
|
||||
.collect(toList());
|
||||
|
||||
return ac.listOffsets(topicPartitions, OffsetSpec.earliest())
|
||||
.zipWith(ac.listOffsets(topicPartitions, OffsetSpec.latest()),
|
||||
return ac.listOffsetsUnsafe(topicPartitions, OffsetSpec.earliest())
|
||||
.zipWith(ac.listOffsetsUnsafe(topicPartitions, OffsetSpec.latest()),
|
||||
(earliest, latest) ->
|
||||
topicPartitions.stream()
|
||||
.filter(tp -> earliest.containsKey(tp) && latest.containsKey(tp))
|
||||
|
|
Loading…
Add table
Reference in a new issue