package de.juplo.kafka.wordcount.splitter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.streams.StreamsConfig;
import org.springframework.boot.SpringApplication;
import org.springframework.boot.autoconfigure.SpringBootApplication;
import org.springframework.boot.context.properties.EnableConfigurationProperties;
+import org.springframework.context.ConfigurableApplicationContext;
+import org.springframework.context.annotation.Bean;
+import org.springframework.kafka.support.serializer.JsonDeserializer;
+import org.springframework.kafka.support.serializer.JsonSerde;
+import org.springframework.kafka.support.serializer.JsonSerializer;
+
+import java.util.Properties;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
@SpringBootApplication
@EnableConfigurationProperties(SplitterApplicationProperties.class)
+@Slf4j
public class SplitterApplication
{
+ @Bean(initMethod = "start", destroyMethod = "stop")
+ public SplitterStreamProcessor streamProcessor(
+ SplitterApplicationProperties properties,
+ ConfigurableApplicationContext context)
+ {
+ Properties propertyMap = new Properties();
+
+ propertyMap.put(StreamsConfig.APPLICATION_ID_CONFIG, properties.getApplicationId());
+ propertyMap.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, properties.getBootstrapServer());
+ propertyMap.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.StringSerde.class.getName());
+ propertyMap.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, JsonSerde.class.getName());
+ propertyMap.put(JsonDeserializer.TRUSTED_PACKAGES, Recording.class.getName() );
+ propertyMap.put(JsonDeserializer.VALUE_DEFAULT_TYPE, Recording.class.getName());
+ propertyMap.put(JsonSerializer.ADD_TYPE_INFO_HEADERS, false);
+ propertyMap.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+
+ SplitterStreamProcessor streamProcessor = new SplitterStreamProcessor(
+ properties.getInputTopic(),
+ properties.getOutputTopic(),
+ propertyMap);
+
+ streamProcessor.streams.setUncaughtExceptionHandler((Throwable e) ->
+ {
+ log.error("Unexpected error!", e);
+ CompletableFuture.runAsync(() ->
+ {
+ log.info("Stopping application...");
+ SpringApplication.exit(context, () -> 1);
+ });
+ return SHUTDOWN_CLIENT;
+ });
+
+
+ return streamProcessor;
+ }
+
public static void main(String[] args)
{
SpringApplication.run(SplitterApplication.class, args);
package de.juplo.kafka.wordcount.splitter;
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.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.kafka.support.serializer.JsonDeserializer;
-import org.springframework.kafka.support.serializer.JsonSerde;
-import org.springframework.kafka.support.serializer.JsonSerializer;
-import org.springframework.stereotype.Component;
-import jakarta.annotation.PostConstruct;
-import jakarta.annotation.PreDestroy;
import java.util.Arrays;
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 SplitterStreamProcessor
{
final static Pattern PATTERN = Pattern.compile("[^\\p{IsAlphabetic}]+");
public SplitterStreamProcessor(
- SplitterApplicationProperties properties,
- ConfigurableApplicationContext context)
+ String inputTopic,
+ String outputTopic,
+ Properties properties)
{
StreamsBuilder builder = new StreamsBuilder();
- KStream<String, Recording> source = builder.stream(properties.getInputTopic());
+ KStream<String, Recording> source = builder.stream(inputTopic);
source
.flatMapValues(recording -> Arrays
.stream(PATTERN.split(recording.getSentence()))
.map(word -> Word.of(recording.getUser(), word))
.toList())
- .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.StringSerde.class.getName());
- props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, JsonSerde.class.getName());
- props.put(JsonDeserializer.TRUSTED_PACKAGES, Recording.class.getName() );
- props.put(JsonDeserializer.VALUE_DEFAULT_TYPE, Recording.class.getName());
- props.put(JsonSerializer.ADD_TYPE_INFO_HEADERS, false);
- props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+ .to(outputTopic);
- 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;
- });
+ streams = new KafkaStreams(builder.build(), properties);
}
- @PostConstruct
public void start()
{
log.info("Starting Stream-Processor");
streams.start();
}
- @PreDestroy
public void stop()
{
log.info("Stopping Stream-Processor");