WIP
[demos/kafka/wordcount] / src / main / java / de / juplo / kafka / wordcount / query / QueryStreamProcessor.java
index 696e088..6dcf8b1 100644 (file)
@@ -2,6 +2,8 @@ package de.juplo.kafka.wordcount.query;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig;
+import io.confluent.kafka.streams.serdes.avro.SpecificAvroSerde;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.serialization.Serdes;
@@ -33,8 +35,7 @@ public class QueryStreamProcessor
        public final KafkaStreams streams;
        public final HostInfo hostInfo;
        public final String storeName = "rankingsByUsername";
-       public final StoreQueryParameters<ReadOnlyKeyValueStore<String, String>> storeParameters;
-       public final ObjectMapper mapper;
+       public final StoreQueryParameters<ReadOnlyKeyValueStore<String, UserRankingTO>> storeParameters;
 
 
        public QueryStreamProcessor(
@@ -44,36 +45,26 @@ public class QueryStreamProcessor
        {
                StreamsBuilder builder = new StreamsBuilder();
 
-               KTable<String, String> users = builder.table(properties.getUsersInputTopic());
-               KStream<String, String> rankings = builder.stream(properties.getRankingInputTopic());
+               KTable<String, UserTO> users = builder.table(properties.getUsersInputTopic());
+               KStream<String, RankingTO> rankings = builder.stream(properties.getRankingInputTopic());
 
                rankings
-                               .join(users, (rankingJson, userJson) ->
-                               {
-                                       try
-                                       {
-                                               Ranking ranking = mapper.readValue(rankingJson, Ranking.class);
-                                               User user = mapper.readValue(userJson, User.class);
-
-                                               return mapper.writeValueAsString(
-                                                               UserRanking.of(
-                                                                               user.getFirstName(),
-                                                                               user.getLastName(),
-                                                                               ranking.getEntries()));
-                                       }
-                                       catch (JsonProcessingException e)
-                                       {
-                                               throw new RuntimeException(e);
-                                       }
-                               })
+                               .join(users, (ranking, user) ->
+                                               UserRankingTO
+                                                               .newBuilder()
+                                                               .setFirstName(user.getFirstName())
+                                                               .setLastName((user.getLastName()))
+                                                               .setTop10(ranking.getEntries())
+                                                               .build())
                                .toTable(Materialized.as(storeName));
 
                Properties props = new Properties();
                props.put(StreamsConfig.APPLICATION_ID_CONFIG, properties.getApplicationId());
                props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, properties.getApplicationServer());
                props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, properties.getBootstrapServer());
-               props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
-               props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
+               props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
+               props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, SpecificAvroSerde.class);
+               props.put(AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, "https://schema-registry:9081/");
                props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
 
                streams = new KafkaStreams(builder.build(), props);
@@ -90,7 +81,6 @@ public class QueryStreamProcessor
 
                hostInfo = HostInfo.buildFromEndpoint(properties.getApplicationServer());
                storeParameters = StoreQueryParameters.fromNameAndType(storeName, QueryableStoreTypes.keyValueStore());;
-               this.mapper = mapper;
        }
 
        public Optional<URI> getRedirect(String username)
@@ -114,17 +104,14 @@ public class QueryStreamProcessor
                return
                                Optional
                                                .ofNullable(streams.store(storeParameters).get(username))
-                                               .map(json ->
-                                               {
-                                                       try
-                                                       {
-                                                               return mapper.readValue(json, UserRanking.class);
-                                                       }
-                                                       catch (JsonProcessingException e)
-                                                       {
-                                                               throw new RuntimeException(e);
-                                                       }
-                                               });
+                                               .map(userRankingTO -> UserRanking.of(
+                                                               userRankingTO.getFirstName().toString(),
+                                                               userRankingTO.getLastName().toString(),
+                                                               userRankingTO
+                                                                               .getTop10()
+                                                                               .stream()
+                                                                               .map(entryTO -> Entry.of(entryTO.getWord().toString(), entryTO.getCount()))
+                                                                               .toArray(size -> new Entry[size])));
        }
 
        @PostConstruct