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;
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;
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;
}
HostInfo applicationServer,
QueryApplicationProperties applicationProperties,
KeyValueBytesStoreSupplier storeSupplier,
- ObjectMapper mapper,
ConfigurableApplicationContext context)
{
QueryStreamProcessor streamProcessor = new QueryStreamProcessor(
applicationServer,
applicationProperties.getUsersInputTopic(),
applicationProperties.getRankingInputTopic(),
- storeSupplier,
- mapper);
+ storeSupplier);
streamProcessor.streams.setUncaughtExceptionHandler((Throwable e) ->
{
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;
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;
public final KafkaStreams streams;
public final HostInfo hostInfo;
- public final StoreQueryParameters<ReadOnlyKeyValueStore<String, String>> storeParameters;
- public final ObjectMapper mapper;
+ public final StoreQueryParameters<ReadOnlyKeyValueStore<String, UserRanking>> storeParameters;
public 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<String, String> users = builder.table(usersInputTopic);
- KStream<String, String> rankings = builder.stream(rankingInputTopic);
+ KTable<String, User> users = builder.table(usersInputTopic);
+ KStream<String, Ranking> 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
+ .<String, UserRanking>as(storeSupplier)
+ .withValueSerde(new JsonSerde().copyWithType(UserRanking.class)));
Topology topology = builder.build();
log.info("\n\n{}", topology.describe());
public Optional<UserRanking> 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