-
Notifications
You must be signed in to change notification settings - Fork 1.6k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
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`**
- Loading branch information
1 parent
781665e
commit 54190df
Showing
2 changed files
with
116 additions
and
5 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
97 changes: 97 additions & 0 deletions
97
...va/org/springframework/kafka/config/StreamsBuilderFactoryBeanInMemoryStateStoreTests.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,97 @@ | ||
/* | ||
* Copyright 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. | ||
* You may obtain a copy of the License at | ||
* | ||
* https://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.springframework.kafka.config; | ||
|
||
import static org.assertj.core.api.Assertions.assertThat; | ||
|
||
import java.io.IOException; | ||
import java.nio.file.Files; | ||
import java.nio.file.Path; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
|
||
import org.apache.kafka.common.serialization.Serdes; | ||
import org.apache.kafka.streams.StreamsBuilder; | ||
import org.apache.kafka.streams.StreamsConfig; | ||
import org.apache.kafka.streams.kstream.KStream; | ||
import org.apache.kafka.streams.kstream.KTable; | ||
import org.apache.kafka.streams.kstream.Materialized; | ||
import org.junit.jupiter.api.BeforeAll; | ||
import org.junit.jupiter.api.Test; | ||
|
||
import org.springframework.beans.factory.annotation.Value; | ||
import org.springframework.context.annotation.Bean; | ||
import org.springframework.context.annotation.Configuration; | ||
import org.springframework.kafka.annotation.EnableKafkaStreams; | ||
import org.springframework.kafka.annotation.KafkaStreamsDefaultConfiguration; | ||
import org.springframework.kafka.test.EmbeddedKafkaBroker; | ||
import org.springframework.kafka.test.context.EmbeddedKafka; | ||
import org.springframework.test.annotation.DirtiesContext; | ||
import org.springframework.test.context.junit.jupiter.SpringJUnitConfig; | ||
|
||
/** | ||
* @author Cédric Schaller | ||
*/ | ||
@SpringJUnitConfig | ||
@DirtiesContext | ||
@EmbeddedKafka | ||
public class StreamsBuilderFactoryBeanInMemoryStateStoreTests { | ||
|
||
private static Path stateStoreDir; | ||
|
||
@BeforeAll | ||
static void beforeAll() throws IOException { | ||
stateStoreDir = Files.createTempDirectory(StreamsBuilderFactoryBeanInMemoryStateStoreTests.class.getSimpleName()); | ||
} | ||
|
||
@Test | ||
void testStateStoreIsInMemory() { | ||
// Testing that an in-memory state store is used requires accessing the internal state of StreamsBuilder via reflection | ||
// Therefore, we check the non-existence of RocksDB files instead | ||
assertThat(stateStoreDir).isEmptyDirectory(); | ||
} | ||
|
||
@Configuration | ||
@EnableKafkaStreams | ||
static class KafkaStreamsConfig { | ||
|
||
@Value("${" + EmbeddedKafkaBroker.SPRING_EMBEDDED_KAFKA_BROKERS + "}") | ||
private String brokerAddresses; | ||
|
||
@Bean(name = KafkaStreamsDefaultConfiguration.DEFAULT_STREAMS_CONFIG_BEAN_NAME) | ||
public KafkaStreamsConfiguration kStreamsConfigWithInMemoryStateStores() { | ||
Map<String, Object> props = new HashMap<>(); | ||
props.put(StreamsConfig.APPLICATION_ID_CONFIG, "should-be-stored-in-memory"); | ||
props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, this.brokerAddresses); | ||
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.StringSerde.class); | ||
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class); | ||
props.put(StreamsConfig.STATE_DIR_CONFIG, stateStoreDir.toString()); | ||
|
||
// Property introduced with KIP-591 (Kafka 3.2) and deprecated (but still supported) with Kafka 3.7 | ||
props.put(StreamsConfig.DEFAULT_DSL_STORE_CONFIG, "in_memory"); | ||
return new KafkaStreamsConfiguration(props); | ||
} | ||
|
||
@Bean | ||
public KTable<?, ?> table(StreamsBuilder builder) { | ||
KStream<Object, Object> stream = builder.stream("source-topic"); | ||
return stream.groupByKey() | ||
.count(Materialized.as("store")); | ||
|
||
} | ||
} | ||
} |