X-Git-Url: https://juplo.de/gitweb/?a=blobdiff_plain;ds=sidebyside;f=src%2Ftest%2Fjava%2Fde%2Fjuplo%2Fkafka%2Fwordcount%2Ftop10%2FTop10StreamProcessorTopologyTest.java;h=90d8e4cb90987d0455c84f67252b6080e5d9bed1;hb=478b25d6700bd0a48b55647f09386f3b1d59fa29;hp=1becd654cc74ab107795a09ddd9f14e5f8fada6a;hpb=b69e1270a308f200b2640a01d37d4636a0a549e1;p=demos%2Fkafka%2Fwordcount diff --git a/src/test/java/de/juplo/kafka/wordcount/top10/Top10StreamProcessorTopologyTest.java b/src/test/java/de/juplo/kafka/wordcount/top10/Top10StreamProcessorTopologyTest.java index 1becd65..90d8e4c 100644 --- a/src/test/java/de/juplo/kafka/wordcount/top10/Top10StreamProcessorTopologyTest.java +++ b/src/test/java/de/juplo/kafka/wordcount/top10/Top10StreamProcessorTopologyTest.java @@ -1,5 +1,9 @@ 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; @@ -11,19 +15,13 @@ 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 @@ -35,8 +33,8 @@ public class Top10StreamProcessorTopologyTest TopologyTestDriver testDriver; - TestInputTopic in; - TestOutputTopic out; + TestInputTopic in; + TestOutputTopic out; @BeforeEach @@ -47,28 +45,21 @@ public class Top10StreamProcessorTopologyTest OUT, Stores.inMemoryKeyValueStore(STORE_NAME)); - Top10ApplicationConfiguration applicationConfiguriation = - new Top10ApplicationConfiguration(); - Properties streamProcessorProperties = - applicationConfiguriation.streamProcessorProperties(new Top10ApplicationProperties()); - Map propertyMap = convertToMap(streamProcessorProperties); - - JsonSerde keySerde = new JsonSerde<>(); - keySerde.configure(propertyMap, true); - JsonSerde valueSerde = new JsonSerde<>(); - valueSerde.configure(propertyMap, false); - - testDriver = new TopologyTestDriver(topology, streamProcessorProperties); + testDriver = new TopologyTestDriver(topology, serializationConfig()); in = testDriver.createInputTopic( IN, - (JsonSerializer)keySerde.serializer(), - (JsonSerializer)valueSerde.serializer()); + jsonSerializer(TestWord.class, true), + jsonSerializer(TestCounter.class,false)); out = testDriver.createOutputTopic( OUT, - (JsonDeserializer)keySerde.deserializer(), - (JsonDeserializer)valueSerde.deserializer()); + new JsonDeserializer() + .copyWithType(TestUser.class) + .ignoreTypeHeaders(), + new JsonDeserializer() + .copyWithType(TestRanking.class) + .ignoreTypeHeaders()); } @@ -76,28 +67,20 @@ public class Top10StreamProcessorTopologyTest @Test public void test() { - 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()))); + TestData + .getInputMessages() + .forEach(kv -> in.pipeInput(kv.key, kv.value)); - MultiValueMap receivedMessages = new LinkedMultiValueMap<>(); + MultiValueMap 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 store = testDriver.getKeyValueStore(STORE_NAME); TestData.assertExpectedState(store); } @@ -107,4 +90,16 @@ public class Top10StreamProcessorTopologyTest { testDriver.close(); } + + private JsonSerializer jsonSerializer(Class type, boolean isKey) + { + JsonSerializer jsonSerializer = new JsonSerializer<>(); + jsonSerializer.configure( + Map.of( + JsonSerializer.TYPE_MAPPINGS, + "word:" + TestWord.class.getName() + "," + + "counter:" + TestCounter.class.getName()), + isKey); + return jsonSerializer; + } }