Fixing infinite polls in ForwardRecordEmitter for topic where no record found for last offset (compact topic) (#2627)

Co-authored-by: iliax <ikuramshin@provectus.com>
This commit is contained in:
Ilya Kuramshin 2022-09-22 09:19:00 +04:00 committed by GitHub
parent b4424fc960
commit d673ea63a1
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
4 changed files with 15 additions and 3 deletions

View file

@ -16,6 +16,13 @@ import reactor.core.publisher.FluxSink;
public abstract class AbstractEmitter { public abstract class AbstractEmitter {
private static final Duration DEFAULT_POLL_TIMEOUT_MS = Duration.ofMillis(1000L); private static final Duration DEFAULT_POLL_TIMEOUT_MS = Duration.ofMillis(1000L);
// In some situations it is hard to say whether records range (between two offsets) was fully polled.
// This happens when we have holes in records sequences that is usual case for compact topics or
// topics with transactional writes. In such cases if you want to poll all records between offsets X and Y
// there is no guarantee that you will ever see record with offset Y.
// To workaround this we can assume that after N consecutive empty polls all target messages were read.
public static final int NO_MORE_DATA_EMPTY_POLLS_COUNT = 3;
private final RecordSerDe recordDeserializer; private final RecordSerDe recordDeserializer;
private final ConsumingStats consumingStats = new ConsumingStats(); private final ConsumingStats consumingStats = new ConsumingStats();

View file

@ -118,7 +118,7 @@ public class BackwardRecordEmitter
var recordsToSend = new ArrayList<ConsumerRecord<Bytes, Bytes>>(); var recordsToSend = new ArrayList<ConsumerRecord<Bytes, Bytes>>();
// we use empty polls counting to verify that partition was fully read // we use empty polls counting to verify that partition was fully read
for (int emptyPolls = 0; recordsToSend.size() < desiredMsgsToPoll && emptyPolls < 3; ) { for (int emptyPolls = 0; recordsToSend.size() < desiredMsgsToPoll && emptyPolls < NO_MORE_DATA_EMPTY_POLLS_COUNT;) {
var polledRecords = poll(sink, consumer, POLL_TIMEOUT); var polledRecords = poll(sink, consumer, POLL_TIMEOUT);
log.debug("{} records polled from {}", polledRecords.count(), tp); log.debug("{} records polled from {}", polledRecords.count(), tp);

View file

@ -33,10 +33,13 @@ public class ForwardRecordEmitter
try (KafkaConsumer<Bytes, Bytes> consumer = consumerSupplier.get()) { try (KafkaConsumer<Bytes, Bytes> consumer = consumerSupplier.get()) {
sendPhase(sink, "Assigning partitions"); sendPhase(sink, "Assigning partitions");
var waitingOffsets = offsetsSeek.assignAndSeek(consumer); var waitingOffsets = offsetsSeek.assignAndSeek(consumer);
while (!sink.isCancelled() && !waitingOffsets.endReached()) { // we use empty polls counting to verify that topic was fully read
int emptyPolls = 0;
while (!sink.isCancelled() && !waitingOffsets.endReached() && emptyPolls < NO_MORE_DATA_EMPTY_POLLS_COUNT) {
sendPhase(sink, "Polling"); sendPhase(sink, "Polling");
ConsumerRecords<Bytes, Bytes> records = poll(sink, consumer); ConsumerRecords<Bytes, Bytes> records = poll(sink, consumer);
log.info("{} records polled", records.count()); log.info("{} records polled", records.count());
emptyPolls = records.isEmpty() ? emptyPolls + 1 : 0;
for (ConsumerRecord<Bytes, Bytes> msg : records) { for (ConsumerRecord<Bytes, Bytes> msg : records) {
if (!sink.isCancelled() && !waitingOffsets.endReached()) { if (!sink.isCancelled() && !waitingOffsets.endReached()) {

View file

@ -1,5 +1,7 @@
package com.provectus.kafka.ui.service.analyze; package com.provectus.kafka.ui.service.analyze;
import static com.provectus.kafka.ui.emitter.AbstractEmitter.NO_MORE_DATA_EMPTY_POLLS_COUNT;
import com.provectus.kafka.ui.exception.TopicAnalysisException; import com.provectus.kafka.ui.exception.TopicAnalysisException;
import com.provectus.kafka.ui.model.KafkaCluster; import com.provectus.kafka.ui.model.KafkaCluster;
import com.provectus.kafka.ui.model.TopicAnalysisDTO; import com.provectus.kafka.ui.model.TopicAnalysisDTO;
@ -118,7 +120,7 @@ public class TopicAnalysisService {
consumer.seekToBeginning(topicPartitions); consumer.seekToBeginning(topicPartitions);
var waitingOffsets = new WaitingOffsets(topicId.topicName, consumer, topicPartitions); var waitingOffsets = new WaitingOffsets(topicId.topicName, consumer, topicPartitions);
for (int emptyPolls = 0; !waitingOffsets.endReached() && emptyPolls < 3; ) { for (int emptyPolls = 0; !waitingOffsets.endReached() && emptyPolls < NO_MORE_DATA_EMPTY_POLLS_COUNT;) {
var polled = consumer.poll(Duration.ofSeconds(3)); var polled = consumer.poll(Duration.ofSeconds(3));
emptyPolls = polled.isEmpty() ? emptyPolls + 1 : 0; emptyPolls = polled.isEmpty() ? emptyPolls + 1 : 0;
polled.forEach(r -> { polled.forEach(r -> {