Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Use custom streams implementation to simplify error handling, writing to topics and serde configuration #265

Draft
wants to merge 10 commits into
base: master
Choose a base branch
from
6 changes: 3 additions & 3 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -57,14 +57,14 @@ Create a subclass of `KafkaStreamsApplication` and implement the abstract method
and `getUniqueAppId()`. You can define the topology of your application in `buildTopology()`.

```java
import com.bakdata.kafka.ImprovedKStream;
import com.bakdata.kafka.KafkaStreamsApplication;
import com.bakdata.kafka.SerdeConfig;
import com.bakdata.kafka.StreamsApp;
import com.bakdata.kafka.StreamsTopicConfig;
import com.bakdata.kafka.TopologyBuilder;
import java.util.Map;
import org.apache.kafka.common.serialization.Serdes.StringSerde;
import org.apache.kafka.streams.kstream.KStream;

public class MyStreamsApplication extends KafkaStreamsApplication<StreamsApp> {
public static void main(final String[] args) {
Expand All @@ -76,11 +76,11 @@ public class MyStreamsApplication extends KafkaStreamsApplication<StreamsApp> {
return new StreamsApp() {
@Override
public void buildTopology(final TopologyBuilder builder) {
final KStream<String, String> input = builder.streamInput();
final ImprovedKStream<String, String> input = builder.streamInput();

// your topology

input.to(builder.getTopics().getOutputTopic());
input.toOutput();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -192,7 +192,7 @@ public StreamsApp createApp() {
return new StreamsApp() {
@Override
public void buildTopology(final TopologyBuilder builder) {
builder.streamInput().to(builder.getTopics().getOutputTopic());
builder.streamInput().toOutputTopic();
}

@Override
Expand Down Expand Up @@ -265,7 +265,7 @@ void shouldExitWithSuccessCodeOnShutdown() {
@Override
public void buildTopology(final TopologyBuilder builder) {
builder.streamInput(Consumed.with(Serdes.ByteArray(), Serdes.ByteArray()))
.to(builder.getTopics().getOutputTopic());
.toOutputTopic();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
/*
* MIT License
*
* Copyright (c) 2024 bakdata
* Copyright (c) 2025 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
Expand All @@ -28,7 +28,6 @@
import lombok.NoArgsConstructor;
import lombok.Setter;
import org.apache.kafka.common.serialization.Serdes.StringSerde;
import org.apache.kafka.streams.kstream.KStream;

@NoArgsConstructor
@Getter
Expand All @@ -49,8 +48,8 @@ public StreamsApp createApp() {
return new StreamsApp() {
@Override
public void buildTopology(final TopologyBuilder builder) {
final KStream<String, String> input = builder.streamInput();
input.to(builder.getTopics().getOutputTopic());
final ImprovedKStream<String, String> input = builder.streamInput();
input.toOutputTopic();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
/*
* MIT License
*
* Copyright (c) 2024 bakdata
* Copyright (c) 2025 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
Expand All @@ -24,20 +24,20 @@

package com.bakdata.kafka.test_applications;

import com.bakdata.kafka.ImprovedKStream;
import com.bakdata.kafka.SerdeConfig;
import com.bakdata.kafka.StreamsApp;
import com.bakdata.kafka.StreamsTopicConfig;
import com.bakdata.kafka.TopologyBuilder;
import lombok.NoArgsConstructor;
import org.apache.kafka.common.serialization.Serdes.StringSerde;
import org.apache.kafka.streams.kstream.KStream;

@NoArgsConstructor
public class Mirror implements StreamsApp {
@Override
public void buildTopology(final TopologyBuilder builder) {
final KStream<String, String> input = builder.streamInput();
input.to(builder.getTopics().getOutputTopic());
final ImprovedKStream<String, String> input = builder.streamInput();
input.toOutputTopic();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
/*
* MIT License
*
* Copyright (c) 2024 bakdata
* Copyright (c) 2025 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
Expand All @@ -24,6 +24,8 @@

package com.bakdata.kafka.test_applications;

import com.bakdata.kafka.ImprovedKStream;
import com.bakdata.kafka.ImprovedKTable;
import com.bakdata.kafka.SerdeConfig;
import com.bakdata.kafka.StreamsApp;
import com.bakdata.kafka.StreamsTopicConfig;
Expand All @@ -33,8 +35,6 @@
import lombok.NoArgsConstructor;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.Serdes.StringSerde;
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;

Expand All @@ -43,15 +43,15 @@ public class WordCount implements StreamsApp {

@Override
public void buildTopology(final TopologyBuilder builder) {
final KStream<String, String> textLines = builder.streamInput();
final ImprovedKStream<String, String> textLines = builder.streamInput();

final Pattern pattern = Pattern.compile("\\W+", Pattern.UNICODE_CHARACTER_CLASS);
final KTable<String, Long> wordCounts = textLines
final ImprovedKTable<String, Long> wordCounts = textLines
.flatMapValues(value -> Arrays.asList(pattern.split(value.toLowerCase())))
.groupBy((key, word) -> word)
.count(Materialized.as("counts"));

wordCounts.toStream().to(builder.getTopics().getOutputTopic(), Produced.valueSerde(Serdes.Long()));
wordCounts.toStream().toOutputTopic(Produced.valueSerde(Serdes.Long()));
}

@Override
Expand Down
1 change: 1 addition & 0 deletions streams-bootstrap-core/build.gradle.kts
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ dependencies {
version = "2.0.16"
)
implementation(group = "org.jooq", name = "jool", version = "0.9.15")
api(group = "com.bakdata.kafka", name = "error-handling-core", version = "1.6.1-SNAPSHOT")

val junitVersion: String by project
testRuntimeOnly(group = "org.junit.jupiter", name = "junit-jupiter-engine", version = junitVersion)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* MIT License
*
* Copyright (c) 2025 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.NonNull;
import lombok.RequiredArgsConstructor;
import lombok.With;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.streams.Topology.AutoOffsetReset;
import org.apache.kafka.streams.kstream.Consumed;
import org.apache.kafka.streams.processor.TimestampExtractor;

@With
@RequiredArgsConstructor(access = AccessLevel.PRIVATE)
public final class ConfiguredConsumed<K, V> {

private final @NonNull Preconfigured<Serde<K>> keySerde;
private final @NonNull Preconfigured<Serde<V>> valueSerde;
private final TimestampExtractor timestampExtractor;
private final AutoOffsetReset offsetResetPolicy;
private final String name;

public static <K, V> ConfiguredConsumed<K, V> keySerde(final Preconfigured<Serde<K>> keySerde) {
return with(keySerde, Preconfigured.defaultSerde());
}

public static <K, V> ConfiguredConsumed<K, V> valueSerde(final Preconfigured<Serde<V>> valueSerde) {
return with(Preconfigured.defaultSerde(), valueSerde);
}

public static <K, V> ConfiguredConsumed<K, V> with(final Preconfigured<Serde<K>> keySerde,
final Preconfigured<Serde<V>> valueSerde) {
return new ConfiguredConsumed<>(keySerde, valueSerde, null, null, null);
}

public static <K, V> ConfiguredConsumed<K, V> as(final String processorName) {
return new ConfiguredConsumed<>(Preconfigured.defaultSerde(), Preconfigured.defaultSerde(), null, null,
processorName);
}

Consumed<K, V> configure(final Configurator configurator) {
return Consumed.<K, V>as(this.name)
.withKeySerde(configurator.configureForKeys(this.keySerde))
.withValueSerde(configurator.configureForValues(this.valueSerde))
.withOffsetResetPolicy(this.offsetResetPolicy)
.withTimestampExtractor(this.timestampExtractor);
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,105 @@
/*
* MIT License
*
* Copyright (c) 2025 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.time.Duration;
import java.util.HashMap;
import java.util.Map;
import lombok.AccessLevel;
import lombok.NonNull;
import lombok.RequiredArgsConstructor;
import lombok.With;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.streams.kstream.Materialized;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.state.DslStoreSuppliers;

@RequiredArgsConstructor(access = AccessLevel.PRIVATE)
public final class ConfiguredMaterialized<K, V, S extends StateStore> {

@With
private final @NonNull Preconfigured<Serde<K>> keySerde;
@With
private final @NonNull Preconfigured<Serde<V>> valueSerde;
private final String storeName;
@With
private final Duration retention;
@With
private final DslStoreSuppliers storeType;
private final Map<String, String> topicConfig;
private final boolean loggingEnabled;
private final boolean cachingEnabled;

public static <K, V, S extends StateStore> ConfiguredMaterialized<K, V, S> keySerde(
final Preconfigured<Serde<K>> keySerde) {
return with(keySerde, Preconfigured.defaultSerde());
}

public static <K, V, S extends StateStore> ConfiguredMaterialized<K, V, S> valueSerde(
final Preconfigured<Serde<V>> valueSerde) {
return with(Preconfigured.defaultSerde(), valueSerde);
}

public static <K, V, S extends StateStore> ConfiguredMaterialized<K, V, S> with(
final Preconfigured<Serde<K>> keySerde,
final Preconfigured<Serde<V>> valueSerde) {
return new ConfiguredMaterialized<>(keySerde, valueSerde, null, null, null, new HashMap<>(), true, true);
}

public static <K, V, S extends StateStore> ConfiguredMaterialized<K, V, S> as(final String storeName) {
return new ConfiguredMaterialized<>(Preconfigured.defaultSerde(), Preconfigured.defaultSerde(), storeName, null,
null, new HashMap<>(), true, true);
}

public static <K, V, S extends StateStore> ConfiguredMaterialized<K, V, S> as(
final DslStoreSuppliers storeSuppliers) {
return new ConfiguredMaterialized<>(Preconfigured.defaultSerde(), Preconfigured.defaultSerde(), null, null,
storeSuppliers, new HashMap<>(), true, true);
}

Materialized<K, V, S> configure(final Configurator configurator) {
final Materialized<K, V, S> materialized = Materialized.<K, V, S>as(this.storeName)
.withKeySerde(configurator.configureForKeys(this.keySerde))
.withValueSerde(configurator.configureForValues(this.valueSerde));
if (this.retention != null) {
materialized.withRetention(this.retention);
}
if (this.storeType != null) {
materialized.withStoreType(this.storeType);
}
if (this.loggingEnabled) {
materialized.withLoggingEnabled(this.topicConfig);
} else {
materialized.withLoggingDisabled();
}
if (this.cachingEnabled) {
materialized.withCachingEnabled();
} else {
materialized.withCachingDisabled();
}
return materialized;
}

}
Loading