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

Create topology before kafka streams start. #3172

Merged
merged 5 commits into from
Apr 4, 2024
Merged
Show file tree
Hide file tree
Changes from 4 commits
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 2017-2023 the original author or authors.
* Copyright 2017-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 Down Expand Up @@ -33,6 +33,7 @@
import org.apache.kafka.streams.processor.internals.DefaultKafkaClientSupplier;

import org.springframework.beans.factory.BeanNameAware;
import org.springframework.beans.factory.SmartInitializingSingleton;
import org.springframework.beans.factory.config.AbstractFactoryBean;
import org.springframework.context.SmartLifecycle;
import org.springframework.core.log.LogAccessor;
Expand All @@ -56,11 +57,12 @@
* @author Denis Washington
* @author Gary Russell
* @author Julien Wittouck
* @author Sanghyeok An
*
* @since 1.1.4
*/
public class StreamsBuilderFactoryBean extends AbstractFactoryBean<StreamsBuilder>
implements SmartLifecycle, BeanNameAware {
implements SmartLifecycle, BeanNameAware, SmartInitializingSingleton {

/**
* The default {@link Duration} of {@code 10 seconds} for close timeout.
Expand Down Expand Up @@ -356,11 +358,7 @@ public void start() {
try {
Assert.state(this.properties != null,
"streams configuration properties must not be null");
Topology topol = getObject().build(this.properties); // NOSONAR: getObject() cannot return null
this.infrastructureCustomizer.configureTopology(topol);
this.topology = topol;
LOGGER.debug(() -> topol.describe().toString());
this.kafkaStreams = new KafkaStreams(topol, this.properties, this.clientSupplier);
this.kafkaStreams = new KafkaStreams(this.topology, this.properties, this.clientSupplier);
this.kafkaStreams.setStateListener(this.stateListener);
this.kafkaStreams.setGlobalStateRestoreListener(this.stateRestoreListener);
if (this.streamsUncaughtExceptionHandler != null) {
Expand Down Expand Up @@ -432,6 +430,18 @@ public boolean isRunning() {
}
}

@Override
public void afterSingletonsInstantiated() {
try {
this.topology = getObject().build(this.properties);
this.infrastructureCustomizer.configureTopology(this.topology);
LOGGER.debug(() -> this.topology.describe().toString());
}
catch (Exception e) {
throw new RuntimeException(e);
}
}

/**
* Called whenever a {@link KafkaStreams} is added or removed.
*
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright 2018-2020 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 Down Expand Up @@ -56,6 +56,7 @@
* @author Gary Russell
* @author Denis Washington
* @author Soby Chacko
* @author Sanghyeok An
*/
@SpringJUnitConfig
@DirtiesContext
Expand Down Expand Up @@ -102,12 +103,35 @@ protected StreamsBuilder createInstance() {
streamsBuilderFactoryBean.afterPropertiesSet();
StreamsBuilder builder = streamsBuilderFactoryBean.getObject();
builder.stream(Pattern.compile("foo"));
streamsBuilderFactoryBean.afterSingletonsInstantiated();
streamsBuilderFactoryBean.start();
StreamsBuilder streamsBuilder = streamsBuilderFactoryBean.getObject();
verify(streamsBuilder).build(kafkaStreamsConfiguration.asProperties());
assertThat(streamsBuilderFactoryBean.getTopology()).isNotNull();
}

@Test
public void testGetTopologyBeforeKafkaStreamsStart() throws Exception {
// Given
streamsBuilderFactoryBean = new StreamsBuilderFactoryBean(kafkaStreamsConfiguration) {
@Override
protected StreamsBuilder createInstance() {
return spy(super.createInstance());
}
};
streamsBuilderFactoryBean.afterPropertiesSet();
StreamsBuilder builder = streamsBuilderFactoryBean.getObject();
builder.stream(Pattern.compile("foo"));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Try not to use names like foo in the new code that we add.


// When
streamsBuilderFactoryBean.afterSingletonsInstantiated();

// Then
assertThat(streamsBuilderFactoryBean.getTopology()).isNotNull();
assertThat(streamsBuilderFactoryBean.isRunning()).isFalse();
}


@Configuration
@EnableKafkaStreams
public static class KafkaStreamsConfig {
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright 2018-2020 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.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Missing author tag.

Expand All @@ -23,6 +23,7 @@
import java.util.Properties;
import java.util.regex.Pattern;

import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.StreamsConfig;
import org.junit.jupiter.api.Test;

Expand All @@ -43,6 +44,7 @@
* @author Soby Chacko
* @author Artem Bilan
* @author Gary Russell
* @author Sanghyeok An
*/
@SpringJUnitConfig
@DirtiesContext
Expand Down Expand Up @@ -72,11 +74,12 @@ public void testStreamBuilderFactoryCannotBeInstantiatedWhenAutoStart() {

@Test
public void testStreamsBuilderFactoryWithConfigProvidedLater() throws Exception {
boolean isAutoStartUp = true;
Properties props = new Properties();
props.put(StreamsConfig.APPLICATION_ID_CONFIG, APPLICATION_ID);
props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, this.brokerAddresses);
streamsBuilderFactoryBean.setStreamsConfiguration(props);
streamsBuilderFactoryBean.getObject().stream(Pattern.compile("foo"));
streamsBuilderFactoryBean.setAutoStartup(isAutoStartUp);

assertThat(streamsBuilderFactoryBean.isRunning()).isFalse();
streamsBuilderFactoryBean.start();
Expand All @@ -95,6 +98,23 @@ public StreamsBuilderFactoryBean defaultKafkaStreamsBuilder() {
return streamsBuilderFactoryBean;
}

@Bean
public KafkaStreamsService kafkaStreamsService(StreamsBuilder streamsBuilder) {
return new KafkaStreamsService(streamsBuilder);
}

}

static class KafkaStreamsService {
private final StreamsBuilder streamsBuilder;

public KafkaStreamsService(StreamsBuilder streamsBuilder) {
this.streamsBuilder = streamsBuilder;
buildPipeline();
}

public void buildPipeline() {
this.streamsBuilder.stream("foo");
}
}
}
Loading