Skip to content

Commit

Permalink
GH-3660: Fix EmbeddedKafkaBroker seekToEnd (#3661)
Browse files Browse the repository at this point in the history
Fixes: #3660

Problem: consumeFromEmbeddedTopics calls Consumer.seekToEnd, which
"evaluates lazily, seeking to the final offset in all partitions only
when poll(Duration) or position(TopicPartition) are called".
This means that the consumer may or may not see future messages,
depending on timing.

This fix calls `position` so that the seek happens before
consumeFromEmbeddedTopics returns. That was it is ensured that any
message sent to the topic after the call to consumeFromEmbeddedTopics
are seen by the consumer.

Issue: #3660
  • Loading branch information
coekie authored and sobychacko committed Dec 13, 2024
1 parent ab17e3d commit 71b571c
Show file tree
Hide file tree
Showing 4 changed files with 61 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,9 @@
* @author Nakul Mishra
* @author Pawel Lozinski
* @author Adrian Chlebosz
* @author Soby Chacko
* @author Sanghyeok An
* @author Wouter Coekaerts
*
* @since 3.1
*/
Expand Down Expand Up @@ -553,6 +556,8 @@ public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
+ (seekToEnd ? "end; " : "beginning"));
if (seekToEnd) {
consumer.seekToEnd(assigned.get());
// seekToEnd is asynchronous. query the position to force the seek to happen now.
assigned.get().forEach(consumer::position);
}
else {
consumer.seekToBeginning(assigned.get());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,10 @@
* @author Nakul Mishra
* @author Pawel Lozinski
* @author Adrian Chlebosz
* @author Soby Chacko
* @author Sanghyeok An
* @author Borahm Lee
* @author Wouter Coekaerts
*
* @since 2.2
*/
Expand Down Expand Up @@ -755,6 +759,8 @@ public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
+ (seekToEnd ? "end; " : "beginning"));
if (seekToEnd) {
consumer.seekToEnd(assigned.get());
// seekToEnd is asynchronous. query the position to force the seek to happen now.
assigned.get().forEach(consumer::position);
}
else {
consumer.seekToBeginning(assigned.get());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,14 +16,21 @@

package org.springframework.kafka.test;

import java.util.Map;

import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.junit.jupiter.api.Test;

import org.springframework.kafka.test.utils.KafkaTestUtils;
import org.springframework.util.StringUtils;

import static org.assertj.core.api.Assertions.assertThat;

/**
* @author Gary Russell
* @author Wouter Coekaerts
* @since 3.1
*
*/
Expand All @@ -37,4 +44,21 @@ void testUpDown() {
kafka.destroy();
}

@Test
void testConsumeFromEmbeddedWithSeekToEnd() {
EmbeddedKafkaKraftBroker kafka = new EmbeddedKafkaKraftBroker(1, 1, "seekTestTopic");
kafka.afterPropertiesSet();
Map<String, Object> producerProps = KafkaTestUtils.producerProps(kafka);
KafkaProducer<Integer, String> producer = new KafkaProducer<>(producerProps);
producer.send(new ProducerRecord<>("seekTestTopic", 0, 1, "beforeSeekToEnd"));
Map<String, Object> consumerProps = KafkaTestUtils.consumerProps("seekTest", "false", kafka);
KafkaConsumer<Integer, String> consumer = new KafkaConsumer<>(consumerProps);
kafka.consumeFromAnEmbeddedTopic(consumer, true /* seekToEnd */, "seekTestTopic");
producer.send(new ProducerRecord<>("seekTestTopic", 0, 1, "afterSeekToEnd"));
producer.close();
assertThat(KafkaTestUtils.getSingleRecord(consumer, "seekTestTopic").value())
.isEqualTo("afterSeekToEnd");
consumer.close();
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,20 @@

package org.springframework.kafka.test;

import java.util.Map;

import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.junit.jupiter.api.Test;

import org.springframework.kafka.test.utils.KafkaTestUtils;

import static org.assertj.core.api.Assertions.assertThat;

/**
* @author Gary Russell
* @author Wouter Coekaerts
* @since 2.3
*
*/
Expand All @@ -42,4 +50,22 @@ void testUpDown() {
assertThat(System.getProperty(EmbeddedKafkaBroker.SPRING_EMBEDDED_KAFKA_BROKERS)).isNull();
}

@Test
void testConsumeFromEmbeddedWithSeekToEnd() {
EmbeddedKafkaZKBroker kafka = new EmbeddedKafkaZKBroker(1);
kafka.afterPropertiesSet();
kafka.addTopics("seekTestTopic");
Map<String, Object> producerProps = KafkaTestUtils.producerProps(kafka);
KafkaProducer<Integer, String> producer = new KafkaProducer<>(producerProps);
producer.send(new ProducerRecord<>("seekTestTopic", 0, 1, "beforeSeekToEnd"));
Map<String, Object> consumerProps = KafkaTestUtils.consumerProps("seekTest", "false", kafka);
KafkaConsumer<Integer, String> consumer = new KafkaConsumer<>(consumerProps);
kafka.consumeFromAnEmbeddedTopic(consumer, true /* seekToEnd */, "seekTestTopic");
producer.send(new ProducerRecord<>("seekTestTopic", 0, 1, "afterSeekToEnd"));
producer.close();
assertThat(KafkaTestUtils.getSingleRecord(consumer, "seekTestTopic").value())
.isEqualTo("afterSeekToEnd");
consumer.close();
}

}

0 comments on commit 71b571c

Please sign in to comment.