query: 2.0.0 - Defined 2 state-stores (all state in-memory in tests)
[demos/kafka/wordcount] / src / main / java / de / juplo / kafka / wordcount / query / QueryStreamProcessor.java
index 886c8cf..4749264 100644 (file)
@@ -1,40 +1,33 @@
 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.KafkaStreams;
-import org.apache.kafka.streams.KeyQueryMetadata;
-import org.apache.kafka.streams.StoreQueryParameters;
-import org.apache.kafka.streams.StreamsBuilder;
+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.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.boot.SpringApplication;
-import org.springframework.context.ConfigurableApplicationContext;
+import org.springframework.kafka.support.serializer.JsonSerde;
 
 import java.net.URI;
 import java.util.Optional;
 import java.util.Properties;
-import java.util.concurrent.CompletableFuture;
-
-import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
 
 
 @Slf4j
 public class QueryStreamProcessor
 {
+       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 String storeName = "rankingsByUsername";
-       public final StoreQueryParameters<ReadOnlyKeyValueStore<String, String>> storeParameters;
-       public final ObjectMapper mapper;
+       public final StoreQueryParameters<ReadOnlyKeyValueStore<String, UserRanking>> storeParameters;
 
 
        public QueryStreamProcessor(
@@ -42,55 +35,60 @@ public class QueryStreamProcessor
                        HostInfo applicationServer,
                        String usersInputTopic,
                        String rankingInputTopic,
-                       ObjectMapper mapper,
-                       ConfigurableApplicationContext context)
+                       KeyValueBytesStoreSupplier userStoreSupplier,
+                       KeyValueBytesStoreSupplier rankingStoreSupplier)
+       {
+               Topology topology = buildTopology(
+                               usersInputTopic,
+                               rankingInputTopic,
+                               userStoreSupplier,
+                               rankingStoreSupplier);
+               streams = new KafkaStreams(topology, props);
+               hostInfo = applicationServer;
+               storeParameters = StoreQueryParameters.fromNameAndType(RANKING_STORE_NAME, QueryableStoreTypes.keyValueStore());;
+       }
+
+       static Topology buildTopology(
+                       String usersInputTopic,
+                       String rankingInputTopic,
+                       KeyValueBytesStoreSupplier userStoreSupplier,
+                       KeyValueBytesStoreSupplier rankingStoreSupplier)
        {
                StreamsBuilder builder = new StreamsBuilder();
 
-               KTable<String, String> users = builder.table(usersInputTopic);
-               KStream<String, String> rankings = builder.stream(rankingInputTopic);
+               KTable<String, User> users = builder
+                               .stream(usersInputTopic)
+                               .toTable(
+                                               Materialized
+                                                               .<String, User>as(userStoreSupplier)
+                                                               .withKeySerde(Serdes.String())
+                                                               .withValueSerde(new JsonSerde().copyWithType(User.class)));
+               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(storeName));
-
-               streams = new KafkaStreams(builder.build(), props);
-               streams.setUncaughtExceptionHandler((Throwable e) ->
-               {
-                       log.error("Unexpected error!", e);
-                       CompletableFuture.runAsync(() ->
-                       {
-                               log.info("Stopping application...");
-                               SpringApplication.exit(context, () -> 1);
-                       });
-                       return SHUTDOWN_CLIENT;
-               });
+                               .join(users, (ranking, user) -> UserRanking.of(
+                                               user.getFirstName(),
+                                               user.getLastName(),
+                                               ranking.getEntries()))
+                               .toTable(
+                                               Materialized
+                                                               .<String, UserRanking>as(rankingStoreSupplier)
+                                                               .withValueSerde(new JsonSerde().copyWithType(UserRanking.class)));
+
+               Topology topology = builder.build();
+               log.info("\n\n{}", topology.describe());
+
+               return topology;
+       }
 
-               hostInfo = applicationServer;
-               storeParameters = StoreQueryParameters.fromNameAndType(storeName, QueryableStoreTypes.keyValueStore());;
-               this.mapper = mapper;
+       ReadOnlyKeyValueStore<String, UserRanking> getStore()
+       {
+               return streams.store(storeParameters);
        }
 
        public Optional<URI> getRedirect(String username)
        {
-               KeyQueryMetadata metadata = streams.queryMetadataForKey(storeName, 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 +104,7 @@ public class QueryStreamProcessor
 
        public Optional<UserRanking> 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