X-Git-Url: https://juplo.de/gitweb/?a=blobdiff_plain;f=src%2Fmain%2Fjava%2Fde%2Fjuplo%2Fkafka%2Fwordcount%2Fcounter%2FCounterStreamProcessor.java;h=97d460fc7a33bd4c211a00e32546642bfbfb37f8;hb=419be8ac0668ecb0e34b3a432cf2dcca1c3642dc;hp=a823e25e0500aeeaffb8dac353caefed99bef06f;hpb=9856b2a34d6e2bda771b83d2825c1db8e2a6916f;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 a823e25..97d460f 100644 --- a/src/main/java/de/juplo/kafka/wordcount/counter/CounterStreamProcessor.java +++ b/src/main/java/de/juplo/kafka/wordcount/counter/CounterStreamProcessor.java @@ -1,19 +1,27 @@ package de.juplo.kafka.wordcount.counter; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import lombok.extern.slf4j.Slf4j; -import org.apache.kafka.streams.KafkaStreams; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.StreamsBuilder; -import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.*; +import org.apache.kafka.streams.kstream.Consumed; +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 STORE_NAME = "counter"; + + public final KafkaStreams streams; @@ -21,31 +29,48 @@ public class CounterStreamProcessor String inputTopic, String outputTopic, Properties properties, - ObjectMapper mapper) + KeyValueBytesStoreSupplier storeSupplier) + { + Topology topology = CounterStreamProcessor.buildTopology( + inputTopic, + outputTopic, + storeSupplier); + + streams = new KafkaStreams(topology, properties); + } + + static Topology buildTopology( + String inputTopic, + String outputTopic, + KeyValueBytesStoreSupplier storeSupplier) { StreamsBuilder builder = new StreamsBuilder(); - KStream source = builder.stream(inputTopic); - source - .map((username, word) -> - { - try - { - String key = mapper.writeValueAsString(Key.of(username, word)); - return new KeyValue<>(key, word); - } - catch (JsonProcessingException e) - { - throw new RuntimeException(e); - } - }) + builder + .stream( + inputTopic, + Consumed.with( + new JsonSerde<>(User.class), + new JsonSerde<>(Word.class))) + .map((key, word) -> new KeyValue<>(word, word)) .groupByKey() - .count() - .mapValues(value->Long.toString(value)) + .count( + Materialized + .as(storeSupplier) + .withKeySerde(new JsonSerde<>().copyWithType(Word.class).forKeys())) .toStream() - .to(outputTopic); + .map((word, counter) -> new KeyValue<>(word, WordCounter.of(word, counter))) + .to(outputTopic, Produced.with(outKeySerde(), outValueSerde())); - streams = new KafkaStreams(builder.build(), properties); + Topology topology = builder.build(); + log.info("\n\n{}", topology.describe()); + + return topology; + } + + ReadOnlyKeyValueStore getStore() + { + return streams.store(StoreQueryParameters.fromNameAndType(STORE_NAME, QueryableStoreTypes.keyValueStore())); } public void start() @@ -59,4 +84,41 @@ public class CounterStreamProcessor log.info("Stopping Stream-Processor"); streams.close(); } + + + + 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 typeMappings() + .entrySet() + .stream() + .map(entry -> entry.getKey() + ":" + entry.getValue().getName()) + .collect(Collectors.joining(",")); + } + + private static Map typeMappings() + { + return Map.of( + "word", Word.class, + "counter", WordCounter.class); + } }