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

Support custom KafkaStreams implementations #3516

Merged
merged 6 commits into from
Sep 27, 2024
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright 2018-2019 the original author or authors.
* Copyright 2018-2024 the original author or authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
Expand All @@ -16,7 +16,11 @@

package org.springframework.kafka.config;

import java.util.Properties;

import org.apache.kafka.streams.KafkaClientSupplier;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.Topology;

/**
* Callback interface that can be used to configure {@link KafkaStreams} directly.
Expand All @@ -30,6 +34,14 @@
@FunctionalInterface
public interface KafkaStreamsCustomizer {
agavra marked this conversation as resolved.
Show resolved Hide resolved

default KafkaStreams initKafkaStreams(
agavra marked this conversation as resolved.
Show resolved Hide resolved
Topology topology,
Properties properties,
KafkaClientSupplier clientSupplier
) {
return new KafkaStreams(topology, properties, clientSupplier);
}

void customize(KafkaStreams kafkaStreams);

}
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ public class StreamsBuilderFactoryBean extends AbstractFactoryBean<StreamsBuilde
private KafkaStreamsInfrastructureCustomizer infrastructureCustomizer = new KafkaStreamsInfrastructureCustomizer() {
};

private KafkaStreamsCustomizer kafkaStreamsCustomizer;
private KafkaStreamsCustomizer kafkaStreamsCustomizer = kafkaStreams -> { };

private KafkaStreams.StateListener stateListener;

Expand Down Expand Up @@ -361,15 +361,15 @@ public void start() {
try {
Assert.state(this.properties != null,
"streams configuration properties must not be null");
this.kafkaStreams = new KafkaStreams(this.topology, this.properties, this.clientSupplier);
this.kafkaStreams = this.kafkaStreamsCustomizer.initKafkaStreams(
this.topology, this.properties, this.clientSupplier
);
this.kafkaStreams.setStateListener(this.stateListener);
this.kafkaStreams.setGlobalStateRestoreListener(this.stateRestoreListener);
if (this.streamsUncaughtExceptionHandler != null) {
this.kafkaStreams.setUncaughtExceptionHandler(this.streamsUncaughtExceptionHandler);
}
if (this.kafkaStreamsCustomizer != null) {
sobychacko marked this conversation as resolved.
Show resolved Hide resolved
this.kafkaStreamsCustomizer.customize(this.kafkaStreams);
}
this.kafkaStreamsCustomizer.customize(this.kafkaStreams);
if (this.cleanupConfig.cleanupOnStart()) {
this.kafkaStreams.cleanUp();
}
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright 2018-2023 the original author or authors.
* Copyright 2018-2024 the original author or authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
Expand All @@ -26,6 +26,7 @@

import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.KafkaClientSupplier;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.StreamsConfig;
Expand Down Expand Up @@ -95,6 +96,7 @@ public void testKafkaStreamsCustomizer(@Autowired KafkaStreamsConfiguration conf
.isEqualTo(1000);
assertThat(this.config.builderConfigured.get()).isTrue();
assertThat(this.config.topologyConfigured.get()).isTrue();
assertThat(this.config.ksInitialized.get()).isTrue();
assertThat(this.meterRegistry.get("kafka.consumer.coordinator.join.total")
.tag("customTag", "stream")
.tag("spring.id", KafkaStreamsDefaultConfiguration.DEFAULT_STREAMS_BUILDER_BEAN_NAME)
Expand All @@ -118,6 +120,8 @@ public static class KafkaStreamsConfig {

final AtomicBoolean topologyConfigured = new AtomicBoolean();

final AtomicBoolean ksInitialized = new AtomicBoolean();

@Autowired
EmbeddedKafkaBroker broker;

Expand Down Expand Up @@ -168,7 +172,26 @@ public KafkaStreamsConfiguration kStreamsConfigs() {
}

private KafkaStreamsCustomizer customizer() {
return kafkaStreams -> kafkaStreams.setStateListener(STATE_LISTENER);
return new KafkaStreamsCustomizer() {
@Override
public KafkaStreams initKafkaStreams(
final Topology topology,
final Properties properties,
final KafkaClientSupplier clientSupplier
) {
ksInitialized.set(true);
return KafkaStreamsCustomizer.super.initKafkaStreams(
topology,
properties,
clientSupplier
);
}

@Override
public void customize(final KafkaStreams kafkaStreams) {
kafkaStreams.setStateListener(STATE_LISTENER);
sobychacko marked this conversation as resolved.
Show resolved Hide resolved
}
};
}

@Bean
Expand Down