cc65fce7452acf68593e1a8a340d652c3018a520
[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 STORE_NAME = "rankings-by-username";
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 storeSupplier)
39         {
40                 Topology topology = buildTopology(
41                                 usersInputTopic,
42                                 rankingInputTopic,
43                                 storeSupplier);
44                 streams = new KafkaStreams(topology, props);
45                 hostInfo = applicationServer;
46                 storeParameters = StoreQueryParameters.fromNameAndType(STORE_NAME, QueryableStoreTypes.keyValueStore());;
47         }
48
49         static Topology buildTopology(
50                         String usersInputTopic,
51                         String rankingInputTopic,
52                         KeyValueBytesStoreSupplier storeSupplier)
53         {
54                 StreamsBuilder builder = new StreamsBuilder();
55
56                 KTable<String, User> users = builder.table(usersInputTopic);
57                 KStream<String, Ranking> rankings = builder.stream(rankingInputTopic);
58
59                 rankings
60                                 .join(users, (ranking, user) -> UserRanking.of(
61                                                 user.getFirstName(),
62                                                 user.getLastName(),
63                                                 ranking.getEntries()))
64                                 .toTable(
65                                                 Materialized
66                                                                 .<String, UserRanking>as(storeSupplier)
67                                                                 .withValueSerde(new JsonSerde().copyWithType(UserRanking.class)));
68
69                 Topology topology = builder.build();
70                 log.info("\n\n{}", topology.describe());
71
72                 return topology;
73         }
74
75         ReadOnlyKeyValueStore<String, UserRanking> getStore()
76         {
77                 return streams.store(storeParameters);
78         }
79
80         public Optional<URI> getRedirect(String username)
81         {
82                 KeyQueryMetadata metadata = streams.queryMetadataForKey(STORE_NAME, username, Serdes.String().serializer());
83                 HostInfo activeHost = metadata.activeHost();
84                 log.debug("Local store for {}: {}, {}:{}", username, metadata.partition(), activeHost.host(), activeHost.port());
85
86                 if (activeHost.equals(this.hostInfo) || activeHost.equals(HostInfo.unavailable()))
87                 {
88                         return Optional.empty();
89                 }
90
91                 URI location = URI.create("http://" + activeHost.host() + ":" + activeHost.port() + "/" + username);
92                 log.debug("Redirecting to {}", location);
93                 return Optional.of(location);
94         }
95
96         public Optional<UserRanking> getUserRanking(String username)
97         {
98                 return Optional.ofNullable(getStore().get(username));
99         }
100
101         @PostConstruct
102         public void start()
103         {
104                 log.info("Starting Stream-Processor");
105                 streams.start();
106         }
107
108         @PreDestroy
109         public void stop()
110         {
111                 log.info("Stopping Stream-Processor");
112                 streams.close();
113         }
114 }