X-Git-Url: https://juplo.de/gitweb/?a=blobdiff_plain;f=src%2Fmain%2Fjava%2Fde%2Fjuplo%2Fkafka%2Fwordcount%2Fquery%2FQueryStreamProcessor.java;h=3a1665fa9f6245c867e7703efdd957874e3d0343;hb=f929a47a66a2c14a291133aaa624ad9c5696bd53;hp=3e205f67beb2d1f331b9a9bde68f7118fd3b88ba;hpb=030c0db664cea0ec3e4307065054c449dbca6be2;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 3e205f6..3a1665f 100644 --- a/src/main/java/de/juplo/kafka/wordcount/query/QueryStreamProcessor.java +++ b/src/main/java/de/juplo/kafka/wordcount/query/QueryStreamProcessor.java @@ -1,19 +1,16 @@ 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.KStream; -import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.kstream.*; 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; @@ -23,12 +20,12 @@ import java.util.Properties; @Slf4j public class QueryStreamProcessor { - public static final String STORE_NAME = "rankings-by-username"; + public static final String USER_STORE_NAME = "users"; + public static final String RANKING_STORE_NAME = "rankings"; public final KafkaStreams streams; public final HostInfo hostInfo; - public final StoreQueryParameters> storeParameters; - public final ObjectMapper mapper; + public final StoreQueryParameters> storeParameters; public QueryStreamProcessor( @@ -36,51 +33,51 @@ public class QueryStreamProcessor HostInfo applicationServer, String usersInputTopic, String rankingInputTopic, - KeyValueBytesStoreSupplier storeSupplier, - ObjectMapper mapper) + KeyValueBytesStoreSupplier userStoreSupplier, + KeyValueBytesStoreSupplier rankingStoreSupplier) { Topology topology = buildTopology( usersInputTopic, rankingInputTopic, - storeSupplier, - mapper); + userStoreSupplier, + rankingStoreSupplier); streams = new KafkaStreams(topology, props); hostInfo = applicationServer; - storeParameters = StoreQueryParameters.fromNameAndType(STORE_NAME, QueryableStoreTypes.keyValueStore());; - this.mapper = mapper; + storeParameters = StoreQueryParameters.fromNameAndType(RANKING_STORE_NAME, QueryableStoreTypes.keyValueStore());; } static Topology buildTopology( String usersInputTopic, String rankingInputTopic, - KeyValueBytesStoreSupplier storeSupplier, - ObjectMapper mapper) + KeyValueBytesStoreSupplier userStoreSupplier, + KeyValueBytesStoreSupplier rankingStoreSupplier) { StreamsBuilder builder = new StreamsBuilder(); - KTable users = builder.table(usersInputTopic); - KStream rankings = builder.stream(rankingInputTopic); + KTable users = builder + .stream( + usersInputTopic, + Consumed.with(Serdes.String(), new JsonSerde().copyWithType(User.class))) + .toTable( + Materialized + .as(userStoreSupplier) + .withKeySerde(Serdes.String()) + .withValueSerde(new JsonSerde().copyWithType(User.class))); + KStream rankings = builder + .stream(rankingInputTopic) + .map((key, value) -> new KeyValue<>(key.getUsername(), value)); 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()), + Joined.keySerde(Serdes.String())) + .toTable( + Materialized + .as(rankingStoreSupplier) + .withKeySerde(Serdes.String()) + .withValueSerde(new JsonSerde().copyWithType(UserRanking.class))); Topology topology = builder.build(); log.info("\n\n{}", topology.describe()); @@ -88,9 +85,14 @@ public class QueryStreamProcessor return topology; } + ReadOnlyKeyValueStore getStore() + { + return streams.store(storeParameters); + } + public Optional getRedirect(String username) { - KeyQueryMetadata metadata = streams.queryMetadataForKey(STORE_NAME, username, Serdes.String().serializer()); + KeyQueryMetadata metadata = streams.queryMetadataForKey(RANKING_STORE_NAME, username, Serdes.String().serializer()); HostInfo activeHost = metadata.activeHost(); log.debug("Local store for {}: {}, {}:{}", username, metadata.partition(), activeHost.host(), activeHost.port()); @@ -106,20 +108,7 @@ public class QueryStreamProcessor public Optional getUserRanking(String username) { - return - Optional - .ofNullable(streams.store(storeParameters).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