Skip to content

Commit 54190df

Browse files
cedric-schallersobychacko
authored andcommitted
GH-3176: Take default.dsl.store into account
Fixes: #3176 Currently, the applications need to use special customizers to use in-memory state stores. Kafka Streams provides a property to configure that - `default.dsl.store` (KIP-591), which the framework does not take into account when creating the `StreamsBuilder` since it ignores any config and always relies on customizations after the fact. This change allows this property to work in the framework so that applications can easily switch to in-memory state stores. **Auto-cherry-pick to `3.1.x` & `3.0.x`**
1 parent 781665e commit 54190df

File tree

2 files changed

+116
-5
lines changed

2 files changed

+116
-5
lines changed

spring-kafka/src/main/java/org/springframework/kafka/config/StreamsBuilderFactoryBean.java

Lines changed: 19 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,5 @@
11
/*
2-
* Copyright 2017-2022 the original author or authors.
2+
* Copyright 2017-2024 the original author or authors.
33
*
44
* Licensed under the Apache License, Version 2.0 (the "License");
55
* you may not use this file except in compliance with the License.
@@ -26,7 +26,9 @@
2626
import org.apache.kafka.streams.KafkaClientSupplier;
2727
import org.apache.kafka.streams.KafkaStreams;
2828
import org.apache.kafka.streams.StreamsBuilder;
29+
import org.apache.kafka.streams.StreamsConfig;
2930
import org.apache.kafka.streams.Topology;
31+
import org.apache.kafka.streams.TopologyConfig;
3032
import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler;
3133
import org.apache.kafka.streams.processor.StateRestoreListener;
3234
import org.apache.kafka.streams.processor.internals.DefaultKafkaClientSupplier;
@@ -54,6 +56,7 @@
5456
* @author Nurettin Yilmaz
5557
* @author Denis Washington
5658
* @author Gary Russell
59+
* @author Cédric Schaller
5760
*
5861
* @since 1.1.4
5962
*/
@@ -303,10 +306,10 @@ protected synchronized StreamsBuilder createInstance() {
303306
if (this.autoStartup) {
304307
Assert.state(this.properties != null,
305308
"streams configuration properties must not be null");
306-
}
307-
StreamsBuilder builder = new StreamsBuilder();
308-
this.infrastructureCustomizer.configureBuilder(builder);
309-
return builder;
309+
}
310+
StreamsBuilder builder = createStreamBuilder();
311+
this.infrastructureCustomizer.configureBuilder(builder);
312+
return builder;
310313
}
311314

312315
@Override
@@ -385,6 +388,17 @@ public synchronized boolean isRunning() {
385388
return this.running;
386389
}
387390

391+
private StreamsBuilder createStreamBuilder() {
392+
if (this.properties == null) {
393+
return new StreamsBuilder();
394+
}
395+
else {
396+
StreamsConfig streamsConfig = new StreamsConfig(this.properties);
397+
TopologyConfig topologyConfig = new TopologyConfig(streamsConfig);
398+
return new StreamsBuilder(topologyConfig);
399+
}
400+
}
401+
388402
/**
389403
* Called whenever a {@link KafkaStreams} is added or removed.
390404
*
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,97 @@
1+
/*
2+
* Copyright 2024 the original author or authors.
3+
*
4+
* Licensed under the Apache License, Version 2.0 (the "License");
5+
* you may not use this file except in compliance with the License.
6+
* You may obtain a copy of the License at
7+
*
8+
* https://www.apache.org/licenses/LICENSE-2.0
9+
*
10+
* Unless required by applicable law or agreed to in writing, software
11+
* distributed under the License is distributed on an "AS IS" BASIS,
12+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
* See the License for the specific language governing permissions and
14+
* limitations under the License.
15+
*/
16+
17+
package org.springframework.kafka.config;
18+
19+
import static org.assertj.core.api.Assertions.assertThat;
20+
21+
import java.io.IOException;
22+
import java.nio.file.Files;
23+
import java.nio.file.Path;
24+
import java.util.HashMap;
25+
import java.util.Map;
26+
27+
import org.apache.kafka.common.serialization.Serdes;
28+
import org.apache.kafka.streams.StreamsBuilder;
29+
import org.apache.kafka.streams.StreamsConfig;
30+
import org.apache.kafka.streams.kstream.KStream;
31+
import org.apache.kafka.streams.kstream.KTable;
32+
import org.apache.kafka.streams.kstream.Materialized;
33+
import org.junit.jupiter.api.BeforeAll;
34+
import org.junit.jupiter.api.Test;
35+
36+
import org.springframework.beans.factory.annotation.Value;
37+
import org.springframework.context.annotation.Bean;
38+
import org.springframework.context.annotation.Configuration;
39+
import org.springframework.kafka.annotation.EnableKafkaStreams;
40+
import org.springframework.kafka.annotation.KafkaStreamsDefaultConfiguration;
41+
import org.springframework.kafka.test.EmbeddedKafkaBroker;
42+
import org.springframework.kafka.test.context.EmbeddedKafka;
43+
import org.springframework.test.annotation.DirtiesContext;
44+
import org.springframework.test.context.junit.jupiter.SpringJUnitConfig;
45+
46+
/**
47+
* @author Cédric Schaller
48+
*/
49+
@SpringJUnitConfig
50+
@DirtiesContext
51+
@EmbeddedKafka
52+
public class StreamsBuilderFactoryBeanInMemoryStateStoreTests {
53+
54+
private static Path stateStoreDir;
55+
56+
@BeforeAll
57+
static void beforeAll() throws IOException {
58+
stateStoreDir = Files.createTempDirectory(StreamsBuilderFactoryBeanInMemoryStateStoreTests.class.getSimpleName());
59+
}
60+
61+
@Test
62+
void testStateStoreIsInMemory() {
63+
// Testing that an in-memory state store is used requires accessing the internal state of StreamsBuilder via reflection
64+
// Therefore, we check the non-existence of RocksDB files instead
65+
assertThat(stateStoreDir).isEmptyDirectory();
66+
}
67+
68+
@Configuration
69+
@EnableKafkaStreams
70+
static class KafkaStreamsConfig {
71+
72+
@Value("${" + EmbeddedKafkaBroker.SPRING_EMBEDDED_KAFKA_BROKERS + "}")
73+
private String brokerAddresses;
74+
75+
@Bean(name = KafkaStreamsDefaultConfiguration.DEFAULT_STREAMS_CONFIG_BEAN_NAME)
76+
public KafkaStreamsConfiguration kStreamsConfigWithInMemoryStateStores() {
77+
Map<String, Object> props = new HashMap<>();
78+
props.put(StreamsConfig.APPLICATION_ID_CONFIG, "should-be-stored-in-memory");
79+
props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, this.brokerAddresses);
80+
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.StringSerde.class);
81+
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class);
82+
props.put(StreamsConfig.STATE_DIR_CONFIG, stateStoreDir.toString());
83+
84+
// Property introduced with KIP-591 (Kafka 3.2) and deprecated (but still supported) with Kafka 3.7
85+
props.put(StreamsConfig.DEFAULT_DSL_STORE_CONFIG, "in_memory");
86+
return new KafkaStreamsConfiguration(props);
87+
}
88+
89+
@Bean
90+
public KTable<?, ?> table(StreamsBuilder builder) {
91+
KStream<Object, Object> stream = builder.stream("source-topic");
92+
return stream.groupByKey()
93+
.count(Materialized.as("store"));
94+
95+
}
96+
}
97+
}

0 commit comments

Comments
 (0)