X-Git-Url: https://juplo.de/gitweb/?a=blobdiff_plain;f=src%2Fmain%2Fjava%2Fde%2Fjuplo%2Fkafka%2Fwordcount%2Fquery%2FQueryStreamProcessor.java;fp=src%2Fmain%2Fjava%2Fde%2Fjuplo%2Fkafka%2Fwordcount%2Fquery%2FQueryStreamProcessor.java;h=cc65fce7452acf68593e1a8a340d652c3018a520;hb=278c7b8125c82120e1d80fa640bd16d375d4bf86;hp=ff7c1501c0a7cae4e2373d729bf240f18677ff47;hpb=57f47fca712981116b726e437f589ac727c5d0a7;p=demos%2Fkafka%2Fwordcount diff --git a/src/main/java/de/juplo/kafka/wordcount/query/QueryStreamProcessor.java b/src/main/java/de/juplo/kafka/wordcount/query/QueryStreamProcessor.java index ff7c150..cc65fce 100644 --- a/src/main/java/de/juplo/kafka/wordcount/query/QueryStreamProcessor.java +++ b/src/main/java/de/juplo/kafka/wordcount/query/QueryStreamProcessor.java @@ -1,12 +1,11 @@ package de.juplo.kafka.wordcount.query; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import jakarta.annotation.PostConstruct; import jakarta.annotation.PreDestroy; 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.KTable; import org.apache.kafka.streams.kstream.Materialized; @@ -14,6 +13,7 @@ import org.apache.kafka.streams.state.HostInfo; 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 java.net.URI; import java.util.Optional; @@ -27,8 +27,7 @@ public class QueryStreamProcessor public final KafkaStreams streams; public final HostInfo hostInfo; - public final StoreQueryParameters> storeParameters; - public final ObjectMapper mapper; + public final StoreQueryParameters> storeParameters; public QueryStreamProcessor( @@ -36,51 +35,36 @@ public class QueryStreamProcessor HostInfo applicationServer, String usersInputTopic, String rankingInputTopic, - KeyValueBytesStoreSupplier storeSupplier, - ObjectMapper mapper) + KeyValueBytesStoreSupplier storeSupplier) { Topology topology = buildTopology( usersInputTopic, rankingInputTopic, - storeSupplier, - mapper); + storeSupplier); streams = new KafkaStreams(topology, props); hostInfo = applicationServer; storeParameters = StoreQueryParameters.fromNameAndType(STORE_NAME, QueryableStoreTypes.keyValueStore());; - this.mapper = mapper; } static Topology buildTopology( String usersInputTopic, String rankingInputTopic, - KeyValueBytesStoreSupplier storeSupplier, - ObjectMapper mapper) + KeyValueBytesStoreSupplier storeSupplier) { StreamsBuilder builder = new StreamsBuilder(); - KTable users = builder.table(usersInputTopic); - KStream rankings = builder.stream(rankingInputTopic); + KTable users = builder.table(usersInputTopic); + KStream rankings = builder.stream(rankingInputTopic); 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); - } - }) - .toTable(Materialized.as(storeSupplier)); + .join(users, (ranking, user) -> UserRanking.of( + user.getFirstName(), + user.getLastName(), + ranking.getEntries())) + .toTable( + Materialized + .as(storeSupplier) + .withValueSerde(new JsonSerde().copyWithType(UserRanking.class))); Topology topology = builder.build(); log.info("\n\n{}", topology.describe()); @@ -111,20 +95,7 @@ public class QueryStreamProcessor public Optional getUserRanking(String username) { - return - Optional - .ofNullable(getStore().get(username)) - .map(json -> - { - try - { - return mapper.readValue(json, UserRanking.class); - } - catch (JsonProcessingException e) - { - throw new RuntimeException(e); - } - }); + return Optional.ofNullable(getStore().get(username)); } @PostConstruct