Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
27 commits
Select commit Hold shift + click to select a range
23ce8e8
refactor startup state stores initialization
eduwercamacaro Oct 22, 2025
2bfc943
merge trunk
eduwercamacaro Oct 27, 2025
8dc87f2
Remove rocksdb temp files
eduwercamacaro Nov 21, 2025
2b1e77b
Merge branch 'trunk' into refactor-startup-store-initialization
eduwercamacaro Nov 21, 2025
0e9e40b
Fix merge conflicts
eduwercamacaro Nov 21, 2025
13c9e91
Replace RocksDBStore#openDB to StateStore#preInit
eduwercamacaro Nov 21, 2025
050e725
Remove unused classes
eduwercamacaro Nov 21, 2025
a81eaed
Discover startup stores during topology initialization
eduwercamacaro Nov 25, 2025
4ad6553
Allow closing state stores during preInit phase
eduwercamacaro Dec 2, 2025
fd93623
Add a new method that register startup stores in a ProcessorStateManager
eduwercamacaro Dec 15, 2025
233dccf
Merge branch 'trunk' into refactor-startup-store-initialization
eduwercamacaro Dec 15, 2025
abac041
Merge branch 'trunk' into refactor-startup-store-initialization
eduwercamacaro Jan 12, 2026
f8febe7
Set a log prefix for startup stores during initialization
eduwercamacaro Jan 12, 2026
49e786f
Properly assign a StreamThread to a ProcessorStateManager created dur…
eduwercamacaro Jan 16, 2026
b4cb0ff
Implement preInit method for InMemory stores
eduwercamacaro Jan 16, 2026
7b614ea
Revert "Implement preInit method for InMemory stores"
eduwercamacaro Jan 23, 2026
cd0dc0a
wip
eduwercamacaro Feb 1, 2026
6078803
Refactor: Open and close state stores during pre-existing stores init…
eduwercamacaro Feb 2, 2026
97dea13
Refacttor: remove startup stores from state manager
eduwercamacaro Feb 3, 2026
cc8d4f1
Revert changes in StandbyTaskCreator
eduwercamacaro Feb 3, 2026
60d8aa9
rename methods
eduwercamacaro Feb 3, 2026
65ffd3f
Fix unit tests
eduwercamacaro Feb 3, 2026
525b280
Remove unused method
eduwercamacaro Feb 3, 2026
191e9de
Merge branch 'trunk' into refactor-startup-store-initialization
eduwercamacaro Feb 4, 2026
2c0bb94
streams: simplify startup state handling and locking
eduwercamacaro Feb 10, 2026
a8efdea
Merge branch 'trunk' into refactor-startup-store-initialization
eduwercamacaro Feb 10, 2026
a395b3d
- Rename unit tests related to startup states
eduwercamacaro Feb 11, 2026
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
Expand Up @@ -281,6 +281,11 @@ public void shouldPassMetrics(final String topologyType, final String groupProto
streamsApplicationProperties = props(groupProtocol);
final Topology topology = topologyType.equals("simple") ? simpleTopology(false) : complexTopology();

shouldPassMetrics(topology, FIRST_INSTANCE_CLIENT);
shouldPassMetrics(topology, SECOND_INSTANCE_CLIENT);
}

