counter: 1.4.2 - RocksDB does nor work in Alpine-Linux
[demos/kafka/wordcount] / src / test / java / de / juplo / kafka / wordcount / counter / CounterApplicationIT.java
index 0d09a76..ab395fd 100644 (file)
@@ -1,12 +1,14 @@
 package de.juplo.kafka.wordcount.counter;
 
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import lombok.RequiredArgsConstructor;
-import lombok.Value;
+import de.juplo.kafka.wordcount.splitter.TestInputUser;
+import de.juplo.kafka.wordcount.splitter.TestInputWord;
+import de.juplo.kafka.wordcount.top10.TestOutputWord;
+import de.juplo.kafka.wordcount.top10.TestOutputWordCounter;
 import lombok.extern.slf4j.Slf4j;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.junit.jupiter.api.BeforeEach;
+import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier;
+import org.apache.kafka.streams.state.Stores;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.DisplayName;
 import org.junit.jupiter.api.Test;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.boot.test.context.SpringBootTest;
@@ -14,138 +16,150 @@ import org.springframework.boot.test.context.TestConfiguration;
 import org.springframework.context.annotation.Bean;
 import org.springframework.kafka.annotation.KafkaListener;
 import org.springframework.kafka.core.KafkaTemplate;
+import org.springframework.kafka.support.KafkaHeaders;
+import org.springframework.kafka.support.SendResult;
 import org.springframework.kafka.test.context.EmbeddedKafka;
+import org.springframework.messaging.handler.annotation.Header;
+import org.springframework.messaging.handler.annotation.Payload;
 import org.springframework.util.LinkedMultiValueMap;
 import org.springframework.util.MultiValueMap;
 
 import java.time.Duration;
 
-import static de.juplo.kafka.wordcount.counter.CounterApplicationIT.*;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.awaitility.Awaitility.*;
+import static de.juplo.kafka.wordcount.counter.CounterApplicationIT.TOPIC_IN;
+import static de.juplo.kafka.wordcount.counter.CounterApplicationIT.TOPIC_OUT;
+import static de.juplo.kafka.wordcount.counter.CounterStreamProcessor.STORE_NAME;
+import static org.awaitility.Awaitility.await;
 
 
 @SpringBootTest(
                properties = {
-                               "spring.kafka.bootstrap-servers=${spring.embedded.kafka.brokers}",
+                               "spring.main.allow-bean-definition-overriding=true",
+                               "spring.kafka.producer.key-serializer=org.springframework.kafka.support.serializer.JsonSerializer",
+                               "spring.kafka.producer.value-serializer=org.springframework.kafka.support.serializer.JsonSerializer",
+                               "spring.kafka.producer.properties.spring.json.add.type.headers=false",
+                               "spring.kafka.consumer.auto-offset-reset=earliest",
+                               "spring.kafka.consumer.key-deserializer=org.springframework.kafka.support.serializer.JsonDeserializer",
+                               "spring.kafka.consumer.value-deserializer=org.springframework.kafka.support.serializer.JsonDeserializer",
+                               "spring.kafka.consumer.properties.spring.json.type.mapping=key:de.juplo.kafka.wordcount.top10.TestOutputWord,counter:de.juplo.kafka.wordcount.top10.TestOutputWordCounter",
+                               "logging.level.root=WARN",
+                               "logging.level.de.juplo=DEBUG",
                                "juplo.wordcount.counter.bootstrap-server=${spring.embedded.kafka.brokers}",
-                               "juplo.wordcount.counter.commit-interval=0",
-                               "juplo.wordcount.counter.cacheMaxBytes=0",
+                               "juplo.wordcount.counter.commit-interval=100",
+                               "juplo.wordcount.counter.cache-max-bytes=0",
                                "juplo.wordcount.counter.input-topic=" + TOPIC_IN,
                                "juplo.wordcount.counter.output-topic=" + TOPIC_OUT })
