counter: 1.2.0 - Service reads/writes using `JsonSerde`
[demos/kafka/wordcount] / src / main / java / de / juplo / kafka / wordcount / counter / CounterStreamProcessor.java
index 4002df2..324e424 100644 (file)
@@ -1,8 +1,8 @@
 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;
@@ -10,6 +10,8 @@ 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;
 
@@ -44,27 +46,44 @@ public class CounterStreamProcessor
        {
                StreamsBuilder builder = new StreamsBuilder();
 
-               KStream<String, String> source = builder.stream(inputTopic);
+               KStream<String, Word> 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(Materialized.as(storeSupplier))
-                               .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
+                                               .<Word,Long>as(storeSupplier)
+                                               .withKeySerde(
+                                                               new JsonSerde<>(Word.class)
+                                                                               .forKeys()
+                                                                               .noTypeInfo())
+                                               .withValueSerde(
+                                                               Serdes.Long()))
                                .toStream()
-                               .to(outputTopic);
+                               .map((word, count) -> new KeyValue<>(word, WordCount.of(word.getUser(), word.getWord(), 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());
 
-               return builder.build();
+               return topology;
        }
 
        public void start()