X-Git-Url: https://juplo.de/gitweb/?a=blobdiff_plain;f=src%2Fmain%2Fjava%2Fde%2Fjuplo%2Fkafka%2Fwordcount%2Fcounter%2FCounterStreamProcessor.java;h=2304e558c242359f9303d6c2fdbfe33b4b419120;hb=refs%2Fheads%2Fcounter;hp=524860977e15dfed0a1b9a8af0061f08b4ff851b;hpb=aec12e4b0a06b6a8f32c789dd75be45c6e71f343;p=demos%2Fkafka%2Fwordcount diff --git a/src/main/java/de/juplo/kafka/wordcount/counter/CounterStreamProcessor.java b/src/main/java/de/juplo/kafka/wordcount/counter/CounterStreamProcessor.java index 5248609..455d895 100644 --- a/src/main/java/de/juplo/kafka/wordcount/counter/CounterStreamProcessor.java +++ b/src/main/java/de/juplo/kafka/wordcount/counter/CounterStreamProcessor.java @@ -1,92 +1,132 @@ package de.juplo.kafka.wordcount.counter; -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.apache.kafka.streams.kstream.KStream; -import org.springframework.boot.SpringApplication; -import org.springframework.context.ConfigurableApplicationContext; -import org.springframework.stereotype.Component; - -import jakarta.annotation.PostConstruct; -import jakarta.annotation.PreDestroy; -import java.util.Properties; -import java.util.concurrent.CompletableFuture; +import org.apache.kafka.streams.*; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; +import org.apache.kafka.streams.state.QueryableStoreTypes; +import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; +import org.springframework.kafka.support.serializer.JsonSerde; +import org.springframework.kafka.support.serializer.JsonSerializer; -import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; @Slf4j -@Component public class CounterStreamProcessor { + public static final String TYPE = "COUNTER"; + public static final String STORE_NAME = "counter"; + + public final KafkaStreams streams; public CounterStreamProcessor( - CounterApplicationProperties properties, - ObjectMapper mapper, - ConfigurableApplicationContext context) + String inputTopic, + String outputTopic, + Properties properties, + KeyValueBytesStoreSupplier storeSupplier) + { + Topology topology = CounterStreamProcessor.buildTopology( + inputTopic, + outputTopic, + storeSupplier); + + streams = new KafkaStreams(topology, properties); + } + + static Topology buildTopology( + String inputTopic, + String outputTopic, + KeyValueBytesStoreSupplier storeSupplier) { StreamsBuilder builder = new StreamsBuilder(); - KStream source = builder.stream(properties.getInputTopic()); - source - .map((username, word) -> - { - try - { - String key = mapper.writeValueAsString(Key.of(username, word)); - return new KeyValue<>(key, word); - } - catch (JsonProcessingException e) - { - throw new RuntimeException(e); - } - }) + builder + .stream(inputTopic, Consumed.with(inKeySerde(), inValueSerde())) + .mapValues(word -> Word.of(word.getUser(), word.getWord())) + .map((key, word) -> new KeyValue<>(word, word)) .groupByKey() - .count() - .mapValues(value->Long.toString(value)) + .count( + Materialized + .as(storeSupplier) + .withKeySerde(new JsonSerde<>(Word.class))) // No headers are present: fixed typing is needed! .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; - }); + .map((word, counter) -> new KeyValue<>(word, WordCounter.of(word, counter))) + .to(outputTopic, Produced.with(outKeySerde(), outValueSerde())); + + Topology topology = builder.build(); + log.info("\n\n{}", topology.describe()); + + return topology; + } + + ReadOnlyKeyValueStore getStore() + { + return streams.store(StoreQueryParameters.fromNameAndType(STORE_NAME, QueryableStoreTypes.keyValueStore())); } - @PostConstruct public void start() { log.info("Starting Stream-Processor"); streams.start(); } - @PreDestroy public void stop() { log.info("Stopping Stream-Processor"); streams.close(); } + + + + public static JsonSerde inKeySerde() + { + return new JsonSerde<>(User.class); + } + + public static JsonSerde inValueSerde() + { + return new JsonSerde<>(UserWord.class); + } + + public static JsonSerde outKeySerde() + { + return serde(true); + } + + public static JsonSerde outValueSerde() + { + return serde(false); + } + + public static JsonSerde serde(boolean isKey) + { + JsonSerde serde = new JsonSerde<>(); + serde.configure( + Map.of(JsonSerializer.TYPE_MAPPINGS, typeMappingsConfig()), + isKey); + return serde; + } + + private static String typeMappingsConfig() + { + return typeMappingsConfig(Word.class, WordCounter.class); + } + + public static String typeMappingsConfig(Class keyClass, Class counterClass) + { + return Map.of( + "key", keyClass, + "counter", counterClass) + .entrySet() + .stream() + .map(entry -> entry.getKey() + ":" + entry.getValue().getName()) + .collect(Collectors.joining(",")); + } }