Die Implementierung speichert Zustand & Offsets vor _jedem_ `poll()`
[demos/kafka/training] / src / test / java / de / juplo / kafka / GenericApplicationTests.java
1 package de.juplo.kafka;
2
3 import com.mongodb.client.MongoClient;
4 import lombok.extern.slf4j.Slf4j;
5 import org.apache.kafka.clients.consumer.ConsumerRecord;
6 import org.apache.kafka.clients.consumer.KafkaConsumer;
7 import org.apache.kafka.clients.producer.KafkaProducer;
8 import org.apache.kafka.clients.producer.ProducerRecord;
9 import org.apache.kafka.common.TopicPartition;
10 import org.apache.kafka.common.errors.RecordDeserializationException;
11 import org.apache.kafka.common.serialization.*;
12 import org.apache.kafka.common.utils.Bytes;
13 import org.junit.jupiter.api.*;
14 import org.springframework.beans.factory.annotation.Autowired;
15 import org.springframework.boot.autoconfigure.EnableAutoConfiguration;
16 import org.springframework.boot.autoconfigure.mongo.MongoProperties;
17 import org.springframework.boot.test.autoconfigure.data.mongo.AutoConfigureDataMongo;
18 import org.springframework.boot.test.context.ConfigDataApplicationContextInitializer;
19 import org.springframework.boot.test.context.TestConfiguration;
20 import org.springframework.context.annotation.Import;
21 import org.springframework.kafka.test.context.EmbeddedKafka;
22 import org.springframework.test.context.TestPropertySource;
23 import org.springframework.test.context.junit.jupiter.SpringJUnitConfig;
24
25 import java.time.Duration;
26 import java.util.*;
27 import java.util.concurrent.ExecutorService;
28 import java.util.function.BiConsumer;
29 import java.util.function.Consumer;
30 import java.util.stream.Collectors;
31 import java.util.stream.IntStream;
32
33 import static de.juplo.kafka.GenericApplicationTests.PARTITIONS;
34 import static de.juplo.kafka.GenericApplicationTests.TOPIC;
35 import static org.assertj.core.api.Assertions.*;
36 import static org.awaitility.Awaitility.*;
37
38
39 @SpringJUnitConfig(initializers = ConfigDataApplicationContextInitializer.class)
40 @TestPropertySource(
41                 properties = {
42                                 "sumup.adder.bootstrap-server=${spring.embedded.kafka.brokers}",
43                                 "sumup.adder.topic=" + TOPIC,
44                                 "sumup.adder.commit-interval=500ms",
45                                 "spring.mongodb.embedded.version=4.4.13" })
46 @EmbeddedKafka(topics = TOPIC, partitions = PARTITIONS)
47 @EnableAutoConfiguration
48 @AutoConfigureDataMongo
49 @Slf4j
50 abstract class GenericApplicationTests<K, V>
51 {
52         public static final String TOPIC = "FOO";
53         public static final int PARTITIONS = 10;
54
55
56         @Autowired
57         KafkaConsumer<K, V> kafkaConsumer;
58         @Autowired
59         Consumer<ConsumerRecord<K, V>> consumer;
60         @Autowired
61         ApplicationProperties properties;
62         @Autowired
63         ExecutorService executor;
64         @Autowired
65         MongoClient mongoClient;
66         @Autowired
67         MongoProperties mongoProperties;
68         @Autowired
69         RebalanceListener rebalanceListener;
70         @Autowired
71         RecordHandler<K, V> recordHandler;
72
73         KafkaProducer<Bytes, Bytes> testRecordProducer;
74         KafkaConsumer<Bytes, Bytes> offsetConsumer;
75         EndlessConsumer<K, V> endlessConsumer;
76         Map<TopicPartition, Long> oldOffsets;
77         Map<TopicPartition, Long> seenOffsets;
78         Set<ConsumerRecord<K, V>> receivedRecords;
79
80
81         final RecordGenerator recordGenerator;
82         final Consumer<ProducerRecord<Bytes, Bytes>> messageSender;
83
84         public GenericApplicationTests(RecordGenerator recordGenerator)
85         {
86                 this.recordGenerator = recordGenerator;
87                 this.messageSender = (record) -> sendMessage(record);
88         }
89
90
91         /** Tests methods */
92
93         @Test
94         void commitsCurrentOffsetsOnSuccess() throws Exception
95         {
96                 int numberOfGeneratedMessages =
97                                 recordGenerator.generate(false, false, messageSender);
98
99                 await(numberOfGeneratedMessages + " records received")
100                                 .atMost(Duration.ofSeconds(30))
101                                 .pollInterval(Duration.ofSeconds(1))
102                                 .until(() -> receivedRecords.size() >= numberOfGeneratedMessages);
103
104                 await("Offsets committed")
105                                 .atMost(Duration.ofSeconds(10))
106                                 .pollInterval(Duration.ofSeconds(1))
107                                 .untilAsserted(() ->
108                                 {
109                                         checkSeenOffsetsForProgress();
110                                         assertSeenOffsetsEqualCommittedOffsets(seenOffsets);
111                                 });
112
113                 assertThatExceptionOfType(IllegalStateException.class)
114                                 .isThrownBy(() -> endlessConsumer.exitStatus())
115                                 .describedAs("Consumer should still be running");
116
117                 endlessConsumer.stop();
118                 recordGenerator.assertBusinessLogic();
119         }
120
121         @Test
122         @SkipWhenErrorCannotBeGenerated(poisonPill = true)
123         void commitsOffsetOfErrorForReprocessingOnDeserializationError()
124         {
125                 int numberOfGeneratedMessages =
126                                 recordGenerator.generate(true, false, messageSender);
127
128                 await("Consumer failed")
129                                 .atMost(Duration.ofSeconds(30))
130                                 .pollInterval(Duration.ofSeconds(1))
131                                 .until(() -> !endlessConsumer.running());
132
133                 checkSeenOffsetsForProgress();
134                 assertSeenOffsetsEqualCommittedOffsets(seenOffsets);
135
136                 endlessConsumer.start();
137                 await("Consumer failed")
138                                 .atMost(Duration.ofSeconds(30))
139                                 .pollInterval(Duration.ofSeconds(1))
140                                 .until(() -> !endlessConsumer.running());
141
142                 checkSeenOffsetsForProgress();
143                 assertSeenOffsetsEqualCommittedOffsets(seenOffsets);
144                 assertThat(receivedRecords.size())
145                                 .describedAs("Received not all sent events")
146                                 .isLessThan(numberOfGeneratedMessages);
147
148                 assertThatNoException()
149                                 .describedAs("Consumer should not be running")
150                                 .isThrownBy(() -> endlessConsumer.exitStatus());
151                 assertThat(endlessConsumer.exitStatus())
152                                 .describedAs("Consumer should have exited abnormally")
153                                 .containsInstanceOf(RecordDeserializationException.class);
154
155                 recordGenerator.assertBusinessLogic();
156         }
157
158         @Test
159         @SkipWhenErrorCannotBeGenerated(logicError = true)
160         void doesNotCommitOffsetsOnLogicError()
161         {
162                 int numberOfGeneratedMessages =
163                                 recordGenerator.generate(false, true, messageSender);
164
165                 await("Consumer failed")
166                                 .atMost(Duration.ofSeconds(30))
167                                 .pollInterval(Duration.ofSeconds(1))
168                                 .until(() -> !endlessConsumer.running());
169
170                 checkSeenOffsetsForProgress();
171                 assertSeenOffsetsAreBehindCommittedOffsets(seenOffsets);
172
173                 endlessConsumer.start();
174                 await("Consumer failed")
175                                 .atMost(Duration.ofSeconds(30))
176                                 .pollInterval(Duration.ofSeconds(1))
177                                 .until(() -> !endlessConsumer.running());
178
179                 assertSeenOffsetsAreBehindCommittedOffsets(seenOffsets);
180
181                 assertThatNoException()
182                                 .describedAs("Consumer should not be running")
183                                 .isThrownBy(() -> endlessConsumer.exitStatus());
184                 assertThat(endlessConsumer.exitStatus())
185                                 .describedAs("Consumer should have exited abnormally")
186                                 .containsInstanceOf(RuntimeException.class);
187
188                 recordGenerator.assertBusinessLogic();
189         }
190
191
192         /** Helper methods for the verification of expectations */
193
194         void assertSeenOffsetsEqualCommittedOffsets(Map<TopicPartition, Long> offsetsToCheck)
195         {
196                 doForCurrentOffsets((tp, offset) ->
197                 {
198                         Long expected = offsetsToCheck.get(tp) + 1;
199                         log.debug("Checking, if the offset {} for {} is exactly {}", offset, tp, expected);
200                         assertThat(offset)
201                                         .describedAs("Committed offset corresponds to the offset of the consumer")
202                                         .isEqualTo(expected);
203                 });
204         }
205
206         void assertSeenOffsetsAreBehindCommittedOffsets(Map<TopicPartition, Long> offsetsToCheck)
207         {
208                 List<Boolean> isOffsetBehindSeen = new LinkedList<>();
209
210                 doForCurrentOffsets((tp, offset) ->
211                 {
212                         Long expected = offsetsToCheck.get(tp) + 1;
213                         log.debug("Checking, if the offset {} for {} is at most {}", offset, tp, expected);
214                         assertThat(offset)
215                                         .describedAs("Committed offset must be at most equal to the offset of the consumer")
216                                         .isLessThanOrEqualTo(expected);
217                         isOffsetBehindSeen.add(offset < expected);
218                 });
219
220                 assertThat(isOffsetBehindSeen.stream().reduce(false, (result, next) -> result | next))
221                                 .describedAs("Committed offsets are behind seen offsets")
222                                 .isTrue();
223         }
224
225         void checkSeenOffsetsForProgress()
226         {
227                 // Be sure, that some messages were consumed...!
228                 Set<TopicPartition> withProgress = new HashSet<>();
229                 partitions().forEach(tp ->
230                 {
231                         Long oldOffset = oldOffsets.get(tp) + 1;
232                         Long newOffset = seenOffsets.get(tp) + 1;
233                         if (!oldOffset.equals(newOffset))
234                         {
235                                 log.debug("Progress for {}: {} -> {}", tp, oldOffset, newOffset);
236                                 withProgress.add(tp);
237                         }
238                 });
239                 assertThat(withProgress)
240                                 .describedAs("Some offsets must have changed, compared to the old offset-positions")
241                                 .isNotEmpty();
242         }
243
244
245         /** Helper methods for setting up and running the tests */
246
247         void seekToEnd()
248         {
249                 offsetConsumer.assign(partitions());
250                 offsetConsumer.seekToEnd(partitions());
251                 partitions().forEach(tp ->
252                 {
253                         // seekToEnd() works lazily: it only takes effect on poll()/position()
254                         Long offset = offsetConsumer.position(tp);
255                         log.info("New position for {}: {}", tp, offset);
256                 });
257                 // The new positions must be commited!
258                 offsetConsumer.commitSync();
259                 offsetConsumer.unsubscribe();
260         }
261
262         void doForCurrentOffsets(BiConsumer<TopicPartition, Long> consumer)
263         {
264                 offsetConsumer.assign(partitions());
265                 partitions().forEach(tp -> consumer.accept(tp, offsetConsumer.position(tp)));
266                 offsetConsumer.unsubscribe();
267         }
268
269         List<TopicPartition> partitions()
270         {
271                 return
272                                 IntStream
273                                                 .range(0, PARTITIONS)
274                                                 .mapToObj(partition -> new TopicPartition(TOPIC, partition))
275                                                 .collect(Collectors.toList());
276         }
277
278
279         public interface RecordGenerator
280         {
281                 int generate(
282                                 boolean poisonPills,
283                                 boolean logicErrors,
284                                 Consumer<ProducerRecord<Bytes, Bytes>> messageSender);
285
286                 default boolean canGeneratePoisonPill()
287                 {
288                         return true;
289                 }
290
291                 default boolean canGenerateLogicError()
292                 {
293                         return true;
294                 }
295
296                 default void assertBusinessLogic()
297                 {
298                         log.debug("No business-logic to assert");
299                 }
300         }
301
302         void sendMessage(ProducerRecord<Bytes, Bytes> record)
303         {
304                 testRecordProducer.send(record, (metadata, e) ->
305                 {
306                         if (metadata != null)
307                         {
308                                 log.debug(
309                                                 "{}|{} - {}={}",
310                                                 metadata.partition(),
311                                                 metadata.offset(),
312                                                 record.key(),
313                                                 record.value());
314                         }
315                         else
316                         {
317                                 log.warn(
318                                                 "Exception for {}={}: {}",
319                                                 record.key(),
320                                                 record.value(),
321                                                 e.toString());
322                         }
323                 });
324         }
325
326
327         @BeforeEach
328         public void init()
329         {
330                 Properties props;
331                 props = new Properties();
332                 props.put("bootstrap.servers", properties.getBootstrapServer());
333                 props.put("linger.ms", 100);
334                 props.put("key.serializer", BytesSerializer.class.getName());
335                 props.put("value.serializer", BytesSerializer.class.getName());
336                 testRecordProducer = new KafkaProducer<>(props);
337
338                 props = new Properties();
339                 props.put("bootstrap.servers", properties.getBootstrapServer());
340                 props.put("client.id", "OFFSET-CONSUMER");
341                 props.put("group.id", properties.getGroupId());
342                 props.put("key.deserializer", BytesDeserializer.class.getName());
343                 props.put("value.deserializer", BytesDeserializer.class.getName());
344                 offsetConsumer = new KafkaConsumer<>(props);
345
346                 mongoClient.getDatabase(mongoProperties.getDatabase()).drop();
347                 seekToEnd();
348
349                 oldOffsets = new HashMap<>();
350                 seenOffsets = new HashMap<>();
351                 receivedRecords = new HashSet<>();
352
353                 doForCurrentOffsets((tp, offset) ->
354                 {
355                         oldOffsets.put(tp, offset - 1);
356                         seenOffsets.put(tp, offset - 1);
357                 });
358
359                 TestRecordHandler<K, V> captureOffsetAndExecuteTestHandler =
360                                 new TestRecordHandler<K, V>(recordHandler)
361                                 {
362                                         @Override
363                                         public void onNewRecord(ConsumerRecord<K, V> record)
364                                         {
365                                                 seenOffsets.put(
366                                                                 new TopicPartition(record.topic(), record.partition()),
367                                                                 record.offset());
368                                                 receivedRecords.add(record);
369                                         }
370                                 };
371
372                 endlessConsumer =
373                                 new EndlessConsumer<>(
374                                                 executor,
375                                                 properties.getClientId(),
376                                                 properties.getTopic(),
377                                                 kafkaConsumer,
378                                                 rebalanceListener,
379                                                 captureOffsetAndExecuteTestHandler);
380
381                 endlessConsumer.start();
382         }
383
384         @AfterEach
385         public void deinit()
386         {
387                 try
388                 {
389                         testRecordProducer.close();
390                         offsetConsumer.close();
391                 }
392                 catch (Exception e)
393                 {
394                         log.info("Exception while stopping the consumer: {}", e.toString());
395                 }
396         }
397
398
399         @TestConfiguration
400         @Import(ApplicationConfiguration.class)
401         public static class Configuration
402         {
403         }
404 }