query: 2.0.0 - (GREEN) Adjusted the implementation to the new expectations
[demos/kafka/wordcount] / src / main / java / de / juplo / kafka / wordcount / query / QueryStreamProcessor.java
1 package de.juplo.kafka.wordcount.query;
2
3 import jakarta.annotation.PostConstruct;
4 import jakarta.annotation.PreDestroy;
5 import lombok.extern.slf4j.Slf4j;
6 import org.apache.kafka.common.serialization.Serdes;
7 import org.apache.kafka.streams.*;
8 import org.apache.kafka.streams.kstream.Consumed;
9 import org.apache.kafka.streams.kstream.KStream;
10 import org.apache.kafka.streams.kstream.KTable;
11 import org.apache.kafka.streams.kstream.Materialized;
12 import org.apache.kafka.streams.state.HostInfo;
13 import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier;
14 import org.apache.kafka.streams.state.QueryableStoreTypes;
15 import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
16 import org.springframework.kafka.support.serializer.JsonSerde;
17
18 import java.net.URI;
19 import java.util.Optional;
20 import java.util.Properties;
21
22
23 @Slf4j
24 public class QueryStreamProcessor
25 {
26         public static final String USER_STORE_NAME = "users";
27         public static final String RANKING_STORE_NAME = "rankings";
28
29         public final KafkaStreams streams;
30         public final HostInfo hostInfo;
31         public final StoreQueryParameters<ReadOnlyKeyValueStore<String, UserRanking>> storeParameters;
32
33
34         public QueryStreamProcessor(
35                         Properties props,
36                         HostInfo applicationServer,
37                         String usersInputTopic,
38                         String rankingInputTopic,
39                         KeyValueBytesStoreSupplier userStoreSupplier,
40                         KeyValueBytesStoreSupplier rankingStoreSupplier)
41         {
42                 Topology topology = buildTopology(
43                                 usersInputTopic,
44                                 rankingInputTopic,
45                                 userStoreSupplier,
46                                 rankingStoreSupplier);
47                 streams = new KafkaStreams(topology, props);
48                 hostInfo = applicationServer;
49                 storeParameters = StoreQueryParameters.fromNameAndType(RANKING_STORE_NAME, QueryableStoreTypes.keyValueStore());;
50         }
51
52         static Topology buildTopology(
53                         String usersInputTopic,
54                         String rankingInputTopic,
55                         KeyValueBytesStoreSupplier userStoreSupplier,
56                         KeyValueBytesStoreSupplier rankingStoreSupplier)
57         {
58                 StreamsBuilder builder = new StreamsBuilder();
59
60                 KTable<String, User> users = builder
61                                 .stream(
62                                                 usersInputTopic,
63                                                 Consumed.with(null, new JsonSerde().copyWithType(User.class)))
64                                 .toTable(
65                                                 Materialized
66                                                                 .<String, User>as(userStoreSupplier)
67                                                                 .withKeySerde(Serdes.String())
68                                                                 .withValueSerde(new JsonSerde().copyWithType(User.class)));
69                 KStream<String, Ranking> rankings = builder.stream(rankingInputTopic);
70
71                 rankings
72                                 .join(users, (ranking, user) -> UserRanking.of(
73                                                 user.getFirstName(),
74                                                 user.getLastName(),
75                                                 ranking.getEntries()))
76                                 .toTable(
77                                                 Materialized
78                                                                 .<String, UserRanking>as(rankingStoreSupplier)
79                                                                 .withValueSerde(new JsonSerde().copyWithType(UserRanking.class)));
80
81                 Topology topology = builder.build();
82                 log.info("\n\n{}", topology.describe());
83
84                 return topology;
85         }
86
87         ReadOnlyKeyValueStore<String, UserRanking> getStore()
88         {
89                 return streams.store(storeParameters);
90         }
91
92         public Optional<URI> getRedirect(String username)
93         {
94                 KeyQueryMetadata metadata = streams.queryMetadataForKey(RANKING_STORE_NAME, username, Serdes.String().serializer());
95                 HostInfo activeHost = metadata.activeHost();
96                 log.debug("Local store for {}: {}, {}:{}", username, metadata.partition(), activeHost.host(), activeHost.port());
97
98                 if (activeHost.equals(this.hostInfo) || activeHost.equals(HostInfo.unavailable()))
99                 {
100                         return Optional.empty();
101                 }
102
103                 URI location = URI.create("http://" + activeHost.host() + ":" + activeHost.port() + "/" + username);
104                 log.debug("Redirecting to {}", location);
105                 return Optional.of(location);
106         }
107
108         public Optional<UserRanking> getUserRanking(String username)
109         {
110                 return Optional.ofNullable(getStore().get(username));
111         }
112
113         @PostConstruct
114         public void start()
115         {
116                 log.info("Starting Stream-Processor");
117                 streams.start();
118         }
119
120         @PreDestroy
121         public void stop()
122         {
123                 log.info("Stopping Stream-Processor");
124                 streams.close();
125         }
126 }