-@EmbeddedKafka(topics = { TOPIC_IN, TOPIC_OUT }, partitions = PARTITIONS)
+@EmbeddedKafka(topics = { TOPIC_IN, TOPIC_OUT })
 @Slf4j
 public class CounterApplicationIT
 {
-       public final static String TOPIC_IN = "in";
-       public final static String TOPIC_OUT = "out";
-       static final int PARTITIONS = 2;
+       public static final String TOPIC_IN = "in";
+       public static final String TOPIC_OUT = "out";
 
-       @Autowired
-       KafkaTemplate<String, String> kafkaTemplate;
-       @Autowired
-       ObjectMapper mapper;
        @Autowired
        Consumer consumer;
+       @Autowired
+       CounterStreamProcessor streamProcessor;
 
 
-       @BeforeEach
-       public void clear()
+       @BeforeAll
+       public static void testSendMessage(
+                       @Autowired KafkaTemplate<TestInputUser, TestInputWord> kafkaTemplate)
        {
-               consumer.received.clear();
+               TestData
+                               .getInputMessages()
+                               .forEach(kv ->
+                               {
+                                       try
+                                       {
+                                               SendResult<TestInputUser, TestInputWord> result = kafkaTemplate.send(TOPIC_IN, kv.key, kv.value).get();
+                                               log.info(
+                                                               "Sent: {}={}, partition={}, offset={}",
+                                                               result.getProducerRecord().key(),
+                                                               result.getProducerRecord().value(),
+                                                               result.getRecordMetadata().partition(),
+                                                               result.getRecordMetadata().offset());
+                                       }
+                                       catch (Exception e)
+                                       {
+                                               throw new RuntimeException(e);
+                                       }
+                               });
        }
 
+       @DisplayName("Await the expected number of messages")
+       @Test
+       public void testAwaitExpectedNumberOfMessagesForUsers()
+       {
+               await("Expected number of messages")
+                               .atMost(Duration.ofSeconds(5))
+                               .untilAsserted(() -> consumer.enforceAssertion(
+                                               receivedMessages -> TestData.assertExpectedNumberOfMessagesForWord(receivedMessages)));
+       }
 
+       @DisplayName("Await the expected output messages")
        @Test
-       void testSendMessage() throws Exception
+       void testSendMessage()
        {
-               kafkaTemplate.send(TOPIC_IN, "peter", "Hallo");
-               kafkaTemplate.send(TOPIC_IN, "klaus", "Müsch");
-               kafkaTemplate.send(TOPIC_IN, "peter", "Welt");
-               kafkaTemplate.send(TOPIC_IN, "klaus", "Müsch");
-               kafkaTemplate.send(TOPIC_IN, "klaus", "s");
-               kafkaTemplate.send(TOPIC_IN, "peter", "Boäh");
-               kafkaTemplate.send(TOPIC_IN, "peter", "Welt");
-               kafkaTemplate.send(TOPIC_IN, "peter", "Boäh");
-               kafkaTemplate.send(TOPIC_IN, "klaus", "s");
-               kafkaTemplate.send(TOPIC_IN, "peter", "Boäh");
-               kafkaTemplate.send(TOPIC_IN, "klaus", "s");
-
-               Message peter1 = Message.of(
-                               "{\"username\":\"peter\",\"word\":\"Hallo\"}",
-                               "1");
-               Message peter2 = Message.of(
-                               "{\"username\":\"peter\",\"word\":\"Welt\"}",
-                               "1");
-               Message peter3 = Message.of(
-                               "{\"username\":\"peter\",\"word\":\"Boäh\"}",
-                               "1");
-               Message peter4 = Message.of(
-                               "{\"username\":\"peter\",\"word\":\"Welt\"}",
-                               "2");
-               Message peter5 = Message.of(
-                               "{\"username\":\"peter\",\"word\":\"Boäh\"}",
-                               "2");
-               Message peter6 = Message.of(
-                               "{\"username\":\"peter\",\"word\":\"Boäh\"}",
-                               "3");
-
-               Message klaus1 = Message.of(
-                               "{\"username\":\"klaus\",\"word\":\"Müsch\"}",
-                               "1");
-               Message klaus2 = Message.of(
-                               "{\"username\":\"klaus\",\"word\":\"Müsch\"}",
-                               "2");
-               Message klaus3 = Message.of(
-                               "{\"username\":\"klaus\",\"word\":\"s\"}",
-                               "1");
-               Message klaus4 = Message.of(
-                               "{\"username\":\"klaus\",\"word\":\"s\"}",
-                               "2");
-               Message klaus5 = Message.of(
-                               "{\"username\":\"klaus\",\"word\":\"s\"}",
-                               "3");
-
-               await("Expexted converted data")
+               await("Expected messages")
                                .atMost(Duration.ofSeconds(10))
-                               .untilAsserted(() ->
-                               {
-                                       assertThat(consumer.received).hasSize(2);
-                                       assertThat(consumer.received.get("klaus")).containsExactly(klaus1, klaus2, klaus3, klaus4, klaus5);
-                                       assertThat(consumer.received.get("peter")).containsExactly(peter1, peter2, peter3, peter4, peter5, peter6);
-                               });
+                               .untilAsserted(() -> consumer.enforceAssertion(
+                                               receivedMessages -> TestData.assertExpectedMessages(receivedMessages)));
+       }
+
+       @DisplayName("Await the expected final output messages")
+       @Test
+       public void testAwaitExpectedLastMessagesForUsers()
+       {
+               await("Expected final output messages")
+                               .atMost(Duration.ofSeconds(5))
+                               .untilAsserted(() -> consumer.enforceAssertion(
+                                               receivedMessages -> TestData.assertExpectedLastMessagesForWord(receivedMessages)));
+       }
+
+       @DisplayName("Await the expected state in the state-store")
+       @Test
+       public void testAwaitExpectedState()
+       {
+               await("Expected state")
+                               .atMost(Duration.ofSeconds(5))
+                               .untilAsserted(() -> TestData.assertExpectedState(streamProcessor.getStore()));
        }
 
 
-       @RequiredArgsConstructor
        static class Consumer
        {
-               private final MultiValueMap<String, Message> received = new LinkedMultiValueMap<>();
-               private final ObjectMapper mapper;
+               private final MultiValueMap<TestOutputWord, TestOutputWordCounter> received = new LinkedMultiValueMap<>();
 
                @KafkaListener(groupId = "TEST", topics = TOPIC_OUT)
-               public void receive(ConsumerRecord<String, String> record) throws JsonProcessingException
+               public synchronized void receive(
+                               @Header(KafkaHeaders.RECEIVED_KEY) TestOutputWord word,
+                               @Payload TestOutputWordCounter counter)
                {
-                       log.debug("Received message: {}", record);
-                       Key key = mapper.readValue(record.key(), Key.class);
-                       received.add(key.getUsername(), Message.of(record.key(),record.value()));
+                       log.debug("Received message: {} -> {}", word, counter);
+                       received.add(word, counter);
                }
-       }
 
-       @Value(staticConstructor = "of")
-       static class Message
-       {
-               String key;
-               String value;
+               synchronized void enforceAssertion(
+                               java.util.function.Consumer<MultiValueMap<TestOutputWord, TestOutputWordCounter>> assertion)
+               {
+                       assertion.accept(received);
+               }
        }
 
        @TestConfiguration
        static class Configuration
        {
                @Bean
-               Consumer consumer(ObjectMapper mapper)
+               Consumer consumer()
+               {
+                       return new Consumer();
+               }
+
+               @Bean
+               KeyValueBytesStoreSupplier storeSupplier()
                {
-                       return new Consumer(mapper);
+                       return Stores.inMemoryKeyValueStore(STORE_NAME);
                }
        }
 }