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