Skip to content

Commit

Permalink
GH-2942: Kafka Streams queryable stores
Browse files Browse the repository at this point in the history
Fixes: #2942

This is an initial iteration for providing a basic API around interactive query service in Kafka Streams.
In this iteration, we introduce a single API for retrieving the queryable state store from the Kafka Streams topology, namely, `retrieveQueryableStore`.

* Adding docs
  • Loading branch information
sobychacko authored Feb 12, 2024
1 parent e6affce commit f2e77e6
Show file tree
Hide file tree
Showing 4 changed files with 402 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -315,10 +315,61 @@ public DeadLetterPublishingRecoverer recoverer() {

Of course, the `recoverer()` bean can be your own implementation of `ConsumerRecordRecoverer`.

[[kafka-streams-iq-support]]
== Interactive Query Support

Starting with version 3.2, Spring for Apache Kafka provides basic facilities required for interactive queries in Kafka Streams.
Interactive queries are useful in stateful Kafka Streams applications since they provide a way to constantly query the stateful stores in the application.
Thus, if an application wants to materialize the current view of the system under consideration, interactive queries provide a way to do that.
To learn more about interacive queries, see this https://kafka.apache.org/36/documentation/streams/developer-guide/interactive-queries.html[article].
The support in Spring for Apache Kafka is centered around an API called `KafkaStreamsInteractiveQueryService` which is a facade around interactive queries APIs in Kafka Streams library.
An application can create an instance of this service as a bean and then later on use it to retrieve the state store by its name.

The following code snippet shows an example.

[source, java]
----
@Bean
public KafkaStreamsInteractiveQueryService kafkaStreamsInteractiveQueryService(StreamsBuilderFactoryBean streamsBuilderFactoryBean) {
final KafkaStreamsInteractiveQueryService kafkaStreamsInteractiveQueryService =
new KafkaStreamsInteractiveQueryService(streamsBuilderFactoryBean);
return kafkaStreamsInteractiveQueryService;
}
----

Assuming that a Kafka Streams application has a state store called `app-store`, then that store can be retrieved via the `KafkStreamsInteractiveQuery` API as show below.

[source, java]
----
@Autowired
private KafkaStreamsInteractiveQueryService interactiveQueryService;
ReadOnlyKeyValueStore<Object, Object> appStore = interactiveQueryService.retrieveQueryableStore("app-store", QueryableStoreTypes.keyValueStore());
----

Once an application gains access to the state store, then it can query from it for key-value information.

In this case, the state store that the application uses is a read-only key value store.
There are other types of state stores that a Kafka Streams application can use.
For instance, if an application prefers to query a window based store, it can build that store in the Kafka Streams application business logic and then later on retrieve it.
Because of this reason, the API to retrieve the queryable store in `KafkaStreamsInteractiveQueryService` has a generic store type signature, so that the end-user can assign the proper type.

Here is the type signature from the API.

[source, java]
----
public <T> T retrieveQueryableStore(String storeName, QueryableStoreType<T> storeType)
----

When calling this method, the user can specifially ask for the proper state store type, as we have done in the above example.

NOTE: `KafkaStreamsInteractiveQueryService` API in Spring for Apache Kafka only supports providing access to local key-value stores at the moment.


[[kafka-streams-example]]
== Kafka Streams Example

The following example combines all the topics we have covered in this chapter:
The following example combines the various topics we have covered in this chapter:

[source, java]
----
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,14 @@
This section covers the changes made from version 3.1 to version 3.2.
For changes in earlier version, see xref:appendix/change-history.adoc[Change History].

[[x32-kafka-streams-iqs-support]]
=== Kafka Streams Interactive Query Support

A new API `KafkaStreamsInteractiveQuerySupport` for accessing queryable stores used in Kafka Streams interactive queries.
See xref:streams.adoc#kafka-streams-iq-support[Kafka Streams Interactive Support] for more details.



[[x32-tiss]]
=== TransactionIdSuffixStrategy

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,95 @@
/*
* Copyright 2024-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.streams;

import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.StoreQueryParameters;
import org.apache.kafka.streams.state.QueryableStoreType;

import org.springframework.kafka.config.StreamsBuilderFactoryBean;
import org.springframework.retry.support.RetryTemplate;
import org.springframework.util.Assert;

/**
* Provide a wrapper API around the interactive query stores in Kafka Streams.
* Using this API, an application can gain access to a named state store in the
* {@link KafkaStreams} under consideration.
*
* @author Soby Chacko
* @since 3.2
*/
public class KafkaStreamsInteractiveQueryService {

/**
* {@link StreamsBuilderFactoryBean} that provides {@link KafkaStreams} where the state store is retrieved from.
*/
private final StreamsBuilderFactoryBean streamsBuilderFactoryBean;

/**
* {@link RetryTemplate} to be used by the interative query service.
*/
private RetryTemplate retryTemplate = new RetryTemplate();

/**
* Underlying {@link KafkaStreams} from {@link StreamsBuilderFactoryBean}.
*/
private volatile KafkaStreams kafkaStreams;

/**
* Construct an instance for querying state stores from the KafkaStreams in the {@link StreamsBuilderFactoryBean}.
* @param streamsBuilderFactoryBean {@link StreamsBuilderFactoryBean} for {@link KafkaStreams}.
*/
public KafkaStreamsInteractiveQueryService(StreamsBuilderFactoryBean streamsBuilderFactoryBean) {
Assert.notNull(streamsBuilderFactoryBean, "StreamsBuildFactoryBean instance cannot be null.");
this.streamsBuilderFactoryBean = streamsBuilderFactoryBean;
}

/**
* Custom {@link RetryTemplate} provided by the end users.
* @param retryTemplate {@link RetryTemplate}
*/
public void setRetryTemplate(RetryTemplate retryTemplate) {
Assert.notNull(retryTemplate, "The provided RetryTemplate instance must not be null");
this.retryTemplate = retryTemplate;
}

/**
* Retrieve and return a queryable store by name created in the application.
* @param storeName name of the queryable store
* @param storeType type of the queryable store
* @param <T> generic type for the queryable store
* @return queryable store.
*/
public <T> T retrieveQueryableStore(String storeName, QueryableStoreType<T> storeType) {
if (this.kafkaStreams == null) {
this.kafkaStreams = this.streamsBuilderFactoryBean.getKafkaStreams();
}
Assert.notNull(this.kafkaStreams, "KafkaStreams cannot be null. " +
"Make sure that the corresponding StreamsBuilderFactoryBean has started properly.");
StoreQueryParameters<T> storeQueryParams = StoreQueryParameters.fromNameAndType(storeName, storeType);

return this.retryTemplate.execute(context -> {
try {
return this.kafkaStreams.store(storeQueryParams);
}
catch (Exception e) {
throw new IllegalStateException("Error retrieving state store: " + storeName, e);
}
});
}

}
Loading

0 comments on commit f2e77e6

Please sign in to comment.