X-Git-Url: https://juplo.de/gitweb/?a=blobdiff_plain;ds=sidebyside;f=src%2Fmain%2Fjava%2Fde%2Fjuplo%2Fkafka%2Fwordcount%2Fcounter%2FCounterStreamProcessor.java;h=2304e558c242359f9303d6c2fdbfe33b4b419120;hb=refs%2Fheads%2Fcounter;hp=712ab6573be5b7d555a1d96efc895b14bd0e457e;hpb=3eb1ec997478982fffb31f09c21e756b529e474a;p=demos%2Fkafka%2Fwordcount diff --git a/src/main/java/de/juplo/kafka/wordcount/counter/CounterStreamProcessor.java b/src/main/java/de/juplo/kafka/wordcount/counter/CounterStreamProcessor.java index 712ab65..455d895 100644 --- a/src/main/java/de/juplo/kafka/wordcount/counter/CounterStreamProcessor.java +++ b/src/main/java/de/juplo/kafka/wordcount/counter/CounterStreamProcessor.java @@ -1,21 +1,25 @@ package de.juplo.kafka.wordcount.counter; import lombok.extern.slf4j.Slf4j; -import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.*; import org.apache.kafka.streams.kstream.Consumed; -import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.kstream.Produced; import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; import org.apache.kafka.streams.state.QueryableStoreTypes; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; +import org.springframework.kafka.support.serializer.JsonSerde; +import org.springframework.kafka.support.serializer.JsonSerializer; +import java.util.Map; import java.util.Properties; +import java.util.stream.Collectors; @Slf4j public class CounterStreamProcessor { + public static final String TYPE = "COUNTER"; public static final String STORE_NAME = "counter"; @@ -43,17 +47,18 @@ public class CounterStreamProcessor { StreamsBuilder builder = new StreamsBuilder(); - KStream source = builder.stream( - inputTopic, - Consumed.with(Serdes.String(), null)); - - source + builder + .stream(inputTopic, Consumed.with(inKeySerde(), inValueSerde())) + .mapValues(word -> Word.of(word.getUser(), word.getWord())) .map((key, word) -> new KeyValue<>(word, word)) .groupByKey() - .count(Materialized.as(storeSupplier)) + .count( + Materialized + .as(storeSupplier) + .withKeySerde(new JsonSerde<>(Word.class))) // No headers are present: fixed typing is needed! .toStream() .map((word, counter) -> new KeyValue<>(word, WordCounter.of(word, counter))) - .to(outputTopic); + .to(outputTopic, Produced.with(outKeySerde(), outValueSerde())); Topology topology = builder.build(); log.info("\n\n{}", topology.describe()); @@ -77,4 +82,51 @@ public class CounterStreamProcessor log.info("Stopping Stream-Processor"); streams.close(); } + + + + public static JsonSerde inKeySerde() + { + return new JsonSerde<>(User.class); + } + + public static JsonSerde inValueSerde() + { + return new JsonSerde<>(UserWord.class); + } + + public static JsonSerde outKeySerde() + { + return serde(true); + } + + public static JsonSerde outValueSerde() + { + return serde(false); + } + + public static JsonSerde serde(boolean isKey) + { + JsonSerde serde = new JsonSerde<>(); + serde.configure( + Map.of(JsonSerializer.TYPE_MAPPINGS, typeMappingsConfig()), + isKey); + return serde; + } + + private static String typeMappingsConfig() + { + return typeMappingsConfig(Word.class, WordCounter.class); + } + + public static String typeMappingsConfig(Class keyClass, Class counterClass) + { + return Map.of( + "key", keyClass, + "counter", counterClass) + .entrySet() + .stream() + .map(entry -> entry.getKey() + ":" + entry.getValue().getName()) + .collect(Collectors.joining(",")); + } }