top10: 1.2.1 - Refined `TestData` clearified concerns
[demos/kafka/wordcount] / src / test / java / de / juplo / kafka / wordcount / top10 / Top10StreamProcessorTopologyTest.java
index 8ecf9fa..90d8e4c 100644 (file)
 package de.juplo.kafka.wordcount.top10;
 
+import de.juplo.kafka.wordcount.counter.TestCounter;
+import de.juplo.kafka.wordcount.counter.TestWord;
+import de.juplo.kafka.wordcount.query.TestRanking;
+import de.juplo.kafka.wordcount.query.TestUser;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.kafka.streams.TestInputTopic;
 import org.apache.kafka.streams.TestOutputTopic;
 import org.apache.kafka.streams.Topology;
 import org.apache.kafka.streams.TopologyTestDriver;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.Stores;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.springframework.kafka.support.serializer.JsonDeserializer;
-import org.springframework.kafka.support.serializer.JsonSerde;
 import org.springframework.kafka.support.serializer.JsonSerializer;
 import org.springframework.util.LinkedMultiValueMap;
 import org.springframework.util.MultiValueMap;
 
 import java.util.Map;
-import java.util.Properties;
-import java.util.stream.Stream;
 
-import static de.juplo.kafka.wordcount.top10.TestData.convertToMap;
-import static de.juplo.kafka.wordcount.top10.TestData.parseHeader;
-import static org.springframework.kafka.support.mapping.AbstractJavaTypeMapper.DEFAULT_CLASSID_FIELD_NAME;
-import static org.springframework.kafka.support.mapping.AbstractJavaTypeMapper.KEY_DEFAULT_CLASSID_FIELD_NAME;
+import static de.juplo.kafka.wordcount.top10.Top10ApplicationConfiguration.serializationConfig;
 
 
 @Slf4j
 public class Top10StreamProcessorTopologyTest
 {
-  public final static String IN = "TEST-IN";
-  public final static String OUT = "TEST-OUT";
+  public static final String IN = "TEST-IN";
+  public static final String OUT = "TEST-OUT";
+  public static final String STORE_NAME = "TOPOLOGY-TEST";
 
-  @Test
-  public void test()
-  {
-    Topology topology = Top10StreamProcessor.buildTopology(IN, OUT);
 
-    Top10ApplicationConfiguration applicationConfiguriation =
-        new Top10ApplicationConfiguration();
-    Properties streamProcessorProperties =
-        applicationConfiguriation.streamProcessorProperties(new Top10ApplicationProperties());
-    Map<String, Object> propertyMap = convertToMap(streamProcessorProperties);
+  TopologyTestDriver testDriver;
+  TestInputTopic<TestWord, TestCounter> in;
+  TestOutputTopic<TestUser, TestRanking> out;
 
-    JsonSerde<?> keySerde = new JsonSerde<>();
-    keySerde.configure(propertyMap, true);
-    JsonSerde<?> valueSerde = new JsonSerde<>();
-    valueSerde.configure(propertyMap, false);
 
-    TopologyTestDriver testDriver = new TopologyTestDriver(topology, streamProcessorProperties);
+  @BeforeEach
+  public void setUp()
+  {
+    Topology topology = Top10StreamProcessor.buildTopology(
+        IN,
+        OUT,
+        Stores.inMemoryKeyValueStore(STORE_NAME));
+
+    testDriver = new TopologyTestDriver(topology, serializationConfig());
 
-    TestInputTopic<Key, Entry> in = testDriver.createInputTopic(
+    in = testDriver.createInputTopic(
         IN,
-        (JsonSerializer<Key>)keySerde.serializer(),
-        (JsonSerializer<Entry>)valueSerde.serializer());
+        jsonSerializer(TestWord.class, true),
+        jsonSerializer(TestCounter.class,false));
 
-    TestOutputTopic<String, Ranking> out = testDriver.createOutputTopic(
+    out = testDriver.createOutputTopic(
         OUT,
-        (JsonDeserializer<String>)keySerde.deserializer(),
-        (JsonDeserializer<Ranking>)valueSerde.deserializer());
+        new JsonDeserializer()
+            .copyWithType(TestUser.class)
+            .ignoreTypeHeaders(),
+        new JsonDeserializer()
+            .copyWithType(TestRanking.class)
+            .ignoreTypeHeaders());
+
+  }
+
 
-    Stream
-        .of(TestData.INPUT_MESSAGES)
-        .forEach(kv -> in.pipeInput(
-            Key.of(kv.key.getUser(), kv.key.getWord()),
-            Entry.of(kv.value.getWord(), kv.value.getCounter())));
+  @Test
+  public void test()
+  {
+    TestData
+        .getInputMessages()
+        .forEach(kv -> in.pipeInput(kv.key, kv.value));
 
-    MultiValueMap<String, Ranking> receivedMessages = new LinkedMultiValueMap<>();
+    MultiValueMap<TestUser, TestRanking> receivedMessages = new LinkedMultiValueMap<>();
     out
         .readRecordsToList()
-        .forEach(record ->
-        {
-          log.debug(
-              "OUT: {} -> {}, {}, {}",
-              record.key(),
-              record.value(),
-              parseHeader(record.headers(), KEY_DEFAULT_CLASSID_FIELD_NAME),
-              parseHeader(record.headers(), DEFAULT_CLASSID_FIELD_NAME));
-          receivedMessages.add(record.key(), record.value());
-        });
+        .forEach(record -> receivedMessages.add(record.key(), record.value()));
 
     TestData.assertExpectedMessages(receivedMessages);
 
+    TestData.assertExpectedNumberOfMessagesForUsers(receivedMessages);
+    TestData.assertExpectedLastMessagesForUsers(receivedMessages);
+
+    KeyValueStore<User, Ranking> store = testDriver.getKeyValueStore(STORE_NAME);
+    TestData.assertExpectedState(store);
+  }
+
+  @AfterEach
+  public void tearDown()
+  {
     testDriver.close();
   }
+
+  private <T> JsonSerializer<T> jsonSerializer(Class<T> type, boolean isKey)
+  {
+    JsonSerializer<T> jsonSerializer = new JsonSerializer<>();
+    jsonSerializer.configure(
+        Map.of(
+            JsonSerializer.TYPE_MAPPINGS,
+            "word:" + TestWord.class.getName() + "," +
+            "counter:" + TestCounter.class.getName()),
+        isKey);
+    return jsonSerializer;
+  }
 }