private void shouldPassMetrics(final Topology topology, final int clientInstance) throws Exception {
try (final KafkaStreams streams = new KafkaStreams(topology, streamsApplicationProperties)) {
IntegrationTestUtils.startApplicationAndWaitUntilRunning(streams);

Expand All @@ -292,8 +297,8 @@ public void shouldPassMetrics(final String topologyType, final String groupProto



final List<MetricName> consumerPassedStreamThreadMetricNames = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CLIENT).passedMetrics().stream().map(KafkaMetric::metricName).toList();
final List<MetricName> adminPassedStreamClientMetricNames = INTERCEPTING_ADMIN_CLIENTS.get(FIRST_INSTANCE_CLIENT).passedMetrics.stream().map(KafkaMetric::metricName).toList();
final List<MetricName> consumerPassedStreamThreadMetricNames = INTERCEPTING_CONSUMERS.get(clientInstance).passedMetrics().stream().map(KafkaMetric::metricName).toList();
final List<MetricName> adminPassedStreamClientMetricNames = INTERCEPTING_ADMIN_CLIENTS.get(clientInstance).passedMetrics.stream().map(KafkaMetric::metricName).toList();


assertEquals(streamsThreadMetrics.size(), consumerPassedStreamThreadMetricNames.size());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1376,7 +1376,7 @@ private static HostInfo parseHostInfo(final String endPoint) {
public synchronized void start() throws IllegalStateException, StreamsException {
if (setState(State.REBALANCING)) {
log.debug("Initializing STANDBY tasks for existing local state");
stateDirectory.initializeStartupTasks(topologyMetadata, streamsMetrics, logContext);
stateDirectory.initializeStartupTasks(topologyMetadata, logContext);

log.debug("Starting Streams client");

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,8 @@ public interface StateStore {
*/
void init(final StateStoreContext stateStoreContext, final StateStore root);

default void preInit(final StateStoreContext stateStoreContext) {};

/**
* Flush any cached data
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
import java.util.Map;
import java.util.OptionalLong;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;

import static java.lang.String.format;
Expand Down Expand Up @@ -176,6 +177,7 @@ public String toString() {

// must be maintained in topological order
private final FixedOrderMap<String, StateStoreMetadata> stores = new FixedOrderMap<>();
private final Map<String, StateStore> startupStores = new HashMap<>();
private final FixedOrderMap<String, StateStore> globalStores = new FixedOrderMap<>();

private final File baseDir;
Expand All @@ -185,6 +187,7 @@ public String toString() {
private TaskType taskType;
private Logger log;
private Task.State taskState;
private final AtomicBoolean startupState;

public static String storeChangelogTopic(final String prefix, final String storeName, final String namedTopology) {
if (namedTopology == null) {
Expand All @@ -205,7 +208,8 @@ public ProcessorStateManager(final TaskId taskId,
final ChangelogRegister changelogReader,
final Map<String, String> storeToChangelogTopic,
final Collection<TopicPartition> sourcePartitions,
final boolean stateUpdaterEnabled) throws ProcessorStateException {
final boolean stateUpdaterEnabled,
final boolean startupState) throws ProcessorStateException {
this.storeToChangelogTopic = storeToChangelogTopic;
this.log = logContext.logger(ProcessorStateManager.class);
this.logPrefix = logContext.logPrefix();
Expand All @@ -220,6 +224,22 @@ public ProcessorStateManager(final TaskId taskId,
this.checkpointFile = new OffsetCheckpoint(stateDirectory.checkpointFileFor(taskId));

log.debug("Created state store manager for task {}", taskId);
this.startupState = new AtomicBoolean(startupState);
Copy link
Member

Choose a reason for hiding this comment

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

same would apply to the AtomicBoolean variable

}

/**
* @throws ProcessorStateException if the task directory does not exist and could not be created
*/
public ProcessorStateManager(final TaskId taskId,
final TaskType taskType,
final boolean eosEnabled,
final LogContext logContext,
final StateDirectory stateDirectory,
final ChangelogRegister changelogReader,
final Map<String, String> storeToChangelogTopic,
final Collection<TopicPartition> sourcePartitions,
final boolean stateUpdaterEnabled) throws ProcessorStateException {
this(taskId, taskType, eosEnabled, logContext, stateDirectory, changelogReader, storeToChangelogTopic, sourcePartitions, stateUpdaterEnabled, false);
}

/**
Expand All @@ -234,7 +254,7 @@ static ProcessorStateManager createStartupTaskStateManager(final TaskId taskId,
final Map<String, String> storeToChangelogTopic,
final Set<TopicPartition> sourcePartitions,
final boolean stateUpdaterEnabled) {
return new ProcessorStateManager(taskId, TaskType.STANDBY, eosEnabled, logContext, stateDirectory, null, storeToChangelogTopic, sourcePartitions, stateUpdaterEnabled);
return new ProcessorStateManager(taskId, TaskType.STANDBY, eosEnabled, logContext, stateDirectory, null, storeToChangelogTopic, sourcePartitions, stateUpdaterEnabled, true);
}

/**
Expand All @@ -255,6 +275,10 @@ void assignToStreamThread(final LogContext logContext,
this.sourcePartitions.addAll(sourcePartitions);
}

void reuseState() {
startupState.set(false);
}

void registerStateStores(final List<StateStore> allStores, final InternalProcessorContext<?, ?> processorContext) {
processorContext.uninitialize();
for (final StateStore store : allStores) {
Expand All @@ -263,7 +287,13 @@ void registerStateStores(final List<StateStore> allStores, final InternalProcess
maybeRegisterStoreWithChangelogReader(store.name());
}
} else {
store.init(processorContext, store);
if (startupState.get()) {
store.preInit(processorContext);
startupStores.put(store.name(), store);
} else {
store.init(processorContext, store);
startupStores.remove(store.name());
}
}
log.trace("Registered state store {}", store.name());
}
Expand Down Expand Up @@ -649,9 +679,19 @@ else if (exception instanceof StreamsException)
}
}


stores.clear();
}

if (!startupStores.isEmpty()) {
for (final Map.Entry<String, StateStore> entry : startupStores.entrySet()) {
final StateStore store = entry.getValue();
store.close();
}
startupStores.clear();
}


if (firstException != null) {
throw firstException;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -165,6 +165,35 @@ StandbyTask createStandbyTask(final TaskId taskId,
return task;
}

StandbyTask createStandbyTaskFromStartupLocalStore(final TaskId taskId,
final Set<TopicPartition> inputPartitions,
final ProcessorTopology topology,
final ProcessorStateManager stateManager) {
stateManager.reuseState();
final InternalProcessorContext<Object, Object> context = new ProcessorContextImpl(
taskId,
applicationConfig,
stateManager,
streamsMetrics,
dummyCache
);
final StandbyTask task = new StandbyTask(
taskId,
inputPartitions,
topology,
topologyMetadata.taskConfig(taskId),
streamsMetrics,
stateManager,
stateDirectory,
dummyCache,
context
);

log.trace("Created standby task {} with assigned partitions {}", taskId, inputPartitions);
createTaskSensor.record();
return task;
}

private LogContext getLogContext(final TaskId taskId) {
final String threadIdPrefix = String.format("stream-thread [%s] ", Thread.currentThread().getName());
final String logPrefix = threadIdPrefix + String.format("%s [%s] ", "standby-task", taskId);
Expand Down
Loading