From: Kai Moritz Date: Sun, 9 Jun 2024 18:44:35 +0000 (+0200) Subject: query: 2.0.0 - (GREEN) Values are serialized as JSON X-Git-Tag: query-with-kafkaproducer~7 X-Git-Url: http://juplo.de/gitweb/?a=commitdiff_plain;h=278c7b8125c82120e1d80fa640bd16d375d4bf86;p=demos%2Fkafka%2Fwordcount query: 2.0.0 - (GREEN) Values are serialized as JSON -- works __only__, if a default-type is defined * The default-type is needed, to deserialized values that are read from the state-store. * Without it, the deserialization fails, because not type-information is available. * The type-information gets lost, when the values are stored in the state- store, because the message-headers are _not_ stored along with the value! --- diff --git a/pom.xml b/pom.xml index 60ea716..3edc6d3 100644 --- a/pom.xml +++ b/pom.xml @@ -29,6 +29,10 @@ org.apache.kafka kafka-streams + + org.springframework.kafka + spring-kafka + org.springframework.boot @@ -51,11 +55,6 @@ spring-boot-starter-test test - - org.springframework.kafka - spring-kafka - test - org.springframework.kafka spring-kafka-test diff --git a/src/main/java/de/juplo/kafka/wordcount/query/QueryApplicationConfiguration.java b/src/main/java/de/juplo/kafka/wordcount/query/QueryApplicationConfiguration.java index 50a5364..7da1712 100644 --- a/src/main/java/de/juplo/kafka/wordcount/query/QueryApplicationConfiguration.java +++ b/src/main/java/de/juplo/kafka/wordcount/query/QueryApplicationConfiguration.java @@ -1,6 +1,5 @@ package de.juplo.kafka.wordcount.query; -import com.fasterxml.jackson.databind.ObjectMapper; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.serialization.Serdes; @@ -14,6 +13,8 @@ import org.springframework.boot.context.properties.EnableConfigurationProperties import org.springframework.context.ConfigurableApplicationContext; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; +import org.springframework.kafka.support.serializer.JsonDeserializer; +import org.springframework.kafka.support.serializer.JsonSerde; import java.io.IOException; import java.net.InetSocketAddress; @@ -78,7 +79,14 @@ public class QueryApplicationConfiguration Properties props = new Properties(); 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_VALUE_SERDE_CLASS_CONFIG, JsonSerde.class.getName()); + props.put(JsonDeserializer.VALUE_DEFAULT_TYPE, User.class.getName()); // << Does not work without this! + props.put( + JsonDeserializer.TYPE_MAPPINGS, + "user:" + Key.class.getName() + "," + + "ranking:" + Ranking.class.getName() + "," + + "userdata:" + User.class.getName() + "," + + "userranking:" + UserRanking.class.getName()); return props; } @@ -89,7 +97,6 @@ public class QueryApplicationConfiguration HostInfo applicationServer, QueryApplicationProperties applicationProperties, KeyValueBytesStoreSupplier storeSupplier, - ObjectMapper mapper, ConfigurableApplicationContext context) { QueryStreamProcessor streamProcessor = new QueryStreamProcessor( @@ -97,8 +104,7 @@ public class QueryApplicationConfiguration applicationServer, applicationProperties.getUsersInputTopic(), applicationProperties.getRankingInputTopic(), - storeSupplier, - mapper); + storeSupplier); streamProcessor.streams.setUncaughtExceptionHandler((Throwable e) -> { 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 diff --git a/src/test/java/de/juplo/kafka/wordcount/query/QueryStreamProcessorTopologyTest.java b/src/test/java/de/juplo/kafka/wordcount/query/QueryStreamProcessorTopologyTest.java index eef1eec..845792c 100644 --- a/src/test/java/de/juplo/kafka/wordcount/query/QueryStreamProcessorTopologyTest.java +++ b/src/test/java/de/juplo/kafka/wordcount/query/QueryStreamProcessorTopologyTest.java @@ -1,6 +1,5 @@ package de.juplo.kafka.wordcount.query; -import com.fasterxml.jackson.databind.ObjectMapper; import de.juplo.kafka.wordcount.top10.TestRanking; import de.juplo.kafka.wordcount.users.TestUserData; import lombok.extern.slf4j.Slf4j; @@ -39,8 +38,7 @@ public class QueryStreamProcessorTopologyTest Topology topology = QueryStreamProcessor.buildTopology( USERS_IN, TOP10_IN, - Stores.inMemoryKeyValueStore(STORE_NAME), - new ObjectMapper()); + Stores.inMemoryKeyValueStore(STORE_NAME)); testDriver = new TopologyTestDriver(topology, serializationConfig());