diff --git a/README.md b/README.md
index 0504a9bd..91688f9d 100644
--- a/README.md
+++ b/README.md
@@ -26,7 +26,13 @@ You can add streams-bootstrap via Maven Central.
#### Gradle
```gradle
-compile group: 'com.bakdata.kafka', name: 'streams-bootstrap', version: '2.1.1'
+implementation group: 'com.bakdata.kafka', name: 'streams-bootstrap-cli', version: '3.0.0'
+```
+
+With Kotlin DSL
+
+```gradle
+implementation(group = "com.bakdata.kafka", name = "streams-bootstrap-cli", version = "3.0.0")
```
#### Maven
@@ -35,8 +41,8 @@ compile group: 'com.bakdata.kafka', name: 'streams-bootstrap', version: '2.1.1'
com.bakdata.kafka
- streams-bootstrap
- 2.1.1
+ streams-bootstrap-cli
+ 3.0.0
```
@@ -52,8 +58,10 @@ and `getUniqueAppId()`. You can define the topology of your application in `buil
```java
import com.bakdata.kafka.KafkaStreamsApplication;
-import java.util.Properties;
-import org.apache.kafka.streams.StreamsBuilder;
+import com.bakdata.kafka.StreamsApp;
+import com.bakdata.kafka.StreamsTopicConfig;
+import com.bakdata.kafka.TopologyBuilder;
+import java.util.Map;
import org.apache.kafka.streams.kstream.KStream;
public class StreamsBootstrapApplication extends KafkaStreamsApplication {
@@ -62,26 +70,30 @@ public class StreamsBootstrapApplication extends KafkaStreamsApplication {
}
@Override
- public void buildTopology(final StreamsBuilder builder) {
- final KStream input =
- builder.stream(this.getInputTopics());
+ public StreamsApp createApp(final boolean cleanUp) {
+ return new StreamsApp() {
+ @Override
+ public void buildTopology(final TopologyBuilder builder) {
+ final KStream input = builder.streamInput();
- // your topology
+ // your topology
- input.to(this.getOutputTopic());
- }
-
- @Override
- public String getUniqueAppId() {
- return "streams-bootstrap-app";
- }
+ input.to(builder.getTopics().getOutputTopic());
+ }
- // Optionally you can override the default streams bootstrap Kafka properties
- @Override
- protected Properties createKafkaProperties() {
- final Properties kafkaProperties = super.createKafkaProperties();
+ @Override
+ public String getUniqueAppId(final StreamsTopicConfig topics) {
+ return "streams-bootstrap-app-" + topics.getOutputTopic();
+ }
- return kafkaProperties;
+ // Optionally you can define custom Kafka properties
+ @Override
+ public Map createKafkaProperties() {
+ return Map.of(
+ // your config
+ );
+ }
+ };
}
}
```
@@ -92,6 +104,8 @@ The following configuration options are available:
- `--schema-registry-url`: The URL of the Schema Registry
+- `--kafka-config`: Kafka Streams configuration (`[,...]`)
+
- `--input-topics`: List of input topics (comma-separated)
- `--input-pattern`: Pattern of input topics
@@ -100,8 +114,6 @@ The following configuration options are available:
- `--error-topic`: A topic to write errors to
-- `--streams-config`: Kafka Streams configuration (`[,...]`)
-
- `--extra-input-topics`: Additional named input topics if you need to specify multiple topics with different message
types (`[,...]`)
@@ -113,13 +125,15 @@ The following configuration options are available:
- `--volatile-group-instance-id`: Whether the group instance id is volatile, i.e., it will change on a Streams shutdown.
-- `--clean-up`: Whether the state of the Kafka Streams app, i.e., offsets and state stores and auto-created topics,
- should be cleared instead of running the app
+- `--debug`: Configure logging to debug
+
+Additionally, the following commands are available:
-- `--delete-output`: Whether the output topics with their associated schemas and the consumer group should be deleted
- during the cleanup
+- `clean`: Reset the Kafka Streams application. Additionally, delete the consumer group and all output and intermediate
+ topics associated with the Kafka Streams application.
-- `--debug`: Configure logging to debug
+- `reset`: Clear all state stores, consumer group offsets, and internal topics associated with the Kafka Streams
+ application.
#### Kafka producer
@@ -127,8 +141,11 @@ Create a subclass of `KafkaProducerApplication`.
```java
import com.bakdata.kafka.KafkaProducerApplication;
-import java.util.Properties;
-import org.apache.kafka.clients.producer.KafkaProducer;
+import com.bakdata.kafka.ProducerApp;
+import com.bakdata.kafka.ProducerBuilder;
+import com.bakdata.kafka.ProducerRunnable;
+import java.util.Map;
+import org.apache.kafka.clients.producer.Producer;
public class StreamsBootstrapApplication extends KafkaProducerApplication {
public static void main(final String[] args) {
@@ -136,18 +153,25 @@ public class StreamsBootstrapApplication extends KafkaProducerApplication {
}
@Override
- protected void runApplication() {
- try (final KafkaProducer producer = this.createProducer()) {
- // your producer
+ public ProducerApp createApp(final boolean cleanUp) {
+ return new ProducerApp() {
+ @Override
+ public ProducerRunnable buildRunnable(final ProducerBuilder builder) {
+ return () -> {
+ try (final Producer producer = builder.createProducer()) {
+ // your producer
+ }
+ };
}
- }
-
- // Optionally you can override the default streams bootstrap Kafka properties
- @Override
- protected Properties createKafkaProperties() {
- final Properties kafkaProperties = super.createKafkaProperties();
- return kafkaProperties;
+ // Optionally you can define custom Kafka properties
+ @Override
+ public Map createKafkaProperties() {
+ return Map.of(
+ // your config
+ );
+ }
+ };
}
}
```
@@ -158,17 +182,18 @@ The following configuration options are available:
- `--schema-registry-url`: The URL of the Schema Registry
-- `--output-topic`: The output topic
+- `--kafka-config`: Kafka producer configuration (`[,...]`)
-- `--streams-config`: Kafka producer configuration (`[,...]`)
+- `--output-topic`: The output topic
- `--extra-output-topics`: Additional named output topics (`String=String>[,...]`)
-- `--clean-up`: Whether the output topics and associated schemas of the producer app should be deleted instead of
- running the app
-
- `--debug`: Configure logging to debug
+Additionally, the following commands are available:
+
+- `clean`: Delete all output topics associated with the Kafka Producer application.
+
### Helm Charts
For the configuration and deployment to Kubernetes, you can use
diff --git a/build.gradle.kts b/build.gradle.kts
index 693286fe..5cfa071e 100644
--- a/build.gradle.kts
+++ b/build.gradle.kts
@@ -1,7 +1,7 @@
plugins {
id("com.bakdata.release") version "1.4.0"
id("com.bakdata.sonar") version "1.4.0"
- id("com.bakdata.sonatype") version "1.4.0"
+ id("com.bakdata.sonatype") version "1.4.1"
id("io.freefair.lombok") version "8.4"
}
@@ -16,6 +16,7 @@ allprojects {
repositories {
mavenCentral()
maven(url = "https://packages.confluent.io/maven/")
+ maven(url = "https://s01.oss.sonatype.org/content/repositories/snapshots")
}
}
diff --git a/charts/producer-app-cleanup-job/templates/job.yaml b/charts/producer-app-cleanup-job/templates/job.yaml
index 94d0e5f3..65f11c2d 100644
--- a/charts/producer-app-cleanup-job/templates/job.yaml
+++ b/charts/producer-app-cleanup-job/templates/job.yaml
@@ -55,6 +55,8 @@ spec:
imagePullPolicy: "{{ .Values.imagePullPolicy }}"
resources:
{{ toYaml .Values.resources | indent 12 }}
+ args:
+ - clean
env:
- name: ENV_PREFIX
value: {{ .Values.configurationEnvPrefix }}_
@@ -74,8 +76,6 @@ spec:
- name: "{{ .Values.configurationEnvPrefix }}_DEBUG"
value: {{ .Values.debug | quote }}
{{- end }}
- - name: "{{ .Values.configurationEnvPrefix }}_CLEAN_UP"
- value: "true"
{{- if hasKey .Values.streams "outputTopic" }}
- name: "{{ .Values.configurationEnvPrefix }}_OUTPUT_TOPIC"
value: {{ .Values.streams.outputTopic | quote }}
diff --git a/charts/producer-app/README.md b/charts/producer-app/README.md
index daf5a657..2366b8db 100644
--- a/charts/producer-app/README.md
+++ b/charts/producer-app/README.md
@@ -47,7 +47,7 @@ Alternatively, a YAML file that specifies the values for the parameters can be p
### Streams
| Parameter | Description | Default |
-| --------------------------- | ---------------------------------------------------------------------------------------------------------- | ------- |
+|-----------------------------|------------------------------------------------------------------------------------------------------------|---------|
| `streams.brokers` | Comma separated list of Kafka brokers to connect to. | |
| `streams.schemaRegistryUrl` | URL of Schema Registry to connect to. | `null` |
| `streams.config` | Configurations for your [Kafka producer app](https://kafka.apache.org/documentation/#producerconfigs). | `{}` |
diff --git a/charts/streams-app-cleanup-job/templates/job.yaml b/charts/streams-app-cleanup-job/templates/job.yaml
index e2e315c4..7b19f207 100644
--- a/charts/streams-app-cleanup-job/templates/job.yaml
+++ b/charts/streams-app-cleanup-job/templates/job.yaml
@@ -55,6 +55,12 @@ spec:
imagePullPolicy: "{{ .Values.imagePullPolicy }}"
resources:
{{ toYaml .Values.resources | indent 12 }}
+ args:
+ {{- if .Values.streams.deleteOutput }}
+ - clean
+ {{- else }}
+ - reset
+ {{- end }}
env:
- name: ENV_PREFIX
value: {{ .Values.configurationEnvPrefix }}_
@@ -70,20 +76,10 @@ spec:
- name: "{{ .Values.configurationEnvPrefix }}_SCHEMA_REGISTRY_URL"
value: {{ .Values.streams.schemaRegistryUrl | quote }}
{{- end }}
- {{- if hasKey .Values.streams "productive" }}
- - name: "{{ .Values.configurationEnvPrefix }}_PRODUCTIVE"
- value: {{ .Values.streams.productive | quote }}
- {{- end }}
{{- if hasKey .Values "debug" }}
- name: "{{ .Values.configurationEnvPrefix }}_DEBUG"
value: {{ .Values.debug | quote }}
{{- end }}
- - name: "{{ .Values.configurationEnvPrefix }}_CLEAN_UP"
- value: "true"
- {{- if hasKey .Values.streams "deleteOutput" }}
- - name: "{{ .Values.configurationEnvPrefix }}_DELETE_OUTPUT"
- value: {{ .Values.streams.deleteOutput | quote }}
- {{- end }}
{{- if and (hasKey .Values.streams "inputTopics") (.Values.streams.inputTopics) }}
- name: "{{ .Values.configurationEnvPrefix }}_INPUT_TOPICS"
value: {{ .Values.streams.inputTopics | join "," | quote }}
diff --git a/charts/streams-app-cleanup-job/values.yaml b/charts/streams-app-cleanup-job/values.yaml
index 19426361..b3464e2a 100644
--- a/charts/streams-app-cleanup-job/values.yaml
+++ b/charts/streams-app-cleanup-job/values.yaml
@@ -36,7 +36,6 @@ streams:
extraOutputTopics: {}
# role: output
# errorTopic: error
-# productive: true
deleteOutput: false
commandLine: {}
diff --git a/charts/streams-app/README.md b/charts/streams-app/README.md
index 4698348c..77fa5803 100644
--- a/charts/streams-app/README.md
+++ b/charts/streams-app/README.md
@@ -50,21 +50,20 @@ Alternatively, a YAML file that specifies the values for the parameters can be p
### Streams
-| Parameter | Description | Default |
-| ------------------------------------ | -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ------- |
-| `streams.brokers` | Comma separated list of Kafka brokers to connect to. | |
-| `streams.schemaRegistryUrl` | URL of Schema Registry to connect to. | `null` |
-| `streams.staticMembership` | Whether to use [Kafka Static Group Membership](https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances). | `false` |
-| `streams.optimizeLeaveGroupBehavior` | Enabling this optimizes the leave group behavior when a pod is terminated. Depends on the deployment kind, i.e., `statefulSet`. Requires the app to use streams-bootstrap 2.7+. | `true` |
-| `streams.config` | Configurations for your [Kafka Streams app](https://kafka.apache.org/documentation/#streamsconfigs). | `{}` |
-| `streams.inputTopics` | List of input topics for your streams application. | `[]` |
-| `streams.extraInputTopics` | Map of additional named input topics if you need to specify multiple topics with different message types. | `{}` |
-| `streams.inputPattern` | Input pattern of topics for your streams application. | |
-| `streams.extraInputPatterns` | Map of additional named input patterns if you need to specify multiple topics with different message types. | `{}` |
-| `streams.outputTopic` | Output topic for your streams application. | |
-| `streams.extraOutputTopics` | Map of additional named output topics if you need to specify multiple topics with different message types. | `{}` |
-| `streams.errorTopic` | Error topic for your streams application. | |
-| `streams.productive` | Whether to use Kafka configuration values that are more suitable for production environments. | `true` |
+| Parameter | Description | Default |
+|------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|---------|
+| `streams.brokers` | Comma separated list of Kafka brokers to connect to. | |
+| `streams.schemaRegistryUrl` | URL of Schema Registry to connect to. | `null` |
+| `streams.staticMembership` | Whether to use [Kafka Static Group Membership](https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances). | `false` |
+| `streams.config` | Configurations for your [Kafka Streams app](https://kafka.apache.org/documentation/#streamsconfigs). | `{}` |
+| `streams.inputTopics` | List of input topics for your streams application. | `[]` |
+| `streams.extraInputTopics` | Map of additional named input topics if you need to specify multiple topics with different message types. | `{}` |
+| `streams.inputPattern` | Input pattern of topics for your streams application. | |
+| `streams.extraInputPatterns` | Map of additional named input patterns if you need to specify multiple topics with different message types. | `{}` |
+| `streams.outputTopic` | Output topic for your streams application. | |
+| `streams.extraOutputTopics` | Map of additional named output topics if you need to specify multiple topics with different message types. | `{}` |
+| `streams.errorTopic` | Error topic for your streams application. | |
+| `streams.productive` | Whether to use Kafka configuration values that are more suitable for production environments. | `true` |
### Other
diff --git a/charts/streams-app/templates/deployment.yaml b/charts/streams-app/templates/deployment.yaml
index 5b98e856..c13b080c 100644
--- a/charts/streams-app/templates/deployment.yaml
+++ b/charts/streams-app/templates/deployment.yaml
@@ -111,7 +111,7 @@ spec:
- name: KAFKA_JMX_PORT
value: "{{ .Values.jmx.port }}"
{{- end }}
- {{- if and (.Values.streams.optimizeLeaveGroupBehavior) (not .Values.statefulSet) }}
+ {{- if not .Values.statefulSet }}
- name: "{{ .Values.configurationEnvPrefix }}_VOLATILE_GROUP_INSTANCE_ID"
value: "true"
{{- end }}
@@ -123,10 +123,6 @@ spec:
- name: "{{ .Values.configurationEnvPrefix }}_SCHEMA_REGISTRY_URL"
value: {{ .Values.streams.schemaRegistryUrl | quote }}
{{- end }}
- {{- if hasKey .Values.streams "productive" }}
- - name: "{{ .Values.configurationEnvPrefix }}_PRODUCTIVE"
- value: {{ .Values.streams.productive | quote }}
- {{- end }}
{{- if hasKey .Values "debug" }}
- name: "{{ .Values.configurationEnvPrefix }}_DEBUG"
value: {{ .Values.debug | quote }}
diff --git a/charts/streams-app/values.yaml b/charts/streams-app/values.yaml
index 43749141..b3627e9d 100644
--- a/charts/streams-app/values.yaml
+++ b/charts/streams-app/values.yaml
@@ -27,7 +27,6 @@ streams:
# brokers: "test:9092"
# schemaRegistryUrl: "url:1234"
staticMembership: false
- optimizeLeaveGroupBehavior: true
config: {}
# max.poll.records: 500
# Note that YAML may convert large integers to scientific notation. Use Strings to avoid this.
@@ -46,7 +45,6 @@ streams:
extraOutputTopics: {}
# role: output
# errorTopic: error
- # productive: true
commandLine: {}
# MY_CLI_PARAM: "foo-bar"
diff --git a/gradle.properties b/gradle.properties
index 9a9e33a9..2293871c 100644
--- a/gradle.properties
+++ b/gradle.properties
@@ -1,7 +1,13 @@
-version=2.23.1-SNAPSHOT
+version=3.0.0-SNAPSHOT
org.gradle.caching=true
-org.gradle.parallel=true
+# running Kafka JUnit in parallel causes problems
+org.gradle.parallel=false
kafkaVersion=3.6.1
+kafkaJunitVersion=3.6.0
confluentVersion=7.6.0
-fluentKafkaVersion=2.13.1
-org.gradle.jvmargs=-Xmx2048m
+fluentKafkaVersion=2.14.0
+junitVersion=5.10.2
+mockitoVersion=5.11.0
+assertJVersion=3.25.3
+log4jVersion=2.23.1
+org.gradle.jvmargs=-Xmx4096m
diff --git a/settings.gradle b/settings.gradle
index c7e6e4b6..e9057df9 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -7,7 +7,8 @@ pluginManagement {
rootProject.name = 'streams-bootstrap'
include(
- ":streams-bootstrap",
+ ":streams-bootstrap-core",
":streams-bootstrap-test",
":streams-bootstrap-large-messages",
+ ":streams-bootstrap-cli",
)
diff --git a/streams-bootstrap-cli/build.gradle.kts b/streams-bootstrap-cli/build.gradle.kts
new file mode 100644
index 00000000..da3fc86c
--- /dev/null
+++ b/streams-bootstrap-cli/build.gradle.kts
@@ -0,0 +1,34 @@
+description = "Base classes to create standalone Java applications using picocli"
+
+plugins {
+ id("com.github.davidmc24.gradle.plugin.avro") version "1.9.1"
+}
+
+dependencies {
+ api(project(":streams-bootstrap-core"))
+ api(group = "info.picocli", name = "picocli", version = "4.7.5")
+ val log4jVersion: String by project
+ implementation(group = "org.apache.logging.log4j", name = "log4j-core", version = log4jVersion)
+ implementation(group = "org.apache.logging.log4j", name = "log4j-slf4j2-impl", version = log4jVersion)
+
+ val junitVersion: String by project
+ testRuntimeOnly(group = "org.junit.jupiter", name = "junit-jupiter-engine", version = junitVersion)
+ testImplementation(group = "org.junit.jupiter", name = "junit-jupiter-api", version = junitVersion)
+ testImplementation(group = "org.junit.jupiter", name = "junit-jupiter-params", version = junitVersion)
+ val assertJVersion: String by project
+ testImplementation(group = "org.assertj", name = "assertj-core", version = assertJVersion)
+ val mockitoVersion: String by project
+ testImplementation(group = "org.mockito", name = "mockito-core", version = mockitoVersion)
+ testImplementation(group = "org.mockito", name = "mockito-junit-jupiter", version = mockitoVersion)
+ val kafkaJunitVersion: String by project
+ testImplementation(group = "net.mguenther.kafka", name = "kafka-junit", version = kafkaJunitVersion) {
+ exclude(group = "org.slf4j", module = "slf4j-log4j12")
+ }
+ testImplementation(group = "com.ginsberg", name = "junit5-system-exit", version = "1.1.2")
+ val fluentKafkaVersion: String by project
+ testImplementation(
+ group = "com.bakdata.fluent-kafka-streams-tests",
+ name = "schema-registry-mock-junit5",
+ version = fluentKafkaVersion
+ )
+}
diff --git a/streams-bootstrap/src/main/java/com/bakdata/kafka/EnvironmentArgumentsParser.java b/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/EnvironmentArgumentsParser.java
similarity index 98%
rename from streams-bootstrap/src/main/java/com/bakdata/kafka/EnvironmentArgumentsParser.java
rename to streams-bootstrap-cli/src/main/java/com/bakdata/kafka/EnvironmentArgumentsParser.java
index f105a308..d93d7dc8 100644
--- a/streams-bootstrap/src/main/java/com/bakdata/kafka/EnvironmentArgumentsParser.java
+++ b/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/EnvironmentArgumentsParser.java
@@ -1,7 +1,7 @@
/*
* MIT License
*
- * Copyright (c) 2023 bakdata
+ * Copyright (c) 2024 bakdata
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
@@ -36,7 +36,7 @@
/**
* This class is primarily used to inject environment variables to the passed in command line arguments
- * in {@link KafkaStreamsApplication}.
+ * in {@link KafkaPropertiesFactory}.
*
* In general a usage would look like this:
* {@code
diff --git a/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/KafkaApplication.java b/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/KafkaApplication.java
new file mode 100644
index 00000000..e0a1f84c
--- /dev/null
+++ b/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/KafkaApplication.java
@@ -0,0 +1,382 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import static java.util.Collections.emptyMap;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.function.Consumer;
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.core.config.Configurator;
+import picocli.CommandLine;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.ParseResult;
+
+/**
+ * The base class for creating Kafka applications.
+ * This class provides the following configuration options:
+ *
+ * {@link #brokers}
+ * {@link #outputTopic}
+ * {@link #extraOutputTopics}
+ * {@link #brokers}
+ * {@link #debug}
+ * {@link #schemaRegistryUrl}
+ * {@link #kafkaConfig}
+ *
+ * To implement your Kafka application inherit from this class and add your custom options. Run it by calling
+ * {@link #startApplication(KafkaApplication, String[])} with a instance of your class from your main.
+ *
+ * @param type of {@link Runner} used by this app
+ * @param type of {@link CleanUpRunner} used by this app
+ * @param type of execution options to create runner
+ * @param type of {@link ExecutableApp} used by this app
+ * @param type of {@link ConfiguredApp} used by this app
+ * @param type of topic config used by this app
+ * @param type of app
+ */
+@ToString
+@Getter
+@Setter
+@RequiredArgsConstructor
+@Slf4j
+@Command(mixinStandardHelpOptions = true)
+public abstract class KafkaApplication, CA extends ConfiguredApp, T, A>
+ implements Runnable, AutoCloseable {
+ private static final String ENV_PREFIX = Optional.ofNullable(System.getenv("ENV_PREFIX")).orElse("APP_");
+ @ToString.Exclude
+ @Getter(AccessLevel.NONE)
+ // ConcurrentLinkedDeque required because calling #stop() causes asynchronous #run() calls to finish and thus
+ // concurrently iterating and removing from #runners
+ private final ConcurrentLinkedDeque activeApps = new ConcurrentLinkedDeque<>();
+ @CommandLine.Option(names = "--output-topic", description = "Output topic")
+ private String outputTopic;
+ @CommandLine.Option(names = "--extra-output-topics", split = ",", description = "Additional named output topics")
+ private Map extraOutputTopics = emptyMap();
+ @CommandLine.Option(names = "--brokers", required = true, description = "Broker addresses to connect to")
+ private String brokers;
+ @CommandLine.Option(names = "--debug", arity = "0..1", description = "Configure logging to debug")
+ private boolean debug;
+ @CommandLine.Option(names = "--schema-registry-url", description = "URL of Schema Registry")
+ private String schemaRegistryUrl;
+ @CommandLine.Option(names = "--kafka-config", split = ",", description = "Additional Kafka properties")
+ private Map kafkaConfig = emptyMap();
+
+ /**
+ * This methods needs to be called in the executable custom application class inheriting from
+ * {@code KafkaApplication}.
+ * This method calls System exit
+ *
+ * @param app An instance of the custom application class.
+ * @param args Arguments passed in by the custom application class.
+ * @see #startApplicationWithoutExit(KafkaApplication, String[])
+ */
+ public static void startApplication(final KafkaApplication, ?, ?, ?, ?, ?, ?> app, final String[] args) {
+ final int exitCode = startApplicationWithoutExit(app, args);
+ System.exit(exitCode);
+ }
+
+ /**
+ * This methods needs to be called in the executable custom application class inheriting from
+ * {@code KafkaApplication}.
+ *
+ * @param app An instance of the custom application class.
+ * @param args Arguments passed in by the custom application class.
+ * @return Exit code of application
+ */
+ public static int startApplicationWithoutExit(final KafkaApplication, ?, ?, ?, ?, ?, ?> app,
+ final String[] args) {
+ final String[] populatedArgs = addEnvironmentVariablesArguments(args);
+ final CommandLine commandLine = new CommandLine(app)
+ .setExecutionStrategy(app::execute);
+ return commandLine.execute(populatedArgs);
+ }
+
+ private static String[] addEnvironmentVariablesArguments(final String[] args) {
+ Preconditions.checkArgument(!ENV_PREFIX.equals(EnvironmentStreamsConfigParser.PREFIX),
+ "Prefix '" + EnvironmentStreamsConfigParser.PREFIX + "' is reserved for Streams config");
+ final List environmentArguments = new EnvironmentArgumentsParser(ENV_PREFIX)
+ .parseVariables(System.getenv());
+ final Collection allArgs = new ArrayList<>(environmentArguments);
+ allArgs.addAll(Arrays.asList(args));
+ return allArgs.toArray(String[]::new);
+ }
+
+ /**
+ * Create options for running the app
+ * @return run options if available
+ * @see ExecutableApp#createRunner(Object)
+ */
+ public abstract Optional createExecutionOptions();
+
+ /**
+ * Topics used by app
+ * @return topic configuration
+ */
+ public abstract T createTopicConfig();
+
+ /**
+ * Create a new app that will be configured and executed according to this application.
+ *
+ * @param cleanUp whether app is created for clean up purposes. In that case, the user might want
+ * to skip initialization of expensive resources.
+ * @return app
+ */
+ public abstract A createApp(boolean cleanUp);
+
+ /**
+ * Create a new app that will be configured and executed according to this application.
+ *
+ * @return app
+ */
+ public A createApp() {
+ return this.createApp(false);
+ }
+
+ /**
+ * Clean all resources associated with this application
+ */
+ public void clean() {
+ try (final CleanableApp cleanableApp = this.createCleanableApp()) {
+ final CR cleanUpRunner = cleanableApp.getCleanUpRunner();
+ cleanUpRunner.clean();
+ }
+ }
+
+ /**
+ * @see #stop()
+ */
+ @Override
+ public void close() {
+ this.stop();
+ }
+
+ /**
+ * Stop all applications that have been started asynchronously, e.g., by using {@link #run()} or {@link #clean()}.
+ */
+ public final void stop() {
+ this.activeApps.forEach(Stoppable::stop);
+ }
+
+ /**
+ * Run the application.
+ */
+ @Override
+ public void run() {
+ try (final RunnableApp runnableApp = this.createRunnableApp()) {
+ final R runner = runnableApp.getRunner();
+ runner.run();
+ }
+ }
+
+ public KafkaEndpointConfig getEndpointConfig() {
+ return KafkaEndpointConfig.builder()
+ .brokers(this.brokers)
+ .schemaRegistryUrl(this.schemaRegistryUrl)
+ .build();
+ }
+
+ /**
+ * Create a new {@code ExecutableApp} that will be executed according to the requested command.
+ * @return {@code ExecutableApp}
+ */
+ public final E createExecutableApp() {
+ return this.createExecutableApp(false);
+ }
+
+ /**
+ * Create a new {@code ExecutableApp} that will be executed according to the requested command.
+ *
+ * @param cleanUp whether app is created for clean up purposes. In that case, the user might want to skip
+ * initialization of expensive resources.
+ * @return {@code ExecutableApp}
+ */
+ public final E createExecutableApp(final boolean cleanUp) {
+ final ConfiguredApp configuredStreamsApp = this.createConfiguredApp(cleanUp);
+ final KafkaEndpointConfig endpointConfig = this.getEndpointConfig();
+ return configuredStreamsApp.withEndpoint(endpointConfig);
+ }
+
+ /**
+ * Create a new {@code ConfiguredApp} that will be executed according to this application.
+ * @return {@code ConfiguredApp}
+ */
+ public final CA createConfiguredApp() {
+ return this.createConfiguredApp(false);
+ }
+
+ /**
+ * Create a new {@code ConfiguredApp} that will be executed according to this application.
+ *
+ * @param cleanUp whether {@code ConfiguredApp} is created for clean up purposes. In that case, the user might want
+ * to skip initialization of expensive resources.
+ * @return {@code ConfiguredApp}
+ */
+ public final CA createConfiguredApp(final boolean cleanUp) {
+ final AppConfiguration configuration = this.createConfiguration();
+ final A app = this.createApp(cleanUp);
+ return this.createConfiguredApp(app, configuration);
+ }
+
+ /**
+ * Create configuration to configure app
+ * @return configuration
+ */
+ public final AppConfiguration createConfiguration() {
+ final T topics = this.createTopicConfig();
+ return new AppConfiguration<>(topics, this.kafkaConfig);
+ }
+
+ /**
+ * Create a new {@code RunnableApp}
+ * @return {@code RunnableApp}
+ */
+ public final RunnableApp createRunnableApp() {
+ final ExecutableApp app = this.createExecutableApp(false);
+ final Optional executionOptions = this.createExecutionOptions();
+ final R runner = executionOptions.map(app::createRunner).orElseGet(app::createRunner);
+ final RunnableApp runnableApp = new RunnableApp<>(app, runner, this.activeApps::remove);
+ this.activeApps.add(runnableApp);
+ return runnableApp;
+ }
+
+ /**
+ * Create a new {@code CleanableApp}
+ * @return {@code CleanableApp}
+ */
+ public final CleanableApp createCleanableApp() {
+ final ExecutableApp executableApp = this.createExecutableApp(true);
+ final CR cleanUpRunner = executableApp.createCleanUpRunner();
+ final CleanableApp cleanableApp = new CleanableApp<>(executableApp, cleanUpRunner, this.activeApps::remove);
+ this.activeApps.add(cleanableApp);
+ return cleanableApp;
+ }
+
+ /**
+ * Create a new {@code ConfiguredApp} that will be executed according to the given config.
+ *
+ * @param app app to configure.
+ * @param configuration configuration for app
+ * @return {@code ConfiguredApp}
+ */
+ protected abstract CA createConfiguredApp(final A app, AppConfiguration configuration);
+
+ /**
+ * Configure application when running in debug mode. By default, Log4j2 log level is configured to debug for
+ * {@code com.bakdata} and the applications package.
+ */
+ protected void configureDebug() {
+ Configurator.setLevel("com.bakdata", Level.DEBUG);
+ Configurator.setLevel(this.getClass().getPackageName(), Level.DEBUG);
+ }
+
+ private void startApplication() {
+ Runtime.getRuntime().addShutdownHook(new Thread(this::close));
+ log.info("Starting application");
+ if (this.debug) {
+ this.configureDebug();
+ }
+ log.debug("Starting application: {}", this);
+ }
+
+ private int execute(final ParseResult parseResult) {
+ this.startApplication();
+ final int exitCode = new CommandLine.RunLast().execute(parseResult);
+ this.close();
+ return exitCode;
+ }
+
+ @FunctionalInterface
+ private interface Stoppable {
+ void stop();
+ }
+
+ /**
+ * Provides access to a {@link CleanUpRunner} and closes the associated {@link ExecutableApp}
+ */
+ @RequiredArgsConstructor(access = AccessLevel.PROTECTED)
+ public static class CleanableApp implements AutoCloseable, Stoppable {
+ private final @NonNull ExecutableApp, ?, ?> app;
+ @Getter
+ private final @NonNull CR cleanUpRunner;
+ private final @NonNull Consumer onClose;
+
+ @Override
+ public void close() {
+ this.stop();
+ this.onClose.accept(this);
+ }
+
+ /**
+ * Close the app
+ */
+ @Override
+ public void stop() {
+ this.app.close();
+ }
+ }
+
+ /**
+ * Provides access to a {@link Runner} and closes the associated {@link ExecutableApp}
+ */
+ @RequiredArgsConstructor(access = AccessLevel.PROTECTED)
+ public static final class RunnableApp implements AutoCloseable, Stoppable {
+ private final @NonNull ExecutableApp, ?, ?> app;
+ @Getter
+ private final @NonNull R runner;
+ private final @NonNull Consumer onClose;
+
+ @Override
+ public void close() {
+ this.stop();
+ this.onClose.accept(this);
+ }
+
+ /**
+ * Close the runner and app
+ */
+ @Override
+ public void stop() {
+ this.runner.close();
+ // close app after runner because messages currently processed might depend on resources
+ this.app.close();
+ }
+ }
+}
diff --git a/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/KafkaProducerApplication.java b/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/KafkaProducerApplication.java
new file mode 100644
index 00000000..e38f3408
--- /dev/null
+++ b/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/KafkaProducerApplication.java
@@ -0,0 +1,80 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import java.util.Optional;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import picocli.CommandLine.Command;
+
+
+/**
+ * The base class for creating Kafka Producer applications.
+ * This class provides all configuration options provided by {@link KafkaApplication}.
+ * To implement your Kafka Producer application inherit from this class and add your custom options. Run it by
+ * calling {@link #startApplication(KafkaApplication, String[])} with a instance of your class from your main.
+ */
+@ToString(callSuper = true)
+@Getter
+@Setter
+@RequiredArgsConstructor
+@Slf4j
+@Command(description = "Run a Kafka Producer application")
+public abstract class KafkaProducerApplication extends
+ KafkaApplication, ConfiguredProducerApp, ProducerTopicConfig,
+ ProducerApp> {
+
+ /**
+ * Delete all output topics associated with the Kafka Producer application.
+ */
+ @Command(description = "Delete all output topics associated with the Kafka Producer application.")
+ @Override
+ public void clean() {
+ super.clean();
+ }
+
+ @Override
+ public final Optional createExecutionOptions() {
+ return Optional.empty();
+ }
+
+ @Override
+ public final ProducerTopicConfig createTopicConfig() {
+ return ProducerTopicConfig.builder()
+ .outputTopic(this.getOutputTopic())
+ .extraOutputTopics(this.getExtraOutputTopics())
+ .build();
+ }
+
+ @Override
+ public final ConfiguredProducerApp createConfiguredApp(final ProducerApp app,
+ final AppConfiguration configuration) {
+ return new ConfiguredProducerApp<>(app, configuration);
+ }
+}
diff --git a/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/KafkaStreamsApplication.java b/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/KafkaStreamsApplication.java
new file mode 100644
index 00000000..a6e62f21
--- /dev/null
+++ b/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/KafkaStreamsApplication.java
@@ -0,0 +1,168 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import static java.util.Collections.emptyList;
+import static java.util.Collections.emptyMap;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.regex.Pattern;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KafkaStreams.StateListener;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler;
+import picocli.CommandLine;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.UseDefaultConverter;
+
+
+/**
+ * The base class for creating Kafka Streams applications.
+ * This class provides the following configuration options in addition to those provided by {@link KafkaApplication}:
+ *
+ * {@link #inputTopics}
+ * {@link #inputPattern}
+ * {@link #errorTopic}
+ * {@link #extraInputTopics}
+ * {@link #extraInputPatterns}
+ * {@link #volatileGroupInstanceId}
+ *
+ * To implement your Kafka Streams application inherit from this class and add your custom options. Run it by calling
+ * {@link #startApplication(KafkaApplication, String[])} with a instance of your class from your main.
+ */
+@ToString(callSuper = true)
+@Getter
+@Setter
+@RequiredArgsConstructor
+@Slf4j
+@Command(description = "Run a Kafka Streams application.")
+public abstract class KafkaStreamsApplication extends
+ KafkaApplication, ConfiguredStreamsApp, StreamsTopicConfig, StreamsApp> {
+ @CommandLine.Option(names = "--input-topics", description = "Input topics", split = ",")
+ private List inputTopics = emptyList();
+ @CommandLine.Option(names = "--input-pattern", description = "Input pattern")
+ private Pattern inputPattern;
+ @CommandLine.Option(names = "--error-topic", description = "Error topic")
+ private String errorTopic;
+ @CommandLine.Option(names = "--extra-input-topics", split = ",", description = "Additional named input topics",
+ converter = {UseDefaultConverter.class, StringListConverter.class})
+ private Map> extraInputTopics = emptyMap();
+ @CommandLine.Option(names = "--extra-input-patterns", split = ",", description = "Additional named input patterns")
+ private Map extraInputPatterns = emptyMap();
+ @CommandLine.Option(names = "--volatile-group-instance-id", arity = "0..1",
+ description = "Whether the group instance id is volatile, i.e., it will change on a Streams shutdown.")
+ private boolean volatileGroupInstanceId;
+
+ /**
+ * Reset the Kafka Streams application. Additionally, delete the consumer group and all output and intermediate
+ * topics associated with the Kafka Streams application.
+ */
+ @Command(description = "Reset the Kafka Streams application. Additionally, delete the consumer group and all "
+ + "output and intermediate topics associated with the Kafka Streams application.")
+ @Override
+ public void clean() {
+ super.clean();
+ }
+
+ /**
+ * Clear all state stores, consumer group offsets, and internal topics associated with the Kafka Streams
+ * application.
+ */
+ @Command(description = "Clear all state stores, consumer group offsets, and internal topics associated with the "
+ + "Kafka Streams application.")
+ public void reset() {
+ try (final CleanableApp app = this.createCleanableApp()) {
+ final StreamsCleanUpRunner runner = app.getCleanUpRunner();
+ runner.reset();
+ }
+ }
+
+ @Override
+ public final Optional createExecutionOptions() {
+ final StreamsExecutionOptions options = StreamsExecutionOptions.builder()
+ .volatileGroupInstanceId(this.volatileGroupInstanceId)
+ .uncaughtExceptionHandler(this::createUncaughtExceptionHandler)
+ .stateListener(this::createStateListener)
+ .onStart(this::onStreamsStart)
+ .build();
+ return Optional.of(options);
+ }
+
+ @Override
+ public final StreamsTopicConfig createTopicConfig() {
+ return StreamsTopicConfig.builder()
+ .inputTopics(this.inputTopics)
+ .extraInputTopics(this.extraInputTopics)
+ .inputPattern(this.inputPattern)
+ .extraInputPatterns(this.extraInputPatterns)
+ .outputTopic(this.getOutputTopic())
+ .extraOutputTopics(this.getExtraOutputTopics())
+ .errorTopic(this.errorTopic)
+ .build();
+ }
+
+ @Override
+ public final ConfiguredStreamsApp createConfiguredApp(final StreamsApp app,
+ final AppConfiguration configuration) {
+ return new ConfiguredStreamsApp<>(app, configuration);
+ }
+
+ /**
+ * Create a {@link StateListener} to use for Kafka Streams.
+ *
+ * @return {@code StateListener}. {@link NoOpStateListener} by default
+ * @see KafkaStreams#setStateListener(StateListener)
+ */
+ protected StateListener createStateListener() {
+ return new NoOpStateListener();
+ }
+
+ /**
+ * Create a {@link StreamsUncaughtExceptionHandler} to use for Kafka Streams.
+ *
+ * @return {@code StreamsUncaughtExceptionHandler}. {@link DefaultStreamsUncaughtExceptionHandler} by default
+ * @see KafkaStreams#setUncaughtExceptionHandler(StreamsUncaughtExceptionHandler)
+ */
+ protected StreamsUncaughtExceptionHandler createUncaughtExceptionHandler() {
+ return new DefaultStreamsUncaughtExceptionHandler();
+ }
+
+ /**
+ * Called after starting Kafka Streams
+ * @param runningStreams running {@link KafkaStreams} instance along with its {@link StreamsConfig} and
+ * {@link org.apache.kafka.streams.Topology}
+ */
+ protected void onStreamsStart(final RunningStreams runningStreams) {
+ // do nothing by default
+ }
+}
diff --git a/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/SimpleKafkaProducerApplication.java b/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/SimpleKafkaProducerApplication.java
new file mode 100644
index 00000000..e14e764a
--- /dev/null
+++ b/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/SimpleKafkaProducerApplication.java
@@ -0,0 +1,51 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import java.util.function.Function;
+import java.util.function.Supplier;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+
+/**
+ * {@code KafkaProducerApplication} without any additional configuration options.
+ */
+@RequiredArgsConstructor
+public final class SimpleKafkaProducerApplication extends KafkaProducerApplication {
+ private final @NonNull Function appFactory;
+
+ /**
+ * Create new {@code SimpleKafkaProducerApplication}
+ * @param appFactory factory to create {@code ProducerApp} without any parameters
+ */
+ public SimpleKafkaProducerApplication(final Supplier extends ProducerApp> appFactory) {
+ this(cleanUp -> appFactory.get());
+ }
+
+ @Override
+ public ProducerApp createApp(final boolean cleanUp) {
+ return this.appFactory.apply(cleanUp);
+ }
+}
diff --git a/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/SimpleKafkaStreamsApplication.java b/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/SimpleKafkaStreamsApplication.java
new file mode 100644
index 00000000..1279cdd8
--- /dev/null
+++ b/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/SimpleKafkaStreamsApplication.java
@@ -0,0 +1,52 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import java.util.function.Function;
+import java.util.function.Supplier;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+
+/**
+ * {@code KafkaStreamsApplication} without any additional configuration options.
+ */
+@RequiredArgsConstructor
+public final class SimpleKafkaStreamsApplication extends KafkaStreamsApplication {
+
+ private final @NonNull Function appFactory;
+
+ /**
+ * Create new {@code SimpleKafkaStreamsApplication}
+ * @param appFactory factory to create {@code StreamsApp} without any parameters
+ */
+ public SimpleKafkaStreamsApplication(final Supplier extends StreamsApp> appFactory) {
+ this(cleanUp -> appFactory.get());
+ }
+
+ @Override
+ public StreamsApp createApp(final boolean cleanUp) {
+ return this.appFactory.apply(cleanUp);
+ }
+}
diff --git a/streams-bootstrap/src/main/java/com/bakdata/kafka/StringListConverter.java b/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/StringListConverter.java
similarity index 89%
rename from streams-bootstrap/src/main/java/com/bakdata/kafka/StringListConverter.java
rename to streams-bootstrap-cli/src/main/java/com/bakdata/kafka/StringListConverter.java
index 9dceec13..a31657bc 100644
--- a/streams-bootstrap/src/main/java/com/bakdata/kafka/StringListConverter.java
+++ b/streams-bootstrap-cli/src/main/java/com/bakdata/kafka/StringListConverter.java
@@ -1,7 +1,7 @@
/*
* MIT License
*
- * Copyright (c) 2023 bakdata
+ * Copyright (c) 2024 bakdata
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
@@ -32,10 +32,10 @@
* Converter for lists inside collection type parsed by PicoCLI. List members need to be separated by {@code ;}
*/
public class StringListConverter implements ITypeConverter> {
- private static final Splitter TOPIC_SPLITTER = Splitter.on(";").omitEmptyStrings().trimResults();
+ private static final Splitter SPLITTER = Splitter.on(";").omitEmptyStrings().trimResults();
@Override
public List convert(final String value) {
- return TOPIC_SPLITTER.splitToList(value);
+ return SPLITTER.splitToList(value);
}
}
diff --git a/streams-bootstrap/src/test/avro/TestRecord.avsc b/streams-bootstrap-cli/src/test/avro/TestRecord.avsc
similarity index 100%
rename from streams-bootstrap/src/test/avro/TestRecord.avsc
rename to streams-bootstrap-cli/src/test/avro/TestRecord.avsc
diff --git a/streams-bootstrap/src/test/java/com/bakdata/kafka/CliTest.java b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/CliTest.java
similarity index 52%
rename from streams-bootstrap/src/test/java/com/bakdata/kafka/CliTest.java
rename to streams-bootstrap-cli/src/test/java/com/bakdata/kafka/CliTest.java
index 4593062e..cdede819 100644
--- a/streams-bootstrap/src/test/java/com/bakdata/kafka/CliTest.java
+++ b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/CliTest.java
@@ -1,7 +1,7 @@
/*
* MIT License
*
- * Copyright (c) 2023 bakdata
+ * Copyright (c) 2024 bakdata
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
@@ -24,8 +24,7 @@
package com.bakdata.kafka;
-import static net.mguenther.kafka.junit.EmbeddedKafkaCluster.provisionWith;
-import static net.mguenther.kafka.junit.EmbeddedKafkaClusterConfig.defaultClusterConfig;
+import static com.bakdata.kafka.TestUtil.newKafkaCluster;
import static net.mguenther.kafka.junit.Wait.delay;
import static org.assertj.core.api.Assertions.assertThat;
@@ -39,7 +38,6 @@
import net.mguenther.kafka.junit.SendKeyValues;
import net.mguenther.kafka.junit.TopicConfig;
import org.apache.kafka.common.serialization.Serdes;
-import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.kstream.Consumed;
import org.junit.jupiter.api.Test;
@@ -54,13 +52,18 @@ private static void runApp(final KafkaStreamsApplication app, final String... ar
void shouldExitWithSuccessCode() {
KafkaApplication.startApplication(new KafkaStreamsApplication() {
@Override
- public void buildTopology(final StreamsBuilder builder) {
- throw new UnsupportedOperationException();
- }
+ public StreamsApp createApp(final boolean cleanUp) {
+ return new StreamsApp() {
+ @Override
+ public void buildTopology(final TopologyBuilder builder) {
+ throw new UnsupportedOperationException();
+ }
- @Override
- public String getUniqueAppId() {
- throw new UnsupportedOperationException();
+ @Override
+ public String getUniqueAppId(final StreamsTopicConfig topics) {
+ throw new UnsupportedOperationException();
+ }
+ };
}
@Override
@@ -78,22 +81,17 @@ public void run() {
@Test
@ExpectSystemExitWithStatus(1)
void shouldExitWithErrorCodeOnRunError() {
- KafkaApplication.startApplication(new KafkaStreamsApplication() {
+ KafkaApplication.startApplication(new SimpleKafkaStreamsApplication(() -> new StreamsApp() {
@Override
- public void buildTopology(final StreamsBuilder builder) {
+ public void buildTopology(final TopologyBuilder builder) {
throw new UnsupportedOperationException();
}
@Override
- public String getUniqueAppId() {
+ public String getUniqueAppId(final StreamsTopicConfig topics) {
throw new UnsupportedOperationException();
}
-
- @Override
- public void run() {
- throw new RuntimeException();
- }
- }, new String[]{
+ }), new String[]{
"--brokers", "localhost:9092",
"--schema-registry-url", "http://localhost:8081",
"--input-topics", "input",
@@ -106,17 +104,22 @@ public void run() {
void shouldExitWithErrorCodeOnCleanupError() {
KafkaApplication.startApplication(new KafkaStreamsApplication() {
@Override
- public void buildTopology(final StreamsBuilder builder) {
- throw new UnsupportedOperationException();
- }
+ public StreamsApp createApp(final boolean cleanUp) {
+ return new StreamsApp() {
+ @Override
+ public void buildTopology(final TopologyBuilder builder) {
+ throw new UnsupportedOperationException();
+ }
- @Override
- public String getUniqueAppId() {
- throw new UnsupportedOperationException();
+ @Override
+ public String getUniqueAppId(final StreamsTopicConfig topics) {
+ throw new UnsupportedOperationException();
+ }
+ };
}
@Override
- protected void runCleanUp() {
+ public void clean() {
throw new RuntimeException();
}
}, new String[]{
@@ -124,7 +127,7 @@ protected void runCleanUp() {
"--schema-registry-url", "http://localhost:8081",
"--input-topics", "input",
"--output-topic", "output",
- "--clean-up",
+ "clean",
});
}
@@ -133,13 +136,18 @@ protected void runCleanUp() {
void shouldExitWithErrorCodeOnMissingBrokerParameter() {
KafkaApplication.startApplication(new KafkaStreamsApplication() {
@Override
- public void buildTopology(final StreamsBuilder builder) {
- throw new UnsupportedOperationException();
- }
+ public StreamsApp createApp(final boolean cleanUp) {
+ return new StreamsApp() {
+ @Override
+ public void buildTopology(final TopologyBuilder builder) {
+ throw new UnsupportedOperationException();
+ }
- @Override
- public String getUniqueAppId() {
- throw new UnsupportedOperationException();
+ @Override
+ public String getUniqueAppId(final StreamsTopicConfig topics) {
+ throw new UnsupportedOperationException();
+ }
+ };
}
@Override
@@ -157,21 +165,21 @@ public void run() {
@ExpectSystemExitWithStatus(1)
void shouldExitWithErrorInTopology() throws InterruptedException {
final String input = "input";
- try (final EmbeddedKafkaCluster kafkaCluster = provisionWith(defaultClusterConfig());
- final KafkaStreamsApplication app = new KafkaStreamsApplication() {
+ try (final EmbeddedKafkaCluster kafkaCluster = newKafkaCluster();
+ final KafkaStreamsApplication app = new SimpleKafkaStreamsApplication(() -> new StreamsApp() {
@Override
- public void buildTopology(final StreamsBuilder builder) {
- builder.stream(this.getInputTopics(), Consumed.with(Serdes.ByteArray(), Serdes.ByteArray()))
+ public void buildTopology(final TopologyBuilder builder) {
+ builder.streamInput(Consumed.with(Serdes.ByteArray(), Serdes.ByteArray()))
.peek((k, v) -> {
throw new RuntimeException();
});
}
@Override
- public String getUniqueAppId() {
+ public String getUniqueAppId(final StreamsTopicConfig topics) {
return "app";
}
- }) {
+ })) {
kafkaCluster.start();
kafkaCluster.createTopic(TopicConfig.withName(input).build());
@@ -190,19 +198,19 @@ public String getUniqueAppId() {
void shouldExitWithSuccessCodeOnShutdown() throws InterruptedException {
final String input = "input";
final String output = "output";
- try (final EmbeddedKafkaCluster kafkaCluster = provisionWith(defaultClusterConfig());
- final KafkaStreamsApplication app = new KafkaStreamsApplication() {
+ try (final EmbeddedKafkaCluster kafkaCluster = newKafkaCluster();
+ final KafkaStreamsApplication app = new SimpleKafkaStreamsApplication(() -> new StreamsApp() {
@Override
- public void buildTopology(final StreamsBuilder builder) {
- builder.stream(this.getInputTopics(), Consumed.with(Serdes.ByteArray(), Serdes.ByteArray()))
- .to(this.getOutputTopic());
+ public void buildTopology(final TopologyBuilder builder) {
+ builder.streamInput(Consumed.with(Serdes.ByteArray(), Serdes.ByteArray()))
+ .to(builder.getTopics().getOutputTopic());
}
@Override
- public String getUniqueAppId() {
+ public String getUniqueAppId(final StreamsTopicConfig topics) {
return "app";
}
- }) {
+ })) {
kafkaCluster.start();
kafkaCluster.createTopic(TopicConfig.withName(input).build());
kafkaCluster.createTopic(TopicConfig.withName(output).build());
@@ -227,81 +235,87 @@ public String getUniqueAppId() {
@Test
@ExpectSystemExitWithStatus(1)
- void shouldExitWithSuccessCodeOnCleanupError() {
+ void shouldExitWithErrorOnCleanupError() {
KafkaApplication.startApplication(new KafkaStreamsApplication() {
@Override
- public void buildTopology(final StreamsBuilder builder) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public String getUniqueAppId() {
- throw new UnsupportedOperationException();
- }
+ public StreamsApp createApp(final boolean cleanUp) {
+ return new StreamsApp() {
+ @Override
+ public void buildTopology(final TopologyBuilder builder) {
+ throw new UnsupportedOperationException();
+ }
- @Override
- protected void runCleanUp() {
- // do nothing
+ @Override
+ public String getUniqueAppId(final StreamsTopicConfig topics) {
+ throw new UnsupportedOperationException();
+ }
+ };
}
}, new String[]{
"--brokers", "localhost:9092",
"--schema-registry-url", "http://localhost:8081",
"--input-topics", "input",
"--output-topic", "output",
- "--clean-up",
+ "clean",
});
}
@Test
void shouldParseArguments() {
- final KafkaStreamsApplication app = new KafkaStreamsApplication() {
+ try (final KafkaStreamsApplication app = new KafkaStreamsApplication() {
@Override
- public void buildTopology(final StreamsBuilder builder) {
- throw new UnsupportedOperationException();
- }
+ public StreamsApp createApp(final boolean cleanUp) {
+ return new StreamsApp() {
+ @Override
+ public void buildTopology(final TopologyBuilder builder) {
+ throw new UnsupportedOperationException();
+ }
- @Override
- public String getUniqueAppId() {
- throw new UnsupportedOperationException();
+ @Override
+ public String getUniqueAppId(final StreamsTopicConfig topics) {
+ throw new UnsupportedOperationException();
+ }
+ };
}
@Override
public void run() {
// do nothing
}
- };
- KafkaApplication.startApplicationWithoutExit(app, new String[]{
- "--brokers", "brokers",
- "--schema-registry-url", "schema-registry",
- "--input-topics", "input1,input2",
- "--extra-input-topics", "role1=input3,role2=input4;input5",
- "--input-pattern", ".*",
- "--extra-input-patterns", "role1=.+,role2=\\d+",
- "--output-topic", "output1",
- "--extra-output-topics", "role1=output2,role2=output3",
- });
- assertThat(app.getInputTopics()).containsExactly("input1", "input2");
- assertThat(app.getExtraInputTopics())
- .hasSize(2)
- .containsEntry("role1", List.of("input3"))
- .containsEntry("role2", List.of("input4", "input5"));
- assertThat(app.getInputTopics("role1")).isEqualTo(List.of("input3"));
- assertThat(app.getInputTopic("role2")).isEqualTo("input4");
- assertThat(app.getInputTopics("role2")).isEqualTo(List.of("input4", "input5"));
- assertThat(app.getInputPattern())
- .satisfies(pattern -> assertThat(pattern.pattern()).isEqualTo(Pattern.compile(".*").pattern()));
- assertThat(app.getExtraInputPatterns())
- .hasSize(2)
- .hasEntrySatisfying("role1",
- pattern -> assertThat(pattern.pattern()).isEqualTo(Pattern.compile(".+").pattern()))
- .hasEntrySatisfying("role2",
- pattern -> assertThat(pattern.pattern()).isEqualTo(Pattern.compile("\\d+").pattern()));
- assertThat(app.getInputPattern("role1").pattern()).isEqualTo(Pattern.compile(".+").pattern());
- assertThat(app.getInputPattern("role2").pattern()).isEqualTo(Pattern.compile("\\d+").pattern());
- assertThat(app.getOutputTopic()).isEqualTo("output1");
- assertThat(app.getExtraOutputTopics())
- .hasSize(2)
- .containsEntry("role1", "output2")
- .containsEntry("role2", "output3");
+ }) {
+ KafkaApplication.startApplicationWithoutExit(app, new String[]{
+ "--brokers", "brokers",
+ "--schema-registry-url", "schema-registry",
+ "--input-topics", "input1,input2",
+ "--extra-input-topics", "role1=input3,role2=input4;input5",
+ "--input-pattern", ".*",
+ "--extra-input-patterns", "role1=.+,role2=\\d+",
+ "--output-topic", "output1",
+ "--extra-output-topics", "role1=output2,role2=output3",
+ "--kafka-config", "foo=1,bar=2",
+ });
+ assertThat(app.getInputTopics()).containsExactly("input1", "input2");
+ assertThat(app.getExtraInputTopics())
+ .hasSize(2)
+ .containsEntry("role1", List.of("input3"))
+ .containsEntry("role2", List.of("input4", "input5"));
+ assertThat(app.getInputPattern())
+ .satisfies(pattern -> assertThat(pattern.pattern()).isEqualTo(Pattern.compile(".*").pattern()));
+ assertThat(app.getExtraInputPatterns())
+ .hasSize(2)
+ .hasEntrySatisfying("role1",
+ pattern -> assertThat(pattern.pattern()).isEqualTo(Pattern.compile(".+").pattern()))
+ .hasEntrySatisfying("role2",
+ pattern -> assertThat(pattern.pattern()).isEqualTo(Pattern.compile("\\d+").pattern()));
+ assertThat(app.getOutputTopic()).isEqualTo("output1");
+ assertThat(app.getExtraOutputTopics())
+ .hasSize(2)
+ .containsEntry("role1", "output2")
+ .containsEntry("role2", "output3");
+ assertThat(app.getKafkaConfig())
+ .hasSize(2)
+ .containsEntry("foo", "1")
+ .containsEntry("bar", "2");
+ }
}
}
diff --git a/streams-bootstrap/src/test/java/com/bakdata/kafka/CloseFlagApp.java b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/CloseFlagApp.java
similarity index 66%
rename from streams-bootstrap/src/test/java/com/bakdata/kafka/CloseFlagApp.java
rename to streams-bootstrap-cli/src/test/java/com/bakdata/kafka/CloseFlagApp.java
index fdeb73cf..6e928d43 100644
--- a/streams-bootstrap/src/test/java/com/bakdata/kafka/CloseFlagApp.java
+++ b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/CloseFlagApp.java
@@ -1,7 +1,7 @@
/*
* MIT License
*
- * Copyright (c) 2023 bakdata
+ * Copyright (c) 2024 bakdata
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
@@ -27,7 +27,6 @@
import lombok.Getter;
import lombok.NoArgsConstructor;
import lombok.Setter;
-import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.kstream.KStream;
@NoArgsConstructor
@@ -36,28 +35,32 @@
public class CloseFlagApp extends KafkaStreamsApplication {
private boolean closed = false;
- private Boolean leaveGroup;
+ private boolean appClosed = false;
@Override
public void close() {
- this.closed = true;
super.close();
+ this.closed = true;
}
@Override
- public void buildTopology(final StreamsBuilder builder) {
- final KStream input = builder.stream(this.getInputTopics());
- input.to(this.getOutputTopic());
- }
+ public StreamsApp createApp(final boolean cleanUp) {
+ return new StreamsApp() {
+ @Override
+ public void buildTopology(final TopologyBuilder builder) {
+ final KStream input = builder.streamInput();
+ input.to(builder.getTopics().getOutputTopic());
+ }
- @Override
- public String getUniqueAppId() {
- return this.getClass().getSimpleName() + "-" + this.getOutputTopic();
- }
+ @Override
+ public String getUniqueAppId(final StreamsTopicConfig topics) {
+ return CloseFlagApp.this.getClass().getSimpleName() + "-" + topics.getOutputTopic();
+ }
- @Override
- void closeStreams(final boolean leaveGroup) {
- this.leaveGroup = leaveGroup;
- super.closeStreams(leaveGroup);
+ @Override
+ public void close() {
+ CloseFlagApp.this.appClosed = true;
+ }
+ };
}
}
diff --git a/streams-bootstrap/src/test/java/com/bakdata/kafka/EnvironmentArgumentsParserTest.java b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/EnvironmentArgumentsParserTest.java
similarity index 99%
rename from streams-bootstrap/src/test/java/com/bakdata/kafka/EnvironmentArgumentsParserTest.java
rename to streams-bootstrap-cli/src/test/java/com/bakdata/kafka/EnvironmentArgumentsParserTest.java
index 7a2395d3..64d5ac4f 100644
--- a/streams-bootstrap/src/test/java/com/bakdata/kafka/EnvironmentArgumentsParserTest.java
+++ b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/EnvironmentArgumentsParserTest.java
@@ -1,7 +1,7 @@
/*
* MIT License
*
- * Copyright (c) 2023 bakdata
+ * Copyright (c) 2024 bakdata
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
diff --git a/streams-bootstrap/src/test/java/com/bakdata/kafka/StringListConverterTest.java b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/StringListConverterTest.java
similarity index 98%
rename from streams-bootstrap/src/test/java/com/bakdata/kafka/StringListConverterTest.java
rename to streams-bootstrap-cli/src/test/java/com/bakdata/kafka/StringListConverterTest.java
index 19f86232..0f08cc1b 100644
--- a/streams-bootstrap/src/test/java/com/bakdata/kafka/StringListConverterTest.java
+++ b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/StringListConverterTest.java
@@ -1,7 +1,7 @@
/*
* MIT License
*
- * Copyright (c) 2023 bakdata
+ * Copyright (c) 2024 bakdata
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
diff --git a/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/TestUtil.java b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/TestUtil.java
new file mode 100644
index 00000000..c89ffa4f
--- /dev/null
+++ b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/TestUtil.java
@@ -0,0 +1,44 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import static net.mguenther.kafka.junit.EmbeddedKafkaCluster.provisionWith;
+import static net.mguenther.kafka.junit.EmbeddedKafkaClusterConfig.newClusterConfig;
+import static net.mguenther.kafka.junit.EmbeddedKafkaConfig.brokers;
+
+import lombok.experimental.UtilityClass;
+import net.mguenther.kafka.junit.EmbeddedKafkaCluster;
+
+@UtilityClass
+public class TestUtil {
+ public static EmbeddedKafkaCluster newKafkaCluster() {
+ return provisionWith(newClusterConfig()
+ .configure(brokers()
+ .with("transaction.state.log.num.partitions", 10)
+ .with("offsets.topic.num.partitions", 10)
+ .build())
+ .build());
+ }
+}
diff --git a/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/integration/RunProducerAppTest.java b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/integration/RunProducerAppTest.java
new file mode 100644
index 00000000..d460658b
--- /dev/null
+++ b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/integration/RunProducerAppTest.java
@@ -0,0 +1,120 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka.integration;
+
+import static com.bakdata.kafka.TestUtil.newKafkaCluster;
+import static net.mguenther.kafka.junit.Wait.delay;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import com.bakdata.kafka.KafkaProducerApplication;
+import com.bakdata.kafka.ProducerApp;
+import com.bakdata.kafka.ProducerBuilder;
+import com.bakdata.kafka.ProducerRunnable;
+import com.bakdata.kafka.SimpleKafkaProducerApplication;
+import com.bakdata.kafka.TestRecord;
+import com.bakdata.schemaregistrymock.junit5.SchemaRegistryMockExtension;
+import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig;
+import io.confluent.kafka.streams.serdes.avro.SpecificAvroDeserializer;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import net.mguenther.kafka.junit.EmbeddedKafkaCluster;
+import net.mguenther.kafka.junit.ReadKeyValues;
+import net.mguenther.kafka.junit.TopicConfig;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+class RunProducerAppTest {
+ private static final int TIMEOUT_SECONDS = 10;
+ @RegisterExtension
+ final SchemaRegistryMockExtension schemaRegistryMockExtension = new SchemaRegistryMockExtension();
+ private final EmbeddedKafkaCluster kafkaCluster = newKafkaCluster();
+
+ @BeforeEach
+ void setup() {
+ this.kafkaCluster.start();
+ }
+
+ @AfterEach
+ void tearDown() {
+ this.kafkaCluster.stop();
+ }
+
+ @Test
+ void shouldRunApp() throws InterruptedException {
+ final String output = "output";
+ this.kafkaCluster.createTopic(TopicConfig.withName(output).useDefaults());
+ try (final KafkaProducerApplication app = new SimpleKafkaProducerApplication(() -> new ProducerApp() {
+ @Override
+ public ProducerRunnable buildRunnable(final ProducerBuilder builder) {
+ return () -> {
+ try (final Producer producer = builder.createProducer()) {
+ final TestRecord testRecord = TestRecord.newBuilder().setContent("bar").build();
+ producer.send(new ProducerRecord<>(builder.getTopics().getOutputTopic(), "foo", testRecord));
+ }
+ };
+ }
+
+ @Override
+ public Map createKafkaProperties() {
+ return Map.of(
+ ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class
+ );
+ }
+ })) {
+ app.setBrokers(this.kafkaCluster.getBrokerList());
+ app.setSchemaRegistryUrl(this.schemaRegistryMockExtension.getUrl());
+ app.setOutputTopic(output);
+ app.setKafkaConfig(Map.of(
+ ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "10000"
+ ));
+ app.run();
+ delay(TIMEOUT_SECONDS, TimeUnit.SECONDS);
+ assertThat(this.kafkaCluster.read(ReadKeyValues.from(output, String.class, TestRecord.class)
+ .with(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class)
+ .with(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, SpecificAvroDeserializer.class)
+ .with(AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG,
+ this.schemaRegistryMockExtension.getUrl())
+ .build()))
+ .hasSize(1)
+ .anySatisfy(kv -> {
+ assertThat(kv.getKey()).isEqualTo("foo");
+ assertThat(kv.getValue().getContent()).isEqualTo("bar");
+ });
+ app.clean();
+ delay(TIMEOUT_SECONDS, TimeUnit.SECONDS);
+ assertThat(this.kafkaCluster.exists(app.getOutputTopic()))
+ .as("Output topic is deleted")
+ .isFalse();
+ }
+ }
+}
diff --git a/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/integration/RunStreamsAppTest.java b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/integration/RunStreamsAppTest.java
new file mode 100644
index 00000000..57111db8
--- /dev/null
+++ b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/integration/RunStreamsAppTest.java
@@ -0,0 +1,96 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka.integration;
+
+import static com.bakdata.kafka.TestUtil.newKafkaCluster;
+import static net.mguenther.kafka.junit.Wait.delay;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import com.bakdata.kafka.KafkaStreamsApplication;
+import com.bakdata.kafka.SimpleKafkaStreamsApplication;
+import com.bakdata.kafka.test_applications.Mirror;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import net.mguenther.kafka.junit.EmbeddedKafkaCluster;
+import net.mguenther.kafka.junit.KeyValue;
+import net.mguenther.kafka.junit.ReadKeyValues;
+import net.mguenther.kafka.junit.SendKeyValuesTransactional;
+import net.mguenther.kafka.junit.TopicConfig;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+class RunStreamsAppTest {
+ private static final int TIMEOUT_SECONDS = 10;
+ private final EmbeddedKafkaCluster kafkaCluster = newKafkaCluster();
+
+ @BeforeEach
+ void setup() {
+ this.kafkaCluster.start();
+ }
+
+ @AfterEach
+ void tearDown() {
+ this.kafkaCluster.stop();
+ }
+
+ @Test
+ void shouldRunApp() throws InterruptedException {
+ final String input = "input";
+ final String output = "output";
+ this.kafkaCluster.createTopic(TopicConfig.withName(input).useDefaults());
+ this.kafkaCluster.createTopic(TopicConfig.withName(output).useDefaults());
+ try (final KafkaStreamsApplication app = new SimpleKafkaStreamsApplication(Mirror::new)) {
+ app.setBrokers(this.kafkaCluster.getBrokerList());
+ app.setKafkaConfig(Map.of(
+ ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "10000"
+ ));
+ app.setInputTopics(List.of(input));
+ app.setOutputTopic(output);
+ // run in Thread because the application blocks indefinitely
+ new Thread(app).start();
+ final SendKeyValuesTransactional kvSendKeyValuesTransactionalBuilder =
+ SendKeyValuesTransactional.inTransaction(input, List.of(new KeyValue<>("foo", "bar")))
+ .with(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class)
+ .with(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class)
+ .build();
+ this.kafkaCluster.send(kvSendKeyValuesTransactionalBuilder);
+ delay(TIMEOUT_SECONDS, TimeUnit.SECONDS);
+ assertThat(this.kafkaCluster.read(ReadKeyValues.from(output, String.class, String.class)
+ .with(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class)
+ .with(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class)
+ .build()))
+ .hasSize(1);
+ }
+ }
+}
diff --git a/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/integration/StreamsCleanUpTest.java b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/integration/StreamsCleanUpTest.java
new file mode 100644
index 00000000..b687f602
--- /dev/null
+++ b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/integration/StreamsCleanUpTest.java
@@ -0,0 +1,208 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka.integration;
+
+
+import static com.bakdata.kafka.TestUtil.newKafkaCluster;
+import static net.mguenther.kafka.junit.Wait.delay;
+
+import com.bakdata.kafka.CloseFlagApp;
+import com.bakdata.kafka.KafkaStreamsApplication;
+import com.bakdata.kafka.SimpleKafkaStreamsApplication;
+import com.bakdata.kafka.test_applications.WordCount;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import lombok.extern.slf4j.Slf4j;
+import net.mguenther.kafka.junit.EmbeddedKafkaCluster;
+import net.mguenther.kafka.junit.KeyValue;
+import net.mguenther.kafka.junit.ReadKeyValues;
+import net.mguenther.kafka.junit.SendValuesTransactional;
+import net.mguenther.kafka.junit.TopicConfig;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.serialization.LongDeserializer;
+import org.apache.kafka.streams.StreamsConfig;
+import org.assertj.core.api.SoftAssertions;
+import org.assertj.core.api.junit.jupiter.InjectSoftAssertions;
+import org.assertj.core.api.junit.jupiter.SoftAssertionsExtension;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.mockito.junit.jupiter.MockitoSettings;
+import org.mockito.quality.Strictness;
+
+@Slf4j
+@ExtendWith(SoftAssertionsExtension.class)
+@ExtendWith(MockitoExtension.class)
+@MockitoSettings(strictness = Strictness.STRICT_STUBS)
+class StreamsCleanUpTest {
+ private static final int TIMEOUT_SECONDS = 10;
+ private final EmbeddedKafkaCluster kafkaCluster = newKafkaCluster();
+ @InjectSoftAssertions
+ private SoftAssertions softly;
+
+ private static void runAppAndClose(final KafkaStreamsApplication app) throws InterruptedException {
+ runApp(app);
+ app.stop();
+ }
+
+ private static void runApp(final KafkaStreamsApplication app) throws InterruptedException {
+ // run in Thread because the application blocks indefinitely
+ new Thread(app).start();
+ // Wait until stream application has consumed all data
+ delay(TIMEOUT_SECONDS, TimeUnit.SECONDS);
+ }
+
+ @BeforeEach
+ void setup() throws InterruptedException {
+ this.kafkaCluster.start();
+ delay(TIMEOUT_SECONDS, TimeUnit.SECONDS);
+ }
+
+ @AfterEach
+ void tearDown() throws InterruptedException {
+ delay(TIMEOUT_SECONDS, TimeUnit.SECONDS);
+ this.kafkaCluster.stop();
+ }
+
+ @Test
+ void shouldClean() throws InterruptedException {
+ try (final KafkaStreamsApplication app = this.createWordCountApplication()) {
+ final SendValuesTransactional sendRequest =
+ SendValuesTransactional.inTransaction(app.getInputTopics().get(0),
+ List.of("blub", "bla", "blub")).useDefaults();
+ this.kafkaCluster.send(sendRequest);
+
+ final List> expectedValues = List.of(
+ new KeyValue<>("blub", 1L),
+ new KeyValue<>("bla", 1L),
+ new KeyValue<>("blub", 2L)
+ );
+ this.runAndAssertContent(expectedValues, "All entries are once in the input topic after the 1st run", app);
+
+ // Wait until all stream application are completely stopped before triggering cleanup
+ delay(TIMEOUT_SECONDS, TimeUnit.SECONDS);
+ app.clean();
+
+ this.softly.assertThat(this.kafkaCluster.exists(app.getOutputTopic()))
+ .as("Output topic is deleted")
+ .isFalse();
+
+ this.runAndAssertContent(expectedValues, "All entries are once in the input topic after the 2nd run", app);
+ }
+ }
+
+ @Test
+ void shouldReset() throws InterruptedException {
+ try (final KafkaStreamsApplication app = this.createWordCountApplication()) {
+ final SendValuesTransactional sendRequest =
+ SendValuesTransactional.inTransaction(app.getInputTopics().get(0),
+ List.of("blub", "bla", "blub")).useDefaults();
+ this.kafkaCluster.send(sendRequest);
+
+ final List> expectedValues = List.of(
+ new KeyValue<>("blub", 1L),
+ new KeyValue<>("bla", 1L),
+ new KeyValue<>("blub", 2L)
+ );
+ this.runAndAssertContent(expectedValues, "All entries are once in the input topic after the 1st run", app);
+
+ // Wait until all stream application are completely stopped before triggering cleanup
+ delay(TIMEOUT_SECONDS, TimeUnit.SECONDS);
+ app.reset();
+
+ final List> entriesTwice = expectedValues.stream()
+ .flatMap(entry -> Stream.of(entry, entry))
+ .collect(Collectors.toList());
+ this.runAndAssertContent(entriesTwice, "All entries are twice in the input topic after the 2nd run", app);
+ }
+ }
+
+ @Test
+ void shouldCallClose() throws InterruptedException {
+ try (final CloseFlagApp app = this.createCloseFlagApplication()) {
+ this.kafkaCluster.createTopic(TopicConfig.withName(app.getInputTopics().get(0)).useDefaults());
+ delay(TIMEOUT_SECONDS, TimeUnit.SECONDS);
+ this.softly.assertThat(app.isClosed()).isFalse();
+ this.softly.assertThat(app.isAppClosed()).isFalse();
+ // if we don't run the app, the coordinator will be unavailable
+ runAppAndClose(app);
+ this.softly.assertThat(app.isAppClosed()).isTrue();
+ app.setAppClosed(false);
+ delay(TIMEOUT_SECONDS, TimeUnit.SECONDS);
+ app.clean();
+ this.softly.assertThat(app.isAppClosed()).isTrue();
+ app.setAppClosed(false);
+ delay(TIMEOUT_SECONDS, TimeUnit.SECONDS);
+ app.reset();
+ this.softly.assertThat(app.isAppClosed()).isTrue();
+ }
+ }
+
+ private CloseFlagApp createCloseFlagApplication() {
+ final CloseFlagApp app = new CloseFlagApp();
+ app.setInputTopics(List.of("input"));
+ app.setOutputTopic("output");
+ return this.configure(app);
+ }
+
+ private List> readOutputTopic(final String outputTopic) throws InterruptedException {
+ final ReadKeyValues readRequest = ReadKeyValues.from(outputTopic, Long.class)
+ .with(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, LongDeserializer.class).build();
+ return this.kafkaCluster.read(readRequest);
+ }
+
+ private void runAndAssertContent(final Iterable extends KeyValue> expectedValues,
+ final String description, final KafkaStreamsApplication app)
+ throws InterruptedException {
+ runAppAndClose(app);
+
+ final List> output = this.readOutputTopic(app.getOutputTopic());
+ this.softly.assertThat(output)
+ .as(description)
+ .containsExactlyInAnyOrderElementsOf(expectedValues);
+ }
+
+ private KafkaStreamsApplication createWordCountApplication() {
+ final KafkaStreamsApplication application = new SimpleKafkaStreamsApplication(WordCount::new);
+ application.setOutputTopic("word_output");
+ application.setInputTopics(List.of("word_input"));
+ return this.configure(application);
+ }
+
+ private T configure(final T application) {
+ application.setBrokers(this.kafkaCluster.getBrokerList());
+ application.setKafkaConfig(Map.of(
+ StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, "0",
+ ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "10000"
+ ));
+ return application;
+ }
+
+}
diff --git a/streams-bootstrap/src/test/java/com/bakdata/kafka/test_applications/Mirror.java b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/test_applications/Mirror.java
similarity index 70%
rename from streams-bootstrap/src/test/java/com/bakdata/kafka/test_applications/Mirror.java
rename to streams-bootstrap-cli/src/test/java/com/bakdata/kafka/test_applications/Mirror.java
index 5dfad604..212c7611 100644
--- a/streams-bootstrap/src/test/java/com/bakdata/kafka/test_applications/Mirror.java
+++ b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/test_applications/Mirror.java
@@ -1,7 +1,7 @@
/*
* MIT License
*
- * Copyright (c) 2023 bakdata
+ * Copyright (c) 2024 bakdata
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
@@ -24,22 +24,23 @@
package com.bakdata.kafka.test_applications;
-import com.bakdata.kafka.KafkaStreamsApplication;
+import com.bakdata.kafka.StreamsApp;
+import com.bakdata.kafka.StreamsTopicConfig;
+import com.bakdata.kafka.TopologyBuilder;
import lombok.NoArgsConstructor;
-import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.kstream.KStream;
@NoArgsConstructor
-public class Mirror extends KafkaStreamsApplication {
+public class Mirror implements StreamsApp {
@Override
- public void buildTopology(final StreamsBuilder builder) {
- final KStream input = builder.stream(this.getInputTopics());
- input.to(this.getOutputTopic());
+ public void buildTopology(final TopologyBuilder builder) {
+ final KStream input = builder.streamInput();
+ input.to(builder.getTopics().getOutputTopic());
}
@Override
- public String getUniqueAppId() {
- return this.getClass().getSimpleName() + "-" + this.getOutputTopic();
+ public String getUniqueAppId(final StreamsTopicConfig topics) {
+ return this.getClass().getSimpleName() + "-" + topics.getOutputTopic();
}
}
diff --git a/streams-bootstrap/src/test/java/com/bakdata/kafka/test_applications/WordCount.java b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/test_applications/WordCount.java
similarity index 77%
rename from streams-bootstrap/src/test/java/com/bakdata/kafka/test_applications/WordCount.java
rename to streams-bootstrap-cli/src/test/java/com/bakdata/kafka/test_applications/WordCount.java
index d67d80d4..2bcdc095 100644
--- a/streams-bootstrap/src/test/java/com/bakdata/kafka/test_applications/WordCount.java
+++ b/streams-bootstrap-cli/src/test/java/com/bakdata/kafka/test_applications/WordCount.java
@@ -1,7 +1,7 @@
/*
* MIT License
*
- * Copyright (c) 2023 bakdata
+ * Copyright (c) 2024 bakdata
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
@@ -24,26 +24,24 @@
package com.bakdata.kafka.test_applications;
-import com.bakdata.kafka.KafkaStreamsApplication;
+import com.bakdata.kafka.StreamsApp;
+import com.bakdata.kafka.StreamsTopicConfig;
+import com.bakdata.kafka.TopologyBuilder;
import java.util.Arrays;
import java.util.regex.Pattern;
import lombok.NoArgsConstructor;
import org.apache.kafka.common.serialization.Serdes;
-import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.KTable;
import org.apache.kafka.streams.kstream.Materialized;
import org.apache.kafka.streams.kstream.Produced;
@NoArgsConstructor
-public class WordCount extends KafkaStreamsApplication {
- public static void main(final String[] args) {
- startApplication(new WordCount(), args);
- }
+public class WordCount implements StreamsApp {
@Override
- public void buildTopology(final StreamsBuilder builder) {
- final KStream textLines = builder.stream(this.getInputTopics());
+ public void buildTopology(final TopologyBuilder builder) {
+ final KStream textLines = builder.streamInput();
final Pattern pattern = Pattern.compile("\\W+", Pattern.UNICODE_CHARACTER_CLASS);
final KTable wordCounts = textLines
@@ -51,11 +49,11 @@ public void buildTopology(final StreamsBuilder builder) {
.groupBy((key, word) -> word)
.count(Materialized.as("counts"));
- wordCounts.toStream().to(this.outputTopic, Produced.valueSerde(Serdes.Long()));
+ wordCounts.toStream().to(builder.getTopics().getOutputTopic(), Produced.valueSerde(Serdes.Long()));
}
@Override
- public String getUniqueAppId() {
- return this.getClass().getSimpleName() + "-" + this.getOutputTopic();
+ public String getUniqueAppId(final StreamsTopicConfig topics) {
+ return this.getClass().getSimpleName() + "-" + topics.getOutputTopic();
}
}
diff --git a/streams-bootstrap-cli/src/test/resources/log4j2.xml b/streams-bootstrap-cli/src/test/resources/log4j2.xml
new file mode 100644
index 00000000..0d4071ce
--- /dev/null
+++ b/streams-bootstrap-cli/src/test/resources/log4j2.xml
@@ -0,0 +1,34 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/streams-bootstrap/build.gradle.kts b/streams-bootstrap-core/build.gradle.kts
similarity index 78%
rename from streams-bootstrap/build.gradle.kts
rename to streams-bootstrap-core/build.gradle.kts
index fb4613e6..4bd028b1 100644
--- a/streams-bootstrap/build.gradle.kts
+++ b/streams-bootstrap-core/build.gradle.kts
@@ -8,15 +8,11 @@ dependencies {
val kafkaVersion: String by project
implementation(group = "org.apache.kafka", name = "kafka-tools", version = kafkaVersion)
- implementation(group = "info.picocli", name = "picocli", version = "4.7.5")
api(group = "org.apache.kafka", name = "kafka-streams", version = kafkaVersion)
api(group = "org.apache.kafka", name = "kafka-clients", version = kafkaVersion)
val confluentVersion: String by project
implementation(group = "io.confluent", name = "kafka-streams-avro-serde", version = confluentVersion)
api(group = "io.confluent", name = "kafka-schema-registry-client", version = confluentVersion)
- val log4jVersion = "2.23.0"
- implementation(group = "org.apache.logging.log4j", name = "log4j-core", version = log4jVersion)
- implementation(group = "org.apache.logging.log4j", name = "log4j-slf4j2-impl", version = log4jVersion)
api(
group = "org.slf4j",
name = "slf4j-api",
@@ -25,29 +21,30 @@ dependencies {
implementation(group = "com.google.guava", name = "guava", version = "33.0.0-jre")
implementation(group = "org.jooq", name = "jool", version = "0.9.14")
- val junitVersion = "5.10.2"
+ val junitVersion: String by project
+ testRuntimeOnly(group = "org.junit.jupiter", name = "junit-jupiter-engine", version = junitVersion)
testImplementation(group = "org.junit.jupiter", name = "junit-jupiter-api", version = junitVersion)
testImplementation(group = "org.junit.jupiter", name = "junit-jupiter-params", version = junitVersion)
testImplementation(group = "org.junit-pioneer", name = "junit-pioneer", version = "2.2.0")
- testRuntimeOnly(group = "org.junit.jupiter", name = "junit-jupiter-engine", version = junitVersion)
- testImplementation(group = "org.assertj", name = "assertj-core", version = "3.25.3")
- val mockitoVersion = "5.10.0"
+ val assertJVersion: String by project
+ testImplementation(group = "org.assertj", name = "assertj-core", version = assertJVersion)
+ val mockitoVersion: String by project
testImplementation(group = "org.mockito", name = "mockito-core", version = mockitoVersion)
testImplementation(group = "org.mockito", name = "mockito-junit-jupiter", version = mockitoVersion)
val fluentKafkaVersion: String by project
testImplementation(project(":streams-bootstrap-test"))
- testImplementation(group = "org.apache.kafka", name = "kafka-streams-test-utils", version = kafkaVersion)
testImplementation(
group = "com.bakdata.fluent-kafka-streams-tests",
name = "schema-registry-mock-junit5",
version = fluentKafkaVersion
)
- testImplementation(group = "net.mguenther.kafka", name = "kafka-junit", version = "3.6.0") {
+ val kafkaJunitVersion: String by project
+ testImplementation(group = "net.mguenther.kafka", name = "kafka-junit", version = kafkaJunitVersion) {
exclude(group = "org.slf4j", module = "slf4j-log4j12")
}
-
- testImplementation(group = "com.ginsberg", name = "junit5-system-exit", version = "1.1.2")
+ val log4jVersion: String by project
+ testImplementation(group = "org.apache.logging.log4j", name = "log4j-slf4j2-impl", version = log4jVersion)
}
tasks.withType {
@@ -55,4 +52,5 @@ tasks.withType {
"--add-opens=java.base/java.lang=ALL-UNNAMED",
"--add-opens=java.base/java.util=ALL-UNNAMED"
)
+ maxHeapSize = "4g"
}
diff --git a/streams-bootstrap/lombok.config b/streams-bootstrap-core/lombok.config
similarity index 100%
rename from streams-bootstrap/lombok.config
rename to streams-bootstrap-core/lombok.config
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/App.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/App.java
new file mode 100644
index 00000000..65ed48bb
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/App.java
@@ -0,0 +1,66 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import static java.util.Collections.emptyMap;
+
+import java.util.Map;
+
+/**
+ * Kafka application that defines necessary configurations
+ * @param type of topic config
+ * @param type of clean up config
+ */
+@FunctionalInterface
+public interface App extends AutoCloseable {
+
+ /**
+ * Configure clean up behavior
+ * @param configuration provides all runtime application configurations
+ * @return clean up configuration
+ */
+ C setupCleanUp(final EffectiveAppConfiguration configuration);
+
+ @Override
+ default void close() {
+ // do nothing by default
+ }
+
+ /**
+ * This method should give a default configuration to run your application with.
+ * @return Returns a default Kafka configuration. Empty by default
+ */
+ default Map createKafkaProperties() {
+ return emptyMap();
+ }
+
+ /**
+ * Setup Kafka resources, such as topics, before running this app
+ * @param configuration provides all runtime application configurations
+ */
+ default void setup(final EffectiveAppConfiguration configuration) {
+ // do nothing by default
+ }
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/AppConfiguration.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/AppConfiguration.java
new file mode 100644
index 00000000..06477203
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/AppConfiguration.java
@@ -0,0 +1,55 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import static java.util.Collections.emptyMap;
+
+import java.util.Map;
+import lombok.EqualsAndHashCode;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+import lombok.Value;
+
+/**
+ * Configuration of an app. This includes topics and Kafka configuration
+ * @param type of topic config
+ */
+@Value
+@RequiredArgsConstructor
+@EqualsAndHashCode
+public class AppConfiguration {
+ @NonNull
+ T topics;
+ @NonNull
+ Map kafkaConfig;
+
+ /**
+ * Create a new {@code AppConfiguration} with empty Kafka configuration
+ * @param topics topics to use for app
+ */
+ public AppConfiguration(final T topics) {
+ this(topics, emptyMap());
+ }
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/CapturingStreamsUncaughtExceptionHandler.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/CapturingStreamsUncaughtExceptionHandler.java
new file mode 100644
index 00000000..93146792
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/CapturingStreamsUncaughtExceptionHandler.java
@@ -0,0 +1,50 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler;
+
+@RequiredArgsConstructor
+class CapturingStreamsUncaughtExceptionHandler implements StreamsUncaughtExceptionHandler {
+
+ private final @NonNull StreamsUncaughtExceptionHandler wrapped;
+ private Throwable lastException;
+
+ @Override
+ public StreamThreadExceptionResponse handle(final Throwable exception) {
+ final StreamThreadExceptionResponse response = this.wrapped.handle(exception);
+ this.lastException = exception;
+ return response;
+ }
+
+ void throwException() {
+ if (this.lastException instanceof RuntimeException) {
+ throw (RuntimeException) this.lastException;
+ }
+ throw new StreamsApplicationException("Kafka Streams has transitioned to error", this.lastException);
+ }
+}
diff --git a/streams-bootstrap/src/main/java/com/bakdata/kafka/CleanUpException.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/CleanUpException.java
similarity index 93%
rename from streams-bootstrap/src/main/java/com/bakdata/kafka/CleanUpException.java
rename to streams-bootstrap-core/src/main/java/com/bakdata/kafka/CleanUpException.java
index 2ded2c75..e230deef 100644
--- a/streams-bootstrap/src/main/java/com/bakdata/kafka/CleanUpException.java
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/CleanUpException.java
@@ -1,7 +1,7 @@
/*
* MIT License
*
- * Copyright (c) 2023 bakdata
+ * Copyright (c) 2024 bakdata
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
@@ -25,7 +25,7 @@
package com.bakdata.kafka;
/**
- * Exception thrown if running streams clean up was unsuccessful
+ * Exception thrown if running clean up was unsuccessful
*/
public class CleanUpException extends RuntimeException {
public CleanUpException(final String message) {
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/CleanUpRunner.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/CleanUpRunner.java
new file mode 100644
index 00000000..40ef7cfa
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/CleanUpRunner.java
@@ -0,0 +1,36 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+/**
+ * Cleans all resources associated with an application
+ */
+@FunctionalInterface
+public interface CleanUpRunner {
+ /**
+ * Clean all resources associated with an application
+ */
+ void clean();
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/Configurable.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/Configurable.java
new file mode 100644
index 00000000..0cb5c6f3
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/Configurable.java
@@ -0,0 +1,38 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import java.util.Map;
+
+@FunctionalInterface
+interface Configurable {
+ /**
+ * Configure this class
+ * @param config configs in key/value pairs
+ * @param isKey whether is for key or value
+ * @return configured instance
+ */
+ T configure(Map config, boolean isKey);
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ConfigurableSerde.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ConfigurableSerde.java
new file mode 100644
index 00000000..76cb7147
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ConfigurableSerde.java
@@ -0,0 +1,44 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import java.util.Map;
+import lombok.AccessLevel;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+import org.apache.kafka.common.serialization.Serde;
+
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+final class ConfigurableSerde, T> implements Configurable {
+
+ private final @NonNull S serde;
+
+ @Override
+ public S configure(final Map config, final boolean isKey) {
+ this.serde.configure(config, isKey);
+ return this.serde;
+ }
+
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ConfigurableSerializer.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ConfigurableSerializer.java
new file mode 100644
index 00000000..9cd8ffef
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ConfigurableSerializer.java
@@ -0,0 +1,43 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import java.util.Map;
+import lombok.AccessLevel;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+import org.apache.kafka.common.serialization.Serializer;
+
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+final class ConfigurableSerializer, T> implements Configurable {
+
+ private final @NonNull S serializer;
+
+ @Override
+ public S configure(final Map config, final boolean isKey) {
+ this.serializer.configure(config, isKey);
+ return this.serializer;
+ }
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/Configurator.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/Configurator.java
new file mode 100644
index 00000000..1fd308fb
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/Configurator.java
@@ -0,0 +1,149 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import static com.bakdata.kafka.Preconfigured.create;
+
+import java.util.Map;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serializer;
+
+/**
+ * Configure {@link Serde} and {@link Serializer} using base properties
+ */
+@RequiredArgsConstructor
+public class Configurator {
+
+ private final @NonNull Map kafkaProperties;
+
+ /**
+ * Configure a {@code Serde} for values using {@link #kafkaProperties}
+ * @param serde serde to configure
+ * @return configured {@code Serde}
+ * @param type to be (de-)serialized
+ */
+ public Serde configureForValues(final Serde serde) {
+ return this.configureForValues(create(serde));
+ }
+
+ /**
+ * Configure a {@code Serde} for values using {@link #kafkaProperties} and config overrides
+ * @param serde serde to configure
+ * @param configOverrides configuration overrides
+ * @return configured {@code Serde}
+ * @param type to be (de-)serialized
+ */
+ public Serde configureForValues(final Serde serde, final Map configOverrides) {
+ return this.configureForValues(create(serde, configOverrides));
+ }
+
+ /**
+ * Configure a {@code Serde} for keys using {@link #kafkaProperties}
+ * @param serde serde to configure
+ * @return configured {@code Serde}
+ * @param type to be (de-)serialized
+ */
+ public Serde configureForKeys(final Serde serde) {
+ return this.configureForKeys(create(serde));
+ }
+
+ /**
+ * Configure a {@code Serde} for keys using {@link #kafkaProperties} and config overrides
+ * @param serde serde to configure
+ * @param configOverrides configuration overrides
+ * @return configured {@code Serde}
+ * @param type to be (de-)serialized
+ */
+ public Serde configureForKeys(final Serde serde, final Map configOverrides) {
+ return this.configureForKeys(create(serde, configOverrides));
+ }
+
+ /**
+ * Configure a {@code Serializer} for values using {@link #kafkaProperties}
+ * @param serializer serializer to configure
+ * @return configured {@code Serializer}
+ * @param type to be (de-)serialized
+ */
+ public Serializer configureForValues(final Serializer serializer) {
+ return this.configureForValues(create(serializer));
+ }
+
+ /**
+ * Configure a {@code Serializer} for values using {@link #kafkaProperties} and config overrides
+ * @param serializer serializer to configure
+ * @param configOverrides configuration overrides
+ * @return configured {@code Serializer}
+ * @param type to be (de-)serialized
+ */
+ public Serializer configureForValues(final Serializer serializer,
+ final Map configOverrides) {
+ return this.configureForValues(create(serializer, configOverrides));
+ }
+
+ /**
+ * Configure a {@code Serializer} for keys using {@link #kafkaProperties}
+ * @param serializer serializer to configure
+ * @return configured {@code Serializer}
+ * @param type to be (de-)serialized
+ */
+ public Serializer configureForKeys(final Serializer serializer) {
+ return this.configureForKeys(create(serializer));
+ }
+
+ /**
+ * Configure a {@code Serializer} for keys using {@link #kafkaProperties} and config overrides
+ * @param serializer serializer to configure
+ * @param configOverrides configuration overrides
+ * @return configured {@code Serializer}
+ * @param type to be (de-)serialized
+ */
+ public Serializer configureForKeys(final Serializer serializer,
+ final Map configOverrides) {
+ return this.configureForKeys(create(serializer, configOverrides));
+ }
+
+ /**
+ * Configure a {@code Preconfigured} for values object using {@link #kafkaProperties}
+ * @param preconfigured pre-configured {@link Serde} or {@link Serializer}
+ * @return configured instance
+ * @param type of configured instance
+ */
+ public T configureForValues(final Preconfigured preconfigured) {
+ return preconfigured.configureForValues(this.kafkaProperties);
+ }
+
+ /**
+ * Configure a {@code Preconfigured} for keys object using {@link #kafkaProperties}
+ * @param preconfigured pre-configured {@link Serde} or {@link Serializer}
+ * @return configured instance
+ * @param type of configured instance
+ */
+ public T configureForKeys(final Preconfigured preconfigured) {
+ return preconfigured.configureForKeys(this.kafkaProperties);
+ }
+
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ConfiguredApp.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ConfiguredApp.java
new file mode 100644
index 00000000..9bdab008
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ConfiguredApp.java
@@ -0,0 +1,42 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+/**
+ * An application with a corresponding configuration
+ *
+ * @param type of executable app after configuring {@link KafkaEndpointConfig}
+ */
+public interface ConfiguredApp extends AutoCloseable {
+ /**
+ * Create an executable app using the provided {@code KafkaEndpointConfig}
+ * @param endpointConfig endpoint to run app on
+ * @return executable streams app
+ */
+ E withEndpoint(KafkaEndpointConfig endpointConfig);
+
+ @Override
+ void close();
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ConfiguredProducerApp.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ConfiguredProducerApp.java
new file mode 100644
index 00000000..de2f409d
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ConfiguredProducerApp.java
@@ -0,0 +1,141 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import static java.util.Collections.emptyMap;
+
+import io.confluent.kafka.streams.serdes.avro.SpecificAvroSerializer;
+import java.util.HashMap;
+import java.util.Map;
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.StringSerializer;
+
+/**
+ * A {@link ProducerApp} with a corresponding {@link AppConfiguration}
+ * @param type of {@link ProducerApp}
+ */
+@RequiredArgsConstructor
+public class ConfiguredProducerApp implements ConfiguredApp> {
+ @Getter
+ private final @NonNull T app;
+ private final @NonNull AppConfiguration configuration;
+
+ private static Map createBaseConfig(final KafkaEndpointConfig endpointConfig) {
+ final Map kafkaConfig = new HashMap<>();
+
+ if (endpointConfig.isSchemaRegistryConfigured()) {
+ kafkaConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, SpecificAvroSerializer.class);
+ kafkaConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, SpecificAvroSerializer.class);
+ } else {
+ kafkaConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+ kafkaConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+ }
+
+ kafkaConfig.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, 1);
+ kafkaConfig.put(ProducerConfig.ACKS_CONFIG, "all");
+
+ // compression
+ kafkaConfig.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, "gzip");
+
+ return kafkaConfig;
+ }
+
+ /**
+ * This method creates the configuration to run a {@link ProducerApp}.
+ * Configuration is created in the following order
+ *
+ *
+ * @param endpointConfig endpoint to run app on
+ * @return Kafka configuration
+ */
+ public Map getKafkaProperties(final KafkaEndpointConfig endpointConfig) {
+ final KafkaPropertiesFactory propertiesFactory = this.createPropertiesFactory(endpointConfig);
+ return propertiesFactory.createKafkaProperties(emptyMap());
+ }
+
+ /**
+ * Create an {@code ExecutableProducerApp} using the provided {@code KafkaEndpointConfig}
+ * @return {@code ExecutableProducerApp}
+ */
+ @Override
+ public ExecutableProducerApp withEndpoint(final KafkaEndpointConfig endpointConfig) {
+ final ProducerTopicConfig topics = this.getTopics();
+ final Map kafkaProperties = this.getKafkaProperties(endpointConfig);
+ return new ExecutableProducerApp<>(topics, kafkaProperties, this.app);
+ }
+
+ /**
+ * Get topic configuration
+ * @return topic configuration
+ */
+ public ProducerTopicConfig getTopics() {
+ return this.configuration.getTopics();
+ }
+
+ @Override
+ public void close() {
+ this.app.close();
+ }
+
+ private KafkaPropertiesFactory createPropertiesFactory(final KafkaEndpointConfig endpointConfig) {
+ final Map baseConfig = createBaseConfig(endpointConfig);
+ return KafkaPropertiesFactory.builder()
+ .baseConfig(baseConfig)
+ .app(this.app)
+ .configuration(this.configuration)
+ .endpointConfig(endpointConfig)
+ .build();
+ }
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ConfiguredStreamsApp.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ConfiguredStreamsApp.java
new file mode 100644
index 00000000..efe0a3d2
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ConfiguredStreamsApp.java
@@ -0,0 +1,181 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import io.confluent.kafka.streams.serdes.avro.SpecificAvroSerde;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.Serdes.StringSerde;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.Topology;
+
+/**
+ * A {@link StreamsApp} with a corresponding {@link AppConfiguration}
+ * @param type of {@link StreamsApp}
+ */
+@RequiredArgsConstructor
+public class ConfiguredStreamsApp implements ConfiguredApp> {
+ @Getter
+ private final @NonNull T app;
+ private final @NonNull AppConfiguration configuration;
+
+ private static Map createBaseConfig(final KafkaEndpointConfig endpointConfig) {
+ final Map kafkaConfig = new HashMap<>();
+
+ if (endpointConfig.isSchemaRegistryConfigured()) {
+ kafkaConfig.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, SpecificAvroSerde.class);
+ kafkaConfig.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, SpecificAvroSerde.class);
+ } else {
+ kafkaConfig.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, StringSerde.class);
+ kafkaConfig.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, StringSerde.class);
+ }
+
+ // exactly once and order
+ kafkaConfig.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_V2);
+ kafkaConfig.put(StreamsConfig.producerPrefix(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION), 1);
+
+ kafkaConfig.put(StreamsConfig.producerPrefix(ProducerConfig.ACKS_CONFIG), "all");
+
+ // compression
+ kafkaConfig.put(StreamsConfig.producerPrefix(ProducerConfig.COMPRESSION_TYPE_CONFIG), "gzip");
+
+ return kafkaConfig;
+ }
+
+ /**
+ * This method creates the configuration to run a {@link StreamsApp}.
+ * Configuration is created in the following order
+ *
+ *
+ * @param endpointConfig endpoint to run app on
+ * @return Kafka configuration
+ */
+ public Map getKafkaProperties(final KafkaEndpointConfig endpointConfig) {
+ final KafkaPropertiesFactory propertiesFactory = this.createPropertiesFactory(endpointConfig);
+ return propertiesFactory.createKafkaProperties(Map.of(
+ StreamsConfig.APPLICATION_ID_CONFIG, this.getUniqueAppId()
+ ));
+ }
+
+ /**
+ * Get unique application identifier of {@code StreamsApp}
+ * @return unique application identifier
+ * @see StreamsApp#getUniqueAppId(StreamsTopicConfig)
+ */
+ public String getUniqueAppId() {
+ return Objects.requireNonNull(this.app.getUniqueAppId(this.getTopics()));
+ }
+
+ /**
+ * Get topic configuration
+ * @return topic configuration
+ */
+ public StreamsTopicConfig getTopics() {
+ return this.configuration.getTopics();
+ }
+
+ /**
+ * Create an {@code ExecutableStreamsApp} using the provided {@code KafkaEndpointConfig}
+ * @return {@code ExecutableStreamsApp}
+ */
+ @Override
+ public ExecutableStreamsApp withEndpoint(final KafkaEndpointConfig endpointConfig) {
+ final Map kafkaProperties = this.getKafkaProperties(endpointConfig);
+ final Topology topology = this.createTopology(kafkaProperties);
+ final EffectiveAppConfiguration effectiveConfiguration =
+ new EffectiveAppConfiguration<>(this.getTopics(), kafkaProperties);
+ return ExecutableStreamsApp.builder()
+ .topology(topology)
+ .config(new StreamsConfig(kafkaProperties))
+ .app(this.app)
+ .effectiveConfig(effectiveConfiguration)
+ .build();
+ }
+
+ /**
+ * Create the topology of the Kafka Streams app
+ *
+ * @param kafkaProperties configuration that should be used by clients to configure Kafka utilities
+ * @return topology of the Kafka Streams app
+ */
+ public Topology createTopology(final Map kafkaProperties) {
+ final TopologyBuilder topologyBuilder = new TopologyBuilder(this.getTopics(), kafkaProperties);
+ this.app.buildTopology(topologyBuilder);
+ return topologyBuilder.build();
+ }
+
+ @Override
+ public void close() {
+ this.app.close();
+ }
+
+ private KafkaPropertiesFactory createPropertiesFactory(final KafkaEndpointConfig endpointConfig) {
+ final Map baseConfig = createBaseConfig(endpointConfig);
+ return KafkaPropertiesFactory.builder()
+ .baseConfig(baseConfig)
+ .app(this.app)
+ .configuration(this.configuration)
+ .endpointConfig(endpointConfig)
+ .build();
+ }
+
+}
diff --git a/streams-bootstrap/src/main/java/com/bakdata/kafka/DefaultStreamsUncaughtExceptionHandler.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/DefaultConfigurable.java
similarity index 78%
rename from streams-bootstrap/src/main/java/com/bakdata/kafka/DefaultStreamsUncaughtExceptionHandler.java
rename to streams-bootstrap-core/src/main/java/com/bakdata/kafka/DefaultConfigurable.java
index 341a1ad4..4ef82eaf 100644
--- a/streams-bootstrap/src/main/java/com/bakdata/kafka/DefaultStreamsUncaughtExceptionHandler.java
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/DefaultConfigurable.java
@@ -1,7 +1,7 @@
/*
* MIT License
*
- * Copyright (c) 2023 bakdata
+ * Copyright (c) 2024 bakdata
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
@@ -24,11 +24,11 @@
package com.bakdata.kafka;
-import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler;
+import java.util.Map;
-class DefaultStreamsUncaughtExceptionHandler implements StreamsUncaughtExceptionHandler {
+class DefaultConfigurable implements Configurable {
@Override
- public StreamThreadExceptionResponse handle(final Throwable e) {
- return StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
+ public T configure(final Map config, final boolean isKey) {
+ return null;
}
}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/DefaultStreamsUncaughtExceptionHandler.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/DefaultStreamsUncaughtExceptionHandler.java
new file mode 100644
index 00000000..ccb4ca52
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/DefaultStreamsUncaughtExceptionHandler.java
@@ -0,0 +1,39 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler;
+
+/**
+ * {@code StreamsUncaughtExceptionHandler} that does not handle the exception and responds with
+ * {@link org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse#SHUTDOWN_CLIENT}. Mimics default behavior of {@link org.apache.kafka.streams.KafkaStreams} if no {@code StreamsUncaughtExceptionHandler} has been configured.
+ * @see org.apache.kafka.streams.KafkaStreams#setUncaughtExceptionHandler(StreamsUncaughtExceptionHandler)
+ */
+class DefaultStreamsUncaughtExceptionHandler implements StreamsUncaughtExceptionHandler {
+ @Override
+ public StreamThreadExceptionResponse handle(final Throwable e) {
+ return StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
+ }
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/EffectiveAppConfiguration.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/EffectiveAppConfiguration.java
new file mode 100644
index 00000000..4c594344
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/EffectiveAppConfiguration.java
@@ -0,0 +1,56 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import com.bakdata.kafka.util.ImprovedAdminClient;
+import java.util.Map;
+import lombok.EqualsAndHashCode;
+import lombok.NonNull;
+import lombok.Value;
+
+/**
+ * Configuration for setting up an app
+ * @param type of topic config
+ * @see StreamsApp#setup(EffectiveAppConfiguration)
+ * @see StreamsApp#setupCleanUp(EffectiveAppConfiguration)
+ * @see ProducerApp#setup(EffectiveAppConfiguration)
+ * @see ProducerApp#setupCleanUp(EffectiveAppConfiguration)
+ */
+@Value
+@EqualsAndHashCode
+public class EffectiveAppConfiguration {
+ @NonNull
+ T topics;
+ @NonNull
+ Map kafkaProperties;
+
+ /**
+ * Create a new {@code ImprovedAdminClient} using {@link #kafkaProperties}
+ * @return {@code ImprovedAdminClient}
+ */
+ public ImprovedAdminClient createAdminClient() {
+ return ImprovedAdminClient.create(this.kafkaProperties);
+ }
+}
diff --git a/streams-bootstrap/src/main/java/com/bakdata/kafka/EnvironmentStreamsConfigParser.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/EnvironmentStreamsConfigParser.java
similarity index 98%
rename from streams-bootstrap/src/main/java/com/bakdata/kafka/EnvironmentStreamsConfigParser.java
rename to streams-bootstrap-core/src/main/java/com/bakdata/kafka/EnvironmentStreamsConfigParser.java
index 8f283b5a..b2bd4f97 100644
--- a/streams-bootstrap/src/main/java/com/bakdata/kafka/EnvironmentStreamsConfigParser.java
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/EnvironmentStreamsConfigParser.java
@@ -1,7 +1,7 @@
/*
* MIT License
*
- * Copyright (c) 2023 bakdata
+ * Copyright (c) 2024 bakdata
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ExecutableApp.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ExecutableApp.java
new file mode 100644
index 00000000..bb55cd39
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ExecutableApp.java
@@ -0,0 +1,56 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+/**
+ * An application with a corresponding topic and Kafka configuration
+ * @param type returned by {@link #createRunner()} and {@link #createRunner(Object)}
+ * @param type returned by {@link #createCleanUpRunner()}
+ * @param type of options to create runner
+ */
+public interface ExecutableApp extends AutoCloseable {
+
+ @Override
+ void close();
+
+ /**
+ * Create {@code Runner} in order to run application with default options
+ * @return {@code Runner}
+ */
+ R createRunner();
+
+ /**
+ * Create {@code Runner} in order to run application
+ * @param options options for creating runner
+ * @return {@code Runner}
+ */
+ R createRunner(O options);
+
+ /**
+ * Create {@code CleanUpRunner} in order to clean application
+ * @return {@code CleanUpRunner}
+ */
+ C createCleanUpRunner();
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ExecutableProducerApp.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ExecutableProducerApp.java
new file mode 100644
index 00000000..d4b1b370
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ExecutableProducerApp.java
@@ -0,0 +1,81 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import java.util.Map;
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+
+/**
+ * A {@link ProducerApp} with a corresponding {@link ProducerTopicConfig} and Kafka configuration
+ * @param type of {@link ProducerApp}
+ */
+@RequiredArgsConstructor(access = AccessLevel.PACKAGE)
+@Getter
+public class ExecutableProducerApp
+ implements ExecutableApp {
+ private final @NonNull ProducerTopicConfig topics;
+ private final @NonNull Map kafkaProperties;
+ private final @NonNull T app;
+
+ /**
+ * Create {@code ProducerCleanUpRunner} in order to clean application
+ * @return {@code ProducerCleanUpRunner}
+ */
+ @Override
+ public ProducerCleanUpRunner createCleanUpRunner() {
+ final EffectiveAppConfiguration configuration = this.createEffectiveConfiguration();
+ final ProducerCleanUpConfiguration configurer = this.app.setupCleanUp(configuration);
+ return ProducerCleanUpRunner.create(this.topics, this.kafkaProperties, configurer);
+ }
+
+ /**
+ * Create {@code ProducerRunner} in order to run application
+ * @return {@code ProducerRunner}
+ */
+ @Override
+ public ProducerRunner createRunner() {
+ return this.createRunner(ProducerExecutionOptions.builder().build());
+ }
+
+ @Override
+ public ProducerRunner createRunner(final ProducerExecutionOptions options) {
+ final ProducerBuilder producerBuilder = new ProducerBuilder(this.topics, this.kafkaProperties);
+ final EffectiveAppConfiguration configuration = this.createEffectiveConfiguration();
+ this.app.setup(configuration);
+ return new ProducerRunner(this.app.buildRunnable(producerBuilder));
+ }
+
+ @Override
+ public void close() {
+ this.app.close();
+ }
+
+ private EffectiveAppConfiguration createEffectiveConfiguration() {
+ return new EffectiveAppConfiguration<>(this.topics, this.kafkaProperties);
+ }
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ExecutableStreamsApp.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ExecutableStreamsApp.java
new file mode 100644
index 00000000..25009786
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ExecutableStreamsApp.java
@@ -0,0 +1,88 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import lombok.AccessLevel;
+import lombok.Builder;
+import lombok.Getter;
+import lombok.NonNull;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.Topology;
+
+/**
+ * A {@link StreamsApp} with a corresponding {@link Topology} and {@link StreamsConfig}
+ * @param type of {@link ProducerApp}
+ */
+@Builder(access = AccessLevel.PACKAGE)
+@Getter
+public class ExecutableStreamsApp
+ implements ExecutableApp {
+
+ @Getter
+ private final @NonNull Topology topology;
+ @Getter
+ private final @NonNull StreamsConfig config;
+ @Getter
+ private final @NonNull T app;
+ private final @NonNull EffectiveAppConfiguration effectiveConfig;
+
+ /**
+ * Create {@code StreamsCleanUpRunner} in order to clean application
+ * @return {@code StreamsCleanUpRunner}
+ */
+ @Override
+ public StreamsCleanUpRunner createCleanUpRunner() {
+ final StreamsCleanUpConfiguration configurer = this.app.setupCleanUp(this.effectiveConfig);
+ return StreamsCleanUpRunner.create(this.topology, this.config, configurer);
+ }
+
+ /**
+ * Create {@code StreamsRunner} in order to run application with default {@link StreamsExecutionOptions}
+ * @return {@code StreamsRunner}
+ * @see StreamsRunner#StreamsRunner(Topology, StreamsConfig)
+ */
+ @Override
+ public StreamsRunner createRunner() {
+ this.app.setup(this.effectiveConfig);
+ return new StreamsRunner(this.topology, this.config);
+ }
+
+ /**
+ * Create {@code StreamsRunner} in order to run application
+ * @param executionOptions options for running Kafka Streams application
+ * @return {@code StreamsRunner}
+ * @see StreamsRunner#StreamsRunner(Topology, StreamsConfig, StreamsExecutionOptions)
+ */
+ @Override
+ public StreamsRunner createRunner(final StreamsExecutionOptions executionOptions) {
+ this.app.setup(this.effectiveConfig);
+ return new StreamsRunner(this.topology, this.config, executionOptions);
+ }
+
+ @Override
+ public void close() {
+ this.app.close();
+ }
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/HasCleanHook.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/HasCleanHook.java
new file mode 100644
index 00000000..d6238982
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/HasCleanHook.java
@@ -0,0 +1,40 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+/**
+ * Interface for performing actions when cleaning apps
+ * @param self for chaining
+ */
+@FunctionalInterface
+public interface HasCleanHook {
+ /**
+ * Register a hook that is invoked when cleaning apps
+ * @param hook factory to create hook from
+ * @return self for chaining
+ */
+ SELF registerCleanHook(Runnable hook);
+
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/HasTopicHooks.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/HasTopicHooks.java
new file mode 100644
index 00000000..f3433e16
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/HasTopicHooks.java
@@ -0,0 +1,53 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+/**
+ * Interface for performing actions on topics
+ * @param self for chaining
+ */
+@FunctionalInterface
+public interface HasTopicHooks {
+ /**
+ * Register a hook that is invoked when performing actions on topics
+ *
+ * @param hook Action to run. Topic is passed as parameter
+ * @return self for chaining
+ */
+ SELF registerTopicHook(TopicHook hook);
+
+ /**
+ * Hook for performing actions on topics
+ */
+ interface TopicHook {
+ /**
+ * Called when a topic is deleted
+ * @param topic name of the topic
+ */
+ default void deleted(final String topic) {
+ // do nothing
+ }
+ }
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ImprovedStreamsConfig.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ImprovedStreamsConfig.java
new file mode 100644
index 00000000..95b9a484
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ImprovedStreamsConfig.java
@@ -0,0 +1,69 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import lombok.NonNull;
+import lombok.Value;
+import org.apache.kafka.streams.StreamsConfig;
+
+/**
+ * Class for simplified access to configs provided by {@link StreamsConfig}
+ */
+@Value
+public class ImprovedStreamsConfig {
+
+ @NonNull
+ StreamsConfig streamsConfig;
+
+ /**
+ * Get the application id of the underlying {@link StreamsConfig}
+ * @return application id
+ * @see StreamsConfig#APPLICATION_ID_CONFIG
+ */
+ public String getAppId() {
+ return this.streamsConfig.getString(StreamsConfig.APPLICATION_ID_CONFIG);
+ }
+
+ /**
+ * Get the bootstrap servers of the underlying {@link StreamsConfig}
+ * @return list of bootstrap servers
+ * @see StreamsConfig#BOOTSTRAP_SERVERS_CONFIG
+ */
+ public List getBoostrapServers() {
+ return this.streamsConfig.getList(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG);
+ }
+
+ /**
+ * Get all configs of the underlying {@link StreamsConfig}
+ * @return Kafka configs
+ * @see StreamsConfig#originals()
+ */
+ public Map getKafkaProperties() {
+ return Collections.unmodifiableMap(this.streamsConfig.originals());
+ }
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/KafkaEndpointConfig.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/KafkaEndpointConfig.java
new file mode 100644
index 00000000..5aa42876
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/KafkaEndpointConfig.java
@@ -0,0 +1,68 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import lombok.Builder;
+import lombok.NonNull;
+import org.apache.kafka.streams.StreamsConfig;
+
+/**
+ * Configuration to connect to Kafka infrastructure, i.e., brokers and optionally schema registry.
+ */
+@Builder
+public class KafkaEndpointConfig {
+ private final @NonNull String brokers;
+ private final String schemaRegistryUrl;
+
+ /**
+ * Create Kafka properties to connect to infrastructure.
+ * The following properties are configured:
+ *
+ * {@code bootstrap.servers}
+ * {@code schema.registry.url}
+ *
+ * @return properties used for connecting to Kafka
+ */
+ public Map createKafkaProperties() {
+ final Map kafkaConfig = new HashMap<>();
+ kafkaConfig.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, this.brokers);
+ if (this.isSchemaRegistryConfigured()) {
+ kafkaConfig.put(AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, this.schemaRegistryUrl);
+ }
+ return Collections.unmodifiableMap(kafkaConfig);
+ }
+
+ /**
+ * Check if schema registry has been configured
+ * @return true if {@link #schemaRegistryUrl} has been configured
+ */
+ public boolean isSchemaRegistryConfigured() {
+ return this.schemaRegistryUrl != null;
+ }
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/KafkaPropertiesFactory.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/KafkaPropertiesFactory.java
new file mode 100644
index 00000000..aecd5dc2
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/KafkaPropertiesFactory.java
@@ -0,0 +1,49 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import lombok.Builder;
+import lombok.NonNull;
+
+@Builder
+class KafkaPropertiesFactory {
+ private final @NonNull Map baseConfig;
+ private final @NonNull App, ?> app;
+ private final @NonNull AppConfiguration> configuration;
+ private final @NonNull KafkaEndpointConfig endpointConfig;
+
+ Map createKafkaProperties(final Map configOverrides) {
+ final Map kafkaConfig = new HashMap<>(this.baseConfig);
+ kafkaConfig.putAll(this.app.createKafkaProperties());
+ kafkaConfig.putAll(EnvironmentStreamsConfigParser.parseVariables(System.getenv()));
+ kafkaConfig.putAll(this.configuration.getKafkaConfig());
+ kafkaConfig.putAll(this.endpointConfig.createKafkaProperties());
+ kafkaConfig.putAll(configOverrides);
+ return Collections.unmodifiableMap(kafkaConfig);
+ }
+}
diff --git a/streams-bootstrap/src/main/java/com/bakdata/kafka/NoOpStateListener.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/NoOpStateListener.java
similarity index 89%
rename from streams-bootstrap/src/main/java/com/bakdata/kafka/NoOpStateListener.java
rename to streams-bootstrap-core/src/main/java/com/bakdata/kafka/NoOpStateListener.java
index 51563b92..cfc312f6 100644
--- a/streams-bootstrap/src/main/java/com/bakdata/kafka/NoOpStateListener.java
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/NoOpStateListener.java
@@ -1,7 +1,7 @@
/*
* MIT License
*
- * Copyright (c) 2023 bakdata
+ * Copyright (c) 2024 bakdata
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
@@ -27,6 +27,10 @@
import org.apache.kafka.streams.KafkaStreams.State;
import org.apache.kafka.streams.KafkaStreams.StateListener;
+/**
+ * {@code StateListener} that does nothing.
+ * @see org.apache.kafka.streams.KafkaStreams#setStateListener(StateListener)
+ */
class NoOpStateListener implements StateListener {
@Override
public void onChange(final State newState, final State oldState) {
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/Preconfigured.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/Preconfigured.java
new file mode 100644
index 00000000..ee9bc3cb
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/Preconfigured.java
@@ -0,0 +1,155 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import static java.util.Collections.emptyMap;
+
+import java.util.HashMap;
+import java.util.Map;
+import lombok.AccessLevel;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serializer;
+
+/**
+ * A pre-configured {@link Serde} or {@link Serializer}, i.e., configs and isKey are set.
+ * @param type of underlying configurable
+ */
+@RequiredArgsConstructor(access = AccessLevel.PRIVATE)
+public final class Preconfigured {
+ private final @NonNull Configurable configurable;
+ private final @NonNull Map configOverrides;
+
+ private Preconfigured(final Configurable configurable) {
+ this(configurable, emptyMap());
+ }
+
+ /**
+ * Create a pre-configured {@code Serde} that returns {@code null} when calling
+ * {@link Preconfigured#configureForKeys(Map)} and {@link Preconfigured#configureForValues(Map)}
+ * @return pre-configured serde
+ * @param type (de-)serialized by the {@code Serde}
+ */
+ public static Preconfigured> defaultSerde() {
+ return new Preconfigured<>(new DefaultConfigurable<>());
+ }
+
+ /**
+ * Pre-configure a {@code Serde}
+ * @param serde {@code Serde} to pre-configure
+ * @return pre-configured serde
+ * @param type of {@link Serde}
+ * @param type (de-)serialized by the {@code Serde}
+ */
+ public static , T> Preconfigured create(final S serde) {
+ return new Preconfigured<>(configurable(serde));
+ }
+
+ /**
+ * Pre-configure a {@code Serde} with config overrides
+ * @param serde {@code Serde} to pre-configure
+ * @param configOverrides configs passed to {@link Serde#configure(Map, boolean)}
+ * @return pre-configured serde
+ * @param type of {@link Serde}
+ * @param type (de-)serialized by the {@code Serde}
+ */
+ public static , T> Preconfigured create(final S serde,
+ final Map configOverrides) {
+ return new Preconfigured<>(configurable(serde), configOverrides);
+ }
+
+ /**
+ * Create a pre-configured {@code Serializer} that returns {@code null} when calling
+ * {@link Preconfigured#configureForKeys(Map)} and {@link Preconfigured#configureForValues(Map)}
+ * @return pre-configured serializer
+ * @param type (de-)serialized by the {@code Serializer}
+ */
+ public static Preconfigured> defaultSerializer() {
+ return new Preconfigured<>(new DefaultConfigurable<>());
+ }
+
+ /**
+ * Pre-configure a {@code Serializer}
+ * @param serializer {@code Serializer} to pre-configure
+ * @return pre-configured serializer
+ * @param type of {@link Serializer}
+ * @param type serialized by the {@code Serializer}
+ */
+ public static , T> Preconfigured create(final S serializer) {
+ return new Preconfigured<>(configurable(serializer));
+ }
+
+ /**
+ * Pre-configure a {@code Serializer}
+ * @param serializer {@code Serializer} to pre-configure
+ * @param configOverrides configs passed to {@link Serializer#configure(Map, boolean)}
+ * @return pre-configured serializer
+ * @param type of {@link Serializer}
+ * @param type serialized by the {@code Serializer}
+ */
+ public static , T> Preconfigured create(final S serializer,
+ final Map configOverrides) {
+ return new Preconfigured<>(configurable(serializer), configOverrides);
+ }
+
+ private static , T> ConfigurableSerde configurable(final S serde) {
+ return new ConfigurableSerde<>(serde);
+ }
+
+ private static , T> ConfigurableSerializer configurable(final S serializer) {
+ return new ConfigurableSerializer<>(serializer);
+ }
+
+ /**
+ * Configure for values using a base config
+ * @param baseConfig Base config. {@link #configOverrides} override properties of base config.
+ * @return configured instance
+ */
+ public T configureForValues(final Map baseConfig) {
+ return this.configure(baseConfig, false);
+ }
+
+ /**
+ * Configure for keys using a base config
+ * @param baseConfig Base config. {@link #configOverrides} override properties of base config.
+ * @return configured instance
+ */
+ public T configureForKeys(final Map baseConfig) {
+ return this.configure(baseConfig, true);
+ }
+
+ private T configure(final Map baseConfig, final boolean isKey) {
+ final Map serializerConfig = this.mergeConfig(baseConfig);
+ return this.configurable.configure(serializerConfig, isKey);
+ }
+
+ private Map mergeConfig(final Map baseConfig) {
+ final Map config = new HashMap<>(baseConfig);
+ config.putAll(this.configOverrides);
+ return config;
+ }
+
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerApp.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerApp.java
new file mode 100644
index 00000000..42cee6e6
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerApp.java
@@ -0,0 +1,49 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+/**
+ * Application that defines how to produce messages to Kafka and necessary configurations
+ */
+@FunctionalInterface
+public interface ProducerApp extends App {
+
+ /**
+ * Create a runnable that produces Kafka messages
+ * @param builder provides all runtime application configurations
+ * @return {@code ProducerRunnable}
+ */
+ ProducerRunnable buildRunnable(ProducerBuilder builder);
+
+ /**
+ * @return {@code ProducerCleanUpConfiguration}
+ * @see ProducerCleanUpRunner
+ */
+ @Override
+ default ProducerCleanUpConfiguration setupCleanUp(
+ final EffectiveAppConfiguration configuration) {
+ return new ProducerCleanUpConfiguration();
+ }
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerBuilder.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerBuilder.java
new file mode 100644
index 00000000..df01fdad
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerBuilder.java
@@ -0,0 +1,90 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import java.util.Map;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+import lombok.Value;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.common.serialization.Serializer;
+
+/**
+ * Provides all runtime configurations when running a {@link ProducerApp}
+ *
+ * @see ProducerApp#buildRunnable(ProducerBuilder)
+ */
+@RequiredArgsConstructor
+@Value
+public class ProducerBuilder {
+
+ @NonNull
+ ProducerTopicConfig topics;
+ @NonNull
+ Map kafkaProperties;
+
+ /**
+ * Create a new {@code Producer} using {@link #kafkaProperties}
+ * @return {@code Producer}
+ * @param type of keys
+ * @param type of values
+ * @see KafkaProducer#KafkaProducer(Map)
+ */
+ public Producer createProducer() {
+ return new KafkaProducer<>(this.kafkaProperties);
+ }
+
+ /**
+ * Create a new {@code Producer} using {@link #kafkaProperties} and provided {@code Serializers}
+ * @param keySerializer {@code Serializer} to use for keys
+ * @param valueSerializer {@code Serializer} to use for values
+ * @return {@code Producer}
+ * @param type of keys
+ * @param type of values
+ * @see KafkaProducer#KafkaProducer(Map, Serializer, Serializer)
+ */
+ public Producer createProducer(final Serializer keySerializer,
+ final Serializer valueSerializer) {
+ return new KafkaProducer<>(this.kafkaProperties, keySerializer, valueSerializer);
+ }
+
+ /**
+ * Create {@code Configurator} to configure {@link org.apache.kafka.common.serialization.Serde} and
+ * {@link org.apache.kafka.common.serialization.Serializer} using {@link #kafkaProperties}.
+ * @return {@code Configurator}
+ */
+ public Configurator createConfigurator() {
+ return new Configurator(this.kafkaProperties);
+ }
+
+ /**
+ * Create {@code EffectiveAppConfiguration} used by this app
+ * @return {@code EffectiveAppConfiguration}
+ */
+ public EffectiveAppConfiguration createEffectiveConfiguration() {
+ return new EffectiveAppConfiguration<>(this.topics, this.kafkaProperties);
+ }
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerCleanUpConfiguration.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerCleanUpConfiguration.java
new file mode 100644
index 00000000..aa563091
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerCleanUpConfiguration.java
@@ -0,0 +1,64 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import lombok.NonNull;
+
+/**
+ * Provides configuration options for {@link ProducerCleanUpRunner}
+ */
+public class ProducerCleanUpConfiguration
+ implements HasTopicHooks, HasCleanHook {
+ private final @NonNull Collection topicHooks = new ArrayList<>();
+ private final @NonNull Collection cleanHooks = new ArrayList<>();
+
+ /**
+ * Register a hook that is executed whenever a topic has been deleted by the cleanup runner.
+ */
+ @Override
+ public ProducerCleanUpConfiguration registerTopicHook(final TopicHook hook) {
+ this.topicHooks.add(hook);
+ return this;
+ }
+
+ /**
+ * Register an action that is executed after {@link ProducerCleanUpRunner#clean()} has finished
+ */
+ @Override
+ public ProducerCleanUpConfiguration registerCleanHook(final Runnable hook) {
+ this.cleanHooks.add(hook);
+ return this;
+ }
+
+ void runCleanHooks() {
+ this.cleanHooks.forEach(Runnable::run);
+ }
+
+ void runTopicDeletionHooks(final String topic) {
+ this.topicHooks.forEach(hook -> hook.deleted(topic));
+ }
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerCleanUpRunner.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerCleanUpRunner.java
new file mode 100644
index 00000000..bfddf6d1
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerCleanUpRunner.java
@@ -0,0 +1,114 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import com.bakdata.kafka.util.ImprovedAdminClient;
+import java.util.Map;
+import lombok.AccessLevel;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.jooq.lambda.Seq;
+
+
+/**
+ * Delete all output topics specified by a {@link ProducerTopicConfig}
+ */
+@Slf4j
+@RequiredArgsConstructor(access = AccessLevel.PRIVATE)
+public final class ProducerCleanUpRunner implements CleanUpRunner {
+ private final @NonNull ProducerTopicConfig topics;
+ private final @NonNull Map kafkaProperties;
+ private final @NonNull ProducerCleanUpConfiguration cleanHooks;
+
+ /**
+ * Create a new {@code ProducerCleanUpRunner} with default {@link ProducerCleanUpConfiguration}
+ *
+ * @param topics topic configuration to infer output topics that require cleaning
+ * @param kafkaProperties configuration to connect to Kafka admin tools
+ * @return {@code ProducerCleanUpRunner}
+ */
+ public static ProducerCleanUpRunner create(@NonNull final ProducerTopicConfig topics,
+ @NonNull final Map kafkaProperties) {
+ return create(topics, kafkaProperties, new ProducerCleanUpConfiguration());
+ }
+
+ /**
+ * Create a new {@code ProducerCleanUpRunner}
+ *
+ * @param topics topic configuration to infer output topics that require cleaning
+ * @param kafkaProperties configuration to connect to Kafka admin tools
+ * @param configuration configuration for hooks that are called when running {@link #clean()}
+ * @return {@code ProducerCleanUpRunner}
+ */
+ public static ProducerCleanUpRunner create(@NonNull final ProducerTopicConfig topics,
+ @NonNull final Map kafkaProperties,
+ @NonNull final ProducerCleanUpConfiguration configuration) {
+ return new ProducerCleanUpRunner(topics, kafkaProperties, configuration);
+ }
+
+ /**
+ * Delete all output topics
+ */
+ @Override
+ public void clean() {
+ try (final ImprovedAdminClient adminClient = this.createAdminClient()) {
+ final Task task = new Task(adminClient);
+ task.clean();
+ }
+ }
+
+ private ImprovedAdminClient createAdminClient() {
+ return ImprovedAdminClient.create(this.kafkaProperties);
+ }
+
+ @RequiredArgsConstructor
+ private class Task {
+
+ private final @NonNull ImprovedAdminClient adminClient;
+
+ private void clean() {
+ this.deleteTopics();
+ ProducerCleanUpRunner.this.cleanHooks.runCleanHooks();
+ }
+
+ private void deleteTopics() {
+ final Iterable outputTopics = this.getAllOutputTopics();
+ outputTopics.forEach(this::deleteTopic);
+ }
+
+ private void deleteTopic(final String topic) {
+ this.adminClient.getSchemaTopicClient()
+ .deleteTopicAndResetSchemaRegistry(topic);
+ ProducerCleanUpRunner.this.cleanHooks.runTopicDeletionHooks(topic);
+ }
+
+ private Iterable getAllOutputTopics() {
+ return Seq.of(ProducerCleanUpRunner.this.topics.getOutputTopic())
+ .concat(ProducerCleanUpRunner.this.topics.getExtraOutputTopics().values());
+ }
+ }
+
+}
diff --git a/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerExecutionOptions.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerExecutionOptions.java
new file mode 100644
index 00000000..95328e2f
--- /dev/null
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerExecutionOptions.java
@@ -0,0 +1,34 @@
+/*
+ * MIT License
+ *
+ * Copyright (c) 2024 bakdata
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package com.bakdata.kafka;
+
+import lombok.Builder;
+
+/**
+ * Options to run a Kafka Producer app using {@link ProducerRunner}
+ */
+@Builder
+public final class ProducerExecutionOptions {
+}
diff --git a/streams-bootstrap-large-messages/src/main/java/com/bakdata/kafka/LargeMessageKafkaProducerApplication.java b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerRunnable.java
similarity index 73%
rename from streams-bootstrap-large-messages/src/main/java/com/bakdata/kafka/LargeMessageKafkaProducerApplication.java
rename to streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerRunnable.java
index 2fea1acf..75588546 100644
--- a/streams-bootstrap-large-messages/src/main/java/com/bakdata/kafka/LargeMessageKafkaProducerApplication.java
+++ b/streams-bootstrap-core/src/main/java/com/bakdata/kafka/ProducerRunnable.java
@@ -1,7 +1,7 @@
/*
* MIT License
*
- * Copyright (c) 2023 bakdata
+ * Copyright (c) 2024 bakdata
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
@@ -24,16 +24,19 @@
package com.bakdata.kafka;
-import java.util.function.Consumer;
-
/**
- * Kafka Producer Application that automatically removes files associated with {@code LargeMessageSerializer}
+ * Produce messages to Kafka
*/
-public abstract class LargeMessageKafkaProducerApplication extends KafkaProducerApplication {
+@FunctionalInterface
+public interface ProducerRunnable extends AutoCloseable {
+
+ /**
+ * Produce messages to Kafka
+ */
+ void run();
@Override
- protected Consumer