<relativePath/> <!-- lookup parent from repository -->
</parent>
<groupId>de.juplo.kafka.wordcount</groupId>
- <artifactId>top10</artifactId>
+ <artifactId>query</artifactId>
<version>1.0.0</version>
- <name>Wordcount-Top-10</name>
- <description>Top-10 stream-processor of the multi-user wordcount-example</description>
+ <name>Wordcount-Query</name>
+ <description>Query stream-processor of the multi-user wordcount-example</description>
<properties>
<docker-maven-plugin.version>0.33.0</docker-maven-plugin.version>
<java.version>11</java.version>
--- /dev/null
+package de.juplo.kafka.wordcount.query;
+
+import lombok.Value;
+
+
+@Value(staticConstructor = "of")
+public class Entry
+{
+ private final String word;
+ private final Long count;
+}
--- /dev/null
+package de.juplo.kafka.wordcount.query;
+
+import lombok.Getter;
+import lombok.Setter;
+
+
+@Getter
+@Setter
+public class Key
+{
+ private String username;
+ private String word;
+}
--- /dev/null
+package de.juplo.kafka.wordcount.query;
+
+import org.springframework.boot.SpringApplication;
+import org.springframework.boot.autoconfigure.SpringBootApplication;
+import org.springframework.boot.context.properties.EnableConfigurationProperties;
+
+
+@SpringBootApplication
+@EnableConfigurationProperties(QueryApplicationProperties.class)
+public class QueryApplication
+{
+ public static void main(String[] args)
+ {
+ SpringApplication.run(QueryApplication.class, args);
+ }
+}
--- /dev/null
+package de.juplo.kafka.wordcount.query;
+
+
+import lombok.Getter;
+import lombok.Setter;
+import lombok.ToString;
+import org.springframework.boot.context.properties.ConfigurationProperties;
+
+
+@ConfigurationProperties("juplo.wordcount.query")
+@Getter
+@Setter
+@ToString
+public class QueryApplicationProperties
+{
+ private String bootstrapServer = "localhost:9092";
+ private String applicationId = "query";
+ private String rankingInputTopic = "top10";
+ private String usersInputTopic = "users";
+ private String applicationServer = "localhost:8080";
+}
--- /dev/null
+package de.juplo.kafka.wordcount.query;
+
+import lombok.RequiredArgsConstructor;
+import org.springframework.http.HttpStatus;
+import org.springframework.http.ResponseEntity;
+import org.springframework.web.bind.annotation.GetMapping;
+import org.springframework.web.bind.annotation.PathVariable;
+import org.springframework.web.bind.annotation.RestController;
+
+import java.net.URI;
+import java.util.Optional;
+
+
+@RestController
+@RequiredArgsConstructor
+public class QueryController
+{
+ private final QueryStreamProcessor processor;
+
+ @GetMapping("{username}")
+ ResponseEntity<Ranking> queryFor(@PathVariable String username)
+ {
+ Optional<URI> redirect = processor.getRedirect(username);
+ if (redirect.isPresent())
+ {
+ return
+ ResponseEntity
+ .status(HttpStatus.TEMPORARY_REDIRECT)
+ .location(redirect.get())
+ .build();
+ }
+
+ return ResponseEntity.of(processor.getRanking(username));
+ }
+}
--- /dev/null
+package de.juplo.kafka.wordcount.query;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.streams.*;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.streams.state.HostInfo;
+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.stereotype.Component;
+
+import javax.annotation.PostConstruct;
+import javax.annotation.PreDestroy;
+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
+@Component
+public class QueryStreamProcessor
+{
+ 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 QueryStreamProcessor(
+ QueryApplicationProperties properties,
+ ObjectMapper mapper,
+ ConfigurableApplicationContext context)
+ {
+ StreamsBuilder builder = new StreamsBuilder();
+
+ builder.table(properties.getRankingInputTopic(), Materialized.as(storeName));
+
+ Properties props = new Properties();
+ props.put(StreamsConfig.APPLICATION_ID_CONFIG, properties.getApplicationId());
+ props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, properties.getApplicationServer());
+ props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, properties.getBootstrapServer());
+ props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
+ props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
+ props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+
+ 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;
+ });
+
+ hostInfo = HostInfo.buildFromEndpoint(properties.getApplicationServer());
+ storeParameters = StoreQueryParameters.fromNameAndType(storeName, QueryableStoreTypes.keyValueStore());;
+ this.mapper = mapper;
+ }
+
+ public Optional<URI> getRedirect(String username)
+ {
+ KeyQueryMetadata metadata = streams.queryMetadataForKey(storeName, username, Serdes.String().serializer());
+ HostInfo activeHost = metadata.activeHost();
+ log.debug("Local store for {}: {}, {}:{}", username, metadata.partition(), activeHost.host(), activeHost.port());
+
+ if (activeHost.equals(this.hostInfo))
+ {
+ return Optional.empty();
+ }
+
+ URI location = URI.create("http://" + activeHost.host() + ":" + activeHost.port() + "/" + username);
+ log.debug("Redirecting to {}", location);
+ return Optional.of(location);
+ }
+
+ public Optional<Ranking> getRanking(String username)
+ {
+ return
+ Optional
+ .ofNullable(streams.store(storeParameters).get(username))
+ .map(json ->
+ {
+ try
+ {
+ return mapper.readValue(json, Ranking.class);
+ }
+ catch (JsonProcessingException e)
+ {
+ throw new RuntimeException(e);
+ }
+ });
+ }
+
+ @PostConstruct
+ public void start()
+ {
+ log.info("Starting Stream-Processor");
+ streams.start();
+ }
+
+ @PreDestroy
+ public void stop()
+ {
+ log.info("Stopping Stream-Processor");
+ streams.close();
+ }
+}
--- /dev/null
+package de.juplo.kafka.wordcount.query;
+
+import lombok.Getter;
+import lombok.Setter;
+
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+
+
+@Getter
+@Setter
+public class Ranking
+{
+ private Entry[] entries;
+}
+++ /dev/null
-package de.juplo.kafka.wordcount.top10;
-
-import lombok.Value;
-
-
-@Value(staticConstructor = "of")
-public class Entry
-{
- private final String word;
- private final Long count;
-}
+++ /dev/null
-package de.juplo.kafka.wordcount.top10;
-
-import lombok.Getter;
-import lombok.Setter;
-
-
-@Getter
-@Setter
-public class Key
-{
- private String username;
- private String word;
-}
+++ /dev/null
-package de.juplo.kafka.wordcount.top10;
-
-import lombok.Getter;
-import lombok.Setter;
-
-import java.util.Arrays;
-import java.util.LinkedList;
-import java.util.List;
-
-
-@Getter
-@Setter
-public class Ranking
-{
- private Entry[] entries = new Entry[0];
-
- public void add(Entry newEntry)
- {
- if (entries.length == 0)
- {
- entries = new Entry[1];
- entries[0] = newEntry;
- return;
- }
-
- List<Entry> list = new LinkedList<>(Arrays.asList(entries));
- for (int i = 0; i < list.size(); i++)
- {
- Entry entry;
-
- entry = list.get(i);
- if (entry.getCount() <= newEntry.getCount())
- {
- list.add(i, newEntry);
- for (int j = i+1; j < list.size(); j++)
- {
- entry = list.get(j);
- if(entry.getWord().equals(newEntry.getWord()))
- {
- list.remove(j);
- break;
- }
- }
- if (list.size() > 10)
- {
- list = list.subList(0,10);
- }
- entries = list.toArray(num -> new Entry[num]);
- return;
- }
- }
- }
-}
+++ /dev/null
-package de.juplo.kafka.wordcount.top10;
-
-import org.springframework.boot.SpringApplication;
-import org.springframework.boot.autoconfigure.SpringBootApplication;
-import org.springframework.boot.context.properties.EnableConfigurationProperties;
-
-
-@SpringBootApplication
-@EnableConfigurationProperties(Top10ApplicationProperties.class)
-public class Top10Application
-{
- public static void main(String[] args)
- {
- SpringApplication.run(Top10Application.class, args);
- }
-}
+++ /dev/null
-package de.juplo.kafka.wordcount.top10;
-
-
-import lombok.Getter;
-import lombok.Setter;
-import lombok.ToString;
-import org.springframework.boot.context.properties.ConfigurationProperties;
-
-
-@ConfigurationProperties("juplo.wordcount.top10")
-@Getter
-@Setter
-@ToString
-public class Top10ApplicationProperties
-{
- private String bootstrapServer = "localhost:9092";
- private String applicationId = "top10";
- private String inputTopic = "countings";
- private String outputTopic = "top10";
-}
+++ /dev/null
-package de.juplo.kafka.wordcount.top10;
-
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import lombok.extern.slf4j.Slf4j;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.common.serialization.Serdes;
-import org.apache.kafka.streams.KafkaStreams;
-import org.apache.kafka.streams.KeyValue;
-import org.apache.kafka.streams.StreamsBuilder;
-import org.apache.kafka.streams.StreamsConfig;
-import org.springframework.boot.SpringApplication;
-import org.springframework.context.ConfigurableApplicationContext;
-import org.springframework.stereotype.Component;
-
-import javax.annotation.PostConstruct;
-import javax.annotation.PreDestroy;
-import java.util.Properties;
-import java.util.concurrent.CompletableFuture;
-import java.util.regex.Pattern;
-
-import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
-
-
-@Slf4j
-@Component
-public class Top10StreamProcessor
-{
- final static Pattern PATTERN = Pattern.compile("\\W+");
-
- public final KafkaStreams streams;
-
-
- public Top10StreamProcessor(
- Top10ApplicationProperties properties,
- ObjectMapper mapper,
- ConfigurableApplicationContext context)
- {
- StreamsBuilder builder = new StreamsBuilder();
-
- builder
- .<String, String>stream(properties.getInputTopic())
- .map((keyJson, countStr) ->
- {
- try
- {
- Key key = mapper.readValue(keyJson, Key.class);
- Long count = Long.parseLong(countStr);
- Entry entry = Entry.of(key.getWord(), count);
- String entryJson = mapper.writeValueAsString(entry);
- return new KeyValue<>(key.getUsername(), entryJson);
- }
- catch (JsonProcessingException e)
- {
- throw new RuntimeException(e);
- }
- })
- .groupByKey()
- .aggregate(
- () -> "{\"entries\" : []}",
- (username, entryJson, rankingJson) ->
- {
- try
- {
- Ranking ranking = mapper.readValue(rankingJson, Ranking.class);
- ranking.add(mapper.readValue(entryJson, Entry.class));
- return mapper.writeValueAsString(ranking);
- }
- catch (JsonProcessingException e)
- {
- throw new RuntimeException(e);
- }
- }
- )
- .toStream()
- .to(properties.getOutputTopic());
-
- Properties props = new Properties();
- props.put(StreamsConfig.APPLICATION_ID_CONFIG, properties.getApplicationId());
- props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, properties.getBootstrapServer());
- props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
- props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
- props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
-
- 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;
- });
- }
-
- @PostConstruct
- public void start()
- {
- log.info("Starting Stream-Processor");
- streams.start();
- }
-
- @PreDestroy
- public void stop()
- {
- log.info("Stopping Stream-Processor");
- streams.close();
- }
-}