From: Kai Moritz Date: Sat, 13 Aug 2022 08:48:49 +0000 (+0200) Subject: Verbesserungen aus 'stored-state' nach 'rebalance-listener' gemerged X-Git-Tag: endless-stream-consumer-DEPRECATED^2^2 X-Git-Url: http://juplo.de/gitweb/?a=commitdiff_plain;h=de0468e4db973312e61ad4894edc092e84655161;p=demos%2Fkafka%2Ftraining Verbesserungen aus 'stored-state' nach 'rebalance-listener' gemerged --- de0468e4db973312e61ad4894edc092e84655161 diff --cc docker-compose.yml index 1b067cd,7bcf68c..df6b321 --- a/docker-compose.yml +++ b/docker-compose.yml @@@ -55,6 -73,6 +54,4 @@@ services environment: server.port: 8080 consumer.bootstrap-server: kafka:9092 - consumer.client-id: my-group consumer.client-id: consumer - consumer.topic: test - spring.data.mongodb.uri: mongodb://juplo:training@mongo:27017 - spring.data.mongodb.database: juplo diff --cc pom.xml index 9db9d9d,701704d..1f5caab --- a/pom.xml +++ b/pom.xml @@@ -47,6 -51,21 +47,16 @@@ spring-boot-starter-test test + + org.springframework.kafka + spring-kafka-test + test + + + org.awaitility + awaitility + test + - - de.flapdoodle.embed - de.flapdoodle.embed.mongo - test - diff --cc src/main/java/de/juplo/kafka/ApplicationConfiguration.java index 0000000,1ea90a2..7a0a8ad mode 000000,100644..100644 --- a/src/main/java/de/juplo/kafka/ApplicationConfiguration.java +++ b/src/main/java/de/juplo/kafka/ApplicationConfiguration.java @@@ -1,0 -1,81 +1,76 @@@ + package de.juplo.kafka; + + import org.apache.kafka.clients.consumer.KafkaConsumer; + import org.apache.kafka.common.serialization.LongDeserializer; + import org.apache.kafka.common.serialization.StringDeserializer; + import org.springframework.boot.context.properties.EnableConfigurationProperties; + import org.springframework.context.annotation.Bean; + import org.springframework.context.annotation.Configuration; + -import java.time.Clock; + import java.util.Properties; + import java.util.concurrent.ExecutorService; + import java.util.concurrent.Executors; + + + @Configuration + @EnableConfigurationProperties(ApplicationProperties.class) + public class ApplicationConfiguration + { + @Bean + public KeyCountingRecordHandler keyCountingRecordHandler() + { + return new KeyCountingRecordHandler(); + } + + @Bean + public KeyCountingRebalanceListener keyCountingRebalanceListener( + KeyCountingRecordHandler keyCountingRecordHandler, - PartitionStatisticsRepository repository, + ApplicationProperties properties) + { + return new KeyCountingRebalanceListener( + keyCountingRecordHandler, - repository, - properties.getClientId(), - Clock.systemDefaultZone(), - properties.getCommitInterval()); ++ properties.getClientId()); + } + + @Bean + public EndlessConsumer endlessConsumer( + KafkaConsumer kafkaConsumer, + ExecutorService executor, + KeyCountingRebalanceListener keyCountingRebalanceListener, + KeyCountingRecordHandler keyCountingRecordHandler, + ApplicationProperties properties) + { + return + new EndlessConsumer<>( + executor, + properties.getClientId(), + properties.getTopic(), + kafkaConsumer, + keyCountingRebalanceListener, + keyCountingRecordHandler); + } + + @Bean + public ExecutorService executor() + { + return Executors.newSingleThreadExecutor(); + } + + @Bean(destroyMethod = "close") + public KafkaConsumer kafkaConsumer(ApplicationProperties properties) + { + Properties props = new Properties(); + + props.put("bootstrap.servers", properties.getBootstrapServer()); + props.put("partition.assignment.strategy", "org.apache.kafka.clients.consumer.CooperativeStickyAssignor"); + props.put("group.id", properties.getGroupId()); + props.put("client.id", properties.getClientId()); + props.put("auto.offset.reset", properties.getAutoOffsetReset()); + props.put("auto.commit.interval.ms", (int)properties.getCommitInterval().toMillis()); + props.put("metadata.max.age.ms", "1000"); + props.put("key.deserializer", StringDeserializer.class.getName()); + props.put("value.deserializer", LongDeserializer.class.getName()); + + return new KafkaConsumer<>(props); + } + } diff --cc src/main/java/de/juplo/kafka/EndlessConsumer.java index c2d4447,047d5cb..c7579b8 --- a/src/main/java/de/juplo/kafka/EndlessConsumer.java +++ b/src/main/java/de/juplo/kafka/EndlessConsumer.java @@@ -20,14 -18,15 +18,15 @@@ import java.util.concurrent.locks.Reent @Slf4j - public class EndlessConsumer implements Runnable + @RequiredArgsConstructor + public class EndlessConsumer implements Runnable { private final ExecutorService executor; - private final String bootstrapServer; - private final String groupId; private final String id; private final String topic; - private final String autoOffsetReset; + private final Consumer consumer; - private final PollIntervalAwareConsumerRebalanceListener pollIntervalAwareRebalanceListener; ++ private final ConsumerRebalanceListener consumerRebalanceListener; + private final RecordHandler handler; private final Lock lock = new ReentrantLock(); private final Condition condition = lock.newCondition(); @@@ -61,49 -41,8 +41,8 @@@ { try { - Properties props = new Properties(); - props.put("bootstrap.servers", bootstrapServer); - props.put("group.id", groupId); - props.put("client.id", id); - props.put("auto.offset.reset", autoOffsetReset); - props.put("metadata.max.age.ms", "1000"); - props.put("key.deserializer", StringDeserializer.class.getName()); - props.put("value.deserializer", StringDeserializer.class.getName()); - - this.consumer = new KafkaConsumer<>(props); - log.info("{} - Subscribing to topic {}", id, topic); - consumer.subscribe(Arrays.asList(topic), new ConsumerRebalanceListener() - { - @Override - public void onPartitionsRevoked(Collection partitions) - { - partitions.forEach(tp -> - { - log.info("{} - removing partition: {}", id, tp); - Map removed = seen.remove(tp.partition()); - for (String key : removed.keySet()) - { - log.info( - "{} - Seen {} messages for partition={}|key={}", - id, - removed.get(key), - tp.partition(), - key); - } - }); - } - - @Override - public void onPartitionsAssigned(Collection partitions) - { - partitions.forEach(tp -> - { - log.info("{} - adding partition: {}", id, tp); - seen.put(tp.partition(), new HashMap<>()); - }); - } - }); - consumer.subscribe(Arrays.asList(topic), pollIntervalAwareRebalanceListener); ++ consumer.subscribe(Arrays.asList(topic), consumerRebalanceListener); while (true) { @@@ -125,17 -63,12 +63,10 @@@ record.value() ); - Integer partition = record.partition(); - String key = record.key() == null ? "NULL" : record.key(); - Map byKey = seen.get(partition); + handler.accept(record); - if (!byKey.containsKey(key)) - byKey.put(key, 0); - - int seenByKey = byKey.get(key); - seenByKey++; - byKey.put(key, seenByKey); + consumed++; } - - pollIntervalAwareRebalanceListener.beforeNextPoll(); } } catch(WakeupException e) diff --cc src/main/java/de/juplo/kafka/KeyCountingRebalanceListener.java index 0000000,636ff86..0ad1f31 mode 000000,100644..100644 --- a/src/main/java/de/juplo/kafka/KeyCountingRebalanceListener.java +++ b/src/main/java/de/juplo/kafka/KeyCountingRebalanceListener.java @@@ -1,0 -1,76 +1,50 @@@ + package de.juplo.kafka; + + import lombok.RequiredArgsConstructor; + import lombok.extern.slf4j.Slf4j; ++import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; + import org.apache.kafka.common.TopicPartition; + -import java.time.Clock; -import java.time.Duration; -import java.time.Instant; + import java.util.Collection; ++import java.util.HashMap; + import java.util.Map; + + + @RequiredArgsConstructor + @Slf4j -public class KeyCountingRebalanceListener implements PollIntervalAwareConsumerRebalanceListener ++public class KeyCountingRebalanceListener implements ConsumerRebalanceListener + { + private final KeyCountingRecordHandler handler; - private final PartitionStatisticsRepository repository; + private final String id; - private final Clock clock; - private final Duration commitInterval; - - private Instant lastCommit = Instant.EPOCH; + + @Override + public void onPartitionsAssigned(Collection partitions) + { + partitions.forEach(tp -> + { + Integer partition = tp.partition(); + log.info("{} - adding partition: {}", id, partition); - StatisticsDocument document = - repository - .findById(Integer.toString(partition)) - .orElse(new StatisticsDocument(partition)); - handler.addPartition(partition, document.statistics); ++ handler.addPartition(partition, new HashMap<>()); + }); + } + + @Override + public void onPartitionsRevoked(Collection partitions) + { + partitions.forEach(tp -> + { + Integer partition = tp.partition(); + log.info("{} - removing partition: {}", id, partition); + Map removed = handler.removePartition(partition); + for (String key : removed.keySet()) + { + log.info( + "{} - Seen {} messages for partition={}|key={}", + id, + removed.get(key), + partition, + key); + } - repository.save(new StatisticsDocument(partition, removed)); + }); + } - - - @Override - public void beforeNextPoll() - { - if (lastCommit.plus(commitInterval).isBefore(clock.instant())) - { - log.debug("Storing data, last commit: {}", lastCommit); - handler.getSeen().forEach((partiton, statistics) -> repository.save( - new StatisticsDocument( - partiton, - statistics))); - lastCommit = clock.instant(); - } - } + } diff --cc src/main/java/de/juplo/kafka/RecordHandler.java index 0000000,3c9dd15..327ac9f mode 000000,100644..100644 --- a/src/main/java/de/juplo/kafka/RecordHandler.java +++ b/src/main/java/de/juplo/kafka/RecordHandler.java @@@ -1,0 -1,11 +1,10 @@@ + package de.juplo.kafka; + + import org.apache.kafka.clients.consumer.ConsumerRecord; + + import java.util.function.Consumer; + + + public interface RecordHandler extends Consumer> + { - default void beforeNextPoll() {} + } diff --cc src/test/java/de/juplo/kafka/ApplicationIT.java index 0000000,d1d8e50..2e6ac7d mode 000000,100644..100644 --- a/src/test/java/de/juplo/kafka/ApplicationIT.java +++ b/src/test/java/de/juplo/kafka/ApplicationIT.java @@@ -1,0 -1,43 +1,41 @@@ + package de.juplo.kafka; + + import org.junit.jupiter.api.Test; + import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.boot.test.autoconfigure.data.mongo.AutoConfigureDataMongo; + import org.springframework.boot.test.context.SpringBootTest; + import org.springframework.boot.test.web.client.TestRestTemplate; + import org.springframework.boot.test.web.server.LocalServerPort; + import org.springframework.kafka.test.context.EmbeddedKafka; + + import static de.juplo.kafka.ApplicationTests.TOPIC; + + + @SpringBootTest( + webEnvironment = SpringBootTest.WebEnvironment.RANDOM_PORT, + properties = { + "consumer.bootstrap-server=${spring.embedded.kafka.brokers}", + "consumer.topic=" + TOPIC, + "spring.mongodb.embedded.version=4.4.13" }) + @EmbeddedKafka(topics = TOPIC) -@AutoConfigureDataMongo + public class ApplicationIT + { + public static final String TOPIC = "FOO"; + + @LocalServerPort + private int port; + + @Autowired + private TestRestTemplate restTemplate; + + + + @Test + public void testApplicationStartup() + { + restTemplate.getForObject( + "http://localhost:" + port + "/actuator/health", + String.class + ) + .contains("UP"); + } + } diff --cc src/test/java/de/juplo/kafka/ApplicationTests.java index 0000000,7f666f6..5b13b7d mode 000000,100644..100644 --- a/src/test/java/de/juplo/kafka/ApplicationTests.java +++ b/src/test/java/de/juplo/kafka/ApplicationTests.java @@@ -1,0 -1,348 +1,345 @@@ + package de.juplo.kafka; + + import lombok.extern.slf4j.Slf4j; + import org.apache.kafka.clients.consumer.ConsumerRecord; + import org.apache.kafka.clients.consumer.KafkaConsumer; + import org.apache.kafka.clients.producer.KafkaProducer; + import org.apache.kafka.clients.producer.ProducerRecord; + import org.apache.kafka.common.TopicPartition; + import org.apache.kafka.common.errors.RecordDeserializationException; + import org.apache.kafka.common.serialization.*; + import org.apache.kafka.common.utils.Bytes; + import org.junit.jupiter.api.*; + import org.springframework.beans.factory.annotation.Autowired; + import org.springframework.boot.autoconfigure.EnableAutoConfiguration; -import org.springframework.boot.test.autoconfigure.data.mongo.AutoConfigureDataMongo; + import org.springframework.boot.test.context.ConfigDataApplicationContextInitializer; + import org.springframework.boot.test.context.TestConfiguration; + import org.springframework.context.annotation.Bean; + import org.springframework.context.annotation.Import; + import org.springframework.kafka.test.context.EmbeddedKafka; + import org.springframework.test.context.TestPropertySource; + import org.springframework.test.context.junit.jupiter.SpringJUnitConfig; + + import java.time.Duration; + import java.util.*; + import java.util.concurrent.ExecutionException; + import java.util.concurrent.ExecutorService; + import java.util.function.BiConsumer; + import java.util.stream.Collectors; + import java.util.stream.IntStream; + + import static de.juplo.kafka.ApplicationTests.PARTITIONS; + import static de.juplo.kafka.ApplicationTests.TOPIC; + import static org.assertj.core.api.Assertions.*; + import static org.awaitility.Awaitility.*; + + + @SpringJUnitConfig(initializers = ConfigDataApplicationContextInitializer.class) + @TestMethodOrder(MethodOrderer.OrderAnnotation.class) + @TestPropertySource( + properties = { + "consumer.bootstrap-server=${spring.embedded.kafka.brokers}", + "consumer.topic=" + TOPIC, - "consumer.commit-interval=1s", - "spring.mongodb.embedded.version=4.4.13" }) ++ "consumer.commit-interval=1s" }) + @EmbeddedKafka(topics = TOPIC, partitions = PARTITIONS) + @EnableAutoConfiguration -@AutoConfigureDataMongo + @Slf4j + class ApplicationTests + { + public static final String TOPIC = "FOO"; + public static final int PARTITIONS = 10; + + + StringSerializer stringSerializer = new StringSerializer(); + + @Autowired + Serializer valueSerializer; + @Autowired + KafkaProducer kafkaProducer; + @Autowired + KafkaConsumer kafkaConsumer; + @Autowired + KafkaConsumer offsetConsumer; + @Autowired + ApplicationProperties properties; + @Autowired + ExecutorService executor; + @Autowired + KeyCountingRebalanceListener keyCountingRebalanceListener; + @Autowired + KeyCountingRecordHandler keyCountingRecordHandler; + + EndlessConsumer endlessConsumer; + Map oldOffsets; + Map newOffsets; + Set> receivedRecords; + + + /** Tests methods */ + + @Test + void commitsCurrentOffsetsOnSuccess() throws ExecutionException, InterruptedException + { + send100Messages((partition, key, counter) -> + { + Bytes value = new Bytes(valueSerializer.serialize(TOPIC, counter)); + return new ProducerRecord<>(TOPIC, partition, key, value); + }); + + await("100 records received") + .atMost(Duration.ofSeconds(30)) + .pollInterval(Duration.ofSeconds(1)) + .until(() -> receivedRecords.size() >= 100); + + await("Offsets committed") + .atMost(Duration.ofSeconds(10)) + .pollInterval(Duration.ofSeconds(1)) + .untilAsserted(() -> + { + checkSeenOffsetsForProgress(); + compareToCommitedOffsets(newOffsets); + }); + + assertThatExceptionOfType(IllegalStateException.class) + .isThrownBy(() -> endlessConsumer.exitStatus()) + .describedAs("Consumer should still be running"); + } + + @Test + void commitsOffsetOfErrorForReprocessingOnDeserializationError() + { + send100Messages((partition, key, counter) -> + { + Bytes value = counter == 77 + ? new Bytes(stringSerializer.serialize(TOPIC, "BOOM!")) + : new Bytes(valueSerializer.serialize(TOPIC, counter)); + return new ProducerRecord<>(TOPIC, partition, key, value); + }); + + await("Consumer failed") + .atMost(Duration.ofSeconds(30)) + .pollInterval(Duration.ofSeconds(1)) + .until(() -> !endlessConsumer.running()); + + checkSeenOffsetsForProgress(); + compareToCommitedOffsets(newOffsets); + + endlessConsumer.start(); + await("Consumer failed") + .atMost(Duration.ofSeconds(30)) + .pollInterval(Duration.ofSeconds(1)) + .until(() -> !endlessConsumer.running()); + + checkSeenOffsetsForProgress(); + compareToCommitedOffsets(newOffsets); + assertThat(receivedRecords.size()) + .describedAs("Received not all sent events") + .isLessThan(100); + + assertThatNoException() + .describedAs("Consumer should not be running") + .isThrownBy(() -> endlessConsumer.exitStatus()); + assertThat(endlessConsumer.exitStatus()) + .describedAs("Consumer should have exited abnormally") + .containsInstanceOf(RecordDeserializationException.class); + } + + + /** Helper methods for the verification of expectations */ + + void compareToCommitedOffsets(Map offsetsToCheck) + { + doForCurrentOffsets((tp, offset) -> + { + Long expected = offsetsToCheck.get(tp) + 1; + log.debug("Checking, if the offset for {} is {}", tp, expected); + assertThat(offset) + .describedAs("Committed offset corresponds to the offset of the consumer") + .isEqualTo(expected); + }); + } + + void checkSeenOffsetsForProgress() + { + // Be sure, that some messages were consumed...! + Set withProgress = new HashSet<>(); + partitions().forEach(tp -> + { + Long oldOffset = oldOffsets.get(tp) + 1; + Long newOffset = newOffsets.get(tp) + 1; + if (!oldOffset.equals(newOffset)) + { + log.debug("Progress for {}: {} -> {}", tp, oldOffset, newOffset); + withProgress.add(tp); + } + }); + assertThat(withProgress) + .describedAs("Some offsets must have changed, compared to the old offset-positions") + .isNotEmpty(); + } + + + /** Helper methods for setting up and running the tests */ + + void seekToEnd() + { + offsetConsumer.assign(partitions()); + offsetConsumer.seekToEnd(partitions()); + partitions().forEach(tp -> + { + // seekToEnd() works lazily: it only takes effect on poll()/position() + Long offset = offsetConsumer.position(tp); + log.info("New position for {}: {}", tp, offset); + }); + // The new positions must be commited! + offsetConsumer.commitSync(); + offsetConsumer.unsubscribe(); + } + + void doForCurrentOffsets(BiConsumer consumer) + { + offsetConsumer.assign(partitions()); + partitions().forEach(tp -> consumer.accept(tp, offsetConsumer.position(tp))); + offsetConsumer.unsubscribe(); + } + + List partitions() + { + return + IntStream + .range(0, PARTITIONS) + .mapToObj(partition -> new TopicPartition(TOPIC, partition)) + .collect(Collectors.toList()); + } + + + public interface RecordGenerator + { + public ProducerRecord generate(int partition, String key, long counter); + } + + void send100Messages(RecordGenerator recordGenerator) + { + long i = 0; + + for (int partition = 0; partition < 10; partition++) + { + for (int key = 0; key < 10; key++) + { + ProducerRecord record = + recordGenerator.generate(partition, Integer.toString(partition*10+key%2), ++i); + + kafkaProducer.send(record, (metadata, e) -> + { + if (metadata != null) + { + log.debug( + "{}|{} - {}={}", + metadata.partition(), + metadata.offset(), + record.key(), + record.value()); + } + else + { + log.warn( + "Exception for {}={}: {}", + record.key(), + record.value(), + e.toString()); + } + }); + } + } + } + + + @BeforeEach + public void init() + { + seekToEnd(); + + oldOffsets = new HashMap<>(); + newOffsets = new HashMap<>(); + receivedRecords = new HashSet<>(); + + doForCurrentOffsets((tp, offset) -> + { + oldOffsets.put(tp, offset - 1); + newOffsets.put(tp, offset - 1); + }); + + TestRecordHandler captureOffsetAndExecuteTestHandler = + new TestRecordHandler(keyCountingRecordHandler) { + @Override + public void onNewRecord(ConsumerRecord record) + { + newOffsets.put( + new TopicPartition(record.topic(), record.partition()), + record.offset()); + receivedRecords.add(record); + } + }; + + endlessConsumer = + new EndlessConsumer<>( + executor, + properties.getClientId(), + properties.getTopic(), + kafkaConsumer, + keyCountingRebalanceListener, + captureOffsetAndExecuteTestHandler); + + endlessConsumer.start(); + } + + @AfterEach + public void deinit() + { + try + { + endlessConsumer.stop(); + } + catch (Exception e) + { + log.info("Exception while stopping the consumer: {}", e.toString()); + } + } + + + @TestConfiguration + @Import(ApplicationConfiguration.class) + public static class Configuration + { + @Bean + Serializer serializer() + { + return new LongSerializer(); + } + + @Bean + KafkaProducer kafkaProducer(ApplicationProperties properties) + { + Properties props = new Properties(); + props.put("bootstrap.servers", properties.getBootstrapServer()); + props.put("linger.ms", 100); + props.put("key.serializer", StringSerializer.class.getName()); + props.put("value.serializer", BytesSerializer.class.getName()); + + return new KafkaProducer<>(props); + } + + @Bean + KafkaConsumer offsetConsumer(ApplicationProperties properties) + { + Properties props = new Properties(); + props.put("bootstrap.servers", properties.getBootstrapServer()); + props.put("client.id", "OFFSET-CONSUMER"); + props.put("group.id", properties.getGroupId()); + props.put("key.deserializer", BytesDeserializer.class.getName()); + props.put("value.deserializer", BytesDeserializer.class.getName()); + + return new KafkaConsumer<>(props); + } + } + } diff --cc src/test/java/de/juplo/kafka/TestRecordHandler.java index 0000000,de28385..b4efdd6 mode 000000,100644..100644 --- a/src/test/java/de/juplo/kafka/TestRecordHandler.java +++ b/src/test/java/de/juplo/kafka/TestRecordHandler.java @@@ -1,0 -1,28 +1,22 @@@ + package de.juplo.kafka; + + import lombok.RequiredArgsConstructor; + import org.apache.kafka.clients.consumer.ConsumerRecord; + + + @RequiredArgsConstructor + public abstract class TestRecordHandler implements RecordHandler + { + private final RecordHandler handler; + + + public abstract void onNewRecord(ConsumerRecord record); + + + @Override + public void accept(ConsumerRecord record) + { + this.onNewRecord(record); + handler.accept(record); + } - @Override - - public void beforeNextPoll() - { - handler.beforeNextPoll(); - } + }