parent
2cb9b30090
commit
d3bf65cfb6
9 changed files with 387 additions and 120 deletions
|
@ -2,6 +2,7 @@ package com.provectus.kafka.ui.controller;
|
||||||
|
|
||||||
import com.provectus.kafka.ui.api.MessagesApi;
|
import com.provectus.kafka.ui.api.MessagesApi;
|
||||||
import com.provectus.kafka.ui.model.ConsumerPosition;
|
import com.provectus.kafka.ui.model.ConsumerPosition;
|
||||||
|
import com.provectus.kafka.ui.model.SeekDirection;
|
||||||
import com.provectus.kafka.ui.model.SeekType;
|
import com.provectus.kafka.ui.model.SeekType;
|
||||||
import com.provectus.kafka.ui.model.TopicMessage;
|
import com.provectus.kafka.ui.model.TopicMessage;
|
||||||
import com.provectus.kafka.ui.service.ClusterService;
|
import com.provectus.kafka.ui.service.ClusterService;
|
||||||
|
@ -40,13 +41,15 @@ public class MessagesController implements MessagesApi {
|
||||||
@Override
|
@Override
|
||||||
public Mono<ResponseEntity<Flux<TopicMessage>>> getTopicMessages(
|
public Mono<ResponseEntity<Flux<TopicMessage>>> getTopicMessages(
|
||||||
String clusterName, String topicName, @Valid SeekType seekType, @Valid List<String> seekTo,
|
String clusterName, String topicName, @Valid SeekType seekType, @Valid List<String> seekTo,
|
||||||
@Valid Integer limit, @Valid String q, ServerWebExchange exchange) {
|
@Valid Integer limit, @Valid String q, @Valid SeekDirection seekDirection,
|
||||||
return parseConsumerPosition(seekType, seekTo)
|
ServerWebExchange exchange) {
|
||||||
|
return parseConsumerPosition(seekType, seekTo, seekDirection)
|
||||||
.map(consumerPosition -> ResponseEntity
|
.map(consumerPosition -> ResponseEntity
|
||||||
.ok(clusterService.getMessages(clusterName, topicName, consumerPosition, q, limit)));
|
.ok(clusterService.getMessages(clusterName, topicName, consumerPosition, q, limit)));
|
||||||
}
|
}
|
||||||
|
|
||||||
private Mono<ConsumerPosition> parseConsumerPosition(SeekType seekType, List<String> seekTo) {
|
private Mono<ConsumerPosition> parseConsumerPosition(
|
||||||
|
SeekType seekType, List<String> seekTo, SeekDirection seekDirection) {
|
||||||
return Mono.justOrEmpty(seekTo)
|
return Mono.justOrEmpty(seekTo)
|
||||||
.defaultIfEmpty(Collections.emptyList())
|
.defaultIfEmpty(Collections.emptyList())
|
||||||
.flatMapIterable(Function.identity())
|
.flatMapIterable(Function.identity())
|
||||||
|
@ -61,7 +64,7 @@ public class MessagesController implements MessagesApi {
|
||||||
})
|
})
|
||||||
.collectMap(Pair::getKey, Pair::getValue)
|
.collectMap(Pair::getKey, Pair::getValue)
|
||||||
.map(positions -> new ConsumerPosition(seekType != null ? seekType : SeekType.BEGINNING,
|
.map(positions -> new ConsumerPosition(seekType != null ? seekType : SeekType.BEGINNING,
|
||||||
positions));
|
positions, seekDirection));
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -8,5 +8,5 @@ public class ConsumerPosition {
|
||||||
|
|
||||||
private SeekType seekType;
|
private SeekType seekType;
|
||||||
private Map<Integer, Long> seekTo;
|
private Map<Integer, Long> seekTo;
|
||||||
|
private SeekDirection seekDirection;
|
||||||
}
|
}
|
||||||
|
|
|
@ -6,12 +6,14 @@ import com.provectus.kafka.ui.deserialization.DeserializationService;
|
||||||
import com.provectus.kafka.ui.deserialization.RecordDeserializer;
|
import com.provectus.kafka.ui.deserialization.RecordDeserializer;
|
||||||
import com.provectus.kafka.ui.model.ConsumerPosition;
|
import com.provectus.kafka.ui.model.ConsumerPosition;
|
||||||
import com.provectus.kafka.ui.model.KafkaCluster;
|
import com.provectus.kafka.ui.model.KafkaCluster;
|
||||||
import com.provectus.kafka.ui.model.SeekType;
|
import com.provectus.kafka.ui.model.SeekDirection;
|
||||||
import com.provectus.kafka.ui.model.TopicMessage;
|
import com.provectus.kafka.ui.model.TopicMessage;
|
||||||
import com.provectus.kafka.ui.util.ClusterUtil;
|
import com.provectus.kafka.ui.util.ClusterUtil;
|
||||||
|
import com.provectus.kafka.ui.util.OffsetsSeek;
|
||||||
|
import com.provectus.kafka.ui.util.OffsetsSeekBackward;
|
||||||
|
import com.provectus.kafka.ui.util.OffsetsSeekForward;
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -53,7 +55,10 @@ public class ConsumingService {
|
||||||
.orElse(DEFAULT_RECORD_LIMIT);
|
.orElse(DEFAULT_RECORD_LIMIT);
|
||||||
RecordEmitter emitter = new RecordEmitter(
|
RecordEmitter emitter = new RecordEmitter(
|
||||||
() -> kafkaService.createConsumer(cluster),
|
() -> kafkaService.createConsumer(cluster),
|
||||||
new OffsetsSeek(topic, consumerPosition));
|
consumerPosition.getSeekDirection().equals(SeekDirection.FORWARD)
|
||||||
|
? new OffsetsSeekForward(topic, consumerPosition)
|
||||||
|
: new OffsetsSeekBackward(topic, consumerPosition, recordsLimit)
|
||||||
|
);
|
||||||
RecordDeserializer recordDeserializer =
|
RecordDeserializer recordDeserializer =
|
||||||
deserializationService.getRecordDeserializerForCluster(cluster);
|
deserializationService.getRecordDeserializerForCluster(cluster);
|
||||||
return Flux.create(emitter)
|
return Flux.create(emitter)
|
||||||
|
@ -79,7 +84,7 @@ public class ConsumingService {
|
||||||
* returns end offsets for partitions where start offset != end offsets.
|
* returns end offsets for partitions where start offset != end offsets.
|
||||||
* This is useful when we need to verify that partition is not empty.
|
* This is useful when we need to verify that partition is not empty.
|
||||||
*/
|
*/
|
||||||
private static Map<TopicPartition, Long> significantOffsets(Consumer<?, ?> consumer,
|
public static Map<TopicPartition, Long> significantOffsets(Consumer<?, ?> consumer,
|
||||||
String topicName,
|
String topicName,
|
||||||
Collection<Integer>
|
Collection<Integer>
|
||||||
partitionsToInclude) {
|
partitionsToInclude) {
|
||||||
|
@ -159,98 +164,4 @@ public class ConsumingService {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@RequiredArgsConstructor
|
|
||||||
static class OffsetsSeek {
|
|
||||||
|
|
||||||
private final String topic;
|
|
||||||
private final ConsumerPosition consumerPosition;
|
|
||||||
|
|
||||||
public WaitingOffsets assignAndSeek(Consumer<Bytes, Bytes> consumer) {
|
|
||||||
SeekType seekType = consumerPosition.getSeekType();
|
|
||||||
log.info("Positioning consumer for topic {} with {}", topic, consumerPosition);
|
|
||||||
switch (seekType) {
|
|
||||||
case OFFSET:
|
|
||||||
assignAndSeekForOffset(consumer);
|
|
||||||
break;
|
|
||||||
case TIMESTAMP:
|
|
||||||
assignAndSeekForTimestamp(consumer);
|
|
||||||
break;
|
|
||||||
case BEGINNING:
|
|
||||||
assignAndSeekFromBeginning(consumer);
|
|
||||||
break;
|
|
||||||
default:
|
|
||||||
throw new IllegalArgumentException("Unknown seekType: " + seekType);
|
|
||||||
}
|
|
||||||
log.info("Assignment: {}", consumer.assignment());
|
|
||||||
return new WaitingOffsets(topic, consumer);
|
|
||||||
}
|
|
||||||
|
|
||||||
private List<TopicPartition> getRequestedPartitions(Consumer<Bytes, Bytes> consumer) {
|
|
||||||
Map<Integer, Long> partitionPositions = consumerPosition.getSeekTo();
|
|
||||||
return consumer.partitionsFor(topic).stream()
|
|
||||||
.filter(
|
|
||||||
p -> partitionPositions.isEmpty() || partitionPositions.containsKey(p.partition()))
|
|
||||||
.map(p -> new TopicPartition(p.topic(), p.partition()))
|
|
||||||
.collect(Collectors.toList());
|
|
||||||
}
|
|
||||||
|
|
||||||
private void assignAndSeekForOffset(Consumer<Bytes, Bytes> consumer) {
|
|
||||||
List<TopicPartition> partitions = getRequestedPartitions(consumer);
|
|
||||||
consumer.assign(partitions);
|
|
||||||
consumerPosition.getSeekTo().forEach((partition, offset) -> {
|
|
||||||
TopicPartition topicPartition = new TopicPartition(topic, partition);
|
|
||||||
consumer.seek(topicPartition, offset);
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
private void assignAndSeekForTimestamp(Consumer<Bytes, Bytes> consumer) {
|
|
||||||
Map<TopicPartition, Long> timestampsToSearch =
|
|
||||||
consumerPosition.getSeekTo().entrySet().stream()
|
|
||||||
.collect(Collectors.toMap(
|
|
||||||
partitionPosition -> new TopicPartition(topic, partitionPosition.getKey()),
|
|
||||||
Map.Entry::getValue
|
|
||||||
));
|
|
||||||
Map<TopicPartition, Long> offsetsForTimestamps = consumer.offsetsForTimes(timestampsToSearch)
|
|
||||||
.entrySet().stream()
|
|
||||||
.filter(e -> e.getValue() != null)
|
|
||||||
.collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset()));
|
|
||||||
|
|
||||||
if (offsetsForTimestamps.isEmpty()) {
|
|
||||||
throw new IllegalArgumentException("No offsets were found for requested timestamps");
|
|
||||||
}
|
|
||||||
|
|
||||||
consumer.assign(offsetsForTimestamps.keySet());
|
|
||||||
offsetsForTimestamps.forEach(consumer::seek);
|
|
||||||
}
|
|
||||||
|
|
||||||
private void assignAndSeekFromBeginning(Consumer<Bytes, Bytes> consumer) {
|
|
||||||
List<TopicPartition> partitions = getRequestedPartitions(consumer);
|
|
||||||
consumer.assign(partitions);
|
|
||||||
consumer.seekToBeginning(partitions);
|
|
||||||
}
|
|
||||||
|
|
||||||
static class WaitingOffsets {
|
|
||||||
final Map<Integer, Long> offsets = new HashMap<>(); // partition number -> offset
|
|
||||||
|
|
||||||
WaitingOffsets(String topic, Consumer<?, ?> consumer) {
|
|
||||||
var partitions = consumer.assignment().stream()
|
|
||||||
.map(TopicPartition::partition)
|
|
||||||
.collect(Collectors.toList());
|
|
||||||
significantOffsets(consumer, topic, partitions)
|
|
||||||
.forEach((tp, offset) -> offsets.put(tp.partition(), offset - 1));
|
|
||||||
}
|
|
||||||
|
|
||||||
void markPolled(ConsumerRecord<?, ?> rec) {
|
|
||||||
Long waiting = offsets.get(rec.partition());
|
|
||||||
if (waiting != null && waiting <= rec.offset()) {
|
|
||||||
offsets.remove(rec.partition());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
boolean endReached() {
|
|
||||||
return offsets.isEmpty();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,84 @@
|
||||||
|
package com.provectus.kafka.ui.util;
|
||||||
|
|
||||||
|
import com.provectus.kafka.ui.model.ConsumerPosition;
|
||||||
|
import com.provectus.kafka.ui.model.SeekType;
|
||||||
|
import com.provectus.kafka.ui.service.ConsumingService;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import lombok.extern.log4j.Log4j2;
|
||||||
|
import org.apache.kafka.clients.consumer.Consumer;
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
||||||
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.utils.Bytes;
|
||||||
|
|
||||||
|
@Log4j2
|
||||||
|
public abstract class OffsetsSeek {
|
||||||
|
protected final String topic;
|
||||||
|
protected final ConsumerPosition consumerPosition;
|
||||||
|
|
||||||
|
public OffsetsSeek(String topic, ConsumerPosition consumerPosition) {
|
||||||
|
this.topic = topic;
|
||||||
|
this.consumerPosition = consumerPosition;
|
||||||
|
}
|
||||||
|
|
||||||
|
public WaitingOffsets assignAndSeek(Consumer<Bytes, Bytes> consumer) {
|
||||||
|
SeekType seekType = consumerPosition.getSeekType();
|
||||||
|
log.info("Positioning consumer for topic {} with {}", topic, consumerPosition);
|
||||||
|
switch (seekType) {
|
||||||
|
case OFFSET:
|
||||||
|
assignAndSeekForOffset(consumer);
|
||||||
|
break;
|
||||||
|
case TIMESTAMP:
|
||||||
|
assignAndSeekForTimestamp(consumer);
|
||||||
|
break;
|
||||||
|
case BEGINNING:
|
||||||
|
assignAndSeekFromBeginning(consumer);
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new IllegalArgumentException("Unknown seekType: " + seekType);
|
||||||
|
}
|
||||||
|
log.info("Assignment: {}", consumer.assignment());
|
||||||
|
return new WaitingOffsets(topic, consumer);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected List<TopicPartition> getRequestedPartitions(Consumer<Bytes, Bytes> consumer) {
|
||||||
|
Map<Integer, Long> partitionPositions = consumerPosition.getSeekTo();
|
||||||
|
return consumer.partitionsFor(topic).stream()
|
||||||
|
.filter(
|
||||||
|
p -> partitionPositions.isEmpty() || partitionPositions.containsKey(p.partition()))
|
||||||
|
.map(p -> new TopicPartition(p.topic(), p.partition()))
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
protected abstract void assignAndSeekFromBeginning(Consumer<Bytes, Bytes> consumer);
|
||||||
|
|
||||||
|
protected abstract void assignAndSeekForTimestamp(Consumer<Bytes, Bytes> consumer);
|
||||||
|
|
||||||
|
protected abstract void assignAndSeekForOffset(Consumer<Bytes, Bytes> consumer);
|
||||||
|
|
||||||
|
public static class WaitingOffsets {
|
||||||
|
final Map<Integer, Long> offsets = new HashMap<>(); // partition number -> offset
|
||||||
|
|
||||||
|
public WaitingOffsets(String topic, Consumer<?, ?> consumer) {
|
||||||
|
var partitions = consumer.assignment().stream()
|
||||||
|
.map(TopicPartition::partition)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
ConsumingService.significantOffsets(consumer, topic, partitions)
|
||||||
|
.forEach((tp, offset) -> offsets.put(tp.partition(), offset - 1));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void markPolled(ConsumerRecord<?, ?> rec) {
|
||||||
|
Long waiting = offsets.get(rec.partition());
|
||||||
|
if (waiting != null && waiting <= rec.offset()) {
|
||||||
|
offsets.remove(rec.partition());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean endReached() {
|
||||||
|
return offsets.isEmpty();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,121 @@
|
||||||
|
package com.provectus.kafka.ui.util;
|
||||||
|
|
||||||
|
import com.provectus.kafka.ui.model.ConsumerPosition;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import lombok.extern.log4j.Log4j2;
|
||||||
|
import org.apache.kafka.clients.consumer.Consumer;
|
||||||
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.utils.Bytes;
|
||||||
|
import reactor.util.function.Tuple2;
|
||||||
|
import reactor.util.function.Tuples;
|
||||||
|
|
||||||
|
@Log4j2
|
||||||
|
public class OffsetsSeekBackward extends OffsetsSeek {
|
||||||
|
|
||||||
|
private final int maxMessages;
|
||||||
|
|
||||||
|
public OffsetsSeekBackward(String topic,
|
||||||
|
ConsumerPosition consumerPosition, int maxMessages) {
|
||||||
|
super(topic, consumerPosition);
|
||||||
|
this.maxMessages = maxMessages;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
protected void assignAndSeekForOffset(Consumer<Bytes, Bytes> consumer) {
|
||||||
|
List<TopicPartition> partitions = getRequestedPartitions(consumer);
|
||||||
|
consumer.assign(partitions);
|
||||||
|
final Map<TopicPartition, Long> offsets =
|
||||||
|
findOffsetsInt(consumer, consumerPosition.getSeekTo());
|
||||||
|
offsets.forEach(consumer::seek);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void assignAndSeekFromBeginning(Consumer<Bytes, Bytes> consumer) {
|
||||||
|
List<TopicPartition> partitions = getRequestedPartitions(consumer);
|
||||||
|
consumer.assign(partitions);
|
||||||
|
final Map<TopicPartition, Long> offsets = findOffsets(consumer, Map.of());
|
||||||
|
offsets.forEach(consumer::seek);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void assignAndSeekForTimestamp(Consumer<Bytes, Bytes> consumer) {
|
||||||
|
Map<TopicPartition, Long> timestampsToSearch =
|
||||||
|
consumerPosition.getSeekTo().entrySet().stream()
|
||||||
|
.collect(Collectors.toMap(
|
||||||
|
partitionPosition -> new TopicPartition(topic, partitionPosition.getKey()),
|
||||||
|
e -> e.getValue() + 1
|
||||||
|
));
|
||||||
|
Map<TopicPartition, Long> offsetsForTimestamps = consumer.offsetsForTimes(timestampsToSearch)
|
||||||
|
.entrySet().stream()
|
||||||
|
.filter(e -> e.getValue() != null)
|
||||||
|
.map(v -> Tuples.of(v.getKey(), v.getValue().offset() - 1))
|
||||||
|
.collect(Collectors.toMap(Tuple2::getT1, Tuple2::getT2));
|
||||||
|
|
||||||
|
if (offsetsForTimestamps.isEmpty()) {
|
||||||
|
throw new IllegalArgumentException("No offsets were found for requested timestamps");
|
||||||
|
}
|
||||||
|
|
||||||
|
consumer.assign(offsetsForTimestamps.keySet());
|
||||||
|
final Map<TopicPartition, Long> offsets = findOffsets(consumer, offsetsForTimestamps);
|
||||||
|
offsets.forEach(consumer::seek);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Map<TopicPartition, Long> findOffsetsInt(
|
||||||
|
Consumer<Bytes, Bytes> consumer, Map<Integer, Long> seekTo) {
|
||||||
|
|
||||||
|
final Map<TopicPartition, Long> seekMap = seekTo.entrySet()
|
||||||
|
.stream().map(p ->
|
||||||
|
Tuples.of(
|
||||||
|
new TopicPartition(topic, p.getKey()),
|
||||||
|
p.getValue()
|
||||||
|
)
|
||||||
|
).collect(Collectors.toMap(Tuple2::getT1, Tuple2::getT2));
|
||||||
|
|
||||||
|
return findOffsets(consumer, seekMap);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Map<TopicPartition, Long> findOffsets(
|
||||||
|
Consumer<Bytes, Bytes> consumer, Map<TopicPartition, Long> seekTo) {
|
||||||
|
|
||||||
|
List<TopicPartition> partitions = getRequestedPartitions(consumer);
|
||||||
|
final Map<TopicPartition, Long> beginningOffsets = consumer.beginningOffsets(partitions);
|
||||||
|
final Map<TopicPartition, Long> endOffsets = consumer.endOffsets(partitions);
|
||||||
|
|
||||||
|
final Map<TopicPartition, Long> seekMap = new HashMap<>(seekTo);
|
||||||
|
int awaitingMessages = maxMessages;
|
||||||
|
|
||||||
|
Set<TopicPartition> waiting = new HashSet<>(partitions);
|
||||||
|
|
||||||
|
while (awaitingMessages > 0 && !waiting.isEmpty()) {
|
||||||
|
final int msgsPerPartition = (int) Math.ceil((double) awaitingMessages / partitions.size());
|
||||||
|
for (TopicPartition partition : partitions) {
|
||||||
|
final Long offset = Optional.ofNullable(seekMap.get(partition))
|
||||||
|
.orElseGet(() -> endOffsets.get(partition));
|
||||||
|
final Long beginning = beginningOffsets.get(partition);
|
||||||
|
|
||||||
|
if (offset - beginning > msgsPerPartition) {
|
||||||
|
seekMap.put(partition, offset - msgsPerPartition);
|
||||||
|
awaitingMessages -= msgsPerPartition;
|
||||||
|
} else {
|
||||||
|
final long num = offset - beginning;
|
||||||
|
if (num > 0) {
|
||||||
|
seekMap.put(partition, offset - num);
|
||||||
|
awaitingMessages -= num;
|
||||||
|
} else {
|
||||||
|
waiting.remove(partition);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (awaitingMessages <= 0) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return seekMap;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,59 @@
|
||||||
|
package com.provectus.kafka.ui.util;
|
||||||
|
|
||||||
|
import com.provectus.kafka.ui.model.ConsumerPosition;
|
||||||
|
import com.provectus.kafka.ui.model.SeekType;
|
||||||
|
import com.provectus.kafka.ui.service.ConsumingService;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import lombok.RequiredArgsConstructor;
|
||||||
|
import lombok.extern.log4j.Log4j2;
|
||||||
|
import org.apache.kafka.clients.consumer.Consumer;
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
||||||
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.utils.Bytes;
|
||||||
|
|
||||||
|
@Log4j2
|
||||||
|
public class OffsetsSeekForward extends OffsetsSeek {
|
||||||
|
|
||||||
|
public OffsetsSeekForward(String topic, ConsumerPosition consumerPosition) {
|
||||||
|
super(topic, consumerPosition);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void assignAndSeekForOffset(Consumer<Bytes, Bytes> consumer) {
|
||||||
|
List<TopicPartition> partitions = getRequestedPartitions(consumer);
|
||||||
|
consumer.assign(partitions);
|
||||||
|
consumerPosition.getSeekTo().forEach((partition, offset) -> {
|
||||||
|
TopicPartition topicPartition = new TopicPartition(topic, partition);
|
||||||
|
consumer.seek(topicPartition, offset);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void assignAndSeekForTimestamp(Consumer<Bytes, Bytes> consumer) {
|
||||||
|
Map<TopicPartition, Long> timestampsToSearch =
|
||||||
|
consumerPosition.getSeekTo().entrySet().stream()
|
||||||
|
.collect(Collectors.toMap(
|
||||||
|
partitionPosition -> new TopicPartition(topic, partitionPosition.getKey()),
|
||||||
|
Map.Entry::getValue
|
||||||
|
));
|
||||||
|
Map<TopicPartition, Long> offsetsForTimestamps = consumer.offsetsForTimes(timestampsToSearch)
|
||||||
|
.entrySet().stream()
|
||||||
|
.filter(e -> e.getValue() != null)
|
||||||
|
.collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset()));
|
||||||
|
|
||||||
|
if (offsetsForTimestamps.isEmpty()) {
|
||||||
|
throw new IllegalArgumentException("No offsets were found for requested timestamps");
|
||||||
|
}
|
||||||
|
|
||||||
|
consumer.assign(offsetsForTimestamps.keySet());
|
||||||
|
offsetsForTimestamps.forEach(consumer::seek);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void assignAndSeekFromBeginning(Consumer<Bytes, Bytes> consumer) {
|
||||||
|
List<TopicPartition> partitions = getRequestedPartitions(consumer);
|
||||||
|
consumer.assign(partitions);
|
||||||
|
consumer.seekToBeginning(partitions);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -1,13 +1,14 @@
|
||||||
package com.provectus.kafka.ui.service;
|
package com.provectus.kafka.ui.service;
|
||||||
|
|
||||||
import static com.provectus.kafka.ui.service.ConsumingService.OffsetsSeek;
|
|
||||||
import static com.provectus.kafka.ui.service.ConsumingService.RecordEmitter;
|
import static com.provectus.kafka.ui.service.ConsumingService.RecordEmitter;
|
||||||
import static org.assertj.core.api.Assertions.assertThat;
|
import static org.assertj.core.api.Assertions.assertThat;
|
||||||
|
|
||||||
import com.provectus.kafka.ui.AbstractBaseTest;
|
import com.provectus.kafka.ui.AbstractBaseTest;
|
||||||
import com.provectus.kafka.ui.model.ConsumerPosition;
|
import com.provectus.kafka.ui.model.ConsumerPosition;
|
||||||
|
import com.provectus.kafka.ui.model.SeekDirection;
|
||||||
import com.provectus.kafka.ui.model.SeekType;
|
import com.provectus.kafka.ui.model.SeekType;
|
||||||
import com.provectus.kafka.ui.producer.KafkaTestProducer;
|
import com.provectus.kafka.ui.producer.KafkaTestProducer;
|
||||||
|
import com.provectus.kafka.ui.util.OffsetsSeekForward;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -65,7 +66,10 @@ class RecordEmitterTest extends AbstractBaseTest {
|
||||||
void pollNothingOnEmptyTopic() {
|
void pollNothingOnEmptyTopic() {
|
||||||
var emitter = new RecordEmitter(
|
var emitter = new RecordEmitter(
|
||||||
this::createConsumer,
|
this::createConsumer,
|
||||||
new OffsetsSeek(EMPTY_TOPIC, new ConsumerPosition(SeekType.BEGINNING, Map.of())));
|
new OffsetsSeekForward(EMPTY_TOPIC,
|
||||||
|
new ConsumerPosition(SeekType.BEGINNING, Map.of(), SeekDirection.FORWARD)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
Long polledValues = Flux.create(emitter)
|
Long polledValues = Flux.create(emitter)
|
||||||
.limitRequest(100)
|
.limitRequest(100)
|
||||||
|
@ -79,7 +83,10 @@ class RecordEmitterTest extends AbstractBaseTest {
|
||||||
void pollFullTopicFromBeginning() {
|
void pollFullTopicFromBeginning() {
|
||||||
var emitter = new RecordEmitter(
|
var emitter = new RecordEmitter(
|
||||||
this::createConsumer,
|
this::createConsumer,
|
||||||
new OffsetsSeek(TOPIC, new ConsumerPosition(SeekType.BEGINNING, Map.of())));
|
new OffsetsSeekForward(TOPIC,
|
||||||
|
new ConsumerPosition(SeekType.BEGINNING, Map.of(), SeekDirection.FORWARD)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
var polledValues = Flux.create(emitter)
|
var polledValues = Flux.create(emitter)
|
||||||
.map(this::deserialize)
|
.map(this::deserialize)
|
||||||
|
@ -101,7 +108,10 @@ class RecordEmitterTest extends AbstractBaseTest {
|
||||||
|
|
||||||
var emitter = new RecordEmitter(
|
var emitter = new RecordEmitter(
|
||||||
this::createConsumer,
|
this::createConsumer,
|
||||||
new OffsetsSeek(TOPIC, new ConsumerPosition(SeekType.OFFSET, targetOffsets)));
|
new OffsetsSeekForward(TOPIC,
|
||||||
|
new ConsumerPosition(SeekType.OFFSET, targetOffsets, SeekDirection.FORWARD)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
var polledValues = Flux.create(emitter)
|
var polledValues = Flux.create(emitter)
|
||||||
.map(this::deserialize)
|
.map(this::deserialize)
|
||||||
|
@ -127,7 +137,10 @@ class RecordEmitterTest extends AbstractBaseTest {
|
||||||
|
|
||||||
var emitter = new RecordEmitter(
|
var emitter = new RecordEmitter(
|
||||||
this::createConsumer,
|
this::createConsumer,
|
||||||
new OffsetsSeek(TOPIC, new ConsumerPosition(SeekType.TIMESTAMP, targetTimestamps)));
|
new OffsetsSeekForward(TOPIC,
|
||||||
|
new ConsumerPosition(SeekType.TIMESTAMP, targetTimestamps, SeekDirection.FORWARD)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
var polledValues = Flux.create(emitter)
|
var polledValues = Flux.create(emitter)
|
||||||
.map(this::deserialize)
|
.map(this::deserialize)
|
||||||
|
|
|
@ -1,8 +1,9 @@
|
||||||
package com.provectus.kafka.ui.service;
|
package com.provectus.kafka.ui.util;
|
||||||
|
|
||||||
import static org.assertj.core.api.Assertions.assertThat;
|
import static org.assertj.core.api.Assertions.assertThat;
|
||||||
|
|
||||||
import com.provectus.kafka.ui.model.ConsumerPosition;
|
import com.provectus.kafka.ui.model.ConsumerPosition;
|
||||||
|
import com.provectus.kafka.ui.model.SeekDirection;
|
||||||
import com.provectus.kafka.ui.model.SeekType;
|
import com.provectus.kafka.ui.model.SeekType;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -51,10 +52,16 @@ class OffsetsSeekTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
void seekToBeginningAllPartitions() {
|
void forwardSeekToBeginningAllPartitions() {
|
||||||
var seek = new ConsumingService.OffsetsSeek(
|
var seek = new OffsetsSeekForward(
|
||||||
topic,
|
topic,
|
||||||
new ConsumerPosition(SeekType.BEGINNING, Map.of(0, 0L, 1, 0L)));
|
new ConsumerPosition(
|
||||||
|
SeekType.BEGINNING,
|
||||||
|
Map.of(0, 0L, 1, 0L),
|
||||||
|
SeekDirection.FORWARD
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
seek.assignAndSeek(consumer);
|
seek.assignAndSeek(consumer);
|
||||||
assertThat(consumer.assignment()).containsExactlyInAnyOrder(tp0, tp1);
|
assertThat(consumer.assignment()).containsExactlyInAnyOrder(tp0, tp1);
|
||||||
assertThat(consumer.position(tp0)).isZero();
|
assertThat(consumer.position(tp0)).isZero();
|
||||||
|
@ -62,10 +69,28 @@ class OffsetsSeekTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
void seekToBeginningWithPartitionsList() {
|
void backwardSeekToBeginningAllPartitions() {
|
||||||
var seek = new ConsumingService.OffsetsSeek(
|
var seek = new OffsetsSeekBackward(
|
||||||
topic,
|
topic,
|
||||||
new ConsumerPosition(SeekType.BEGINNING, Map.of()));
|
new ConsumerPosition(
|
||||||
|
SeekType.BEGINNING,
|
||||||
|
Map.of(2, 0L, 3, 0L),
|
||||||
|
SeekDirection.BACKWARD
|
||||||
|
),
|
||||||
|
10
|
||||||
|
);
|
||||||
|
|
||||||
|
seek.assignAndSeek(consumer);
|
||||||
|
assertThat(consumer.assignment()).containsExactlyInAnyOrder(tp2, tp3);
|
||||||
|
assertThat(consumer.position(tp2)).isEqualTo(15L);
|
||||||
|
assertThat(consumer.position(tp3)).isEqualTo(25L);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void forwardSeekToBeginningWithPartitionsList() {
|
||||||
|
var seek = new OffsetsSeekForward(
|
||||||
|
topic,
|
||||||
|
new ConsumerPosition(SeekType.BEGINNING, Map.of(), SeekDirection.FORWARD));
|
||||||
seek.assignAndSeek(consumer);
|
seek.assignAndSeek(consumer);
|
||||||
assertThat(consumer.assignment()).containsExactlyInAnyOrder(tp0, tp1, tp2, tp3);
|
assertThat(consumer.assignment()).containsExactlyInAnyOrder(tp0, tp1, tp2, tp3);
|
||||||
assertThat(consumer.position(tp0)).isZero();
|
assertThat(consumer.position(tp0)).isZero();
|
||||||
|
@ -75,10 +100,31 @@ class OffsetsSeekTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
void seekToOffset() {
|
void backwardSeekToBeginningWithPartitionsList() {
|
||||||
var seek = new ConsumingService.OffsetsSeek(
|
var seek = new OffsetsSeekBackward(
|
||||||
topic,
|
topic,
|
||||||
new ConsumerPosition(SeekType.OFFSET, Map.of(0, 0L, 1, 1L, 2, 2L)));
|
new ConsumerPosition(SeekType.BEGINNING, Map.of(), SeekDirection.BACKWARD),
|
||||||
|
10
|
||||||
|
);
|
||||||
|
seek.assignAndSeek(consumer);
|
||||||
|
assertThat(consumer.assignment()).containsExactlyInAnyOrder(tp0, tp1, tp2, tp3);
|
||||||
|
assertThat(consumer.position(tp0)).isZero();
|
||||||
|
assertThat(consumer.position(tp1)).isEqualTo(10L);
|
||||||
|
assertThat(consumer.position(tp2)).isEqualTo(15L);
|
||||||
|
assertThat(consumer.position(tp3)).isEqualTo(25L);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void forwardSeekToOffset() {
|
||||||
|
var seek = new OffsetsSeekForward(
|
||||||
|
topic,
|
||||||
|
new ConsumerPosition(
|
||||||
|
SeekType.OFFSET,
|
||||||
|
Map.of(0, 0L, 1, 1L, 2, 2L),
|
||||||
|
SeekDirection.FORWARD
|
||||||
|
)
|
||||||
|
);
|
||||||
seek.assignAndSeek(consumer);
|
seek.assignAndSeek(consumer);
|
||||||
assertThat(consumer.assignment()).containsExactlyInAnyOrder(tp0, tp1, tp2);
|
assertThat(consumer.assignment()).containsExactlyInAnyOrder(tp0, tp1, tp2);
|
||||||
assertThat(consumer.position(tp0)).isZero();
|
assertThat(consumer.position(tp0)).isZero();
|
||||||
|
@ -86,15 +132,34 @@ class OffsetsSeekTest {
|
||||||
assertThat(consumer.position(tp2)).isEqualTo(2L);
|
assertThat(consumer.position(tp2)).isEqualTo(2L);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void backwardSeekToOffset() {
|
||||||
|
var seek = new OffsetsSeekBackward(
|
||||||
|
topic,
|
||||||
|
new ConsumerPosition(
|
||||||
|
SeekType.OFFSET,
|
||||||
|
Map.of(0, 0L, 1, 1L, 2, 2L),
|
||||||
|
SeekDirection.FORWARD
|
||||||
|
),
|
||||||
|
2
|
||||||
|
);
|
||||||
|
seek.assignAndSeek(consumer);
|
||||||
|
assertThat(consumer.assignment()).containsExactlyInAnyOrder(tp0, tp1, tp2);
|
||||||
|
assertThat(consumer.position(tp0)).isZero();
|
||||||
|
assertThat(consumer.position(tp1)).isEqualTo(1L);
|
||||||
|
assertThat(consumer.position(tp2)).isEqualTo(0L);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
@Nested
|
@Nested
|
||||||
class WaitingOffsetsTest {
|
class WaitingOffsetsTest {
|
||||||
|
|
||||||
ConsumingService.OffsetsSeek.WaitingOffsets offsets;
|
OffsetsSeekForward.WaitingOffsets offsets;
|
||||||
|
|
||||||
@BeforeEach
|
@BeforeEach
|
||||||
void assignAndCreateOffsets() {
|
void assignAndCreateOffsets() {
|
||||||
consumer.assign(List.of(tp0, tp1, tp2, tp3));
|
consumer.assign(List.of(tp0, tp1, tp2, tp3));
|
||||||
offsets = new ConsumingService.OffsetsSeek.WaitingOffsets(topic, consumer);
|
offsets = new OffsetsSeek.WaitingOffsets(topic, consumer);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
|
@ -323,6 +323,10 @@ paths:
|
||||||
in: query
|
in: query
|
||||||
schema:
|
schema:
|
||||||
type: string
|
type: string
|
||||||
|
- name: seekDirection
|
||||||
|
in: query
|
||||||
|
schema:
|
||||||
|
$ref: "#/components/schemas/SeekDirection"
|
||||||
responses:
|
responses:
|
||||||
200:
|
200:
|
||||||
description: OK
|
description: OK
|
||||||
|
@ -1448,6 +1452,13 @@ components:
|
||||||
- OFFSET
|
- OFFSET
|
||||||
- TIMESTAMP
|
- TIMESTAMP
|
||||||
|
|
||||||
|
SeekDirection:
|
||||||
|
type: string
|
||||||
|
enum:
|
||||||
|
- FORWARD
|
||||||
|
- BACKWARD
|
||||||
|
default: FORWARD
|
||||||
|
|
||||||
Partition:
|
Partition:
|
||||||
type: object
|
type: object
|
||||||
properties:
|
properties:
|
||||||
|
|
Loading…
Add table
Reference in a new issue