X-Git-Url: https://juplo.de/gitweb/?a=blobdiff_plain;f=src%2Fmain%2Fjava%2Fde%2Fjuplo%2Fkafka%2Fwordcount%2Fcounter%2FCounterStreamProcessor.java;h=4cc0c6866bb7d31613f179debd285f3f3d9c9566;hb=f58f069e556e9782656cf4e1d09eabbfcd6fa1c4;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..4cc0c68 100644 --- a/src/main/java/de/juplo/kafka/wordcount/counter/CounterStreamProcessor.java +++ b/src/main/java/de/juplo/kafka/wordcount/counter/CounterStreamProcessor.java @@ -1,12 +1,17 @@ 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.common.serialization.Serdes; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; +import org.apache.kafka.streams.kstream.*; +import org.springframework.kafka.support.serializer.JsonSerde; import java.util.Properties; @@ -21,31 +26,57 @@ public class CounterStreamProcessor String inputTopic, String outputTopic, Properties properties, + KeyValueBytesStoreSupplier storeSupplier, + ObjectMapper mapper) + { + Topology topology = CounterStreamProcessor.buildTopology( + inputTopic, + outputTopic, + storeSupplier, + mapper); + + streams = new KafkaStreams(topology, properties); + } + + static Topology buildTopology( + String inputTopic, + String outputTopic, + KeyValueBytesStoreSupplier storeSupplier, ObjectMapper mapper) { StreamsBuilder builder = new StreamsBuilder(); - KStream source = builder.stream(inputTopic); + KStream source = builder.stream( + inputTopic, + Consumed.with( + Serdes.String(), + new JsonSerde<>(Word.class) + .ignoreTypeHeaders())); + 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); - } - }) - .groupByKey() - .count() - .mapValues(value->Long.toString(value)) + .map((key, word) -> new KeyValue<>(word, word)) + .groupByKey(Grouped.with( + new JsonSerde<>(Word.class) + .forKeys() + .noTypeInfo(), + new JsonSerde<>(Word.class) + .noTypeInfo())) + .count(Materialized.as(storeSupplier)) .toStream() - .to(outputTopic); + .map((word, count) -> new KeyValue<>(word, WordCount.of(word, count))) + .to( + outputTopic, + Produced.with( + new JsonSerde<>(Word.class) + .forKeys() + .noTypeInfo(), + new JsonSerde<>(WordCount.class) + .noTypeInfo())); + + Topology topology = builder.build(); + log.info("\n\n{}", topology.describe()); - streams = new KafkaStreams(builder.build(), properties); + return topology; } public void start()