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