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