This is the multi-page printable view of this section. Click here to print.
Kafka Streams
- 1: Introduction
- 2: Quick Start
- 3: Write a streams app
- 4: Core Concepts
- 5: Architecture
- 6: Upgrade Guide
- 7: Streams Developer Guide
- 7.1: Writing a Streams Application
- 7.2: Configuring a Streams Application
- 7.3: Streams DSL
- 7.4: Processor API
- 7.5: Naming Operators in a Streams DSL application
- 7.6: Data Types and Serialization
- 7.7: Testing a Streams Application
- 7.8: Interactive Queries
- 7.9: Memory Management
- 7.10: Running Streams Applications
- 7.11: Managing Streams Application Topics
- 7.12: Streams Security
- 7.13: Application Reset Tool
1 - Introduction
Kafka Streams
Introduction Run Demo App Tutorial: Write App Concepts Architecture Developer Guide Javadoc Upgrade
The easiest way to write mission-critical real-time applications and microservices
Kafka Streams is a client library for building applications and microservices, where the input and output data are stored in Kafka clusters. It combines the simplicity of writing and deploying standard Java and Scala applications on the client side with the benefits of Kafka’s server-side cluster technology.
(Clicking the image will load a video from YouTube)
(Clicking the image will load a video from YouTube)
(Clicking the image will load a video from YouTube)
(Clicking the image will load a video from YouTube)
TOUR OF THE STREAMS API
1Intro to Streams
2Creating a Streams Application
3Transforming Data Pt. 1
4Transforming Data Pt. 2
Why you’ll love using Kafka Streams!
- Elastic, highly scalable, fault-tolerant
- Deploy to containers, VMs, bare metal, cloud
- Equally viable for small, medium, & large use cases
- Fully integrated with Kafka security
- Write standard Java and Scala applications
- Exactly-once processing semantics
- No separate processing cluster required
- Develop on Mac, Linux, Windows
Kafka Streams use cases
The New York Times uses Apache Kafka and the Kafka Streams to store and distribute, in real-time, published content to the various applications and systems that make it available to the readers.
As the leading online fashion retailer in Europe, Zalando uses Kafka as an ESB (Enterprise Service Bus), which helps us in transitioning from a monolithic to a micro services architecture. Using Kafka for processing event streams enables our technical team to do near-real time business intelligence.
LINE uses Apache Kafka as a central datahub for our services to communicate to one another. Hundreds of billions of messages are produced daily and are used to execute various business logic, threat detection, search indexing and data analysis. LINE leverages Kafka Streams to reliably transform and filter topics enabling sub topics consumers can efficiently consume, meanwhile retaining easy maintainability thanks to its sophisticated yet minimal code base.
Pinterest uses Apache Kafka and the Kafka Streams at large scale to power the real-time, predictive budgeting system of their advertising infrastructure. With Kafka Streams, spend predictions are more accurate than ever.
Rabobank is one of the 3 largest banks in the Netherlands. Its digital nervous system, the Business Event Bus, is powered by Apache Kafka. It is used by an increasing amount of financial processes and services, one of which is Rabo Alerts. This service alerts customers in real-time upon financial events and is built using Kafka Streams.
Trivago is a global hotel search platform. We are focused on reshaping the way travelers search for and compare hotels, while enabling hotel advertisers to grow their businesses by providing access to a broad audience of travelers via our websites and apps. As of 2017, we offer access to approximately 1.8 million hotels and other accommodations in over 190 countries. We use Kafka, Kafka Connect, and Kafka Streams to enable our developers to access data freely in the company. Kafka Streams powers parts of our analytics pipeline and delivers endless options to explore and operate on the data sources we have at hand.
Hello Kafka Streams
The code example below implements a WordCount application that is elastic, highly scalable, fault-tolerant, stateful, and ready to run in production at large scale
Java Scala
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.streams.KafkaStreams;
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.apache.kafka.streams.kstream.Produced;
import org.apache.kafka.streams.state.KeyValueStore;
import java.util.Arrays;
import java.util.Properties;
public class WordCountApplication {
public static void main(final String[] args) throws Exception {
Properties props = new Properties();
props.put(StreamsConfig.APPLICATION_ID_CONFIG, "wordcount-application");
props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092");
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
StreamsBuilder builder = new StreamsBuilder();
KStream<String, String> textLines = builder.stream("TextLinesTopic");
KTable<String, Long> wordCounts = textLines
.flatMapValues(textLine -> Arrays.asList(textLine.toLowerCase().split("\W+")))
.groupBy((key, word) -> word)
.count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("counts-store"));
wordCounts.toStream().to("WordsWithCountsTopic", Produced.with(Serdes.String(), Serdes.Long()));
KafkaStreams streams = new KafkaStreams(builder.build(), props);
streams.start();
}
}
import java.util.Properties
import java.util.concurrent.TimeUnit
import org.apache.kafka.streams.kstream.Materialized
import org.apache.kafka.streams.scala.ImplicitConversions._
import org.apache.kafka.streams.scala._
import org.apache.kafka.streams.scala.kstream._
import org.apache.kafka.streams.{KafkaStreams, StreamsConfig}
object WordCountApplication extends App {
import Serdes._
val props: Properties = {
val p = new Properties()
p.put(StreamsConfig.APPLICATION_ID_CONFIG, "wordcount-application")
p.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092")
p
}
val builder: StreamsBuilder = new StreamsBuilder
val textLines: KStream[String, String] = builder.stream[String, String]("TextLinesTopic")
val wordCounts: KTable[String, Long] = textLines
.flatMapValues(textLine => textLine.toLowerCase.split("\W+"))
.groupBy((_, word) => word)
.count()(Materialized.as("counts-store"))
wordCounts.toStream.to("WordsWithCountsTopic")
val streams: KafkaStreams = new KafkaStreams(builder.build(), props)
streams.start()
sys.ShutdownHookThread {
streams.close(10, TimeUnit.SECONDS)
}
}
2 - Quick Start
Run Kafka Streams Demo Application
Introduction Run Demo App Tutorial: Write App Concepts Architecture Developer Guide Upgrade
This tutorial assumes you are starting fresh and have no existing Kafka data. However, if you have already started Kafka, feel free to skip the first two steps.
Kafka Streams is a client library for building mission-critical real-time applications and microservices, where the input and/or output data is stored in Kafka clusters. Kafka Streams combines the simplicity of writing and deploying standard Java and Scala applications on the client side with the benefits of Kafka’s server-side cluster technology to make these applications highly scalable, elastic, fault-tolerant, distributed, and much more.
This quickstart example will demonstrate how to run a streaming application coded in this library. Here is the gist of the [WordCountDemo](https://github.com/apache/kafka/blob/4.0/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java)
example code.
// Serializers/deserializers (serde) for String and Long types
final Serde<String> stringSerde = Serdes.String();
final Serde<Long> longSerde = Serdes.Long();
// Construct a `KStream` from the input topic "streams-plaintext-input", where message values
// represent lines of text (for the sake of this example, we ignore whatever may be stored
// in the message keys).
KStream<String, String> textLines = builder.stream(
"streams-plaintext-input",
Consumed.with(stringSerde, stringSerde)
);
KTable<String, Long> wordCounts = textLines
// Split each text line, by whitespace, into words.
.flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\W+")))
// Group the text words as message keys
.groupBy((key, value) -> value)
// Count the occurrences of each word (message key).
.count();
// Store the running counts as a changelog stream to the output topic.
wordCounts.toStream().to("streams-wordcount-output", Produced.with(Serdes.String(), Serdes.Long()));
It implements the WordCount algorithm, which computes a word occurrence histogram from the input text. However, unlike other WordCount examples you might have seen before that operate on bounded data, the WordCount demo application behaves slightly differently because it is designed to operate on an infinite, unbounded stream of data. Similar to the bounded variant, it is a stateful algorithm that tracks and updates the counts of words. However, since it must assume potentially unbounded input data, it will periodically output its current state and results while continuing to process more data because it cannot know when it has processed “all” the input data.
As the first step, we will start Kafka (unless you already have it started) and then we will prepare input data to a Kafka topic, which will subsequently be processed by a Kafka Streams application.
Step 1: Download the code
Download the 4.0.0 release and un-tar it. Note that there are multiple downloadable Scala versions and we choose to use the recommended version (2.13) here:
$ tar -xzf kafka_2.13-4.0.0.tgz
$ cd kafka_2.13-4.0.0
Step 2: Start the Kafka server
Generate a Cluster UUID
$ KAFKA_CLUSTER_ID="$(bin/kafka-storage.sh random-uuid)"
Format Log Directories
$ bin/kafka-storage.sh format --standalone -t $KAFKA_CLUSTER_ID -c config/server.properties
Start the Kafka Server
$ bin/kafka-server-start.sh config/server.properties
Step 3: Prepare input topic and start Kafka producer
Next, we create the input topic named streams-plaintext-input and the output topic named streams-wordcount-output :
$ bin/kafka-topics.sh --create \
--bootstrap-server localhost:9092 \
--replication-factor 1 \
--partitions 1 \
--topic streams-plaintext-input
Created topic "streams-plaintext-input".
Note: we create the output topic with compaction enabled because the output stream is a changelog stream (cf. explanation of application output below).
$ bin/kafka-topics.sh --create \
--bootstrap-server localhost:9092 \
--replication-factor 1 \
--partitions 1 \
--topic streams-wordcount-output \
--config cleanup.policy=compact
Created topic "streams-wordcount-output".
The created topic can be described with the same kafka-topics tool:
$ bin/kafka-topics.sh --bootstrap-server localhost:9092 --describe
Topic:streams-wordcount-output PartitionCount:1 ReplicationFactor:1 Configs:cleanup.policy=compact,segment.bytes=1073741824
Topic: streams-wordcount-output Partition: 0 Leader: 0 Replicas: 0 Isr: 0
Topic:streams-plaintext-input PartitionCount:1 ReplicationFactor:1 Configs:segment.bytes=1073741824
Topic: streams-plaintext-input Partition: 0 Leader: 0 Replicas: 0 Isr: 0
Step 4: Start the Wordcount Application
The following command starts the WordCount demo application:
$ bin/kafka-run-class.sh org.apache.kafka.streams.examples.wordcount.WordCountDemo
The demo application will read from the input topic streams-plaintext-input , perform the computations of the WordCount algorithm on each of the read messages, and continuously write its current results to the output topic streams-wordcount-output. Hence there won’t be any STDOUT output except log entries as the results are written back into in Kafka.
Now we can start the console producer in a separate terminal to write some input data to this topic:
$ bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic streams-plaintext-input
and inspect the output of the WordCount demo application by reading from its output topic with the console consumer in a separate terminal:
$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
--topic streams-wordcount-output \
--from-beginning \
--property print.key=true \
--property print.value=true \
--property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
--property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
Step 5: Process some data
Now let’s write some message with the console producer into the input topic streams-plaintext-input by entering a single line of text and then hit
$ bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic streams-plaintext-input
>all streams lead to kafka
This message will be processed by the Wordcount application and the following output data will be written to the streams-wordcount-output topic and printed by the console consumer:
$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
--topic streams-wordcount-output \
--from-beginning \
--property print.key=true \
--property print.value=true \
--property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
--property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
all 1
streams 1
lead 1
to 1
kafka 1
Here, the first column is the Kafka message key in java.lang.String
format and represents a word that is being counted, and the second column is the message value in java.lang.Long
format, representing the word’s latest count.
Now let’s continue writing one more message with the console producer into the input topic streams-plaintext-input. Enter the text line “hello kafka streams” and hit
$ bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic streams-plaintext-input
>all streams lead to kafka
>hello kafka streams
In your other terminal in which the console consumer is running, you will observe that the WordCount application wrote new output data:
$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
--topic streams-wordcount-output \
--from-beginning \
--property print.key=true \
--property print.value=true \
--property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
--property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
all 1
streams 1
lead 1
to 1
kafka 1
hello 1
kafka 2
streams 2
Here the last printed lines kafka 2 and streams 2 indicate updates to the keys kafka and streams whose counts have been incremented from 1 to 2. Whenever you write further input messages to the input topic, you will observe new messages being added to the streams-wordcount-output topic, representing the most recent word counts as computed by the WordCount application. Let’s enter one final input text line “join kafka summit” and hit
$ bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic streams-plaintext-input
>all streams lead to kafka
>hello kafka streams
>join kafka summit
The streams-wordcount-output topic will subsequently show the corresponding updated word counts (see last three lines):
$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
--topic streams-wordcount-output \
--from-beginning \
--property print.key=true \
--property print.value=true \
--property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
--property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
all 1
streams 1
lead 1
to 1
kafka 1
hello 1
kafka 2
streams 2
join 1
kafka 3
summit 1
As one can see, outputs of the Wordcount application is actually a continuous stream of updates, where each output record (i.e. each line in the original output above) is an updated count of a single word, aka record key such as “kafka”. For multiple records with the same key, each later record is an update of the previous one.
The two diagrams below illustrate what is essentially happening behind the scenes. The first column shows the evolution of the current state of the KTable<String, Long>
that is counting word occurrences for count
. The second column shows the change records that result from state updates to the KTable and that are being sent to the output Kafka topic streams-wordcount-output.
First the text line “all streams lead to kafka” is being processed. The KTable
is being built up as each new word results in a new table entry (highlighted with a green background), and a corresponding change record is sent to the downstream KStream
.
When the second text line “hello kafka streams” is processed, we observe, for the first time, that existing entries in the KTable
are being updated (here: for the words “kafka” and for “streams”). And again, change records are being sent to the output topic.
And so on (we skip the illustration of how the third line is being processed). This explains why the output topic has the contents we showed above, because it contains the full record of changes.
Looking beyond the scope of this concrete example, what Kafka Streams is doing here is to leverage the duality between a table and a changelog stream (here: table = the KTable, changelog stream = the downstream KStream): you can publish every change of the table to a stream, and if you consume the entire changelog stream from beginning to end, you can reconstruct the contents of the table.
Step 6: Teardown the application
You can now stop the console consumer, the console producer, the Wordcount application, the Kafka broker in order via Ctrl-C.
3 - Write a streams app
Tutorial: Write a Kafka Streams Application
Introduction Run Demo App Tutorial: Write App Concepts Architecture Developer Guide Upgrade
In this guide we will start from scratch on setting up your own project to write a stream processing application using Kafka Streams. It is highly recommended to read the quickstart first on how to run a Streams application written in Kafka Streams if you have not done so.
Setting up a Maven Project
We are going to use a Kafka Streams Maven Archetype for creating a Streams project structure with the following commands:
$ mvn archetype:generate \
-DarchetypeGroupId=org.apache.kafka \
-DarchetypeArtifactId=streams-quickstart-java \
-DarchetypeVersion=4.0.0 \
-DgroupId=streams.examples \
-DartifactId=streams-quickstart \
-Dversion=0.1 \
-Dpackage=myapps
You can use a different value for groupId
, artifactId
and package
parameters if you like. Assuming the above parameter values are used, this command will create a project structure that looks like this:
$ tree streams-quickstart
streams-quickstart
|-- pom.xml
|-- src
|-- main
|-- java
| |-- myapps
| |-- LineSplit.java
| |-- Pipe.java
| |-- WordCount.java
|-- resources
|-- log4j.properties
The pom.xml
file included in the project already has the Streams dependency defined. Note, that the generated pom.xml
targets Java 11.
There are already several example programs written with Streams library under src/main/java
. Since we are going to start writing such programs from scratch, we can now delete these examples:
$ cd streams-quickstart
$ rm src/main/java/myapps/*.java
Writing a first Streams application: Pipe
It’s coding time now! Feel free to open your favorite IDE and import this Maven project, or simply open a text editor and create a java file under src/main/java/myapps
. Let’s name it Pipe.java
:
package myapps;
public class Pipe {
public static void main(String[] args) throws Exception {
}
}
We are going to fill in the main
function to write this pipe program. Note that we will not list the import statements as we go since IDEs can usually add them automatically. However if you are using a text editor you need to manually add the imports, and at the end of this section we’ll show the complete code snippet with import statement for you.
The first step to write a Streams application is to create a java.util.Properties
map to specify different Streams execution configuration values as defined in StreamsConfig
. A couple of important configuration values you need to set are: StreamsConfig.BOOTSTRAP_SERVERS_CONFIG
, which specifies a list of host/port pairs to use for establishing the initial connection to the Kafka cluster, and StreamsConfig.APPLICATION_ID_CONFIG
, which gives the unique identifier of your Streams application to distinguish itself with other applications talking to the same Kafka cluster:
Properties props = new Properties();
props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pipe");
props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); // assuming that the Kafka broker this application is talking to runs on local machine with port 9092
In addition, you can customize other configurations in the same map, for example, default serialization and deserialization libraries for the record key-value pairs:
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
For a full list of configurations of Kafka Streams please refer to this table.
Next we will define the computational logic of our Streams application. In Kafka Streams this computational logic is defined as a topology
of connected processor nodes. We can use a topology builder to construct such a topology,
final StreamsBuilder builder = new StreamsBuilder();
And then create a source stream from a Kafka topic named streams-plaintext-input
using this topology builder:
KStream<String, String> source = builder.stream("streams-plaintext-input");
Now we get a KStream
that is continuously generating records from its source Kafka topic streams-plaintext-input
. The records are organized as String
typed key-value pairs. The simplest thing we can do with this stream is to write it into another Kafka topic, say it’s named streams-pipe-output
:
source.to("streams-pipe-output");
Note that we can also concatenate the above two lines into a single line as:
builder.stream("streams-plaintext-input").to("streams-pipe-output");
We can inspect what kind of topology
is created from this builder by doing the following:
final Topology topology = builder.build();
And print its description to standard output as:
System.out.println(topology.describe());
If we just stop here, compile and run the program, it will output the following information:
$ mvn clean package
$ mvn exec:java -Dexec.mainClass=myapps.Pipe
Sub-topologies:
Sub-topology: 0
Source: KSTREAM-SOURCE-0000000000(topics: streams-plaintext-input) --> KSTREAM-SINK-0000000001
Sink: KSTREAM-SINK-0000000001(topic: streams-pipe-output) <-- KSTREAM-SOURCE-0000000000
Global Stores:
none
As shown above, it illustrates that the constructed topology has two processor nodes, a source node KSTREAM-SOURCE-0000000000
and a sink node KSTREAM-SINK-0000000001
. KSTREAM-SOURCE-0000000000
continuously read records from Kafka topic streams-plaintext-input
and pipe them to its downstream node KSTREAM-SINK-0000000001
; KSTREAM-SINK-0000000001
will write each of its received record in order to another Kafka topic streams-pipe-output
(the -->
and <--
arrows dictates the downstream and upstream processor nodes of this node, i.e. “children” and “parents” within the topology graph). It also illustrates that this simple topology has no global state stores associated with it (we will talk about state stores more in the following sections).
Note that we can always describe the topology as we did above at any given point while we are building it in the code, so as a user you can interactively “try and taste” your computational logic defined in the topology until you are happy with it. Suppose we are already done with this simple topology that just pipes data from one Kafka topic to another in an endless streaming manner, we can now construct the Streams client with the two components we have just constructed above: the configuration map specified in a java.util.Properties
instance and the Topology
object.
final KafkaStreams streams = new KafkaStreams(topology, props);
By calling its start()
function we can trigger the execution of this client. The execution won’t stop until close()
is called on this client. We can, for example, add a shutdown hook with a countdown latch to capture a user interrupt and close the client upon terminating this program:
final CountDownLatch latch = new CountDownLatch(1);
// attach shutdown handler to catch control-c
Runtime.getRuntime().addShutdownHook(new Thread("streams-shutdown-hook") {
@Override
public void run() {
streams.close();
latch.countDown();
}
});
try {
streams.start();
latch.await();
} catch (Throwable e) {
System.exit(1);
}
System.exit(0);
The complete code so far looks like this:
package myapps;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.Topology;
import java.util.Properties;
import java.util.concurrent.CountDownLatch;
public class Pipe {
public static void main(String[] args) throws Exception {
Properties props = new Properties();
props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pipe");
props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
final StreamsBuilder builder = new StreamsBuilder();
builder.stream("streams-plaintext-input").to("streams-pipe-output");
final Topology topology = builder.build();
final KafkaStreams streams = new KafkaStreams(topology, props);
final CountDownLatch latch = new CountDownLatch(1);
// attach shutdown handler to catch control-c
Runtime.getRuntime().addShutdownHook(new Thread("streams-shutdown-hook") {
@Override
public void run() {
streams.close();
latch.countDown();
}
});
try {
streams.start();
latch.await();
} catch (Throwable e) {
System.exit(1);
}
System.exit(0);
}
}
If you already have the Kafka broker up and running at localhost:9092
, and the topics streams-plaintext-input
and streams-pipe-output
created on that broker, you can run this code in your IDE or on the command line, using Maven:
$ mvn clean package
$ mvn exec:java -Dexec.mainClass=myapps.Pipe
For detailed instructions on how to run a Streams application and observe its computing results, please read the Play with a Streams Application section. We will not talk about this in the rest of this section.
Writing a second Streams application: Line Split
We have learned how to construct a Streams client with its two key components: the StreamsConfig
and Topology
. Now let’s move on to add some real processing logic by augmenting the current topology. We can first create another program by first copy the existing Pipe.java
class:
$ cp src/main/java/myapps/Pipe.java src/main/java/myapps/LineSplit.java
And change its class name as well as the application id config to distinguish with the original program:
public class LineSplit {
public static void main(String[] args) throws Exception {
Properties props = new Properties();
props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-linesplit");
// ...
}
}
Since each of the source stream’s record is a String
typed key-value pair, let’s treat the value string as a text line and split it into words with a FlatMapValues
operator:
KStream<String, String> source = builder.stream("streams-plaintext-input");
KStream<String, String> words = source.flatMapValues(new ValueMapper<String, Iterable<String>>() {
@Override
public Iterable<String> apply(String value) {
return Arrays.asList(value.split("\W+"));
}
});
The operator will take the source
stream as its input, and generate a new stream named words
by processing each record from its source stream in order and breaking its value string into a list of words, and producing each word as a new record to the output words
stream. This is a stateless operator that does not need to keep track of any previously received records or processed results. Note if you are using JDK 8 you can use lambda expression and simplify the above code as:
KStream<String, String> source = builder.stream("streams-plaintext-input");
KStream<String, String> words = source.flatMapValues(value -> Arrays.asList(value.split("\W+")));
And finally we can write the word stream back into another Kafka topic, say streams-linesplit-output
. Again, these two steps can be concatenated as the following (assuming lambda expression is used):
KStream<String, String> source = builder.stream("streams-plaintext-input");
source.flatMapValues(value -> Arrays.asList(value.split("\W+")))
.to("streams-linesplit-output");
If we now describe this augmented topology as System.out.println(topology.describe())
, we will get the following:
$ mvn clean package
$ mvn exec:java -Dexec.mainClass=myapps.LineSplit
Sub-topologies:
Sub-topology: 0
Source: KSTREAM-SOURCE-0000000000(topics: streams-plaintext-input) --> KSTREAM-FLATMAPVALUES-0000000001
Processor: KSTREAM-FLATMAPVALUES-0000000001(stores: []) --> KSTREAM-SINK-0000000002 <-- KSTREAM-SOURCE-0000000000
Sink: KSTREAM-SINK-0000000002(topic: streams-linesplit-output) <-- KSTREAM-FLATMAPVALUES-0000000001
Global Stores:
none
As we can see above, a new processor node KSTREAM-FLATMAPVALUES-0000000001
is injected into the topology between the original source and sink nodes. It takes the source node as its parent and the sink node as its child. In other words, each record fetched by the source node will first traverse to the newly added KSTREAM-FLATMAPVALUES-0000000001
node to be processed, and one or more new records will be generated as a result. They will continue traverse down to the sink node to be written back to Kafka. Note this processor node is “stateless” as it is not associated with any stores (i.e. (stores: [])
).
The complete code looks like this (assuming lambda expression is used):
package myapps;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.Topology;
import org.apache.kafka.streams.kstream.KStream;
import java.util.Arrays;
import java.util.Properties;
import java.util.concurrent.CountDownLatch;
public class LineSplit {
public static void main(String[] args) throws Exception {
Properties props = new Properties();
props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-linesplit");
props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
final StreamsBuilder builder = new StreamsBuilder();
KStream<String, String> source = builder.stream("streams-plaintext-input");
source.flatMapValues(value -> Arrays.asList(value.split("\W+")))
.to("streams-linesplit-output");
final Topology topology = builder.build();
final KafkaStreams streams = new KafkaStreams(topology, props);
final CountDownLatch latch = new CountDownLatch(1);
// ... same as Pipe.java above
}
}
Writing a third Streams application: Wordcount
Let’s now take a step further to add some “stateful” computations to the topology by counting the occurrence of the words split from the source text stream. Following similar steps let’s create another program based on the LineSplit.java
class:
public class WordCount {
public static void main(String[] args) throws Exception {
Properties props = new Properties();
props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount");
// ...
}
}
In order to count the words we can first modify the flatMapValues
operator to treat all of them as lower case (assuming lambda expression is used):
source.flatMapValues(new ValueMapper<String, Iterable<String>>() {
@Override
public Iterable<String> apply(String value) {
return Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\W+"));
}
});
In order to do the counting aggregation we have to first specify that we want to key the stream on the value string, i.e. the lower cased word, with a groupBy
operator. This operator generate a new grouped stream, which can then be aggregated by a count
operator, which generates a running count on each of the grouped keys:
KTable<String, Long> counts =
source.flatMapValues(new ValueMapper<String, Iterable<String>>() {
@Override
public Iterable<String> apply(String value) {
return Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\W+"));
}
})
.groupBy(new KeyValueMapper<String, String, String>() {
@Override
public String apply(String key, String value) {
return value;
}
})
// Materialize the result into a KeyValueStore named "counts-store".
// The Materialized store is always of type <Bytes, byte[]> as this is the format of the inner most store.
.count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>> as("counts-store"));
Note that the count
operator has a Materialized
parameter that specifies that the running count should be stored in a state store named counts-store
. This counts-store
store can be queried in real-time, with details described in the Developer Manual.
We can also write the counts
KTable’s changelog stream back into another Kafka topic, say streams-wordcount-output
. Because the result is a changelog stream, the output topic streams-wordcount-output
should be configured with log compaction enabled. Note that this time the value type is no longer String
but Long
, so the default serialization classes are not viable for writing it to Kafka anymore. We need to provide overridden serialization methods for Long
types, otherwise a runtime exception will be thrown:
counts.toStream().to("streams-wordcount-output", Produced.with(Serdes.String(), Serdes.Long()));
Note that in order to read the changelog stream from topic streams-wordcount-output
, one needs to set the value deserialization as org.apache.kafka.common.serialization.LongDeserializer
. Details of this can be found in the Play with a Streams Application section. Assuming lambda expression from JDK 8 can be used, the above code can be simplified as:
KStream<String, String> source = builder.stream("streams-plaintext-input");
source.flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\W+")))
.groupBy((key, value) -> value)
.count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("counts-store"))
.toStream()
.to("streams-wordcount-output", Produced.with(Serdes.String(), Serdes.Long()));
If we again describe this augmented topology as System.out.println(topology.describe())
, we will get the following:
$ mvn clean package
$ mvn exec:java -Dexec.mainClass=myapps.WordCount
Sub-topologies:
Sub-topology: 0
Source: KSTREAM-SOURCE-0000000000(topics: streams-plaintext-input) --> KSTREAM-FLATMAPVALUES-0000000001
Processor: KSTREAM-FLATMAPVALUES-0000000001(stores: []) --> KSTREAM-KEY-SELECT-0000000002 <-- KSTREAM-SOURCE-0000000000
Processor: KSTREAM-KEY-SELECT-0000000002(stores: []) --> KSTREAM-FILTER-0000000005 <-- KSTREAM-FLATMAPVALUES-0000000001
Processor: KSTREAM-FILTER-0000000005(stores: []) --> KSTREAM-SINK-0000000004 <-- KSTREAM-KEY-SELECT-0000000002
Sink: KSTREAM-SINK-0000000004(topic: counts-store-repartition) <-- KSTREAM-FILTER-0000000005
Sub-topology: 1
Source: KSTREAM-SOURCE-0000000006(topics: counts-store-repartition) --> KSTREAM-AGGREGATE-0000000003
Processor: KSTREAM-AGGREGATE-0000000003(stores: [counts-store]) --> KTABLE-TOSTREAM-0000000007 <-- KSTREAM-SOURCE-0000000006
Processor: KTABLE-TOSTREAM-0000000007(stores: []) --> KSTREAM-SINK-0000000008 <-- KSTREAM-AGGREGATE-0000000003
Sink: KSTREAM-SINK-0000000008(topic: streams-wordcount-output) <-- KTABLE-TOSTREAM-0000000007
Global Stores:
none
As we can see above, the topology now contains two disconnected sub-topologies. The first sub-topology’s sink node KSTREAM-SINK-0000000004
will write to a repartition topic counts-store-repartition
, which will be read by the second sub-topology’s source node KSTREAM-SOURCE-0000000006
. The repartition topic is used to “shuffle” the source stream by its aggregation key, which is in this case the value string. In addition, inside the first sub-topology a stateless KSTREAM-FILTER-0000000005
node is injected between the grouping KSTREAM-KEY-SELECT-0000000002
node and the sink node to filter out any intermediate record whose aggregate key is empty.
In the second sub-topology, the aggregation node KSTREAM-AGGREGATE-0000000003
is associated with a state store named counts-store
(the name is specified by the user in the count
operator). Upon receiving each record from its upcoming stream source node, the aggregation processor will first query its associated counts-store
store to get the current count for that key, augment by one, and then write the new count back to the store. Each updated count for the key will also be piped downstream to the KTABLE-TOSTREAM-0000000007
node, which interpret this update stream as a record stream before further piping to the sink node KSTREAM-SINK-0000000008
for writing back to Kafka.
The complete code looks like this (assuming lambda expression is used):
package myapps;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.Topology;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.Materialized;
import org.apache.kafka.streams.kstream.Produced;
import org.apache.kafka.streams.state.KeyValueStore;
import java.util.Arrays;
import java.util.Locale;
import java.util.Properties;
import java.util.concurrent.CountDownLatch;
public class WordCount {
public static void main(String[] args) throws Exception {
Properties props = new Properties();
props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount");
props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
final StreamsBuilder builder = new StreamsBuilder();
KStream<String, String> source = builder.stream("streams-plaintext-input");
source.flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\W+")))
.groupBy((key, value) -> value)
.count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("counts-store"))
.toStream()
.to("streams-wordcount-output", Produced.with(Serdes.String(), Serdes.Long()));
final Topology topology = builder.build();
final KafkaStreams streams = new KafkaStreams(topology, props);
final CountDownLatch latch = new CountDownLatch(1);
// ... same as Pipe.java above
}
}
4 - Core Concepts
Core Concepts
Introduction Run Demo App Tutorial: Write App Concepts Architecture Developer Guide Upgrade
Kafka Streams is a client library for processing and analyzing data stored in Kafka. It builds upon important stream processing concepts such as properly distinguishing between event time and processing time, windowing support, and simple yet efficient management and real-time querying of application state.
Kafka Streams has a low barrier to entry : You can quickly write and run a small-scale proof-of-concept on a single machine; and you only need to run additional instances of your application on multiple machines to scale up to high-volume production workloads. Kafka Streams transparently handles the load balancing of multiple instances of the same application by leveraging Kafka’s parallelism model.
Some highlights of Kafka Streams:
- Designed as a simple and lightweight client library , which can be easily embedded in any Java application and integrated with any existing packaging, deployment and operational tools that users have for their streaming applications.
- Has no external dependencies on systems other than Apache Kafka itself as the internal messaging layer; notably, it uses Kafka’s partitioning model to horizontally scale processing while maintaining strong ordering guarantees.
- Supports fault-tolerant local state , which enables very fast and efficient stateful operations like windowed joins and aggregations.
- Supports exactly-once processing semantics to guarantee that each record will be processed once and only once even when there is a failure on either Streams clients or Kafka brokers in the middle of processing.
- Employs one-record-at-a-time processing to achieve millisecond processing latency, and supports event-time based windowing operations with out-of-order arrival of records.
- Offers necessary stream processing primitives, along with a high-level Streams DSL and a low-level Processor API.
We first summarize the key concepts of Kafka Streams.
Stream Processing Topology
- A stream is the most important abstraction provided by Kafka Streams: it represents an unbounded, continuously updating data set. A stream is an ordered, replayable, and fault-tolerant sequence of immutable data records, where a data record is defined as a key-value pair.
- A stream processing application is any program that makes use of the Kafka Streams library. It defines its computational logic through one or more processor topologies , where a processor topology is a graph of stream processors (nodes) that are connected by streams (edges).
- A stream processor is a node in the processor topology; it represents a processing step to transform data in streams by receiving one input record at a time from its upstream processors in the topology, applying its operation to it, and may subsequently produce one or more output records to its downstream processors.
There are two special processors in the topology:
- Source Processor : A source processor is a special type of stream processor that does not have any upstream processors. It produces an input stream to its topology from one or multiple Kafka topics by consuming records from these topics and forwarding them to its down-stream processors.
- Sink Processor : A sink processor is a special type of stream processor that does not have down-stream processors. It sends any received records from its up-stream processors to a specified Kafka topic.
Note that in normal processor nodes other remote systems can also be accessed while processing the current record. Therefore the processed results can either be streamed back into Kafka or written to an external system.
Kafka Streams offers two ways to define the stream processing topology: the Kafka Streams DSL provides the most common data transformation operations such as map
, filter
, join
and aggregations
out of the box; the lower-level Processor API allows developers define and connect custom processors as well as to interact with state stores.
A processor topology is merely a logical abstraction for your stream processing code. At runtime, the logical topology is instantiated and replicated inside the application for parallel processing (see Stream Partitions and Tasks for details).
Time
A critical aspect in stream processing is the notion of time , and how it is modeled and integrated. For example, some operations such as windowing are defined based on time boundaries.
Common notions of time in streams are:
- Event time - The point in time when an event or data record occurred, i.e. was originally created “at the source”. Example: If the event is a geo-location change reported by a GPS sensor in a car, then the associated event-time would be the time when the GPS sensor captured the location change.
- Processing time - The point in time when the event or data record happens to be processed by the stream processing application, i.e. when the record is being consumed. The processing time may be milliseconds, hours, or days etc. later than the original event time. Example: Imagine an analytics application that reads and processes the geo-location data reported from car sensors to present it to a fleet management dashboard. Here, processing-time in the analytics application might be milliseconds or seconds (e.g. for real-time pipelines based on Apache Kafka and Kafka Streams) or hours (e.g. for batch pipelines based on Apache Hadoop or Apache Spark) after event-time.
- Ingestion time - The point in time when an event or data record is stored in a topic partition by a Kafka broker. The difference to event time is that this ingestion timestamp is generated when the record is appended to the target topic by the Kafka broker, not when the record is created “at the source”. The difference to processing time is that processing time is when the stream processing application processes the record. For example, if a record is never processed, there is no notion of processing time for it, but it still has an ingestion time.
The choice between event-time and ingestion-time is actually done through the configuration of Kafka (not Kafka Streams): From Kafka 0.10.x onwards, timestamps are automatically embedded into Kafka messages. Depending on Kafka’s configuration these timestamps represent event-time or ingestion-time. The respective Kafka configuration setting can be specified on the broker level or per topic. The default timestamp extractor in Kafka Streams will retrieve these embedded timestamps as-is. Hence, the effective time semantics of your application depend on the effective Kafka configuration for these embedded timestamps.
Kafka Streams assigns a timestamp to every data record via the TimestampExtractor
interface. These per-record timestamps describe the progress of a stream with regards to time and are leveraged by time-dependent operations such as window operations. As a result, this time will only advance when a new record arrives at the processor. We call this data-driven time the stream time of the application to differentiate with the wall-clock time when this application is actually executing. Concrete implementations of the TimestampExtractor
interface will then provide different semantics to the stream time definition. For example retrieving or computing timestamps based on the actual contents of data records such as an embedded timestamp field to provide event time semantics, and returning the current wall-clock time thereby yield processing time semantics to stream time. Developers can thus enforce different notions of time depending on their business needs.
Finally, whenever a Kafka Streams application writes records to Kafka, then it will also assign timestamps to these new records. The way the timestamps are assigned depends on the context:
- When new output records are generated via processing some input record, for example,
context.forward()
triggered in theprocess()
function call, output record timestamps are inherited from input record timestamps directly. - When new output records are generated via periodic functions such as
Punctuator#punctuate()
, the output record timestamp is defined as the current internal time (obtained throughcontext.timestamp()
) of the stream task. - For aggregations, the timestamp of a result update record will be the maximum timestamp of all input records contributing to the result.
You can change the default behavior in the Processor API by assigning timestamps to output records explicitly when calling #forward()
.
For aggregations and joins, timestamps are computed by using the following rules.
- For joins (stream-stream, table-table) that have left and right input records, the timestamp of the output record is assigned
max(left.ts, right.ts)
. - For stream-table joins, the output record is assigned the timestamp from the stream record.
- For aggregations, Kafka Streams also computes the
max
timestamp over all records, per key, either globally (for non-windowed) or per-window. - For stateless operations, the input record timestamp is passed through. For
flatMap
and siblings that emit multiple records, all output records inherit the timestamp from the corresponding input record.
Duality of Streams and Tables
When implementing stream processing use cases in practice, you typically need both streams and also databases. An example use case that is very common in practice is an e-commerce application that enriches an incoming stream of customer transactions with the latest customer information from a database table. In other words, streams are everywhere, but databases are everywhere, too.
Any stream processing technology must therefore provide first-class support for streams and tables. Kafka’s Streams API provides such functionality through its core abstractions for streams and tables, which we will talk about in a minute. Now, an interesting observation is that there is actually a close relationship between streams and tables , the so-called stream-table duality. And Kafka exploits this duality in many ways: for example, to make your applications elastic, to support fault-tolerant stateful processing, or to run interactive queries against your application’s latest processing results. And, beyond its internal usage, the Kafka Streams API also allows developers to exploit this duality in their own applications.
Before we discuss concepts such as aggregations in Kafka Streams, we must first introduce tables in more detail, and talk about the aforementioned stream-table duality. Essentially, this duality means that a stream can be viewed as a table, and a table can be viewed as a stream. Kafka’s log compaction feature, for example, exploits this duality.
A simple form of a table is a collection of key-value pairs, also called a map or associative array. Such a table may look as follows:
The stream-table duality describes the close relationship between streams and tables.
- Stream as Table : A stream can be considered a changelog of a table, where each data record in the stream captures a state change of the table. A stream is thus a table in disguise, and it can be easily turned into a “real” table by replaying the changelog from beginning to end to reconstruct the table. Similarly, in a more general analogy, aggregating data records in a stream - such as computing the total number of pageviews by user from a stream of pageview events - will return a table (here with the key and the value being the user and its corresponding pageview count, respectively).
- Table as Stream : A table can be considered a snapshot, at a point in time, of the latest value for each key in a stream (a stream’s data records are key-value pairs). A table is thus a stream in disguise, and it can be easily turned into a “real” stream by iterating over each key-value entry in the table.
Let’s illustrate this with an example. Imagine a table that tracks the total number of pageviews by user (first column of diagram below). Over time, whenever a new pageview event is processed, the state of the table is updated accordingly. Here, the state changes between different points in time - and different revisions of the table - can be represented as a changelog stream (second column).
Interestingly, because of the stream-table duality, the same stream can be used to reconstruct the original table (third column):
The same mechanism is used, for example, to replicate databases via change data capture (CDC) and, within Kafka Streams, to replicate its so-called state stores across machines for fault-tolerance. The stream-table duality is such an important concept that Kafka Streams models it explicitly via the KStream, KTable, and GlobalKTable interfaces.
Aggregations
An aggregation operation takes one input stream or table, and yields a new table by combining multiple input records into a single output record. Examples of aggregations are computing counts or sum.
In the Kafka Streams DSL
, an input stream of an aggregation
can be a KStream or a KTable, but the output stream will always be a KTable. This allows Kafka Streams to update an aggregate value upon the out-of-order arrival of further records after the value was produced and emitted. When such out-of-order arrival happens, the aggregating KStream or KTable emits a new aggregate value. Because the output is a KTable, the new value is considered to overwrite the old value with the same key in subsequent processing steps.
Windowing
Windowing lets you control how to group records that have the same key for stateful operations such as aggregations
or joins
into so-called windows. Windows are tracked per record key.
Windowing operations
are available in the Kafka Streams DSL
. When working with windows, you can specify a grace period for the window. This grace period controls how long Kafka Streams will wait for out-of-order data records for a given window. If a record arrives after the grace period of a window has passed, the record is discarded and will not be processed in that window. Specifically, a record is discarded if its timestamp dictates it belongs to a window, but the current stream time is greater than the end of the window plus the grace period.
Out-of-order records are always possible in the real world and should be properly accounted for in your applications. It depends on the effective time semantics
how out-of-order records are handled. In the case of processing-time, the semantics are “when the record is being processed”, which means that the notion of out-of-order records is not applicable as, by definition, no record can be out-of-order. Hence, out-of-order records can only be considered as such for event-time. In both cases, Kafka Streams is able to properly handle out-of-order records.
States
Some stream processing applications don’t require state, which means the processing of a message is independent from the processing of all other messages. However, being able to maintain state opens up many possibilities for sophisticated stream processing applications: you can join input streams, or group and aggregate data records. Many such stateful operators are provided by the Kafka Streams DSL.
Kafka Streams provides so-called state stores , which can be used by stream processing applications to store and query data. This is an important capability when implementing stateful operations. Every task in Kafka Streams embeds one or more state stores that can be accessed via APIs to store and query data required for processing. These state stores can either be a persistent key-value store, an in-memory hashmap, or another convenient data structure. Kafka Streams offers fault-tolerance and automatic recovery for local state stores.
Kafka Streams allows direct read-only queries of the state stores by methods, threads, processes or applications external to the stream processing application that created the state stores. This is provided through a feature called Interactive Queries. All stores are named and Interactive Queries exposes only the read operations of the underlying implementation.
Processing Guarantees
In stream processing, one of the most frequently asked question is “does my stream processing system guarantee that each record is processed once and only once, even if some failures are encountered in the middle of processing?” Failing to guarantee exactly-once stream processing is a deal-breaker for many applications that cannot tolerate any data-loss or data duplicates, and in that case a batch-oriented framework is usually used in addition to the stream processing pipeline, known as the Lambda Architecture. Prior to 0.11.0.0, Kafka only provides at-least-once delivery guarantees and hence any stream processing systems that leverage it as the backend storage could not guarantee end-to-end exactly-once semantics. In fact, even for those stream processing systems that claim to support exactly-once processing, as long as they are reading from / writing to Kafka as the source / sink, their applications cannot actually guarantee that no duplicates will be generated throughout the pipeline.
Since the 0.11.0.0 release, Kafka has added support to allow its producers to send messages to different topic partitions in a transactional and idempotent manner, and Kafka Streams has hence added the end-to-end exactly-once processing semantics by leveraging these features. More specifically, it guarantees that for any record read from the source Kafka topics, its processing results will be reflected exactly once in the output Kafka topic as well as in the state stores for stateful operations. Note the key difference between Kafka Streams end-to-end exactly-once guarantee with other stream processing frameworks’ claimed guarantees is that Kafka Streams tightly integrates with the underlying Kafka storage system and ensure that commits on the input topic offsets, updates on the state stores, and writes to the output topics will be completed atomically instead of treating Kafka as an external system that may have side-effects. For more information on how this is done inside Kafka Streams, see KIP-129.
As of the 2.6.0 release, Kafka Streams supports an improved implementation of exactly-once processing, named “exactly-once v2”, which requires broker version 2.5.0 or newer. This implementation is more efficient, because it reduces client and broker resource utilization, like client threads and used network connections, and it enables higher throughput and improved scalability. As of the 3.0.0 release, the first version of exactly-once has been deprecated. Users are encouraged to use exactly-once v2 for exactly-once processing from now on, and prepare by upgrading their brokers if necessary. For more information on how this is done inside the brokers and Kafka Streams, see KIP-447.
To enable exactly-once semantics when running Kafka Streams applications, set the processing.guarantee
config value (default value is at_least_once) to StreamsConfig.EXACTLY_ONCE_V2 (requires brokers version 2.5 or newer). For more information, see the Kafka Streams Configs section.
Out-of-Order Handling
Besides the guarantee that each record will be processed exactly-once, another issue that many stream processing applications will face is how to handle out-of-order data that may impact their business logic. In Kafka Streams, there are two causes that could potentially result in out-of-order data arrivals with respect to their timestamps:
- Within a topic-partition, a record’s timestamp may not be monotonically increasing along with their offsets. Since Kafka Streams will always try to process records within a topic-partition to follow the offset order, it can cause records with larger timestamps (but smaller offsets) to be processed earlier than records with smaller timestamps (but larger offsets) in the same topic-partition.
- Within a stream task that may be processing multiple topic-partitions, if users configure the application to not wait for all partitions to contain some buffered data and pick from the partition with the smallest timestamp to process the next record, then later on when some records are fetched for other topic-partitions, their timestamps may be smaller than those processed records fetched from another topic-partition.
For stateless operations, out-of-order data will not impact processing logic since only one record is considered at a time, without looking into the history of past processed records; for stateful operations such as aggregations and joins, however, out-of-order data could cause the processing logic to be incorrect. If users want to handle such out-of-order data, generally they need to allow their applications to wait for longer time while bookkeeping their states during the wait time, i.e. making trade-off decisions between latency, cost, and correctness. In Kafka Streams specifically, users can configure their window operators for windowed aggregations to achieve such trade-offs (details can be found in Developer Guide). As for Joins, users may use versioned state stores to address concerns with out-of-order data, but out-of-order data will not be handled by default:
- For Stream-Stream joins, all three types (inner, outer, left) handle out-of-order records correctly.
- For Stream-Table joins, if not using versioned stores, then out-of-order records are not handled (i.e., Streams applications don’t check for out-of-order records and just process all records in offset order), and hence it may produce unpredictable results. With versioned stores, stream-side out-of-order data will be properly handled by performing a timestamp-based lookup in the table. Table-side out-of-order data is still not handled.
- For Table-Table joins, if not using versioned stores, then out-of-order records are not handled (i.e., Streams applications don’t check for out-of-order records and just process all records in offset order). However, the join result is a changelog stream and hence will be eventually consistent. With versioned stores, table-table join semantics change from offset-based semantics to timestamp-based semantics and out-of-order records are handled accordingly.
5 - Architecture
Architecture
Introduction Run Demo App Tutorial: Write App Concepts Architecture Developer Guide Upgrade
Kafka Streams simplifies application development by building on the Kafka producer and consumer libraries and leveraging the native capabilities of Kafka to offer data parallelism, distributed coordination, fault tolerance, and operational simplicity. In this section, we describe how Kafka Streams works underneath the covers.
The picture below shows the anatomy of an application that uses the Kafka Streams library. Let’s walk through some details.
Stream Partitions and Tasks
The messaging layer of Kafka partitions data for storing and transporting it. Kafka Streams partitions data for processing it. In both cases, this partitioning is what enables data locality, elasticity, scalability, high performance, and fault tolerance. Kafka Streams uses the concepts of partitions and tasks as logical units of its parallelism model based on Kafka topic partitions. There are close links between Kafka Streams and Kafka in the context of parallelism:
- Each stream partition is a totally ordered sequence of data records and maps to a Kafka topic partition.
- A data record in the stream maps to a Kafka message from that topic.
- The keys of data records determine the partitioning of data in both Kafka and Kafka Streams, i.e., how data is routed to specific partitions within topics.
An application’s processor topology is scaled by breaking it into multiple tasks. More specifically, Kafka Streams creates a fixed number of tasks based on the input stream partitions for the application, with each task assigned a list of partitions from the input streams (i.e., Kafka topics). The assignment of partitions to tasks never changes so that each task is a fixed unit of parallelism of the application. Tasks can then instantiate their own processor topology based on the assigned partitions; they also maintain a buffer for each of its assigned partitions and process messages one-at-a-time from these record buffers. As a result stream tasks can be processed independently and in parallel without manual intervention.
Slightly simplified, the maximum parallelism at which your application may run is bounded by the maximum number of stream tasks, which itself is determined by maximum number of partitions of the input topic(s) the application is reading from. For example, if your input topic has 5 partitions, then you can run up to 5 applications instances. These instances will collaboratively process the topic’s data. If you run a larger number of app instances than partitions of the input topic, the “excess” app instances will launch but remain idle; however, if one of the busy instances goes down, one of the idle instances will resume the former’s work.
It is important to understand that Kafka Streams is not a resource manager, but a library that “runs” anywhere its stream processing application runs. Multiple instances of the application are executed either on the same machine, or spread across multiple machines and tasks can be distributed automatically by the library to those running application instances. The assignment of partitions to tasks never changes; if an application instance fails, all its assigned tasks will be automatically restarted on other instances and continue to consume from the same stream partitions.
NOTE: Topic partitions are assigned to tasks, and tasks are assigned to all threads over all instances, in a best-effort attempt to trade off load-balancing and stickiness of stateful tasks. For this assignment, Kafka Streams uses the StreamsPartitionAssignor class and doesn’t let you change to a different assignor. If you try to use a different assignor, Kafka Streams ignores it.
The following diagram shows two tasks each assigned with one partition of the input streams.
Threading Model
Kafka Streams allows the user to configure the number of threads that the library can use to parallelize processing within an application instance. Each thread can execute one or more tasks with their processor topologies independently. For example, the following diagram shows one stream thread running two stream tasks.
Starting more stream threads or more instances of the application merely amounts to replicating the topology and having it process a different subset of Kafka partitions, effectively parallelizing processing. It is worth noting that there is no shared state amongst the threads, so no inter-thread coordination is necessary. This makes it very simple to run topologies in parallel across the application instances and threads. The assignment of Kafka topic partitions amongst the various stream threads is transparently handled by Kafka Streams leveraging Kafka’s coordination functionality.
As we described above, scaling your stream processing application with Kafka Streams is easy: you merely need to start additional instances of your application, and Kafka Streams takes care of distributing partitions amongst tasks that run in the application instances. You can start as many threads of the application as there are input Kafka topic partitions so that, across all running instances of an application, every thread (or rather, the tasks it runs) has at least one input partition to process.
As of Kafka 2.8 you can scale stream threads much in the same way you can scale your Kafka Stream clients. Simply add or remove stream threads and Kafka Streams will take care of redistributing the partitions. You may also add threads to replace stream threads that have died removing the need to restart clients to recover the number of thread running.
Local State Stores
Kafka Streams provides so-called state stores , which can be used by stream processing applications to store and query data, which is an important capability when implementing stateful operations. The Kafka Streams DSL, for example, automatically creates and manages such state stores when you are calling stateful operators such as join()
or aggregate()
, or when you are windowing a stream.
Every stream task in a Kafka Streams application may embed one or more local state stores that can be accessed via APIs to store and query data required for processing. Kafka Streams offers fault-tolerance and automatic recovery for such local state stores.
The following diagram shows two stream tasks with their dedicated local state stores.
Fault Tolerance
Kafka Streams builds on fault-tolerance capabilities integrated natively within Kafka. Kafka partitions are highly available and replicated; so when stream data is persisted to Kafka it is available even if the application fails and needs to re-process it. Tasks in Kafka Streams leverage the fault-tolerance capability offered by the Kafka consumer client to handle failures. If a task runs on a machine that fails, Kafka Streams automatically restarts the task in one of the remaining running instances of the application.
In addition, Kafka Streams makes sure that the local state stores are robust to failures, too. For each state store, it maintains a replicated changelog Kafka topic in which it tracks any state updates. These changelog topics are partitioned as well so that each local state store instance, and hence the task accessing the store, has its own dedicated changelog topic partition. Log compaction is enabled on the changelog topics so that old data can be purged safely to prevent the topics from growing indefinitely. If tasks run on a machine that fails and are restarted on another machine, Kafka Streams guarantees to restore their associated state stores to the content before the failure by replaying the corresponding changelog topics prior to resuming the processing on the newly started tasks. As a result, failure handling is completely transparent to the end user.
Note that the cost of task (re)initialization typically depends primarily on the time for restoring the state by replaying the state stores’ associated changelog topics. To minimize this restoration time, users can configure their applications to have standby replicas of local states (i.e. fully replicated copies of the state). When a task migration happens, Kafka Streams will assign a task to an application instance where such a standby replica already exists in order to minimize the task (re)initialization cost. See num.standby.replicas
in the Kafka Streams Configs section. Starting in 2.6, Kafka Streams will guarantee that a task is only ever assigned to an instance with a fully caught-up local copy of the state, if such an instance exists. Standby tasks will increase the likelihood that a caught-up instance exists in the case of a failure.
You can also configure standby replicas with rack awareness. When configured, Kafka Streams will attempt to distribute a standby task on a different “rack” than the active one, thus having a faster recovery time when the rack of the active tasks fails. See rack.aware.assignment.tags
in the Kafka Streams Developer Guide section.
There is also a client config client.rack
which can set the rack for a Kafka consumer. If brokers also have their rack set via broker.rack
, then rack aware task assignment can be enabled via rack.aware.assignment.strategy
(cf. Kafka Streams Developer Guide) to compute a task assignment which can reduce cross rack traffic by trying to assign tasks to clients with the same rack. Note that client.rack
can also be used to distribute standby tasks to different racks from the active ones, which has a similar functionality as rack.aware.assignment.tags
. Currently, rack.aware.assignment.tag
takes precedence in distributing standby tasks which means if both configs present, rack.aware.assignment.tag
will be used for distributing standby tasks on different racks from the active ones because it can configure more tag keys.
6 - Upgrade Guide
Upgrade Guide and API Changes
Introduction Run Demo App Tutorial: Write App Concepts Architecture Developer Guide Upgrade
Upgrading from any older version to 4.0.0 is possible: if upgrading from 3.4 or below, you will need to do two rolling bounces, where during the first rolling bounce phase you set the config upgrade.from="older version"
(possible values are "0.10.0" - "3.4"
) and during the second you remove it. This is required to safely handle 3 changes. The first is introduction of the new cooperative rebalancing protocol of the embedded consumer. The second is a change in foreign-key join serialization format. Note that you will remain using the old eager rebalancing protocol if you skip or delay the second rolling bounce, but you can safely switch over to cooperative at any time once the entire group is on 2.4+ by removing the config value and bouncing. For more details please refer to KIP-429. The third is a change in the serialization format for an internal repartition topic. For more details, please refer to KIP-904:
- prepare your application instances for a rolling bounce and make sure that config
upgrade.from
is set to the version from which it is being upgrade. - bounce each instance of your application once
- prepare your newly deployed 4.0.0 application instances for a second round of rolling bounces; make sure to remove the value for config
upgrade.from
- bounce each instance of your application once more to complete the upgrade
As an alternative, an offline upgrade is also possible. Upgrading from any versions as old as 0.10.0.x to 4.0.0 in offline mode require the following steps:
- stop all old (e.g., 0.10.0.x) application instances
- update your code and swap old code and jar file with new code and new jar file
- restart all new (4.0.0) application instances
Note: The cooperative rebalancing protocol has been the default since 2.4, but we have continued to support the eager rebalancing protocol to provide users an upgrade path. This support will be dropped in a future release, so any users still on the eager protocol should prepare to finish upgrading their applications to the cooperative protocol in version 3.1. This only affects users who are still on a version older than 2.4, and users who have upgraded already but have not yet removed the upgrade.from
config that they set when upgrading from a version below 2.4. Users fitting into the latter case will simply need to unset this config when upgrading beyond 3.1, while users in the former case will need to follow a slightly different upgrade path if they attempt to upgrade from 2.3 or below to a version above 3.1. Those applications will need to go through a bridge release, by first upgrading to a version between 2.4 - 3.1 and setting the upgrade.from
config, then removing that config and upgrading to the final version above 3.1. See KAFKA-8575 for more details.
For a table that shows Streams API compatibility with Kafka broker versions, see Broker Compatibility.
Notable compatibility changes in past releases
Starting in version 4.0.0, Kafka Streams will only be compatible when running against brokers on version 2.1 or higher. Additionally, exactly-once semantics (EOS) will require brokers to be at least version 2.5.
Downgrading from 3.5.x or newer version to 3.4.x or older version needs special attention: Since 3.5.0 release, Kafka Streams uses a new serialization format for repartition topics. This means that older versions of Kafka Streams would not be able to recognize the bytes written by newer versions, and hence it is harder to downgrade Kafka Streams with version 3.5.0 or newer to older versions in-flight. For more details, please refer to KIP-904. For a downgrade, first switch the config from "upgrade.from"
to the version you are downgrading to. This disables writing of the new serialization format in your application. It’s important to wait in this state long enough to make sure that the application has finished processing any “in-flight” messages written into the repartition topics in the new serialization format. Afterwards, you can downgrade your application to a pre-3.5.x version.
Downgrading from 3.0.x or newer version to 2.8.x or older version needs special attention: Since 3.0.0 release, Kafka Streams uses a newer RocksDB version whose on-disk format changed. This means that old versioned RocksDB would not be able to recognize the bytes written by that newer versioned RocksDB, and hence it is harder to downgrade Kafka Streams with version 3.0.0 or newer to older versions in-flight. Users need to wipe out the local RocksDB state stores written by the new versioned Kafka Streams before swapping in the older versioned Kafka Streams bytecode, which would then restore the state stores with the old on-disk format from the changelogs.
Kafka Streams does not support running multiple instances of the same application as different processes on the same physical state directory. Starting in 2.8.0 (as well as 2.7.1 and 2.6.2), this restriction will be enforced. If you wish to run more than one instance of Kafka Streams, you must configure them with different values for state.dir
.
Starting in Kafka Streams 2.6.x, a new processing mode is available, named EOS version 2. This can be configured by setting "processing.guarantee"
to "exactly_once_v2"
for application versions 3.0+, or setting it to "exactly_once_beta"
for versions between 2.6 and 2.8. To use this new feature, your brokers must be on version 2.5.x or newer. If you want to upgrade your EOS application from an older version and enable this feature in version 3.0+, you first need to upgrade your application to version 3.0.x, staying on "exactly_once"
, and then do second round of rolling bounces to switch to "exactly_once_v2"
. If you are upgrading an EOS application from an older (pre-2.6) version to a version between 2.6 and 2.8, follow these same steps but with the config "exactly_once_beta"
instead. No special steps are required to upgrade an application using "exactly_once_beta"
from version 2.6+ to 3.0 or higher: you can just change the config from "exactly_once_beta"
to "exactly_once_v2"
during the rolling upgrade. For a downgrade, do the reverse: first switch the config from "exactly_once_v2"
to "exactly_once"
to disable the feature in your 2.6.x application. Afterward, you can downgrade your application to a pre-2.6.x version.
Since 2.6.0 release, Kafka Streams depends on a RocksDB version that requires MacOS 10.14 or higher.
To run a Kafka Streams application version 2.2.1, 2.3.0, or higher a broker version 0.11.0 or higher is required and the on-disk message format must be 0.11 or higher. Brokers must be on version 0.10.1 or higher to run a Kafka Streams application version 0.10.1 to 2.2.0. Additionally, on-disk message format must be 0.10 or higher to run a Kafka Streams application version 1.0 to 2.2.0. For Kafka Streams 0.10.0, broker version 0.10.0 or higher is required.
In deprecated KStreamBuilder
class, when a KTable
is created from a source topic via KStreamBuilder.table()
, its materialized state store will reuse the source topic as its changelog topic for restoring, and will disable logging to avoid appending new updates to the source topic; in the StreamsBuilder
class introduced in 1.0, this behavior was changed accidentally: we still reuse the source topic as the changelog topic for restoring, but will also create a separate changelog topic to append the update records from source topic to. In the 2.0 release, we have fixed this issue and now users can choose whether or not to reuse the source topic based on the StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG
: if you are upgrading from the old KStreamBuilder
class and hence you need to change your code to use the new StreamsBuilder
, you should set this config value to StreamsConfig#OPTIMIZE
to continue reusing the source topic; if you are upgrading from 1.0 or 1.1 where you are already using StreamsBuilder
and hence have already created a separate changelog topic, you should set this config value to StreamsConfig#NO_OPTIMIZATION
when upgrading to 4.0.0 in order to use that changelog topic for restoring the state store. More details about the new config StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG
can be found in KIP-295.
Streams API changes in 4.0.0
In this release, eos-v1 (Exactly Once Semantics version 1) is no longer supported. To use eos-v2, brokers must be running version 2.5 or later. Additionally, all deprecated methods, classes, APIs, and config parameters up to and including AK 3.5 release have been removed. A few important ones are listed below. The full list can be found in KAFKA-12822.
- Old processor APIs
- KStream#through() in both Java and Scala
- “transformer” methods and classes in both Java and Scala
- kstream.KStream#branch in both Java and Scala
- builder methods for Time/Session/Join/SlidingWindows
- KafkaStreams#setUncaughtExceptionHandler()
In this release the ClientInstanceIds
instance stores the global consumerUuid
for the KIP-714 id with a key of global stream-thread name appended with "-global-consumer"
where before it was only the global stream-thread name.
In this release two configs default.deserialization.exception.handler
and default.production.exception.handler
are deprecated, as they don’t have any overwrites, which is described in KIP-1056 You can refer to new configs via deserialization.exception.handler
and production.exception.handler
.
In previous release, a new version of the Processor API was introduced and the old Processor API was incrementally replaced and deprecated. KIP-1070 follow this path by deprecating MockProcessorContext
, Transformer
, TransformerSupplier
, ValueTransformer
, and ValueTransformerSupplier
.
Previously, the ProductionExceptionHandler
was not invoked on a (retriable) TimeoutException
. With Kafka Streams 4.0, the handler is called, and the default handler would return RETRY
to not change existing behavior. However, a custom handler can now decide to break the infinite retry loop by returning either CONTINUE
or FAIL
(KIP-1065).
In this release, Kafka Streams metrics can be collected broker side via the KIP-714 broker-plugin. For more detailed information, refer to KIP-1076 document please.
KIP-1077 deprecates the ForeachProcessor
class. This change is aimed at improving the organization and clarity of the Kafka Streams API by ensuring that internal classes are not exposed in public packages.
KIP-1078 deprecates the leaking getter methods in the Joined
helper class. These methods are deprecated without a replacement for future removal, as they don’t add any value to Kafka Streams users.
To ensures better encapsulation and organization of configuration documentation within Kafka Streams, KIP-1085 deprecate certain public doc description variables that are only used within the StreamsConfig
or TopologyConfig
classes. Additionally, the unused variable DUMMY_THREAD_INDEX
will also be deprecated.
Due to the removal of the already deprecated #through
method in Kafka Streams, the intermediateTopicsOption
of StreamsResetter
tool in Apache Kafka is not needed any more and therefore is deprecated (KIP-1087).
Since string metrics cannot be collected on the broker side (KIP-714), KIP-1091 introduces numeric counterparts to allow proper broker-side metric collection for Kafka Streams applications. These metrics will be available at the INFO
recording level, and a thread-level metric with a String value will be available for users leveraging Java Management Extensions (JMX
).
In order to reduce storage overhead and improve API usability, a new method in the Java and Scala APIs that accepts a BiFunction for foreign key extraction is introduced by KIP-1104. KIP-1104 allows foreign key extraction from both the key and value in KTable joins in Apache Kafka. Previously, foreign key joins in KTables only allowed extraction from the value, which led to data duplication and potential inconsistencies. This enhancement introduces a new method in the Java and Scala APIs that accepts a BiFunction for foreign key extraction, enabling more intuitive and efficient joins. The existing methods will be deprecated but not removed, ensuring backward compatibility. This change aims to reduce storage overhead and improve API usability.
With introduction of KIP-1106, the existing Topology.AutoOffsetReset
is deprecated and replaced with a new class org.apache.kafka.streams.AutoOffsetReset
to capture the reset strategies. New methods will be added to the org.apache.kafka.streams.Topology
and org.apache.kafka.streams.kstream.Consumed
classes to support the new reset strategy. These changes aim to provide more flexibility and efficiency in managing offsets, especially in scenarios involving long-term storage and infinite retention.
You can now configure your topology with a ProcessorWrapper
, which allows you to access and optionally wrap/replace any processor in the topology by injecting an alternative ProcessorSupplier
in its place. This can be used to peek records and access the processor context even for DSL operators, for example to implement a logging or tracing framework, or to aid in testing or debugging scenarios. You must implement the ProcessorWrapper
interface and then pass the class or class name into the configs via the new StreamsConfig#PROCESSOR_WRAPPER_CLASS_CONFIG
config. NOTE: this config is applied during the topology building phase, and therefore will not take effect unless the config is passed in when creating the StreamsBuilder (DSL) or Topology(PAPI) objects. You MUST use the StreamsBuilder/Topology constructor overload that accepts a TopologyConfig parameter for the StreamsConfig#PROCESSOR_WRAPPER_CLASS_CONFIG
to be picked up. See KIP-1112 for more details.
Upgraded RocksDB dependency to version 9.7.3 (from 7.9.2). This upgrade incorporates various improvements and optimizations within RocksDB. However, it also introduces some API changes. The org.rocksdb.AccessHint
class, along with its associated methods, has been removed. Several methods related to compressed block cache configuration in the BlockBasedTableConfig
class have been removed, including blockCacheCompressedNumShardBits
, blockCacheCompressedSize
, and their corresponding setters. These functionalities are now consolidated under the cache
option, and developers should configure their compressed block cache using the setCache
method instead. The NO_FILE_CLOSES
field has been removed from the org.rocksdb.TickerTypeenum
as a result the number-open-files
metrics does not work as expected. Metric number-open-files
returns constant -1 from now on until it will officially be removed. The org.rocksdb.Options.setLogger()
method now accepts a LoggerInterface
as a parameter instead of the previous Logger
. Some data types used in RocksDB’s Java API have been modified. These changes, along with the removed class, field, and new methods, are primarily relevant to users implementing custom RocksDB configurations. These changes are expected to be largely transparent to most Kafka Streams users. However, those employing advanced RocksDB customizations within their Streams applications, particularly through the rocksdb.config.setter
, are advised to consult the detailed RocksDB 9.7.3 changelog to ensure a smooth transition and adapt their configurations as needed. Specifically, users leveraging the removed AccessHint
class, the removed methods from the BlockBasedTableConfig
class, the NO_FILE_CLOSES
field from TickerType
, or relying on the previous signature of setLogger()
will need to update their implementations.
Streams API changes in 3.9.0
The introduction of KIP-1033 enables you to provide a processing exception handler to manage exceptions during the processing of a record rather than throwing the exception all the way out of your streams application. You can provide the configs via the StreamsConfig
as StreamsConfig#PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG
. The specified handler must implement the org.apache.kafka.streams.errors.ProcessingExceptionHandler
interface.
Kafka Streams now allows to customize the logging interval of stream-thread runtime summary, via the newly added config log.summary.interval.ms
. By default, the summary is logged every 2 minutes. More details can be found in KIP-1049.
Streams API changes in 3.8.0
Kafka Streams now supports customizable task assignment strategies via the task.assignor.class
configuration. The configuration can be set to the fully qualified class name of a custom task assignor implementation that has to extend the new org.apache.kafka.streams.processor.assignment.TaskAssignor
interface. The new configuration also allows users to bring back the behavior of the old task assignor StickyTaskAssignor
that was used before the introduction of the HighAvailabilityTaskAssignor
. If no custom task assignor is configured, the default task assignor HighAvailabilityTaskAssignor
is used. If you were using the internal.task.assignor.class
config, you should switch to using the new task.assignor.class
config instead, as the internal config will be removed in a future release. If you were previously plugging in the StickyTaskAssignor
via the legacy internal.task.assignor.class
config, you will need to make sure that you are importing the new org.apache.kafka.streams.processor.assignment.StickTaskAssignor
when you switch over to the new task.assignor.class
config, which is a version of the StickyTaskAssignor
that implements the new public TaskAssignor
interface. For more details, see the public interface section of KIP-924.
The Processor API now support so-called read-only state stores, added via KIP-813. These stores don’t have a dedicated changelog topic, but use their source topic for fault-tolerance, similar to KTables
with source-topic optimization enabled.
To improve detection of leaked state store iterators, we added new store-level metrics to track the number and age of open iterators. The new metrics are num-open-iterators
, iterator-duration-avg
, iterator-duration-max
and oldest-iterator-open-since-ms
. These metrics are available for all state stores, including RocksDB, in-memory, and custom stores. More details can be found in KIP-989.
Streams API changes in 3.7.0
We added a new method to KafkaStreams
, namely KafkaStreams#setStandbyUpdateListener()
in KIP-988, in which users can provide their customized implementation of the newly added StandbyUpdateListener
interface to continuously monitor changes to standby tasks.
IQv2 supports RangeQuery
that allows to specify unbounded, bounded, or half-open key-ranges, which return data in unordered (byte[]-lexicographical) order (per partition). KIP-985 extends this functionality by adding .withDescendingKeys()
and .withAscendingKeys()
to allow user to receive data in descending or ascending order.
KIP-992 adds two new query types, namely TimestampedKeyQuery
and TimestampedRangeQuery
. Both should be used to query a timestamped key-value store, to retrieve a ValueAndTimestamp
result. The existing KeyQuery
and RangeQuery
are changed to always return the value only for timestamped key-value stores.
IQv2 adds support for MultiVersionedKeyQuery
(introduced in KIP-968) that allows retrieving a set of records from a versioned state store for a given key and a specified time range. Users have to use fromTime(Instant)
and/or toTime(Instant)
to specify a half or a complete time range.
IQv2 adds support for VersionedKeyQuery
(introduced in KIP-960) that allows retrieving a single record from a versioned state store based on its key and timestamp. Users have to use the asOf(Instant)
method to define a query that returns the record’s version for the specified timestamp. To be more precise, the key query returns the record with the greatest timestamp <= Instant
.
The non-null key requirements for Kafka Streams join operators were relaxed as part of KIP-962. The behavior of the following operators changed.
- left join KStream-KStream: no longer drop left records with null-key and call ValueJoiner with ’null’ for right value.
- outer join KStream-KStream: no longer drop left/right records with null-key and call ValueJoiner with ’null’ for right/left value.
- left-foreign-key join KTable-KTable: no longer drop left records with null-foreign-key returned by the ForeignKeyExtractor and call ValueJoiner with ’null’ for right value.
- left join KStream-KTable: no longer drop left records with null-key and call ValueJoiner with ’null’ for right value.
- left join KStream-GlobalTable: no longer drop records when KeyValueMapper returns ’null’ and call ValueJoiner with ’null’ for right value.
Stream-DSL users who want to keep the current behavior can prepend a .filter() operator to the aforementioned operators and filter accordingly. The following snippets illustrate how to keep the old behavior.
//left join KStream-KStream
leftStream
.filter((key, value) -> key != null)
.leftJoin(rightStream, (leftValue, rightValue) -> join(leftValue, rightValue), windows);
//outer join KStream-KStream
rightStream
.filter((key, value) -> key != null);
leftStream
.filter((key, value) -> key != null)
.outerJoin(rightStream, (leftValue, rightValue) -> join(leftValue, rightValue), windows);
//left-foreign-key join KTable-KTable
Function<String;, String> foreignKeyExtractor = leftValue -> ...
leftTable
.filter((key, value) -> foreignKeyExtractor.apply(value) != null)
.leftJoin(rightTable, foreignKeyExtractor, (leftValue, rightValue) -> join(leftValue, rightValue), Named.as("left-foreign-key-table-join"));
//left join KStream-KTable
leftStream
.filter((key, value) -> key != null)
.leftJoin(kTable, (k, leftValue, rightValue) -> join(leftValue, rightValue));
//left join KStream-GlobalTable
KeyValueMapper<String;, String, String> keyValueMapper = (key, value) -> ...;
leftStream
.filter((key, value) -> keyValueMapper.apply(key,value) != null)
.leftJoin(globalTable, keyValueMapper, (leftValue, rightValue) -> join(leftValue, rightValue));
The default.dsl.store
config was deprecated in favor of the new dsl.store.suppliers.class
config to allow for custom state store implementations to be configured as the default. If you currently specify default.dsl.store=ROCKS_DB
or default.dsl.store=IN_MEMORY
replace those configurations with dsl.store.suppliers.class=BuiltInDslStoreSuppliers.RocksDBDslStoreSuppliers.class
and dsl.stores.suppliers.class=BuiltInDslStoreSuppliers.InMemoryDslStoreSuppliers.class
respectively
A new configuration option balance_subtopology
for rack.aware.assignment.strategy
was introduced in 3.7 release. For more information, including how it can be enabled and further configured, see the Kafka Streams Developer Guide.
Streams API changes in 3.6.0
Rack aware task assignment was introduced in KIP-925. Rack aware task assignment can be enabled for StickyTaskAssignor
or HighAvailabilityTaskAssignor
to compute task assignments which can minimize cross rack traffic under certain conditions. For more information, including how it can be enabled and further configured, see the Kafka Streams Developer Guide.
IQv2 supports a RangeQuery
that allows to specify unbounded, bounded, or half-open key-ranges. Users have to use withUpperBound(K)
, withLowerBound(K)
, or withNoBounds()
to specify half-open or unbounded ranges, but cannot use withRange(K lower, K upper)
for the same. KIP-941 closes this gap by allowing to pass in null
as upper and lower bound (with semantics “no bound”) to simplify the usage of the RangeQuery
class.
KStreams-to-KTable joins now have an option for adding a grace period. The grace period is enabled on the Joined
object using with withGracePeriod()
method. This change was introduced in KIP-923. To use the grace period option in the Stream-Table join the table must be versioned. For more information, including how it can be enabled and further configured, see the Kafka Streams Developer Guide.
Streams API changes in 3.5.0
A new state store type, versioned key-value stores, was introduced in KIP-889 and KIP-914. Rather than storing a single record version (value and timestamp) per key, versioned state stores may store multiple record versions per key. This allows versioned state stores to support timestamped retrieval operations to return the latest record (per key) as of a specified timestamp. For more information, including how to upgrade from a non-versioned key-value store to a versioned store in an existing application, see the Developer Guide. Versioned key-value stores are opt-in only; existing applications will not be affected upon upgrading to 3.5 without explicit code changes.
In addition to KIP-899, KIP-914 updates DSL processing semantics if a user opts-in to use the new versioned key-value stores. Using the new versioned key-value stores, DSL processing are able to handle out-of-order data better: For example, late record may be dropped and stream-table joins do a timestamped based lookup into the table. Table aggregations and primary/foreign-key table-table joins are also improved. Note: versioned key-value stores are not supported for global-KTable and don’t work with suppress()
.
KIP-904 improves the implementation of KTable aggregations. In general, an input KTable update triggers a result refinent for two rows; however, prior to KIP-904, if both refinements happen to the same result row, two independent updates to the same row are applied, resulting in spurious itermediate results. KIP-904 allows us to detect this case, and to only apply a single update avoiding spurious intermediate results.
Error handling is improved via KIP-399. The existing ProductionExceptionHandler
now also covers serialization errors.
We added a new Serde type Boolean
in KIP-907
KIP-884 adds a new config default.client.supplier
that allows to use a custom KafkaClientSupplier
without any code changes.
Streams API changes in 3.4.0
KIP-770 deprecates config cache.max.bytes.buffering
in favor of the newly introduced config statestore.cache.max.bytes
. To improve monitoring, two new metrics input-buffer-bytes-total
and cache-size-bytes-total
were added at the DEBUG level. Note, that the KIP is only partially implemented in the 3.4.0 release, and config input.buffer.max.bytes
is not available yet.
KIP-873 enables you to multicast result records to multiple partition of downstream sink topics and adds functionality for choosing to drop result records without sending. The Integer StreamPartitioner.partition()
method is deprecated and replaced by the newly added Optiona≶Set<Integer>>StreamPartitioner.partitions()
method, which enables returning a set of partitions to send the record to.
KIP-862 adds a DSL optimization for stream-stream self-joins. The optimization is enabled via a new option single.store.self.join
which can be set via existing config topology.optimization
. If enabled, the DSL will use a different join processor implementation that uses a single RocksDB store instead of two, to avoid unnecessary data duplication for the self-join case.
KIP-865 updates the Kafka Streams application reset tool’s server parameter name to conform to the other Kafka tooling by deprecating the --bootstrap-servers
parameter and introducing a new --bootstrap-server
parameter in its place.
Streams API changes in 3.3.0
Kafka Streams does not send a “leave group” request when an instance is closed. This behavior implies that a rebalance is delayed until max.poll.interval.ms
passed. KIP-812 introduces KafkaStreams.close(CloseOptions)
overload, which allows forcing an instance to leave the group immediately. Note: Due to internal limitations, CloseOptions
only works for static consumer groups at this point (cf. KAFKA-16514 for more details and a fix in some future release).
KIP-820 adapts the PAPI type-safety improvement of KIP-478 into the DSL. The existing methods KStream.transform
, KStream.flatTransform
, KStream.transformValues
, and KStream.flatTransformValues
as well as all overloads of void KStream.process
are deprecated in favor of the newly added methods
KStream<KOut,VOut> KStream.process(ProcessorSupplier, ...)
KStream<K,VOut> KStream.processValues(FixedKeyProcessorSupplier, ...)
Both new methods have multiple overloads and return a KStream
instead of void
as the deprecated process()
methods did. In addition, FixedKeyProcessor
, FixedKeyRecord
, FixedKeyProcessorContext
, and ContextualFixedKeyProcessor
are introduced to guard against disallowed key modification inside processValues()
. Furthermore, ProcessingContext
is added for a better interface hierarchy.
Emitting a windowed aggregation result only after a window is closed is currently supported via the suppress()
operator. However, suppress()
uses an in-memory implementation and does not support RocksDB. To close this gap, KIP-825 introduces “emit strategies”, which are built into the aggregation operator directly to use the already existing RocksDB store. TimeWindowedKStream.emitStrategy(EmitStrategy)
and SessionWindowedKStream.emitStrategy(EmitStrategy)
allow picking between “emit on window update” (default) and “emit on window close” strategies. Additionally, a few new emit metrics are added, as well as a necessary new method, SessionStore.findSessions(long, long)
.
KIP-834 allows pausing and resuming a Kafka Streams instance. Pausing implies that processing input records and executing punctuations will be skipped; Kafka Streams will continue to poll to maintain its group membership and may commit offsets. In addition to the new methods KafkaStreams.pause()
and KafkaStreams.resume()
, it is also supported to check if an instance is paused via the KafkaStreams.isPaused()
method.
To improve monitoring of Kafka Streams applications, KIP-846 adds four new metrics bytes-consumed-total
, records-consumed-total
, bytes-produced-total
, and records-produced-total
within a new topic level scope. The metrics are collected at INFO level for source and sink nodes, respectively.
Streams API changes in 3.2.0
RocksDB offers many metrics which are critical to monitor and tune its performance. Kafka Streams started to make RocksDB metrics accessible like any other Kafka metric via KIP-471 in 2.4.0 release. However, the KIP was only partially implemented, and is now completed with the 3.2.0 release. For a full list of available RocksDB metrics, please consult the monitoring documentation.
Kafka Streams ships with RocksDB and in-memory store implementations and users can pick which one to use. However, for the DSL, the choice is a per-operator one, making it cumbersome to switch from the default RocksDB store to in-memory store for all operators, especially for larger topologies. KIP-591 adds a new config default.dsl.store
that enables setting the default store for all DSL operators globally. Note that it is required to pass TopologyConfig
to the StreamsBuilder
constructor to make use of this new config.
For multi-AZ deployments, it is desired to assign StandbyTasks to a KafkaStreams instance running in a different AZ than the corresponding active StreamTask. KIP-708 enables configuring Kafka Streams instances with a rack-aware StandbyTask assignment strategy, by using the new added configs rack.aware.assignment.tags
and corresponding client.tag.<myTag>
.
KIP-791 adds a new method Optional<RecordMetadata> StateStoreContext.recordMetadata()
to expose record metadata. This helps for example to provide read-your-writes consistency guarantees in interactive queries.
Interactive Queries allow users to tap into the operational state of Kafka Streams processor nodes. The existing API is tightly coupled with the actual state store interfaces and thus the internal implementation of state store. To break up this tight coupling and allow for building more advanced IQ features, KIP-796 introduces a completely new IQv2 API, via StateQueryRequest
and StateQueryResult
classes, as well as Query
and QueryResult
interfaces (plus additional helper classes). In addition, multiple built-in query types were added: KeyQuery
for key lookups and RangeQuery
(via KIP-805) for key-range queries on key-value stores, as well as WindowKeyQuery
and WindowRangeQuery
(via KIP-806) for key and range lookup into windowed stores.
The Kafka Streams DSL may insert so-called repartition topics for certain DSL operators to ensure correct partitioning of data. These topics are configured with infinite retention time, and Kafka Streams purges old data explicitly via “delete record” requests, when commiting input topic offsets. KIP-811 adds a new config repartition.purge.interval.ms
allowing you to configure the purge interval independently of the commit interval.
Streams API changes in 3.1.0
The semantics of left/outer stream-stream join got improved via KIP-633. Previously, left-/outer stream-stream join might have emitted so-call spurious left/outer results, due to an eager-emit strategy. The implementation was changed to emit left/outer join result records only after the join window is closed. The old API to specify the join window, i.e., JoinWindows.of()
that enables the eager-emit strategy, was deprecated in favor of a JoinWindows.ofTimeDifferenceAndGrace()
and JoinWindows.ofTimeDifferencWithNoGrace()
. The new semantics are only enabled if you use the new join window builders.
Additionally, KIP-633 makes setting a grace period also mandatory for windowed aggregations, i.e., for TimeWindows
(hopping/tumbling), SessionWindows
, and SlidingWindows
. The corresponding builder methods .of(...)
were deprecated in favor of the new .ofTimeDifferenceAndGrace()
and .ofTimeDifferencWithNoGrace()
methods.
KIP-761 adds new metrics that allow to track blocking times on the underlying consumer and producer clients. Check out the section on Kafka Streams metrics for more details.
Interactive Queries were improved via KIP-763 KIP-766. Range queries now accept null
as lower/upper key-range bound to indicate an open-ended lower/upper bound.
Foreign-key table-table joins now support custom partitioners via KIP-775. Previously, if an input table was partitioned by a non-default partitioner, joining records might fail. With KIP-775 you now can pass a custom StreamPartitioner
into the join using the newly added TableJoined
object.
Streams API changes in 3.0.0
We improved the semantics of task idling (max.task.idle.ms
). Now Streams provides stronger in-order join and merge processing semantics. Streams’s new default pauses processing on tasks with multiple input partitions when one of the partitions has no data buffered locally but has a non-zero lag. In other words, Streams will wait to fetch records that are already available on the broker. This results in improved join semantics, since it allows Streams to interleave the two input partitions in timestamp order instead of just processing whichever partition happens to be buffered. There is an option to disable this new behavior, and there is also an option to make Streams wait even longer for new records to be produced to the input partitions, which you can use to get stronger time semantics when you know some of your producers may be slow. See the config reference for more information, and KIP-695 for the larger context of this change.
Interactive Queries may throw new exceptions for different errors:
UnknownStateStoreException
: If the specified store name does not exist in the topology, anUnknownStateStoreException
will be thrown instead of the formerInvalidStateStoreException
.StreamsNotStartedException
: If Streams state isCREATED
, aStreamsNotStartedException
will be thrown.InvalidStateStorePartitionException
: If the specified partition does not exist, aInvalidStateStorePartitionException
will be thrown.
See KIP-216 for more information.
We deprecated the StreamsConfig processing.guarantee
configuration value "exactly_once"
(for EOS version 1) in favor of the improved EOS version 2, formerly configured via "exactly_once_beta
. To avoid confusion about the term “beta” in the config name and highlight the production-readiness of EOS version 2, we have also renamed “eos-beta” to “eos-v2” and deprecated the configuration value "exactly_once_beta"
, replacing it with a new configuration value "exactly_once_v2"
Users of exactly-once semantics should plan to migrate to the eos-v2 config and prepare for the removal of the deprecated configs in 4.0 or after at least a year from the release of 3.0, whichever comes last. Note that eos-v2 requires broker version 2.5 or higher, like eos-beta, so users should begin to upgrade their kafka cluster if necessary. See KIP-732 for more details.
We removed the default implementation of RocksDBConfigSetter#close()
.
We dropped the default 24 hours grace period for windowed operations such as Window or Session aggregates, or stream-stream joins. This period determines how long after a window ends any out-of-order records will still be processed. Records coming in after the grace period has elapsed are considered late and will be dropped. But in operators such as suppression, a large grace period has the drawback of incurring an equally large output latency. The current API made it all too easy to miss the grace period config completely, leading you to wonder why your application seems to produce no output – it actually is, but not for 24 hours.
To prevent accidentally or unknowingly falling back to the default 24hr grace period, we deprecated all of the existing static constructors for the Windows
classes (such as TimeWindows#of
). These are replaced by new static constructors of two flavors: #ofSizeAndGrace
and #ofSizeWithNoGrace
(these are for the TimeWindows
class; analogous APIs exist for the JoinWindows
, SessionWindows
, and SlidingWindows classes). With these new APIs you are forced to set the grace period explicitly, or else consciously choose to opt out by selecting the WithNoGrace
flavor which sets it to 0 for situations where you really don’t care about the grace period, for example during testing or when playing around with Kafka Streams for the first time. Note that using the new APIs for the JoinWindows
class will also enable a fix for spurious left/outer join results, as described in the following paragraph. For more details on the grace period and new static constructors, see KIP-633
Additionally, in older versions Kafka Streams emitted stream-stream left/outer join results eagerly. This behavior may lead to spurious left/outer join result records. In this release, we changed the behavior to avoid spurious results and left/outer join result are only emitted after the join window is closed, i.e., after the grace period elapsed. To maintain backward compatibility, the old API JoinWindows#of(timeDifference)
preserves the old eager-emit behavior and only the new APIs JoinWindows#ofTimeDifferenceAndGrace()
and JoinsWindows#ofTimeDifferenceNoGrace
enable the new behavior. Check out KAFKA-10847 for more information.
The public topicGroupId
and partition
fields on TaskId have been deprecated and replaced with getters. Please migrate to using the new TaskId.subtopology()
(which replaces topicGroupId
) and TaskId.partition()
APIs instead. Also, the TaskId#readFrom
and TaskId#writeTo
methods have been deprecated and will be removed, as they were never intended for public use. We have also deprecated the org.apache.kafka.streams.processor.TaskMetadata
class and introduced a new interface org.apache.kafka.streams.TaskMetadata
to be used instead. This change was introduced to better reflect the fact that TaskMetadata
was not meant to be instantiated outside of Kafka codebase. Please note that the new TaskMetadata
offers APIs that better represent the task id as an actual TaskId
object instead of a String. Please migrate to the new org.apache.kafka.streams.TaskMetadata
which offers these better methods, for example, by using the new ThreadMetadata#activeTasks
and ThreadMetadata#standbyTasks
. org.apache.kafka.streams.processor.ThreadMetadata
class is also now deprecated and the newly introduced interface org.apache.kafka.streams.ThreadMetadata
is to be used instead. In this new ThreadMetadata
interface, any reference to the deprecated TaskMetadata
is replaced by the new interface. Finally, also org.apache.kafka.streams.state.StreamsMetadata
has been deprecated. Please migrate to the new org.apache.kafka.streams.StreamsMetadata
. We have deprecated several methods under org.apache.kafka.streams.KafkaStreams
that returned the aforementioned deprecated classes:
- Users of
KafkaStreams#allMetadata
are meant to migrate to the newKafkaStreams#metadataForAllStreamsClients
. - Users of
KafkaStreams#allMetadataForStore(String)
are meant to migrate to the newKafkaStreams#streamsMetadataForStore(String)
. - Users of
KafkaStreams#localThreadsMetadata
are meant to migrate to the newKafkaStreams#metadataForLocalThreads
.
See KIP-740 and KIP-744 for more details.
We removed the following deprecated APIs:
--zookeeper
flag of the application reset tool: deprecated in Kafka 1.0.0 (KIP-198).--execute
flag of the application reset tool: deprecated in Kafka 1.1.0 (KIP-171).StreamsBuilder#addGlobalStore
(one overload): deprecated in Kafka 1.1.0 (KIP-233).ProcessorContext#forward
(some overloads): deprecated in Kafka 2.0.0 (KIP-251).WindowBytesStoreSupplier#segments
: deprecated in Kafka 2.1.0 (KIP-319).segments, until, maintainMs
onTimeWindows
,JoinWindows
, andSessionWindows
: deprecated in Kafka 2.1.0 (KIP-328).- Overloaded
JoinWindows#of, before, after
,SessionWindows#with
,TimeWindows#of, advanceBy
,UnlimitedWindows#startOn
andKafkaStreams#close
withlong
typed parameters: deprecated in Kafka 2.1.0 (KIP-358). - Overloaded
KStream#groupBy, groupByKey
andKTable#groupBy
withSerialized
parameter: deprecated in Kafka 2.1.0 (KIP-372). Joined#named, name
: deprecated in Kafka 2.3.0 (KIP-307).TopologyTestDriver#pipeInput, readOutput
,OutputVerifier
andConsumerRecordFactory
classes (KIP-470).KafkaClientSupplier#getAdminClient
: deprecated in Kafka 2.4.0 (KIP-476).- Overloaded
KStream#join, leftJoin, outerJoin
withKStream
andJoined
parameters: deprecated in Kafka 2.4.0 (KIP-479). WindowStore#put(K key, V value)
: deprecated in Kafka 2.4.0 (KIP-474).UsePreviousTimeOnInvalidTimestamp
: deprecated in Kafka 2.5.0 as renamed toUsePartitionTimeOnInvalidTimestamp
(KIP-530).- Overloaded
KafkaStreams#metadataForKey
: deprecated in Kafka 2.5.0 (KIP-535). - Overloaded
KafkaStreams#store
: deprecated in Kafka 2.5.0 (KIP-562).
The following dependencies were removed from Kafka Streams:
- Connect-json: As of Kafka Streams no longer has a compile time dependency on “connect:json” module (KAFKA-5146). Projects that were relying on this transitive dependency will have to explicitly declare it.
The default value for configuration parameter replication.factor
was changed to -1
(meaning: use broker default replication factor). The replication.factor
value of -1
requires broker version 2.4 or newer.
The new serde type was introduced ListSerde
:
- Added class
ListSerde
to (de)serializeList
-based objects - Introduced
ListSerializer
andListDeserializer
to power the new functionality
Streams API changes in 2.8.0
We extended StreamJoined
to include the options withLoggingEnabled()
and withLoggingDisabled()
in KIP-689.
We added two new methods to KafkaStreams
, namely KafkaStreams#addStreamThread()
and KafkaStreams#removeStreamThread()
in KIP-663. These methods have enabled adding and removing StreamThreads to a running KafkaStreams client.
We deprecated KafkaStreams#setUncaughtExceptionHandler(final Thread.UncaughtExceptionHandler uncaughtExceptionHandler)
in favor of KafkaStreams#setUncaughtExceptionHandler(final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler)
in KIP-671. The default handler will close the Kafka Streams client and the client will transit to state ERROR. If you implement a custom handler, the new interface allows you to return a StreamThreadExceptionResponse
, which will determine how the application will respond to a stream thread failure.
Changes in KIP-663 necessitated the KafkaStreams client state machine to update, which was done in KIP-696. The ERROR state is now terminal with PENDING_ERROR being a transitional state where the resources are closing. The ERROR state indicates that there is something wrong and the Kafka Streams client should not be blindly restarted without classifying the error that caused the thread to fail. If the error is of a type that you would like to retry, you should have the StreamsUncaughtExceptionHandler
return REPLACE_THREAD
. When all stream threads are dead there is no automatic transition to ERROR as a new stream thread can be added.
The TimeWindowedDeserializer
constructor TimeWindowedDeserializer(final Deserializer inner)
was deprecated to encourage users to properly set their window size through TimeWindowedDeserializer(final Deserializer inner, Long windowSize)
. An additional streams config, window.size.ms
, was added for users that cannot set the window size through the constructor, such as when using the console consumer. KIP-659 has more details.
To simplify testing, two new constructors that don’t require a Properties
parameter have been added to the TopologyTestDriver
class. If Properties
are passed into the constructor, it is no longer required to set mandatory configuration parameters (cf. KIP-680).
We added the prefixScan()
method to interface ReadOnlyKeyValueStore
. The new prefixScan()
allows fetching all values whose keys start with a given prefix. See KIP-614 for more details.
Kafka Streams is now handling TimeoutException
thrown by the consumer, producer, and admin client. If a timeout occurs on a task, Kafka Streams moves to the next task and retries to make progress on the failed task in the next iteration. To bound how long Kafka Streams retries a task, you can set task.timeout.ms
(default is 5 minutes). If a task does not make progress within the specified task timeout, which is tracked on a per-task basis, Kafka Streams throws a TimeoutException
(cf. KIP-572).
We changed the default value of default.key.serde
and default.value.serde
to be null
instead of ByteArraySerde
. Users will now see a ConfigException
if their serdes are not correctly configured through those configs or passed in explicitly. See KIP-741 for more details.
Streams API changes in 2.7.0
In KeyQueryMetadata
we deprecated getActiveHost()
, getStandbyHosts()
as well as getPartition()
and replaced them with activeHost()
, standbyHosts()
and partition()
respectively. KeyQueryMetadata
was introduced in Kafka Streams 2.5 release with getter methods having prefix get
. The intend of this change is to bring the method names to Kafka custom to not use the get
prefix for getter methods. The old methods are deprecated and is not effected. (Cf. KIP-648.)
The StreamsConfig
variable for configuration parameter "topology.optimization"
is renamed from TOPOLOGY_OPTIMIZATION
to TOPOLOGY_OPTIMIZATION_CONFIG
. The old variable is deprecated. Note, that the parameter name itself is not affected. (Cf. KIP-629.)
The configuration parameter retries
is deprecated in favor of the new parameter task.timeout.ms
. Kafka Streams’ runtime ignores retries
if set, however, it would still forward the parameter to its internal clients.
We added SlidingWindows
as an option for windowedBy()
windowed aggregations as described in KIP-450. Sliding windows are fixed-time and data-aligned windows that allow for flexible and efficient windowed aggregations.
The end-to-end latency metrics introduced in 2.6 have been expanded to include store-level metrics. The new store-level metrics are recorded at the TRACE level, a new metrics recording level. Enabling TRACE level metrics will automatically turn on all higher levels, ie INFO and DEBUG. See KIP-613 for more information.
Streams API changes in 2.6.0
We added a new processing mode, EOS version 2, that improves application scalability using exactly-once guarantees (via KIP-447). You can enable this new feature by setting the configuration parameter processing.guarantee
to the new value "exactly_once_beta"
. Note that you need brokers with version 2.5 or newer to use this feature.
For more highly available stateful applications, we’ve modified the task assignment algorithm to delay the movement of stateful active tasks to instances that aren’t yet caught up with that task’s state. Instead, to migrate a task from one instance to another (eg when scaling out), Streams will assign a warmup replica to the target instance so it can begin restoring the state while the active task stays available on an instance that already had the task. The instances warming up tasks will communicate their progress to the group so that, once ready, Streams can move active tasks to their new owners in the background. Check out KIP-441 for full details, including several new configs for control over this new feature.
New end-to-end latency metrics have been added. These task-level metrics will be logged at the INFO level and report the min and max end-to-end latency of a record at the beginning/source node(s) and end/terminal node(s) of a task. See KIP-613 for more information.
As of 2.6.0 Kafka Streams deprecates KStream.through()
in favor of the new KStream.repartition()
operator (as per KIP-221). KStream.repartition()
is similar to KStream.through()
, however Kafka Streams will manage the topic for you. If you need to write into and read back from a topic that you mange, you can fall back to use KStream.to()
in combination with StreamsBuilder#stream()
. Please refer to the developer guide for more details about KStream.repartition()
.
The usability of StateStore
s within the Processor API is improved: ProcessorSupplier
and TransformerSupplier
now extend ConnectedStoreProvider
as per KIP-401, enabling a user to provide StateStore
s with alongside Processor/Transformer logic so that they are automatically added and connected to the processor.
We added a --force
option in StreamsResetter to force remove left-over members on broker side when long session time out was configured as per KIP-571.
We added Suppressed.withLoggingDisabled()
and Suppressed.withLoggingEnabled(config)
methods to allow disabling or configuring of the changelog topic and allows for configuration of the changelog topic as per KIP-446.
Streams API changes in 2.5.0
We add a new cogroup()
operator (via KIP-150) that allows to aggregate multiple streams in a single operation. Cogrouped streams can also be windowed before they are aggregated. Please refer to the developer guide for more details.
We added a new KStream.toTable()
API to translate an input event stream into a changelog stream as per KIP-523.
We added a new Serde type Void
in KIP-527 to represent null keys or null values from input topic.
Deprecated UsePreviousTimeOnInvalidTimestamp
and replaced it with UsePartitionTimeOnInvalidTimeStamp
as per KIP-530.
Deprecated KafkaStreams.store(String, QueryableStoreType)
and replaced it with KafkaStreams.store(StoreQueryParameters)
to allow querying for a store with variety of parameters, including querying a specific task and stale stores, as per KIP-562 and KIP-535 respectively.
Streams API changes in 2.4.0
As of 2.4.0 Kafka Streams offers a KTable-KTable foreign-key join (as per KIP-213). This joiner allows for records to be joined between two KTables with different keys. Both INNER and LEFT foreign-key joins are supported.
In the 2.4 release, you now can name all operators in a Kafka Streams DSL topology via KIP-307. Giving your operators meaningful names makes it easier to understand the topology description (Topology#describe()#toString()
) and understand the full context of what your Kafka Streams application is doing.
There are new overloads on most KStream
and KTable
methods that accept a Named
object. Typically you’ll provide a name for the DSL operation by using Named.as("my operator name")
. Naming of repartition topics for aggregation operations will still use Grouped
and join operations will use either Joined
or the new StreamJoined
object.
Before the 2.4.0 version of Kafka Streams, users of the DSL could not name the state stores involved in a stream-stream join. If users changed their topology and added a operator before the join, the internal names of the state stores would shift, requiring an application reset when redeploying. In the 2.4.0 release, Kafka Streams adds the StreamJoined
class, which gives users the ability to name the join processor, repartition topic(s) (if a repartition is required), and the state stores involved in the join. Also, by naming the state stores, the changelog topics backing the state stores are named as well. It’s important to note that naming the stores will not make them queryable via Interactive Queries.
Another feature delivered by StreamJoined
is that you can now configure the type of state store used in the join. You can elect to use in-memory stores or custom state stores for a stream-stream join. Note that the provided stores will not be available for querying via Interactive Queries. With the addition of StreamJoined
, stream-stream join operations using Joined
have been deprecated. Please switch over to stream-stream join methods using the new overloaded methods. You can get more details from KIP-479.
With the introduction of incremental cooperative rebalancing, Streams no longer requires all tasks be revoked at the beginning of a rebalance. Instead, at the completion of the rebalance only those tasks which are to be migrated to another consumer for overall load balance will need to be closed and revoked. This changes the semantics of the StateListener
a bit, as it will not necessarily transition to REBALANCING
at the beginning of a rebalance anymore. Note that this means IQ will now be available at all times except during state restoration, including while a rebalance is in progress. If restoration is occurring when a rebalance begins, we will continue to actively restore the state stores and/or process standby tasks during a cooperative rebalance. Note that with this new rebalancing protocol, you may sometimes see a rebalance be followed by a second short rebalance that ensures all tasks are safely distributed. For details on please see KIP-429.
The 2.4.0 release contains newly added and reworked metrics. KIP-444 adds new client level (i.e., KafkaStreams
instance level) metrics to the existing thread-level, task-level, and processor-/state-store-level metrics. For a full list of available client level metrics, see the KafkaStreams monitoring section in the operations guide.
Furthermore, RocksDB metrics are exposed via KIP-471. For a full list of available RocksDB metrics, see the RocksDB monitoring section in the operations guide.
Kafka Streams test-utils
got improved via KIP-470 to simplify the process of using TopologyTestDriver
to test your application code. We deprecated ConsumerRecordFactory
, TopologyTestDriver#pipeInput()
, OutputVerifier
, as well as TopologyTestDriver#readOutput()
and replace them with TestInputTopic
and TestOutputTopic
, respectively. We also introduced a new class TestRecord
that simplifies assertion code. For full details see the Testing section in the developer guide.
In 2.4.0, we deprecated WindowStore#put(K key, V value)
that should never be used. Instead the existing WindowStore#put(K key, V value, long windowStartTimestamp)
should be used (KIP-474).
Furthermore, the PartitionGrouper
interface and its corresponding configuration parameter partition.grouper
were deprecated (KIP-528) and will be removed in the next major release (KAFKA-7785. Hence, this feature won’t be supported in the future any longer and you need to updated your code accordingly. If you use a custom PartitionGrouper
and stop to use it, the created tasks might change. Hence, you will need to reset your application to upgrade it.
Streams API changes in 2.3.0
Version 2.3.0 adds the Suppress operator to the kafka-streams-scala
Ktable API.
As of 2.3.0 Streams now offers an in-memory version of the window (KIP-428) and the session (KIP-445) store, in addition to the persistent ones based on RocksDB. The new public interfaces inMemoryWindowStore()
and inMemorySessionStore()
are added to Stores
and provide the built-in in-memory window or session store.
As of 2.3.0 we’ve updated how to turn on optimizations. Now to enable optimizations, you need to do two things. First add this line to your properties properties.setProperty(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE);
, as you have done before. Second, when constructing your KafkaStreams
instance, you’ll need to pass your configuration properties when building your topology by using the overloaded StreamsBuilder.build(Properties)
method. For example KafkaStreams myStream = new KafkaStreams(streamsBuilder.build(properties), properties)
.
In 2.3.0 we have added default implementation to close()
and configure()
for Serializer
, Deserializer
and Serde
so that they can be implemented by lambda expression. For more details please read KIP-331.
To improve operator semantics, new store types are added that allow storing an additional timestamp per key-value pair or window. Some DSL operators (for example KTables) are using those new stores. Hence, you can now retrieve the last update timestamp via Interactive Queries if you specify TimestampedKeyValueStoreType
or TimestampedWindowStoreType
as your QueryableStoreType
. While this change is mainly transparent, there are some corner cases that may require code changes: Caution: If you receive an untyped store and use a cast, you might need to update your code to cast to the correct type. Otherwise, you might get an exception similar tojava.lang.ClassCastException: class org.apache.kafka.streams.state.ValueAndTimestamp cannot be cast to class YOUR-VALUE-TYPE
upon getting a value from the store. Additionally, TopologyTestDriver#getStateStore()
only returns non-built-in stores and throws an exception if a built-in store is accessed. For more details please read KIP-258.
To improve type safety, a new operator KStream#flatTransformValues
is added. For more details please read KIP-313.
Kafka Streams used to set the configuration parameter max.poll.interval.ms
to Integer.MAX_VALUE
. This default value is removed and Kafka Streams uses the consumer default value now. For more details please read KIP-442.
Default configuration for repartition topic was changed: The segment size for index files (segment.index.bytes
) is no longer 50MB, but uses the cluster default. Similarly, the configuration segment.ms
in no longer 10 minutes, but uses the cluster default configuration. Lastly, the retention period (retention.ms
) is changed from Long.MAX_VALUE
to -1
(infinite). For more details please read KIP-443.
To avoid memory leaks, RocksDBConfigSetter
has a new close()
method that is called on shutdown. Users should implement this method to release any memory used by RocksDB config objects, by closing those objects. For more details please read KIP-453.
RocksDB dependency was updated to version 5.18.3
. The new version allows to specify more RocksDB configurations, including WriteBufferManager
which helps to limit RocksDB off-heap memory usage. For more details please read KAFKA-8215.
Streams API changes in 2.2.0
We’ve simplified the KafkaStreams#state
transition diagram during the starting up phase a bit in 2.2.0: in older versions the state will transit from CREATED
to RUNNING
, and then to REBALANCING
to get the first stream task assignment, and then back to RUNNING
; starting in 2.2.0 it will transit from CREATED
directly to REBALANCING
and then to RUNNING
. If you have registered a StateListener
that captures state transition events, you may need to adjust your listener implementation accordingly for this simplification (in practice, your listener logic should be very unlikely to be affected at all).
In WindowedSerdes
, we’ve added a new static constructor to return a TimeWindowSerde
with configurable window size. This is to help users to construct time window serdes to read directly from a time-windowed store’s changelog. More details can be found in KIP-393.
In 2.2.0 we have extended a few public interfaces including KafkaStreams
to extend AutoCloseable
so that they can be used in a try-with-resource statement. For a full list of public interfaces that get impacted please read KIP-376.
Streams API changes in 2.1.0
We updated TopologyDescription
API to allow for better runtime checking. Users are encouraged to use #topicSet()
and #topicPattern()
accordingly on TopologyDescription.Source
nodes, instead of using #topics()
, which has since been deprecated. Similarly, use #topic()
and #topicNameExtractor()
to get descriptions of TopologyDescription.Sink
nodes. For more details, see KIP-321.
We’ve added a new class Grouped
and deprecated Serialized
. The intent of adding Grouped
is the ability to name repartition topics created when performing aggregation operations. Users can name the potential repartition topic using the Grouped#as()
method which takes a String
and is used as part of the repartition topic name. The resulting repartition topic name will still follow the pattern of ${application-id}->name<-repartition
. The Grouped
class is now favored over Serialized
in KStream#groupByKey()
, KStream#groupBy()
, and KTable#groupBy()
. Note that Kafka Streams does not automatically create repartition topics for aggregation operations. Additionally, we’ve updated the Joined
class with a new method Joined#withName
enabling users to name any repartition topics required for performing Stream/Stream or Stream/Table join. For more details repartition topic naming, see KIP-372. As a result we’ve updated the Kafka Streams Scala API and removed the Serialized
class in favor of adding Grouped
. If you just rely on the implicit Serialized
, you just need to recompile; if you pass in Serialized
explicitly, sorry you’ll have to make code changes.
We’ve added a new config named max.task.idle.ms
to allow users specify how to handle out-of-order data within a task that may be processing multiple topic-partitions (see Out-of-Order Handling section for more details). The default value is set to 0
, to favor minimized latency over synchronization between multiple input streams from topic-partitions. If users would like to wait for longer time when some of the topic-partitions do not have data available to process and hence cannot determine its corresponding stream time, they can override this config to a larger value.
We’ve added the missing SessionBytesStoreSupplier#retentionPeriod()
to be consistent with the WindowBytesStoreSupplier
which allows users to get the specified retention period for session-windowed stores. We’ve also added the missing StoreBuilder#withCachingDisabled()
to allow users to turn off caching for their customized stores.
We added a new serde for UUIDs (Serdes.UUIDSerde
) that you can use via Serdes.UUID()
(cf. KIP-206).
We updated a list of methods that take long
arguments as either timestamp (fix point) or duration (time period) and replaced them with Instant
and Duration
parameters for improved semantics. Some old methods base on long
are deprecated and users are encouraged to update their code.
In particular, aggregation windows (hopping/tumbling/unlimited time windows and session windows) as well as join windows now take Duration
arguments to specify window size, hop, and gap parameters. Also, window sizes and retention times are now specified as Duration
type in Stores
class. The Window
class has new methods #startTime()
and #endTime()
that return window start/end timestamp as Instant
. For interactive queries, there are new #fetch(...)
overloads taking Instant
arguments. Additionally, punctuations are now registered via ProcessorContext#schedule(Duration interval, ...)
. For more details, see KIP-358.
We deprecated KafkaStreams#close(...)
and replaced it with KafkaStreams#close(Duration)
that accepts a single timeout argument Note: the new #close(Duration)
method has improved (but slightly different) semantics. For more details, see KIP-358.
The newly exposed AdminClient
metrics are now available when calling the KafkaStream#metrics()
method. For more details on exposing AdminClients
metrics see KIP-324
We deprecated the notion of segments in window stores as those are intended to be an implementation details. Thus, method Windows#segments()
and variable Windows#segments
were deprecated. If you implement custom windows, you should update your code accordingly. Similarly, WindowBytesStoreSupplier#segments()
was deprecated and replaced with WindowBytesStoreSupplier#segmentInterval()
. If you implement custom window store, you need to update your code accordingly. Finally, Stores#persistentWindowStore(...)
were deprecated and replaced with a new overload that does not allow to specify the number of segments any longer. For more details, see KIP-319 (note: KIP-328 and KIP-358 ‘overlap’ with KIP-319).
We’ve added an overloaded StreamsBuilder#build
method that accepts an instance of java.util.Properties
with the intent of using the StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG
config added in Kafka Streams 2.0. Before 2.1, when building a topology with the DSL, Kafka Streams writes the physical plan as the user makes calls on the DSL. Now by providing a java.util.Properties
instance when executing a StreamsBuilder#build
call, Kafka Streams can optimize the physical plan of the topology, provided the StreamsConfig#TOPOLOGY_OPTIMIZATION_CONFIG
config is set to StreamsConfig#OPTIMIZE
. By setting StreamsConfig#OPTIMIZE
in addition to the KTable
optimization of reusing the source topic as the changelog topic, the topology may be optimized to merge redundant repartition topics into one repartition topic. The original no parameter version of StreamsBuilder#build
is still available for those who wish to not optimize their topology. Note that enabling optimization of the topology may require you to do an application reset when redeploying the application. For more details, see KIP-312
We are introducing static membership towards Kafka Streams user. This feature reduces unnecessary rebalances during normal application upgrades or rolling bounces. For more details on how to use it, checkout static membership design. Note, Kafka Streams uses the same ConsumerConfig#GROUP_INSTANCE_ID_CONFIG
, and you only need to make sure it is uniquely defined across different stream instances in one application.
Streams API changes in 2.0.0
In 2.0.0 we have added a few new APIs on the ReadOnlyWindowStore
interface (for details please read Streams API changes below). If you have customized window store implementations that extends the ReadOnlyWindowStore
interface you need to make code changes.
In addition, if you using Java 8 method references in your Kafka Streams code you might need to update your code to resolve method ambiguities. Hot-swapping the jar-file only might not work for this case. See below a complete list of 2.0.0 API and semantic changes that allow you to advance your application and/or simplify your code base.
We moved Consumed
interface from org.apache.kafka.streams
to org.apache.kafka.streams.kstream
as it was mistakenly placed in the previous release. If your code has already used it there is a simple one-liner change needed in your import statement.
We have also removed some public APIs that are deprecated prior to 1.0.x in 2.0.0. See below for a detailed list of removed APIs.
We have removed the skippedDueToDeserializationError-rate
and skippedDueToDeserializationError-total
metrics. Deserialization errors, and all other causes of record skipping, are now accounted for in the pre-existing metrics skipped-records-rate
and skipped-records-total
. When a record is skipped, the event is now logged at WARN level. If these warnings become burdensome, we recommend explicitly filtering out unprocessable records instead of depending on record skipping semantics. For more details, see KIP-274. As of right now, the potential causes of skipped records are:
null
keys in table sourcesnull
keys in table-table inner/left/outer/right joinsnull
keys or values in stream-table joinsnull
keys or values in stream-stream joinsnull
keys or values in aggregations on grouped streamsnull
keys or values in reductions on grouped streamsnull
keys in aggregations on windowed streamsnull
keys in reductions on windowed streamsnull
keys in aggregations on session-windowed streams- Errors producing results, when the configured
default.production.exception.handler
decides toCONTINUE
(the default is toFAIL
and throw an exception). - Errors deserializing records, when the configured
default.deserialization.exception.handler
decides toCONTINUE
(the default is toFAIL
and throw an exception). This was the case previously captured in theskippedDueToDeserializationError
metrics. - Fetched records having a negative timestamp.
We’ve also fixed the metrics name for time and session windowed store operations in 2.0. As a result, our current built-in stores will have their store types in the metric names as in-memory-state
, in-memory-lru-state
, rocksdb-state
, rocksdb-window-state
, and rocksdb-session-state
. For example, a RocksDB time windowed store’s put operation metrics would now be kafka.streams:type=stream-rocksdb-window-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),rocksdb-window-state-id=([-.\w]+)
. Users need to update their metrics collecting and reporting systems for their time and session windowed stores accordingly. For more details, please read the State Store Metrics section.
We have added support for methods in ReadOnlyWindowStore
which allows for querying a single window’s key-value pair. For users who have customized window store implementations on the above interface, they’d need to update their code to implement the newly added method as well. For more details, see KIP-261.
We have added public WindowedSerdes
to allow users to read from / write to a topic storing windowed table changelogs directly. In addition, in StreamsConfig
we have also added default.windowed.key.serde.inner
and default.windowed.value.serde.inner
to let users specify inner serdes if the default serde classes are windowed serdes. For more details, see KIP-265.
We’ve added message header support in the Processor API
in Kafka 2.0.0. In particular, we have added a new API ProcessorContext#headers()
which returns a Headers
object that keeps track of the headers of the source topic’s message that is being processed. Through this object, users can manipulate the headers map that is being propagated throughout the processor topology as well. For more details please feel free to read the Developer Guide section.
We have deprecated constructors of KafkaStreams
that take a StreamsConfig
as parameter. Please use the other corresponding constructors that accept java.util.Properties
instead. For more details, see KIP-245.
Kafka 2.0.0 allows to manipulate timestamps of output records using the Processor API (KIP-251). To enable this new feature, ProcessorContext#forward(...)
was modified. The two existing overloads #forward(Object key, Object value, String childName)
and #forward(Object key, Object value, int childIndex)
were deprecated and a new overload #forward(Object key, Object value, To to)
was added. The new class To
allows you to send records to all or specific downstream processors by name and to set the timestamp for the output record. Forwarding based on child index is not supported in the new API any longer.
We have added support to allow routing records dynamically to Kafka topics. More specifically, in both the lower-level Topology#addSink
and higher-level KStream#to
APIs, we have added variants that take a TopicNameExtractor
instance instead of a specific String
typed topic name, such that for each received record from the upstream processor, the library will dynamically determine which Kafka topic to write to based on the record’s key and value, as well as record context. Note that all the Kafka topics that may possibly be used are still considered as user topics and hence required to be pre-created. In addition to that, we have modified the StreamPartitioner
interface to add the topic name parameter since the topic name now may not be known beforehand; users who have customized implementations of this interface would need to update their code while upgrading their application to use Kafka Streams 2.0.0.
KIP-284 changed the retention time for repartition topics by setting its default value to Long.MAX_VALUE
. Instead of relying on data retention Kafka Streams uses the new purge data API to delete consumed data from those topics and to keep used storage small now.
We have modified the ProcessorStateManger#register(...)
signature and removed the deprecated loggingEnabled
boolean parameter as it is specified in the StoreBuilder
. Users who used this function to register their state stores into the processor topology need to simply update their code and remove this parameter from the caller.
Kafka Streams DSL for Scala is a new Kafka Streams client library available for developers authoring Kafka Streams applications in Scala. It wraps core Kafka Streams DSL types to make it easier to call when interoperating with Scala code. For example, it includes higher order functions as parameters for transformations avoiding the need anonymous classes in Java 7 or experimental SAM type conversions in Scala 2.11, automatic conversion between Java and Scala collection types, a way to implicitly provide Serdes to reduce boilerplate from your application and make it more typesafe, and more! For more information see the Kafka Streams DSL for Scala documentation and KIP-270.
We have removed these deprecated APIs:
KafkaStreams#toString
no longer returns the topology and runtime metadata; to get topology metadata users can callTopology#describe()
and to get thread runtime metadata users can callKafkaStreams#localThreadsMetadata
(they are deprecated since 1.0.0). For detailed guidance on how to update your code please read hereTopologyBuilder
andKStreamBuilder
are removed and replaced byTopology
andStreamsBuidler
respectively (they are deprecated since 1.0.0). For detailed guidance on how to update your code please read hereStateStoreSupplier
are removed and replaced withStoreBuilder
(they are deprecated since 1.0.0); and the correspondingStores#create
andKStream, KTable, KGroupedStream
overloaded functions that use it have also been removed. For detailed guidance on how to update your code please read hereKStream, KTable, KGroupedStream
overloaded functions that requires serde and other specifications explicitly are removed and replaced with simpler overloaded functions that useConsumed, Produced, Serialized, Materialized, Joined
(they are deprecated since 1.0.0). For detailed guidance on how to update your code please read hereProcessor#punctuate
,ValueTransformer#punctuate
,ValueTransformer#punctuate
andProcessorContext#schedule(long)
are removed and replaced byProcessorContext#schedule(long, PunctuationType, Punctuator)
(they are deprecated in 1.0.0).- The second
boolean
typed parameter “loggingEnabled” inProcessorContext#register
has been removed; users can now useStoreBuilder#withLoggingEnabled, withLoggingDisabled
to specify the behavior when they create the state store. KTable#writeAs, print, foreach, to, through
are removed, users can callKTable#tostream()#writeAs
instead for the same purpose (they are deprecated since 0.11.0.0). For detailed list of removed APIs please read hereStreamsConfig#KEY_SERDE_CLASS_CONFIG, VALUE_SERDE_CLASS_CONFIG, TIMESTAMP_EXTRACTOR_CLASS_CONFIG
are removed and replaced withStreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG, DEFAULT_VALUE_SERDE_CLASS_CONFIG, DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG
respectively (they are deprecated since 0.11.0.0).StreamsConfig#ZOOKEEPER_CONNECT_CONFIG
are removed as we do not need ZooKeeper dependency in Streams any more (it is deprecated since 0.10.2.0).
Streams API changes in 1.1.0
We have added support for methods in ReadOnlyWindowStore
which allows for querying WindowStore
s without the necessity of providing keys. For users who have customized window store implementations on the above interface, they’d need to update their code to implement the newly added method as well. For more details, see KIP-205.
There is a new artifact kafka-streams-test-utils
providing a TopologyTestDriver
, ConsumerRecordFactory
, and OutputVerifier
class. You can include the new artifact as a regular dependency to your unit tests and use the test driver to test your business logic of your Kafka Streams application. For more details, see KIP-247.
The introduction of KIP-220 enables you to provide configuration parameters for the embedded admin client created by Kafka Streams, similar to the embedded producer and consumer clients. You can provide the configs via StreamsConfig
by adding the configs with the prefix admin.
as defined by StreamsConfig#adminClientPrefix(String)
to distinguish them from configurations of other clients that share the same config names.
New method in KTable
transformValues
methods have been added toKTable
. Similar to those onKStream
, these methods allow for richer, stateful, value transformation similar to the Processor API.
New method in GlobalKTable
- A method has been provided such that it will return the store name associated with the
GlobalKTable
ornull
if the store name is non-queryable.
New methods in KafkaStreams
:
- added overload for the constructor that allows overriding the
Time
object used for tracking system wall-clock time; this is useful for unit testing your application code.
New methods in KafkaClientSupplier
:
- added
getAdminClient(config)
that allows to override anAdminClient
used for administrative requests such as internal topic creations, etc.
New error handling for exceptions during production:
- added interface
ProductionExceptionHandler
that allows implementors to decide whether or not Streams shouldFAIL
orCONTINUE
when certain exception occur while trying to produce. - provided an implementation,
DefaultProductionExceptionHandler
that always fails, preserving the existing behavior by default. - changing which implementation is used can be done by settings
default.production.exception.handler
to the fully qualified name of a class implementing this interface.
Changes in StreamsResetter
:
- added options to specify input topics offsets to reset according to KIP-171
Streams API changes in 1.0.0
With 1.0 a major API refactoring was accomplished and the new API is cleaner and easier to use. This change includes the five main classes KafkaStreams
, KStreamBuilder
, KStream
, KTable
, and TopologyBuilder
(and some more others). All changes are fully backward compatible as old API is only deprecated but not removed. We recommend to move to the new API as soon as you can. We will summarize all API changes in the next paragraphs.
The two main classes to specify a topology via the DSL (KStreamBuilder
) or the Processor API (TopologyBuilder
) were deprecated and replaced by StreamsBuilder
and Topology
(both new classes are located in package org.apache.kafka.streams
). Note, that StreamsBuilder
does not extend Topology
, i.e., the class hierarchy is different now. The new classes have basically the same methods as the old ones to build a topology via DSL or Processor API. However, some internal methods that were public in KStreamBuilder
and TopologyBuilder
but not part of the actual API are not present in the new classes any longer. Furthermore, some overloads were simplified compared to the original classes. See KIP-120 and KIP-182 for full details.
Changing how a topology is specified also affects KafkaStreams
constructors, that now only accept a Topology
. Using the DSL builder class StreamsBuilder
one can get the constructed Topology
via StreamsBuilder#build()
. Additionally, a new class org.apache.kafka.streams.TopologyDescription
(and some more dependent classes) were added. Those can be used to get a detailed description of the specified topology and can be obtained by calling Topology#describe()
. An example using this new API is shown in the quickstart section.
New methods in KStream
:
- With the introduction of KIP-202 a new method
merge()
has been created inKStream
as the StreamsBuilder class’sStreamsBuilder#merge()
has been removed. The method signature was also changed, too: instead of providing multipleKStream
s into the method at the once, only a singleKStream
is accepted.
New methods in KafkaStreams
:
- retrieve the current runtime information about the local threads via
localThreadsMetadata()
- observe the restoration of all state stores via
setGlobalStateRestoreListener()
, in which users can provide their customized implementation of theorg.apache.kafka.streams.processor.StateRestoreListener
interface
Deprecated / modified methods in KafkaStreams
:
toString()
,toString(final String indent)
were previously used to return static and runtime information. They have been deprecated in favor of using the new classes/methodslocalThreadsMetadata()
/ThreadMetadata
(returning runtime information) andTopologyDescription
/Topology#describe()
(returning static information).- With the introduction of KIP-182 you should no longer pass in
Serde
toKStream#print
operations. If you can’t rely on usingtoString
to print your keys an values, you should instead you provide a customKeyValueMapper
via thePrinted#withKeyValueMapper
call. setStateListener()
now can only be set before the application start running, i.e. beforeKafkaStreams.start()
is called.
Deprecated methods in KGroupedStream
- Windowed aggregations have been deprecated from
KGroupedStream
and moved toWindowedKStream
. You can now perform a windowed aggregation by, for example, usingKGroupedStream#windowedBy(Windows)#reduce(Reducer)
.
Modified methods in Processor
:
- The Processor API was extended to allow users to schedule
punctuate
functions either based on data-driven stream time or wall-clock time. As a result, the originalProcessorContext#schedule
is deprecated with a new overloaded function that accepts a user customizablePunctuator
callback interface, which triggers itspunctuate
API method periodically based on thePunctuationType
. ThePunctuationType
determines what notion of time is used for the punctuation scheduling: either stream time or wall-clock time (by default, stream time is configured to represent event time viaTimestampExtractor
). In addition, thepunctuate
function insideProcessor
is also deprecated.
Before this, users could only schedule based on stream time (i.e. PunctuationType.STREAM_TIME
) and hence the punctuate
function was data-driven only because stream time is determined (and advanced forward) by the timestamps derived from the input data. If there is no data arriving at the processor, the stream time would not advance and hence punctuation will not be triggered. On the other hand, When wall-clock time (i.e. PunctuationType.WALL_CLOCK_TIME
) is used, punctuate
will be triggered purely based on wall-clock time. So for example if the Punctuator
function is scheduled based on PunctuationType.WALL_CLOCK_TIME
, if these 60 records were processed within 20 seconds, punctuate
would be called 2 times (one time every 10 seconds); if these 60 records were processed within 5 seconds, then no punctuate
would be called at all. Users can schedule multiple Punctuator
callbacks with different PunctuationType
s within the same processor by simply calling ProcessorContext#schedule
multiple times inside processor’s init()
method.
If you are monitoring on task level or processor-node / state store level Streams metrics, please note that the metrics sensor name and hierarchy was changed: The task ids, store names and processor names are no longer in the sensor metrics names, but instead are added as tags of the sensors to achieve consistent metrics hierarchy. As a result you may need to make corresponding code changes on your metrics reporting and monitoring tools when upgrading to 1.0.0. Detailed metrics sensor can be found in the Streams Monitoring section.
The introduction of KIP-161 enables you to provide a default exception handler for deserialization errors when reading data from Kafka rather than throwing the exception all the way out of your streams application. You can provide the configs via the StreamsConfig
as StreamsConfig#DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG
. The specified handler must implement the org.apache.kafka.streams.errors.DeserializationExceptionHandler
interface.
The introduction of KIP-173 enables you to provide topic configuration parameters for any topics created by Kafka Streams. This includes repartition and changelog topics. You can provide the configs via the StreamsConfig
by adding the configs with the prefix as defined by StreamsConfig#topicPrefix(String)
. Any properties in the StreamsConfig
with the prefix will be applied when creating internal topics. Any configs that aren’t topic configs will be ignored. If you already use StateStoreSupplier
or Materialized
to provide configs for changelogs, then they will take precedence over those supplied in the config.
Streams API changes in 0.11.0.0
Updates in StreamsConfig
:
- new configuration parameter
processing.guarantee
is added - configuration parameter
key.serde
was deprecated and replaced bydefault.key.serde
- configuration parameter
value.serde
was deprecated and replaced bydefault.value.serde
- configuration parameter
timestamp.extractor
was deprecated and replaced bydefault.timestamp.extractor
- method
keySerde()
was deprecated and replaced bydefaultKeySerde()
- method
valueSerde()
was deprecated and replaced bydefaultValueSerde()
- new method
defaultTimestampExtractor()
was added
New methods in TopologyBuilder
:
- added overloads for
addSource()
that allow to define aTimestampExtractor
per source node - added overloads for
addGlobalStore()
that allow to define aTimestampExtractor
per source node associated with the global store
New methods in KStreamBuilder
:
- added overloads for
stream()
that allow to define aTimestampExtractor
per input stream - added overloads for
table()
that allow to define aTimestampExtractor
per input table - added overloads for
globalKTable()
that allow to define aTimestampExtractor
per global table
Deprecated methods in KTable
:
void foreach(final ForeachAction<? super K, ? super V> action)
void print()
void print(final String streamName)
void print(final Serde<K> keySerde, final Serde<V> valSerde)
void print(final Serde<K> keySerde, final Serde<V> valSerde, final String streamName)
void writeAsText(final String filePath)
void writeAsText(final String filePath, final String streamName)
void writeAsText(final String filePath, final Serde<K> keySerde, final Serde<V> valSerde)
void writeAsText(final String filePath, final String streamName, final Serde<K> keySerde, final Serde<V> valSerde)
The above methods have been deprecated in favor of using the Interactive Queries API. If you want to query the current content of the state store backing the KTable, use the following approach:
- Make a call to
KafkaStreams.store(final String storeName, final QueryableStoreType<T> queryableStoreType)
- Then make a call to
ReadOnlyKeyValueStore.all()
to iterate over the keys of aKTable
.
If you want to view the changelog stream of the KTable
then you could call KTable.toStream().print(Printed.toSysOut)
.
Metrics using exactly-once semantics:
If "exactly_once"
processing (EOS version 1) is enabled via the processing.guarantee
parameter, internally Streams switches from a producer-per-thread to a producer-per-task runtime model. Using "exactly_once_beta"
(EOS version 2) does use a producer-per-thread, so client.id
doesn’t change, compared with "at_least_once"
for this case). In order to distinguish the different producers, the producer’s client.id
additionally encodes the task-ID for this case. Because the producer’s client.id
is used to report JMX metrics, it might be required to update tools that receive those metrics.
Producer’s client.id
naming schema:
- at-least-once (default):
[client.Id]-StreamThread-[sequence-number]
- exactly-once:
[client.Id]-StreamThread-[sequence-number]-[taskId]
- exactly-once-beta:
[client.Id]-StreamThread-[sequence-number]
[client.Id]
is either set via Streams configuration parameter client.id
or defaults to [application.id]-[processId]
([processId]
is a random UUID).
Notable changes in 0.10.2.1
Parameter updates in StreamsConfig
:
- The default config values of embedded producer’s
retries
and consumer’smax.poll.interval.ms
have been changed to improve the resiliency of a Kafka Streams application
Streams API changes in 0.10.2.0
New methods in KafkaStreams
:
- set a listener to react on application state change via
setStateListener(StateListener listener)
- retrieve the current application state via
state()
- retrieve the global metrics registry via
metrics()
- apply a timeout when closing an application via
close(long timeout, TimeUnit timeUnit)
- specify a custom indent when retrieving Kafka Streams information via
toString(String indent)
Parameter updates in StreamsConfig
:
- parameter
zookeeper.connect
was deprecated; a Kafka Streams application does no longer interact with ZooKeeper for topic management but uses the new broker admin protocol (cf. KIP-4, Section “Topic Admin Schema”) - added many new parameters for metrics, security, and client configurations
Changes in StreamsMetrics
interface:
- removed methods:
addLatencySensor()
- added methods:
addLatencyAndThroughputSensor()
,addThroughputSensor()
,recordThroughput()
,addSensor()
,removeSensor()
New methods in TopologyBuilder
:
- added overloads for
addSource()
that allow to define aauto.offset.reset
policy per source node - added methods
addGlobalStore()
to add globalStateStore
s
New methods in KStreamBuilder
:
- added overloads for
stream()
andtable()
that allow to define aauto.offset.reset
policy per input stream/table - added method
globalKTable()
to create aGlobalKTable
New joins for KStream
:
- added overloads for
join()
to join withKTable
- added overloads for
join()
andleftJoin()
to join withGlobalKTable
- note, join semantics in 0.10.2 were improved and thus you might see different result compared to 0.10.0.x and 0.10.1.x (cf. Kafka Streams Join Semantics in the Apache Kafka wiki)
Aligned null
-key handling for KTable
joins:
- like all other KTable operations,
KTable-KTable
joins do not throw an exception onnull
key records anymore, but drop those records silently
New window type Session Windows :
- added class
SessionWindows
to specify session windows - added overloads for
KGroupedStream
methodscount()
,reduce()
, andaggregate()
to allow session window aggregations
Changes to TimestampExtractor
:
- method
extract()
has a second parameter now - new default timestamp extractor class
FailOnInvalidTimestamp
(it gives the same behavior as old (and removed) default extractorConsumerRecordTimestampExtractor
) - new alternative timestamp extractor classes
LogAndSkipOnInvalidTimestamp
andUsePreviousTimeOnInvalidTimestamps
Relaxed type constraints of many DSL interfaces, classes, and methods (cf. KIP-100).
Streams API changes in 0.10.1.0
Stream grouping and aggregation split into two methods:
- old: KStream #aggregateByKey(), #reduceByKey(), and #countByKey()
- new: KStream#groupByKey() plus KGroupedStream #aggregate(), #reduce(), and #count()
- Example: stream.countByKey() changes to stream.groupByKey().count()
Auto Repartitioning:
- a call to through() after a key-changing operator and before an aggregation/join is no longer required
- Example: stream.selectKey(…).through(…).countByKey() changes to stream.selectKey().groupByKey().count()
TopologyBuilder:
- methods #sourceTopics(String applicationId) and #topicGroups(String applicationId) got simplified to #sourceTopics() and #topicGroups()
DSL: new parameter to specify state store names:
- The new Interactive Queries feature requires to specify a store name for all source KTables and window aggregation result KTables (previous parameter “operator/window name” is now the storeName)
- KStreamBuilder#table(String topic) changes to #topic(String topic, String storeName)
- KTable#through(String topic) changes to #through(String topic, String storeName)
- KGroupedStream #aggregate(), #reduce(), and #count() require additional parameter “String storeName”
- Example: stream.countByKey(TimeWindows.of(“windowName”, 1000)) changes to stream.groupByKey().count(TimeWindows.of(1000), “countStoreName”)
Windowing:
- Windows are not named anymore: TimeWindows.of(“name”, 1000) changes to TimeWindows.of(1000) (cf. DSL: new parameter to specify state store names)
- JoinWindows has no default size anymore: JoinWindows.of(“name”).within(1000) changes to JoinWindows.of(1000)
Streams API broker compatibility
The following table shows which versions of the Kafka Streams API are compatible with various Kafka broker versions. For Kafka Stream version older than 2.4.x, please check 3.9 upgrade document.
Kafka Broker (columns) | |
---|---|
Kafka Streams API (rows) | 2.1.x and |
2.2.x and | |
2.3.x and | |
2.4.x and | |
2.5.x and | |
2.6.x and | |
2.7.x and | |
2.8.x and | |
3.0.x and | |
3.1.x and | |
3.2.x and | |
3.3.x and | |
3.4.x and | |
3.5.x and | |
3.6.x and | |
3.7.x and | |
3.8.x and | |
3.9.x | 4.0.x |
2.4.x and | |
2.5.x | compatible |
2.6.x and | |
2.7.x and | |
2.8.x and | |
3.0.x and | |
3.1.x and | |
3.2.x and | |
3.3.x and | |
3.4.x and | |
3.5.x and | |
3.6.x and | |
3.7.x and | |
3.8.x and | |
3.9.x and | |
4.0.x | compatible; enabling exactly-once v2 requires broker version 2.5.x or higher |
Previous Next
7 - Streams Developer Guide
7.1 - Writing a Streams Application
Writing a Streams Application
Table of Contents
- Libraries and Maven artifacts
- Using Kafka Streams within your application code
- Testing a Streams application
Any Java or Scala application that makes use of the Kafka Streams library is considered a Kafka Streams application. The computational logic of a Kafka Streams application is defined as a processor topology, which is a graph of stream processors (nodes) and streams (edges).
You can define the processor topology with the Kafka Streams APIs:
Kafka Streams DSL
A high-level API that provides the most common data transformation operations such as map
, filter
, join
, and aggregations
out of the box. The DSL is the recommended starting point for developers new to Kafka Streams, and should cover many use cases and stream processing needs. If you’re writing a Scala application then you can use the Kafka Streams DSL for Scala library which removes much of the Java/Scala interoperability boilerplate as opposed to working directly with the Java DSL.
Processor API
A low-level API that lets you add and connect processors as well as interact directly with state stores. The Processor API provides you with even more flexibility than the DSL but at the expense of requiring more manual work on the side of the application developer (e.g., more lines of code).
Libraries and Maven artifacts
This section lists the Kafka Streams related libraries that are available for writing your Kafka Streams applications.
You can define dependencies on the following libraries for your Kafka Streams applications.
Group ID | Artifact ID | Version | Description |
---|---|---|---|
org.apache.kafka | kafka-streams | 4.0.0 | (Required) Base library for Kafka Streams. |
org.apache.kafka | kafka-clients | 4.0.0 | (Required) Kafka client library. Contains built-in serializers/deserializers. |
org.apache.kafka | kafka-streams-scala | 4.0.0 | (Optional) Kafka Streams DSL for Scala library to write Scala Kafka Streams applications. When not using SBT you will need to suffix the artifact ID with the correct version of Scala your application is using (_2.12 , _2.13 ) |
Tip
See the section Data Types and Serialization for more information about Serializers/Deserializers.
Example pom.xml
snippet when using Maven:
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-streams</artifactId>
<version>4.0.0</version>
</dependency>
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-clients</artifactId>
<version>4.0.0</version>
</dependency>
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-streams-scala_2.13</artifactId>
<version>4.0.0</version>
</dependency>
Using Kafka Streams within your application code
You can call Kafka Streams from anywhere in your application code, but usually these calls are made within the main()
method of your application, or some variant thereof. The basic elements of defining a processing topology within your application are described below.
First, you must create an instance of KafkaStreams
.
- The first argument of the
KafkaStreams
constructor takes a topology (eitherStreamsBuilder#build()
for the DSL orTopology
for the Processor API) that is used to define a topology. - The second argument is an instance of
java.util.Properties
, which defines the configuration for this specific topology.
Code example:
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.kstream.StreamsBuilder;
import org.apache.kafka.streams.processor.Topology;
// Use the builders to define the actual processing topology, e.g. to specify
// from which input topics to read, which stream operations (filter, map, etc.)
// should be called, and so on. We will cover this in detail in the subsequent
// sections of this Developer Guide.
StreamsBuilder builder = ...; // when using the DSL
Topology topology = builder.build();
//
// OR
//
Topology topology = ...; // when using the Processor API
// Use the configuration to tell your application where the Kafka cluster is,
// which Serializers/Deserializers to use by default, to specify security settings,
// and so on.
Properties props = ...;
KafkaStreams streams = new KafkaStreams(topology, props);
At this point, internal structures are initialized, but the processing is not started yet. You have to explicitly start the Kafka Streams thread by calling the KafkaStreams#start()
method:
// Start the Kafka Streams threads
streams.start();
If there are other instances of this stream processing application running elsewhere (e.g., on another machine), Kafka Streams transparently re-assigns tasks from the existing instances to the new instance that you just started. For more information, see Stream Partitions and Tasks and Threading Model.
To catch any unexpected exceptions, you can set an java.lang.Thread.UncaughtExceptionHandler
before you start the application. This handler is called whenever a stream thread is terminated by an unexpected exception:
streams.setUncaughtExceptionHandler((Thread thread, Throwable throwable) -> {
// here you should examine the throwable/exception and perform an appropriate action!
});
To stop the application instance, call the KafkaStreams#close()
method:
// Stop the Kafka Streams threads
streams.close();
To allow your application to gracefully shutdown in response to SIGTERM, it is recommended that you add a shutdown hook and call KafkaStreams#close
.
Here is a shutdown hook example in Java:
// Add shutdown hook to stop the Kafka Streams threads.
// You can optionally provide a timeout to `close`.
Runtime.getRuntime().addShutdownHook(new Thread(streams::close));
After an application is stopped, Kafka Streams will migrate any tasks that had been running in this instance to available remaining instances.
Testing a Streams application
Kafka Streams comes with a test-utils
module to help you test your application here.
7.2 - Configuring a Streams Application
Configuring a Streams Application
Kafka and Kafka Streams configuration options must be configured before using Streams. You can configure Kafka Streams by specifying parameters in a java.util.Properties
instance.
Create a
java.util.Properties
instance.Set the parameters. For example:
import java.util.Properties; import org.apache.kafka.streams.StreamsConfig;
Properties settings = new Properties();
// Set a few key parameters
settings.put(StreamsConfig.APPLICATION_ID_CONFIG, "my-first-streams-application");
settings.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092");
// Any further settings
settings.put(... , ...);
Configuration parameter reference
This section contains the most common Streams configuration parameters. For a full reference, see the Streams Javadocs.
- Required configuration parameters
- application.id
- bootstrap.servers
- Recommended configuration parameters for resiliency
- acks
- replication.factor
- min.insync.replicas
- num.standby.replicas
- Optional configuration parameters
- acceptable.recovery.lag
- default.deserialization.exception.handler (deprecated since 4.0)
- default.key.serde
- default.production.exception.handler (deprecated since 4.0)
- default.timestamp.extractor
- default.value.serde
- deserialization.exception.handler
- enable.metrics.push
- log.summary.interval.ms
- max.task.idle.ms
- max.warmup.replicas
- num.standby.replicas
- num.stream.threads
- probing.rebalance.interval.ms
- processing.exception.handler
- processing.guarantee
- processor.wrapper.class
- production.exception.handler
- rack.aware.assignment.non_overlap_cost
- rack.aware.assignment.strategy
- rack.aware.assignment.tags
- rack.aware.assignment.traffic_cost
- replication.factor
- rocksdb.config.setter
- state.dir
- task.assignor.class
- topology.optimization
- windowed.inner.class.serde
- Kafka consumers and producer configuration parameters
- Naming
- Default Values
- Parameters controlled by Kafka Streams
- enable.auto.commit
Required configuration parameters
Here are the required Streams configuration parameters.
Parameter Name | Importance | Description | Default Value |
---|---|---|---|
application.id | Required | An identifier for the stream processing application. Must be unique within the Kafka cluster. | None |
bootstrap.servers | Required | A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. | None |
application.id
(Required) The application ID. Each stream processing application must have a unique ID. The same ID must be given to all instances of the application. It is recommended to use only alphanumeric characters,
.
(dot),-
(hyphen), and_
(underscore). Examples:"hello_world"
,"hello_world-v1.0.0"
This ID is used in the following places to isolate resources used by the application from others:
- As the default Kafka consumer and producer
client.id
prefix- As the Kafka consumer
group.id
for coordination- As the name of the subdirectory in the state directory (cf.
state.dir
)- As the prefix of internal Kafka topic names
Tip: When an application is updated, the
application.id
should be changed unless you want to reuse the existing data in internal topics and state stores. For example, you could embed the version information withinapplication.id
, asmy-app-v1.0.0
andmy-app-v1.0.2
.
bootstrap.servers
(Required) The Kafka bootstrap servers. This is the same setting that is used by the underlying producer and consumer clients to connect to the Kafka cluster. Example:
"kafka-broker1:9092,kafka-broker2:9092"
.
Recommended configuration parameters for resiliency
There are several Kafka and Kafka Streams configuration options that need to be configured explicitly for resiliency in face of broker failures:
Parameter Name | Corresponding Client | Default value | Consider setting to |
---|---|---|---|
acks | Producer (for version <=2.8) | acks="1") | acks="all" |
replication.factor (for broker version 2.3 or older) | Streams | -1 | 3 (broker 2.4+: ensure broker config default.replication.factor=3 ) |
min.insync.replicas | Broker | 1 | 2 |
num.standby.replicas | Streams | 0 | 1 |
Increasing the replication factor to 3 ensures that the internal Kafka Streams topic can tolerate up to 2 broker failures. The tradeoff from moving to the default values to the recommended ones is that some performance and more storage space (3x with the replication factor of 3) are sacrificed for more resiliency.
acks
The number of acknowledgments that the leader must have received before considering a request complete. This controls the durability of records that are sent. The possible values are:
acks="0"
The producer does not wait for acknowledgment from the server and the record is immediately added to the socket buffer and considered sent. No guarantee can be made that the server has received the record in this case, and the producer won’t generally know of any failures. The offset returned for each record will always be set to-1
.acks="1"
The leader writes the record to its local log and responds without waiting for full acknowledgement from all followers. If the leader immediately fails after acknowledging the record, but before the followers have replicated it, then the record will be lost.acks="all"
(default since 3.0 release) The leader waits for the full set of in-sync replicas to acknowledge the record. This guarantees that the record will not be lost if there is at least one in-sync replica alive. This is the strongest available guarantee.
For more information, see the Kafka Producer documentation.
replication.factor
See the description here.
min.insync.replicas
The minimum number of in-sync replicas available for replication if the producer is configured with acks="all"
(see topic configs).
num.standby.replicas
See the description here.
Properties streamsSettings = new Properties();
// for broker version 2.3 or older
//streamsSettings.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3);
// for version 2.8 or older
//streamsSettings.put(StreamsConfig.producerPrefix(ProducerConfig.ACKS_CONFIG), "all");
streamsSettings.put(StreamsConfig.topicPrefix(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG), 2);
streamsSettings.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1);
Optional configuration parameters
Here are the optional Streams javadocs, sorted by level of importance:
- High: These are parameters with a default value which is most likely not a good fit for production use. It’s highly recommended to revisit these parameters for production usage.
- Medium: The default values of these parameters should work for production for many cases, but it’s not uncommon that they are changed, for example to tune performance.
- Low: It should rarely be necessary to change the value for these parameters. It’s only recommended to change them if there is a very specific issue you want to address.
Parameter Name | Importance | Description | Default Value |
---|---|---|---|
acceptable.recovery.lag | Medium | The maximum acceptable lag (number of offsets to catch up) for an instance to be considered caught-up and ready for the active task. | 10000 |
application.server | Low | A host:port pair pointing to an embedded user defined endpoint that can be used for discovering the locations of state stores within a single Kafka Streams application. The value of this must be different for each instance of the application. | the empty string |
buffered.records.per.partition | Low | The maximum number of records to buffer per partition. | 1000 |
statestore.cache.max.bytes | Medium | Maximum number of memory bytes to be used for record caches across all threads. | 10485760 |
cache.max.bytes.buffering (Deprecated. Use statestore.cache.max.bytes instead.) | Medium | Maximum number of memory bytes to be used for record caches across all threads. | 10485760 |
client.id | Medium | An ID string to pass to the server when making requests. (This setting is passed to the consumer/producer clients used internally by Kafka Streams.) | the empty string |
commit.interval.ms | Low | The frequency in milliseconds with which to save the position (offsets in source topics) of tasks. | 30000 (30 seconds) |
default.deserialization.exception.handler (Deprecated. Use deserialization.exception.handler instead.) | Medium | Exception handling class that implements the DeserializationExceptionHandler interface. | LogAndContinueExceptionHandler |
default.key.serde | Medium | Default serializer/deserializer class for record keys, implements the Serde interface. Must be set by the user or all serdes must be passed in explicitly (see also default.value.serde). | null |
default.production.exception.handler (Deprecated. Use production.exception.handler instead.) | Medium | Exception handling class that implements the ProductionExceptionHandler interface. | DefaultProductionExceptionHandler |
default.timestamp.extractor | Medium | Timestamp extractor class that implements the TimestampExtractor interface. See Timestamp Extractor | FailOnInvalidTimestamp |
default.value.serde | Medium | Default serializer/deserializer class for record values, implements the Serde interface. Must be set by the user or all serdes must be passed in explicitly (see also default.key.serde). | null |
default.dsl.store | Low | [DEPRECATED] The default state store type used by DSL operators. Deprecated in favor of dsl.store.suppliers.class | "ROCKS_DB" |
deserialization.exception.handler | Medium | Exception handling class that implements the DeserializationExceptionHandler interface. | LogAndContinueExceptionHandler |
dsl.store.suppliers.class | Low | Defines a default state store implementation to be used by any stateful DSL operator that has not explicitly configured the store implementation type. Must implement the org.apache.kafka.streams.state.DslStoreSuppliers interface. | BuiltInDslStoreSuppliers.RocksDBDslStoreSuppliers |
log.summary.interval.ms | Low | The output interval in milliseconds for logging summary information (disabled if negative). | 120000 (2 minutes) |
enable.metrics.push | Low | Whether to enable pushing of client metrics to the cluster, if the cluster has a client metrics subscription which matches this client. | true |
max.task.idle.ms | Medium | This config controls whether joins and merges may produce out-of-order results. The config value is the maximum amount of time in milliseconds a stream task will stay idle when it is fully caught up on some (but not all) input partitions to wait for producers to send additional records and avoid potential out-of-order record processing across multiple input streams. The default (zero) does not wait for producers to send more records, but it does wait to fetch data that is already present on the brokers. This default means that for records that are already present on the brokers, Streams will process them in timestamp order. Set to -1 to disable idling entirely and process any locally available data, even though doing so may produce out-of-order processing. | 0 |
max.warmup.replicas | Medium | The maximum number of warmup replicas (extra standbys beyond the configured num.standbys) that can be assigned at once. | 2 |
metric.reporters | Low | A list of classes to use as metrics reporters. | the empty list |
metrics.num.samples | Low | The number of samples maintained to compute metrics. | 2 |
metrics.recording.level | Low | The highest recording level for metrics. | INFO |
metrics.sample.window.ms | Low | The window of time in milliseconds a metrics sample is computed over. | 30000 (30 seconds) |
num.standby.replicas | High | The number of standby replicas for each task. | 0 |
num.stream.threads | Medium | The number of threads to execute stream processing. | 1 |
probing.rebalance.interval.ms | Low | The maximum time in milliseconds to wait before triggering a rebalance to probe for warmup replicas that have sufficiently caught up. | 600000 (10 minutes) |
processing.exception.handler | Medium | Exception handling class that implements the ProcessingExceptionHandler interface. | LogAndFailProcessingExceptionHandler |
processing.guarantee | Medium | The processing mode. Can be either "at_least_once" or "exactly_once_v2" (for EOS version 2, requires broker version 2.5+). See Processing Guarantee.. | "at_least_once" |
processor.wrapper.class | Medium | A class or class name implementing the ProcessorWrapper interface. Must be passed in when creating the topology, and will not be applied unless passed in to the appropriate constructor as a TopologyConfig. You should use the StreamsBuilder#new(TopologyConfig) constructor for DSL applications, and the Topology#new(TopologyConfig) constructor for PAPI applications. | |
production.exception.handler | Medium | Exception handling class that implements the ProductionExceptionHandler interface. | DefaultProductionExceptionHandler |
poll.ms | Low | The amount of time in milliseconds to block waiting for input. | 100 |
rack.aware.assignment.strategy | Low | The strategy used for rack aware assignment. Acceptable value are "none" (default), "min_traffic" , and "balance_suttopology" . See Rack Aware Assignment Strategy. | "none" |
List of tag keys used to distribute standby replicas across Kafka Streams clients. When configured, Kafka Streams will make a best-effort to distribute the standby tasks over clients with different tag values. See Rack Aware Assignment Tags. the empty list | |||
rack.aware.assignment.non_overlap_cost | Low | Cost associated with moving tasks from existing assignment. See Rack Aware Assignment Non-Overlap-Cost. | null |
rack.aware.assignment.non_overlap_cost | Low | Cost associated with cross rack traffic. See Rack Aware Assignment Traffic-Cost. | null |
replication.factor | Medium | The replication factor for changelog topics and repartition topics created by the application. The default of -1 (meaning: use broker default replication factor) requires broker version 2.4 or newer. | -1 |
retry.backoff.ms | Low | The amount of time in milliseconds, before a request is retried. | 100 |
rocksdb.config.setter | Medium | The RocksDB configuration. | null |
state.cleanup.delay.ms | Low | The amount of time in milliseconds to wait before deleting state when a partition has migrated. | 600000 (10 minutes) |
state.dir | High | Directory location for state stores. | /${java.io.tmpdir}/kafka-streams |
task.assignor.class | Medium | A task assignor class or class name implementing the TaskAssignor interface. | The high-availability task assignor. |
task.timeout.ms | Medium | The maximum amount of time in milliseconds a task might stall due to internal errors and retries until an error is raised. For a timeout of 0 ms , a task would raise an error for the first internal error. For any timeout larger than 0 ms , a task will retry at least once before an error is raised. | 300000 (5 minutes) |
topology.optimization | Medium | A configuration telling Kafka Streams if it should optimize the topology and what optimizations to apply. Acceptable values are: StreamsConfig.NO_OPTIMIZATION (none ), StreamsConfig.OPTIMIZE (all ) or a comma separated list of specific optimizations: StreamsConfig.REUSE_KTABLE_SOURCE_TOPICS (reuse.ktable.source.topics ), StreamsConfig.MERGE_REPARTITION_TOPICS (merge.repartition.topics ), StreamsConfig.SINGLE_STORE_SELF_JOIN (single.store.self.join ). | "NO_OPTIMIZATION" |
upgrade.from | Medium | The version you are upgrading from during a rolling upgrade. See Upgrade From | null |
windowstore.changelog.additional.retention.ms | Low | Added to a windows maintainMs to ensure data is not deleted from the log prematurely. Allows for clock drift. | 86400000 (1 day) |
window.size.ms | Low | Sets window size for the deserializer in order to calculate window end times. | null |
acceptable.recovery.lag
The maximum acceptable lag (total number of offsets to catch up from the changelog) for an instance to be considered caught-up and able to receive an active task. Streams will only assign stateful active tasks to instances whose state stores are within the acceptable recovery lag, if any exist, and assign warmup replicas to restore state in the background for instances that are not yet caught up. Should correspond to a recovery time of well under a minute for a given workload. Must be at least 0.
Note: if you set this to
Long.MAX_VALUE
it effectively disables the warmup replicas and task high availability, allowing Streams to immediately produce a balanced assignment and migrate tasks to a new instance without first warming them up.
deserialization.exception.handler (deprecated: default.deserialization.exception.handler)
The deserialization exception handler allows you to manage record exceptions that fail to deserialize. This can be caused by corrupt data, incorrect serialization logic, or unhandled record types. The implemented exception handler needs to return a
FAIL
orCONTINUE
depending on the record and the exception thrown. ReturningFAIL
will signal that Streams should shut down andCONTINUE
will signal that Streams should ignore the issue and continue processing. The following library built-in exception handlers are available:
- LogAndContinueExceptionHandler: This handler logs the deserialization exception and then signals the processing pipeline to continue processing more records. This log-and-skip strategy allows Kafka Streams to make progress instead of failing if there are records that fail to deserialize.
- LogAndFailExceptionHandler. This handler logs the deserialization exception and then signals the processing pipeline to stop processing more records.
You can also provide your own customized exception handler besides the library provided ones to meet your needs. For example, you can choose to forward corrupt records into a quarantine topic (think: a “dead letter queue”) for further processing. To do this, use the Producer API to write a corrupted record directly to the quarantine topic. To be more concrete, you can create a separate
KafkaProducer
object outside the Streams client, and pass in this object as well as the dead letter queue topic name into theProperties
map, which then can be retrieved from theconfigure
function call. The drawback of this approach is that “manual” writes are side effects that are invisible to the Kafka Streams runtime library, so they do not benefit from the end-to-end processing guarantees of the Streams API:public class SendToDeadLetterQueueExceptionHandler implements DeserializationExceptionHandler { KafkaProducer<byte[], byte[]> dlqProducer; String dlqTopic; @Override public DeserializationHandlerResponse handle(final ErrorHandlerContext context, final ConsumerRecord<byte[], byte[]> record, final Exception exception) { log.warn("Exception caught during Deserialization, sending to the dead queue topic; " + "taskId: {}, topic: {}, partition: {}, offset: {}", context.taskId(), record.topic(), record.partition(), record.offset(), exception); dlqProducer.send(new ProducerRecord<>(dlqTopic, record.timestamp(), record.key(), record.value(), record.headers())).get(); return DeserializationHandlerResponse.CONTINUE; } @Override public void configure(final Map<String, ?> configs) { dlqProducer = .. // get a producer from the configs map dlqTopic = .. // get the topic name from the configs map } }
production.exception.handler (deprecated: default.production.exception.handler)
The production exception handler allows you to manage exceptions triggered when trying to interact with a broker such as attempting to produce a record that is too large. By default, Kafka provides and uses the DefaultProductionExceptionHandler that always fails when these exceptions occur.
An exception handler can return
FAIL
,CONTINUE
, orRETRY
depending on the record and the exception thrown. ReturningFAIL
will signal that Streams should shut down.CONTINUE
will signal that Streams should ignore the issue and continue processing. ForRetriableException
the handler may returnRETRY
to tell the runtime to retry sending the failed record (Note: IfRETRY
is returned for a non-RetriableException
it will be treated asFAIL
.) If you want to provide an exception handler that always ignores records that are too large, you could implement something like the following:import java.util.Properties; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.common.errors.RecordTooLargeException; import org.apache.kafka.streams.errors.ProductionExceptionHandler; import org.apache.kafka.streams.errors.ProductionExceptionHandler.ProductionExceptionHandlerResponse; public class IgnoreRecordTooLargeHandler implements ProductionExceptionHandler { public void configure(Map<String, Object> config) {} public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context, final ProducerRecord<byte[], byte[]> record, final Exception exception) { if (exception instanceof RecordTooLargeException) { return ProductionExceptionHandlerResponse.CONTINUE; } else { return ProductionExceptionHandlerResponse.FAIL; } } } Properties settings = new Properties(); // other various kafka streams settings, e.g. bootstrap servers, application id, etc settings.put(StreamsConfig.PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG, IgnoreRecordTooLargeHandler.class);
default.timestamp.extractor
A timestamp extractor pulls a timestamp from an instance of ConsumerRecord. Timestamps are used to control the progress of streams.
The default extractor is FailOnInvalidTimestamp. This extractor retrieves built-in timestamps that are automatically embedded into Kafka messages by the Kafka producer client since Kafka version 0.10. Depending on the setting of Kafka’s server-side
log.message.timestamp.type
broker andmessage.timestamp.type
topic parameters, this extractor provides you with:
- event-time processing semantics if
log.message.timestamp.type
is set toCreateTime
aka “producer time” (which is the default). This represents the time when a Kafka producer sent the original message. If you use Kafka’s official producer client, the timestamp represents milliseconds since the epoch.- ingestion-time processing semantics if
log.message.timestamp.type
is set toLogAppendTime
aka “broker time”. This represents the time when the Kafka broker received the original message, in milliseconds since the epoch.
The
FailOnInvalidTimestamp
extractor throws an exception if a record contains an invalid (i.e. negative) built-in timestamp, because Kafka Streams would not process this record but silently drop it. Invalid built-in timestamps can occur for various reasons: if for example, you consume a topic that is written to by pre-0.10 Kafka producer clients or by third-party producer clients that don’t support the new Kafka 0.10 message format yet; another situation where this may happen is after upgrading your Kafka cluster from0.9
to0.10
, where all the data that was generated with0.9
does not include the0.10
message timestamps.If you have data with invalid timestamps and want to process it, then there are two alternative extractors available. Both work on built-in timestamps, but handle invalid timestamps differently.
- LogAndSkipOnInvalidTimestamp: This extractor logs a warn message and returns the invalid timestamp to Kafka Streams, which will not process but silently drop the record. This log-and-skip strategy allows Kafka Streams to make progress instead of failing if there are records with an invalid built-in timestamp in your input data.
- UsePartitionTimeOnInvalidTimestamp. This extractor returns the record’s built-in timestamp if it is valid (i.e. not negative). If the record does not have a valid built-in timestamps, the extractor returns the previously extracted valid timestamp from a record of the same topic partition as the current record as a timestamp estimation. In case that no timestamp can be estimated, it throws an exception.
Another built-in extractor is WallclockTimestampExtractor. This extractor does not actually “extract” a timestamp from the consumed record but rather returns the current time in milliseconds from the system clock (think:
System.currentTimeMillis()
), which effectively means Streams will operate on the basis of the so-called processing-time of events.You can also provide your own timestamp extractors, for instance to retrieve timestamps embedded in the payload of messages. If you cannot extract a valid timestamp, you can either throw an exception, return a negative timestamp, or estimate a timestamp. Returning a negative timestamp will result in data loss - the corresponding record will not be processed but silently dropped. If you want to estimate a new timestamp, you can use the value provided via
previousTimestamp
(i.e., a Kafka Streams timestamp estimation). Here is an example of a customTimestampExtractor
implementation:import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.streams.processor.TimestampExtractor; // Extracts the embedded timestamp of a record (giving you "event-time" semantics). public class MyEventTimeExtractor implements TimestampExtractor { @Override public long extract(final ConsumerRecord<Object, Object> record, final long previousTimestamp) { // `Foo` is your own custom class, which we assume has a method that returns // the embedded timestamp (milliseconds since midnight, January 1, 1970 UTC). long timestamp = -1; final Foo myPojo = (Foo) record.value(); if (myPojo != null) { timestamp = myPojo.getTimestampInMillis(); } if (timestamp < 0) { // Invalid timestamp! Attempt to estimate a new timestamp, // otherwise fall back to wall-clock time (processing-time). if (previousTimestamp >= 0) { return previousTimestamp; } else { return System.currentTimeMillis(); } } } }
You would then define the custom timestamp extractor in your Streams configuration as follows:
import java.util.Properties; import org.apache.kafka.streams.StreamsConfig; Properties streamsConfiguration = new Properties(); streamsConfiguration.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MyEventTimeExtractor.class);
default.key.serde
The default Serializer/Deserializer class for record keys, null unless set by user. Serialization and deserialization in Kafka Streams happens whenever data needs to be materialized, for example:
- Whenever data is read from or written to a Kafka topic (e.g., via the
StreamsBuilder#stream()
andKStream#to()
methods).- Whenever data is read from or written to a state store.
This is discussed in more detail in Data types and serialization.
default.value.serde
The default Serializer/Deserializer class for record values, null unless set by user. Serialization and deserialization in Kafka Streams happens whenever data needs to be materialized, for example:
- Whenever data is read from or written to a Kafka topic (e.g., via the
StreamsBuilder#stream()
andKStream#to()
methods).- Whenever data is read from or written to a state store.
This is discussed in more detail in Data types and serialization.
rack.aware.assignment.non_overlap_cost
This configuration sets the cost of moving a task from the original assignment computed either by
StickyTaskAssignor
orHighAvailabilityTaskAssignor
. Together withrack.aware.assignment.traffic_cost
, they control whether the optimizer favors minimizing cross rack traffic or minimizing the movement of tasks in the existing assignment. If this config is set to a larger value thanrack.aware.assignment.traffic_cost
, the optimizer will try to maintain the existing assignment computed by the task assignor. Note that the optimizer takes the ratio of these two configs into consideration of favoring maintaining existing assignment or minimizing traffic cost. For example, settingrack.aware.assignment.non_overlap_cost
to 10 andrack.aware.assignment.traffic_cost
to 1 is more likely to maintain existing assignment than settingrack.aware.assignment.non_overlap_cost
to 100 andrack.aware.assignment.traffic_cost
to 50.The default value is null which means default
non_overlap_cost
in different assignors will be used. InStickyTaskAssignor
, it has a default value of 10 andrack.aware.assignment.traffic_cost
has a default value of 1, which means maintaining stickiness is preferred inStickyTaskAssignor
. InHighAvailabilityTaskAssignor
, it has a default value of 1 andrack.aware.assignment.traffic_cost
has a default value of 10, which means minimizing cross rack traffic is preferred inHighAvailabilityTaskAssignor
.
rack.aware.assignment.strategy
This configuration sets the strategy Kafka Streams uses for rack aware task assignment so that cross traffic from broker to client can be reduced. This config will only take effect when
broker.rack
is set on the brokers andclient.rack
is set on Kafka Streams side. There are two settings for this config:
none
. This is the default value which means rack aware task assignment will be disabled.min_traffic
. This settings means that the rack aware task assigner will compute an assignment which tries to minimize cross rack traffic.balance_subtopology
. This settings means that the rack aware task assigner will compute an assignment which will try to balance tasks from same subtopology to different clients and minimize cross rack traffic on top of that.
This config can be used together with rack.aware.assignment.non_overlap_cost and rack.aware.assignment.traffic_cost to balance reducing cross rack traffic and maintaining the existing assignment.
rack.aware.assignment.tags
This configuration sets a list of tag keys used to distribute standby replicas across Kafka Streams clients. When configured, Kafka Streams will make a best-effort to distribute the standby tasks over clients with different tag values.
Tags for the Kafka Streams clients can be set via
client.tag.
prefix. Example:Client-1 | Client-2 _______________________________________________________________________ client.tag.zone: eu-central-1a | client.tag.zone: eu-central-1b client.tag.cluster: k8s-cluster1 | client.tag.cluster: k8s-cluster1 rack.aware.assignment.tags: zone,cluster | rack.aware.assignment.tags: zone,cluster Client-3 | Client-4 _______________________________________________________________________ client.tag.zone: eu-central-1a | client.tag.zone: eu-central-1b client.tag.cluster: k8s-cluster2 | client.tag.cluster: k8s-cluster2 rack.aware.assignment.tags: zone,cluster | rack.aware.assignment.tags: zone,cluster
In the above example, we have four Kafka Streams clients across two zones (
eu-central-1a
,eu-central-1b
) and across two clusters (k8s-cluster1
,k8s-cluster2
). For an active task located onClient-1
, Kafka Streams will allocate a standby task onClient-4
, sinceClient-4
has a differentzone
and a differentcluster
thanClient-1
.
rack.aware.assignment.traffic_cost
This configuration sets the cost of cross rack traffic. Together with
rack.aware.assignment.non_overlap_cost
, they control whether the optimizer favors minimizing cross rack traffic or minimizing the movement of tasks in the existing assignment. If this config is set to a larger value thanrack.aware.assignment.non_overlap_cost
, the optimizer will try to compute an assignment which minimize the cross rack traffic. Note that the optimizer takes the ratio of these two configs into consideration of favoring maintaining existing assignment or minimizing traffic cost. For example, settingrack.aware.assignment.traffic_cost
to 10 andrack.aware.assignment.non_overlap_cost
to 1 is more likely to minimize cross rack traffic than settingrack.aware.assignment.traffic_cost
to 100 andrack.aware.assignment.non_overlap_cost
to 50.The default value is null which means default traffic cost in different assignors will be used. In
StickyTaskAssignor
, it has a default value of 1 andrack.aware.assignment.non_overlap_cost
has a default value of 10. InHighAvailabilityTaskAssignor
, it has a default value of 10 andrack.aware.assignment.non_overlap_cost
has a default value of 1.
log.summary.interval.ms
This configuration controls the output interval for summary information. If greater or equal to 0, the summary log will be output according to the set time interval; If less than 0, summary output is disabled.
enable.metrics.push
Kafka Streams metrics can be pushed to the brokers similar to client metrics. Additionally, Kafka Streams allows to enable/disable metric pushing for each embedded client individually. However, pushing Kafka Streams metrics requires that
enable.metric.push
is enabled on the main-consumer and admin client.
max.task.idle.ms
This configuration controls how long Streams will wait to fetch data in order to provide in-order processing semantics.
When processing a task that has multiple input partitions (as in a join or merge), Streams needs to choose which partition to process the next record from. When all input partitions have locally buffered data, Streams picks the partition whose next record has the lowest timestamp. This has the desirable effect of collating the input partitions in timestamp order, which is generally what you want in a streaming join or merge. However, when Streams does not have any data buffered locally for one of the partitions, it does not know whether the next record for that partition will have a lower or higher timestamp than the remaining partitions’ records.
There are two cases to consider: either there is data in that partition on the broker that Streams has not fetched yet, or Streams is fully caught up with that partition on the broker, and the producers simply haven’t produced any new records since Streams polled the last batch.
The default value of
0
causes Streams to delay processing a task when it detects that it has no locally buffered data for a partition, but there is data available on the brokers. Specifically, when there is an empty partition in the local buffer, but Streams has a non-zero lag for that partition. However, as soon as Streams catches up to the broker, it will continue processing, even if there is no data in one of the partitions. That is, it will not wait for new data to be produced. This default is designed to sacrifice some throughput in exchange for intuitively correct join semantics.Any config value greater than zero indicates the number of extra milliseconds that Streams will wait if it has a caught-up but empty partition. In other words, this is the amount of time to wait for new data to be produced to the input partitions to ensure in-order processing of data in the event of a slow producer.
The config value of
-1
indicates that Streams will never wait to buffer empty partitions before choosing the next record by timestamp, which achieves maximum throughput at the expense of introducing out-of-order processing.
max.warmup.replicas
The maximum number of warmup replicas (extra standbys beyond the configured
num.standbys
) that can be assigned at once for the purpose of keeping the task available on one instance while it is warming up on another instance it has been reassigned to. Used to throttle how much extra broker traffic and cluster state can be used for high availability. Increasing this will allow Streams to warm up more tasks at once, speeding up the time for the reassigned warmups to restore sufficient state for them to be transitioned to active tasks. Must be at least 1.Note that one warmup replica corresponds to one Stream Task. Furthermore, note that each warmup task can only be promoted to an active task during a rebalance (normally during a so-called probing rebalance, which occur at a frequency specified by the
probing.rebalance.interval.ms
config). This means that the maximum rate at which active tasks can be migrated from one Kafka Streams instance to another instance can be determined by (max.warmup.replicas
/probing.rebalance.interval.ms
).
num.standby.replicas
The number of standby replicas. Standby replicas are shadow copies of local state stores. Kafka Streams attempts to create the specified number of replicas per store and keep them up to date as long as there are enough instances running. Standby replicas are used to minimize the latency of task failover. A task that was previously running on a failed instance is preferred to restart on an instance that has standby replicas so that the local state store restoration process from its changelog can be minimized. Details about how Kafka Streams makes use of the standby replicas to minimize the cost of resuming tasks on failover can be found in the State section.
Recommendation: Increase the number of standbys to 1 to get instant fail-over, i.e., high-availability. Increasing the number of standbys requires more client-side storage space. For example, with 1 standby, 2x space is required.
Note: If you enable n standby tasks, you need to provision n+1
KafkaStreams
instances.
num.stream.threads
This specifies the number of stream threads in an instance of the Kafka Streams application. The stream processing code runs in these thread. For more information about Kafka Streams threading model, see Threading Model.
probing.rebalance.interval.ms
The maximum time to wait before triggering a rebalance to probe for warmup replicas that have restored enough to be considered caught up. Streams will only assign stateful active tasks to instances that are caught up and within the acceptable.recovery.lag, if any exist. Probing rebalances are used to query the latest total lag of warmup replicas and transition them to active tasks if ready. They will continue to be triggered as long as there are warmup tasks, and until the assignment is balanced. Must be at least 1 minute.
processing.exception.handler
The processing exception handler allows you to manage exceptions triggered during the processing of a record. The implemented exception handler needs to return a
FAIL
orCONTINUE
depending on the record and the exception thrown. ReturningFAIL
will signal that Streams should shut down andCONTINUE
will signal that Streams should ignore the issue and continue processing. The following library built-in exception handlers are available:
- LogAndContinueProcessingExceptionHandler: This handler logs the processing exception and then signals the processing pipeline to continue processing more records. This log-and-skip strategy allows Kafka Streams to make progress instead of failing if there are records that fail to be processed.
- LogAndFailProcessingExceptionHandler. This handler logs the processing exception and then signals the processing pipeline to stop processing more records.
You can also provide your own customized exception handler besides the library provided ones to meet your needs. For example, you can choose to forward corrupt records into a quarantine topic (think: a “dead letter queue”) for further processing. To do this, use the Producer API to write a corrupted record directly to the quarantine topic. To be more concrete, you can create a separate
KafkaProducer
object outside the Streams client, and pass in this object as well as the dead letter queue topic name into theProperties
map, which then can be retrieved from theconfigure
function call. The drawback of this approach is that “manual” writes are side effects that are invisible to the Kafka Streams runtime library, so they do not benefit from the end-to-end processing guarantees of the Streams API:public class SendToDeadLetterQueueExceptionHandler implements ProcessingExceptionHandler { KafkaProducer<byte[], byte[]> dlqProducer; String dlqTopic; @Override public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { log.warn("Exception caught during message processing, sending to the dead queue topic; " + "processor node: {}, taskId: {}, source topic: {}, source partition: {}, source offset: {}", context.processorNodeId(), context.taskId(), context.topic(), context.partition(), context.offset(), exception); dlqProducer.send(new ProducerRecord<>(dlqTopic, null, record.timestamp(), (byte[]) record.key(), (byte[]) record.value(), record.headers())); return ProcessingHandlerResponse.CONTINUE; } @Override public void configure(final Map<String, ?> configs) { dlqProducer = .. // get a producer from the configs map dlqTopic = .. // get the topic name from the configs map } }
processing.guarantee
The processing guarantee that should be used. Possible values are
"at_least_once"
(default) and"exactly_once_v2"
(for EOS version 2). Deprecated config options are"exactly_once"
(for EOS alpha), and"exactly_once_beta"
(for EOS version 2). Using"exactly_once_v2"
(or the deprecated"exactly_once_beta"
) requires broker version 2.5 or newer, while using the deprecated"exactly_once"
requires broker version 0.11.0 or newer. Note that if exactly-once processing is enabled, the default for parametercommit.interval.ms
changes to 100ms. Additionally, consumers are configured withisolation.level="read_committed"
and producers are configured withenable.idempotence=true
per default. Note that by default exactly-once processing requires a cluster of at least three brokers what is the recommended setting for production. For development, you can change this configuration by adjusting broker settingtransaction.state.log.replication.factor
andtransaction.state.log.min.isr
to the number of brokers you want to use. For more details see Processing Guarantees.Recommendation: While it is technically possible to use EOS with any replication factor, using a replication factor lower than 3 effectively voids EOS. Thus it is strongly recommended to use a replication factor of 3 (together with
min.in.sync.replicas=2
). This recommendation applies to all topics (i.e.__transaction_state
,__consumer_offsets
, Kafka Streams internal topics, and user topics).
processor.wrapper.class
A class or class name implementing the
ProcessorWrapper
interface. This feature allows you to wrap any of the processors in the compiled topology, including both custom processor implementations and those created by Streams for DSL operators. This can be useful for logging or tracing implementations since it allows access to the otherwise-hidden processor context for DSL operators, and also allows for injecting additional debugging information to an entire application topology with just a single config.IMPORTANT: This MUST be passed in when creating the topology, and will not be applied unless passed in to the appropriate topology-building constructor. You should use the
StreamsBuilder#new(TopologyConfig)
constructor for DSL applications, and theTopology#new(TopologyConfig)
constructor for PAPI applications.
replication.factor
This specifies the replication factor of internal topics that Kafka Streams creates when local states are used or a stream is repartitioned for aggregation. Replication is important for fault tolerance. Without replication even a single broker failure may prevent progress of the stream processing application. It is recommended to use a similar replication factor as source topics.
Recommendation: Increase the replication factor to 3 to ensure that the internal Kafka Streams topic can tolerate up to 2 broker failures. Note that you will require more storage space as well (3x with the replication factor of 3).
rocksdb.config.setter
The RocksDB configuration. Kafka Streams uses RocksDB as the default storage engine for persistent stores. To change the default configuration for RocksDB, you can implement
RocksDBConfigSetter
and provide your custom class via rocksdb.config.setter.Here is an example that adjusts the memory size consumed by RocksDB.
public static class CustomRocksDBConfig implements RocksDBConfigSetter { // This object should be a member variable so it can be closed in RocksDBConfigSetter#close. private org.rocksdb.Cache cache = new org.rocksdb.LRUCache(16 * 1024L * 1024L); @Override public void setConfig(final String storeName, final Options options, final Map<String, Object> configs) { // See #1 below. BlockBasedTableConfig tableConfig = (BlockBasedTableConfig) options.tableFormatConfig(); tableConfig.setBlockCache(cache); // See #2 below. tableConfig.setBlockSize(16 * 1024L); // See #3 below. tableConfig.setCacheIndexAndFilterBlocks(true); options.setTableFormatConfig(tableConfig); // See #4 below. options.setMaxWriteBufferNumber(2); } @Override public void close(final String storeName, final Options options) { // See #5 below. cache.close(); } } Properties streamsSettings = new Properties(); streamsConfig.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, CustomRocksDBConfig.class);
Notes for example:
BlockBasedTableConfig tableConfig = (BlockBasedTableConfig) options.tableFormatConfig();
Get a reference to the existing table config rather than create a new one, so you don’t accidentally overwrite defaults such as theBloomFilter
, which is an important optimization.tableConfig.setBlockSize(16 * 1024L);
Modify the default block size per these instructions from the RocksDB GitHub.tableConfig.setCacheIndexAndFilterBlocks(true);
Do not let the index and filter blocks grow unbounded. For more information, see the RocksDB GitHub.options.setMaxWriteBufferNumber(2);
See the advanced options in the RocksDB GitHub.cache.close();
To avoid memory leaks, you must close any objects you constructed that extend org.rocksdb.RocksObject. See RocksJava docs for more details.
state.dir
The state directory. Kafka Streams persists local states under the state directory. Each application has a subdirectory on its hosting machine that is located under the state directory. The name of the subdirectory is the application ID. The state stores associated with the application are created under this subdirectory. When running multiple instances of the same application on a single machine, this path must be unique for each such instance.
task.assignor.class
A task assignor class or class name implementing the
org.apache.kafka.streams.processor.assignment.TaskAssignor
interface. Defaults to the high-availability task assignor. One possible alternative implementation provided in Apache Kafka is theorg.apache.kafka.streams.processor.assignment.assignors.StickyTaskAssignor
, which was the default task assignor before KIP-441 and minimizes task movement at the cost of stateful task availability. Alternative implementations of the task assignment algorithm can be plugged into the application by implementing a customTaskAssignor
and setting this config to the name of the custom task assignor class.
topology.optimization
A configuration telling Kafka Streams if it should optimize the topology and what optimizations to apply. Acceptable values are:
StreamsConfig.NO_OPTIMIZATION
(none
),StreamsConfig.OPTIMIZE
(all
) or a comma separated list of specific optimizations:StreamsConfig.REUSE_KTABLE_SOURCE_TOPICS
(reuse.ktable.source.topics
),StreamsConfig.MERGE_REPARTITION_TOPICS
(merge.repartition.topics
),StreamsConfig.SINGLE_STORE_SELF_JOIN
(single.store.self.join
).
We recommend listing specific optimizations in the config for production code so that the structure of your topology will not change unexpectedly during upgrades of the Streams library.
These optimizations include moving/reducing repartition topics and reusing the source topic as the changelog for source KTables. These optimizations will save on network traffic and storage in Kafka without changing the semantics of your applications. Enabling them is recommended.
Note that as of 2.3, you need to do two things to enable optimizations. In addition to setting this config to StreamsConfig.OPTIMIZE
, you’ll need to pass in your configuration properties when building your topology by using the overloaded StreamsBuilder.build(Properties)
method. For example KafkaStreams myStream = new KafkaStreams(streamsBuilder.build(properties), properties)
.
windowed.inner.class.serde
Serde for the inner class of a windowed record. Must implement the org.apache.kafka.common.serialization.Serde interface.
Note that this config is only used by plain consumer/producer clients that set a windowed de/serializer type via configs. For Kafka Streams applications that deal with windowed types, you must pass in the inner serde type when you instantiate the windowed serde object for your topology.
upgrade.from
The version you are upgrading from. It is important to set this config when performing a rolling upgrade to certain versions, as described in the upgrade guide. You should set this config to the appropriate version before bouncing your instances and upgrading them to the newer version. Once everyone is on the newer version, you should remove this config and do a second rolling bounce. It is only necessary to set this config and follow the two-bounce upgrade path when upgrading from below version 2.0, or when upgrading to 2.4+ from any version lower than 2.4.
Kafka consumers, producer and admin client configuration parameters
You can specify parameters for the Kafka consumers, producers, and admin client that are used internally. The consumer, producer and admin client settings are defined by specifying parameters in a StreamsConfig
instance.
In this example, the Kafka consumer session timeout is configured to be 60000 milliseconds in the Streams settings:
Properties streamsSettings = new Properties();
// Example of a "normal" setting for Kafka Streams
streamsSettings.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker-01:9092");
// Customize the Kafka consumer settings of your Streams application
streamsSettings.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 60000);
Naming
Some consumer, producer and admin client configuration parameters use the same parameter name, and Kafka Streams library itself also uses some parameters that share the same name with its embedded client. For example, send.buffer.bytes
and receive.buffer.bytes
are used to configure TCP buffers; request.timeout.ms
and retry.backoff.ms
control retries for client request. You can avoid duplicate names by prefix parameter names with consumer.
, producer.
, or admin.
(e.g., consumer.send.buffer.bytes
and producer.send.buffer.bytes
).
Properties streamsSettings = new Properties();
// same value for consumer, producer, and admin client
streamsSettings.put("PARAMETER_NAME", "value");
// different values for consumer and producer
streamsSettings.put("consumer.PARAMETER_NAME", "consumer-value");
streamsSettings.put("producer.PARAMETER_NAME", "producer-value");
streamsSettings.put("admin.PARAMETER_NAME", "admin-value");
// alternatively, you can use
streamsSettings.put(StreamsConfig.consumerPrefix("PARAMETER_NAME"), "consumer-value");
streamsSettings.put(StreamsConfig.producerPrefix("PARAMETER_NAME"), "producer-value");
streamsSettings.put(StreamsConfig.adminClientPrefix("PARAMETER_NAME"), "admin-value");
You could further separate consumer configuration by adding different prefixes:
main.consumer.
for main consumer which is the default consumer of stream source.restore.consumer.
for restore consumer which is in charge of state store recovery.global.consumer.
for global consumer which is used in global KTable construction.
For example, if you only want to set restore consumer config without touching other consumers’ settings, you could simply use restore.consumer.
to set the config.
Properties streamsSettings = new Properties();
// same config value for all consumer types
streamsSettings.put("consumer.PARAMETER_NAME", "general-consumer-value");
// set a different restore consumer config. This would make restore consumer take restore-consumer-value,
// while main consumer and global consumer stay with general-consumer-value
streamsSettings.put("restore.consumer.PARAMETER_NAME", "restore-consumer-value");
// alternatively, you can use
streamsSettings.put(StreamsConfig.restoreConsumerPrefix("PARAMETER_NAME"), "restore-consumer-value");
Same applied to main.consumer.
and main.consumer.
, if you only want to specify one consumer type config.
Additionally, to configure the internal repartition/changelog topics, you could use the topic.
prefix, followed by any of the standard topic configs.
Properties streamsSettings = new Properties();
// Override default for both changelog and repartition topics
streamsSettings.put("topic.PARAMETER_NAME", "topic-value");
// alternatively, you can use
streamsSettings.put(StreamsConfig.topicPrefix("PARAMETER_NAME"), "topic-value");
Default Values
Kafka Streams uses different default values for some of the underlying client configs, which are summarized below. For detailed descriptions of these configs, see Producer Configs and Consumer Configs.
Parameter Name | Corresponding Client | Streams Default |
---|---|---|
auto.offset.reset | Consumer | earliest |
linger.ms | Producer | 100 |
max.poll.records | Consumer | 1000 |
client.id | - | <application.id>-<random-UUID> |
If EOS is enabled, other parameters have the following default values.
Parameter Name | Corresponding Client | Streams Default |
---|---|---|
transaction.timeout.ms | Producer | 10000 |
delivery.timeout.ms | Producer | Integer.MAX_VALUE |
Parameters controlled by Kafka Streams
Some parameters are not configurable by the user. If you supply a value that is different from the default value, your value is ignored. Below is a list of some of these parameters.
Parameter Name | Corresponding Client | Streams Default |
---|---|---|
allow.auto.create.topics | Consumer | false |
group.id | Consumer | application.id |
enable.auto.commit | Consumer | false |
partition.assignment.strategy | Consumer | StreamsPartitionAssignor |
If EOS is enabled, other parameters are set with the following values.
Parameter Name | Corresponding Client | Streams Default |
---|---|---|
isolation.level | Consumer | READ_COMMITTED |
enable.idempotence | Producer | true |
client.id
Kafka Streams uses the client.id
parameter to compute derived client IDs for internal clients. If you don’t set client.id
, Kafka Streams sets it to <application.id>-<random-UUID>
.
This value will be used to derive the client IDs of the following internal clients.
Client | client.id |
---|---|
Consumer | <client.id>-StreamThread-<threadIdx>-consumer |
Restore consumer | <client.id>-StreamThread-<threadIdx>-restore-consumer |
Global consumer | <client.id>-global-consumer |
Producer | For Non-EOS and EOS v2: <client.id>-StreamThread-<threadIdx>-producer |
For EOS v1: <client.id>-StreamThread-<threadIdx>-<taskId>-producer | |
Admin | <client.id>-admin |
enable.auto.commit
The consumer auto commit. To guarantee at-least-once processing semantics and turn off auto commits, Kafka Streams overrides this consumer config value to
false
. Consumers will only commit explicitly via commitSync calls when the Kafka Streams library or a user decides to commit the current processing state.
7.3 - Streams DSL
Streams DSL
The Kafka Streams DSL (Domain Specific Language) is built on top of the Streams Processor API. It is the recommended for most users, especially beginners. Most data processing operations can be expressed in just a few lines of DSL code.
Table of Contents
- Overview
- Creating source streams from Kafka
- Transform a stream
- Stateless transformations
- Stateful transformations
- Aggregating
- Joining
- Join co-partitioning requirements
- KStream-KStream Join
- KTable-KTable Equi-Join
- KTable-KTable Foreign-Key Join
- KStream-KTable Join
- KStream-GlobalKTable Join
- Windowing
- Hopping time windows
- Tumbling time windows
- Sliding time windows
- Session Windows
- Window Final Results
- Applying processors (Processor API integration)
- Transformers removal and migration to processors
- Naming Operators in a Streams DSL application
- Controlling KTable update rate
- Using timestamp-based semantics for table processors
- Writing streams back to Kafka
- Testing a Streams application
- Kafka Streams DSL for Scala
- Sample Usage
- Implicit Serdes
- User-Defined Serdes
Overview
In comparison to the Processor API, only the DSL supports:
- Built-in abstractions for streams and tables in the form of KStream, KTable, and GlobalKTable. Having first-class support for streams and tables is crucial because, in practice, most use cases require not just either streams or databases/tables, but a combination of both. For example, if your use case is to create a customer 360-degree view that is updated in real-time, what your application will be doing is transforming many input streams of customer-related events into an output table that contains a continuously updated 360-degree view of your customers.
- Declarative, functional programming style with stateless transformations (e.g.
map
andfilter
) as well as stateful transformations such as aggregations (e.g.count
andreduce
), joins (e.g.leftJoin
), and windowing (e.g. session windows).
With the DSL, you can define processor topologies (i.e., the logical processing plan) in your application. The steps to accomplish this are:
- Specify one or more input streams that are read from Kafka topics.
- Compose transformations on these streams.
- Write the resulting output streams back to Kafka topics, or expose the processing results of your application directly to other applications through interactive queries (e.g., via a REST API).
After the application is run, the defined processor topologies are continuously executed (i.e., the processing plan is put into action). A step-by-step guide for writing a stream processing application using the DSL is provided below.
For a complete list of available API functionality, see also the Streams API docs.
KStream
Only the Kafka Streams DSL has the notion of a KStream
.
A KStream is an abstraction of a record stream , where each data record represents a self-contained datum in the unbounded data set. Using the table analogy, data records in a record stream are always interpreted as an “INSERT” -- think: adding more entries to an append-only ledger – because no record replaces an existing row with the same key. Examples are a credit card transaction, a page view event, or a server log entry.
To illustrate, let’s imagine the following two data records are being sent to the stream:
(“alice”, 1) –> (“alice”, 3)
If your stream processing application were to sum the values per user, it would return 4
for alice
. Why? Because the second data record would not be considered an update of the previous record. Compare this behavior of KStream to KTable
below, which would return 3
for alice
.
KTable
Only the Kafka Streams DSL has the notion of a KTable
.
A KTable is an abstraction of a changelog stream , where each data record represents an update. More precisely, the value in a data record is interpreted as an “UPDATE” of the last value for the same record key, if any (if a corresponding key doesn’t exist yet, the update will be considered an INSERT). Using the table analogy, a data record in a changelog stream is interpreted as an UPSERT aka INSERT/UPDATE because any existing row with the same key is overwritten. Also, null
values are interpreted in a special way: a record with a null
value represents a “DELETE” or tombstone for the record’s key.
To illustrate, let’s imagine the following two data records are being sent to the stream:
(“alice”, 1) –> (“alice”, 3)
If your stream processing application were to sum the values per user, it would return 3
for alice
. Why? Because the second data record would be considered an update of the previous record.
Effects of Kafka’s log compaction: Another way of thinking about KStream and KTable is as follows: If you were to store a KTable into a Kafka topic, you’d probably want to enable Kafka’s log compaction feature, e.g. to save storage space.
However, it would not be safe to enable log compaction in the case of a KStream because, as soon as log compaction would begin purging older data records of the same key, it would break the semantics of the data. To pick up the illustration example again, you’d suddenly get a 3
for alice
instead of a 4
because log compaction would have removed the ("alice", 1)
data record. Hence log compaction is perfectly safe for a KTable (changelog stream) but it is a mistake for a KStream (record stream).
We have already seen an example of a changelog stream in the section streams and tables. Another example are change data capture (CDC) records in the changelog of a relational database, representing which row in a database table was inserted, updated, or deleted.
KTable also provides an ability to look up current values of data records by keys. This table-lookup functionality is available through join operations (see also Joining in the Developer Guide) as well as through Interactive Queries.
GlobalKTable
Only the Kafka Streams DSL has the notion of a GlobalKTable.
Like a KTable , a GlobalKTable is an abstraction of a changelog stream , where each data record represents an update.
A GlobalKTable differs from a KTable in the data that they are being populated with, i.e. which data from the underlying Kafka topic is being read into the respective table. Slightly simplified, imagine you have an input topic with 5 partitions. In your application, you want to read this topic into a table. Also, you want to run your application across 5 application instances for maximum parallelism.
- If you read the input topic into a KTable , then the “local” KTable instance of each application instance will be populated with data from only 1 partition of the topic’s 5 partitions.
- If you read the input topic into a GlobalKTable , then the local GlobalKTable instance of each application instance will be populated with data from all partitions of the topic.
GlobalKTable provides the ability to look up current values of data records by keys. This table-lookup functionality is available through join operations
. Note that a GlobalKTable has no notion of time in contrast to a KTable.
Benefits of global tables:
- More convenient and/or efficient joins : Notably, global tables allow you to perform star joins, they support “foreign-key” lookups (i.e., you can lookup data in the table not just by record key, but also by data in the record values), and they are more efficient when chaining multiple joins. Also, when joining against a global table, the input data does not need to be co-partitioned.
- Can be used to “broadcast” information to all the running instances of your application.
Downsides of global tables:
- Increased local storage consumption compared to the (partitioned) KTable because the entire topic is tracked.
- Increased network and Kafka broker load compared to the (partitioned) KTable because the entire topic is read.
Creating source streams from Kafka
You can easily read data from Kafka topics into your application. The following operations are supported.
Reading from Kafka | Description |
---|---|
Stream |
- input topics -> KStream
| Creates a KStream from the specified Kafka input topics and interprets the data as a record stream. A KStream
represents a partitioned record stream. (details) In the case of a KStream, the local KStream instance of every application instance will be populated with data from only a subset of the partitions of the input topic. Collectively, across all application instances, all input topic partitions are read and processed.
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.kstream.KStream;
StreamsBuilder builder = new StreamsBuilder();
KStream<String, Long> wordCounts = builder.stream(
"word-counts-input-topic", /* input topic */
Consumed.with(
Serdes.String(), /* key serde */
Serdes.Long() /* value serde */
);
If you do not specify Serdes explicitly, the default Serdes from the configuration are used. You must specify Serdes explicitly if the key or value types of the records in the Kafka input topics do not match the configured default Serdes. For information about configuring default Serdes, available Serdes, and implementing your own custom Serdes see Data Types and Serialization. Several variants of stream
exist. For example, you can specify a regex pattern for input topics to read from (note that all matching topics will be part of the same input topic group, and the work will not be parallelized for different topics if subscribed to in this way).
Table
- input topic -> KTable
| Reads the specified Kafka input topic into a KTable. The topic is interpreted as a changelog stream, where records with the same key are interpreted as UPSERT aka INSERT/UPDATE (when the record value is not null
) or as DELETE (when the value is null
) for that key. (details) In the case of a KTable, the local KTable instance of every application instance will be populated with data from only a subset of the partitions of the input topic. Collectively, across all application instances, all input topic partitions are read and processed. You must provide a name for the table (more precisely, for the internal state store that backs the table). This is required for supporting interactive queries against the table. When a name is not provided the table will not be queryable and an internal name will be provided for the state store. If you do not specify Serdes explicitly, the default Serdes from the configuration are used. You must specify Serdes explicitly if the key or value types of the records in the Kafka input topics do not match the configured default Serdes. For information about configuring default Serdes, available Serdes, and implementing your own custom Serdes see Data Types and Serialization. Several variants of table
exist, for example to specify the auto.offset.reset
policy to be used when reading from the input topic.
Global Table
- input topic -> GlobalKTable
| Reads the specified Kafka input topic into a GlobalKTable. The topic is interpreted as a changelog stream, where records with the same key are interpreted as UPSERT aka INSERT/UPDATE (when the record value is not null
) or as DELETE (when the value is null
) for that key. (details) In the case of a GlobalKTable, the local GlobalKTable instance of every application instance will be populated with data from all the partitions of the input topic. You must provide a name for the table (more precisely, for the internal state store that backs the table). This is required for supporting interactive queries against the table. When a name is not provided the table will not be queryable and an internal name will be provided for the state store.
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.kstream.GlobalKTable;
StreamsBuilder builder = new StreamsBuilder();
GlobalKTable<String, Long> wordCounts = builder.globalTable(
"word-counts-input-topic",
Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as(
"word-counts-global-store" /* table/store name */)
.withKeySerde(Serdes.String()) /* key serde */
.withValueSerde(Serdes.Long()) /* value serde */
);
You must specify Serdes explicitly if the key or value types of the records in the Kafka input topics do not match the configured default Serdes. For information about configuring default Serdes, available Serdes, and implementing your own custom Serdes see Data Types and Serialization. Several variants of globalTable
exist to e.g. specify explicit Serdes.
Transform a stream
The KStream and KTable interfaces support a variety of transformation operations. Each of these operations can be translated into one or more connected processors into the underlying processor topology. Since KStream and KTable are strongly typed, all of these transformation operations are defined as generic functions where users could specify the input and output data types.
Some KStream transformations may generate one or more KStream objects, for example: - filter
and map
on a KStream will generate another KStream - split
on KStream can generate multiple KStreams
Some others may generate a KTable object, for example an aggregation of a KStream also yields a KTable. This allows Kafka Streams to continuously update the computed value upon arrivals of out-of-order records after it has already been produced to the downstream transformation operators.
All KTable transformation operations can only generate another KTable. However, the Kafka Streams DSL does provide a special function that converts a KTable representation into a KStream. All of these transformation methods can be chained together to compose a complex processor topology.
These transformation operations are described in the following subsections:
- Stateless transformations
- Stateful transformations
Stateless transformations
Stateless transformations do not require state for processing and they do not require a state store associated with the stream processor. Kafka 0.11.0 and later allows you to materialize the result from a stateless KTable
transformation. This allows the result to be queried through interactive queries. To materialize a KTable
, each of the below stateless operations can be augmented with an optional queryableStoreName
argument.
Transformation | Description |
---|---|
Branch |
- KStream -> BranchedKStream
| Branch (or split) a KStream
based on the supplied predicates into one or more KStream
instances. (details) Predicates are evaluated in order. A record is placed to one and only one output stream on the first match: if the n-th predicate evaluates to true, the record is placed to n-th stream. If a record does not match any predicates, it will be routed to the default branch, or dropped if no default branch is created. Branching is useful, for example, to route records to different downstream topics.
KStream<String, Long> stream = ...;
Map<String, KStream<String, Long>> branches =
stream.split(Named.as("Branch-"))
.branch((key, value) -> key.startsWith("A"), /* first predicate */
Branched.as("A"))
.branch((key, value) -> key.startsWith("B"), /* second predicate */
Branched.as("B"))
.defaultBranch(Branched.as("C")) /* default branch */
);
// KStream branches.get("Branch-A") contains all records whose keys start with "A"
// KStream branches.get("Branch-B") contains all records whose keys start with "B"
// KStream branches.get("Branch-C") contains all other records
Broadcast/Multicast
- no operator
| Broadcasting a KStream
into multiple downstream operators. A record is sent to more than one operator by applying multiple operators to the same KStream
instance.
KStream<String, Long> stream = ...;
KStream<...> stream1 = stream.map(...);
KStream<...> stream2 = stream.mapValue(...);
KStream<...> stream3 = stream.flatMap(...);
Multicasting a KStream
into multiple downstream operators. In contrast to branching , which sends each record to at most one downstream branch, a multicast may send a record to any number of downstream KStream
instances. A multicast is implemented as a broadcast plus filters.
KStream<String, Long> stream = ...;
KStream<...> stream1 = stream.filter((key, value) -> key.startsWith("A")); // contains all records whose keys start with "A"
KStream<...> stream2 = stream.filter((key, value) -> key.startsWith("AB")); // contains all records whose keys start with "AB" (subset of stream1)
KStream<...> stream3 = stream.filter((key, value) -> key.contains("B")); // contains all records whose keys contains a "B" (superset of stream2)
Filter
- KStream -> KStream
- KTable -> KTable
| Evaluates a boolean function for each element and retains those for which the function returns true. (KStream details, KTable details)
KStream<String, Long> stream = ...;
// A filter that selects (keeps) only positive numbers
KStream<String, Long> onlyPositives = stream.filter((key, value) -> value > 0);
Inverse Filter
- KStream -> KStream
- KTable -> KTable
| Evaluates a boolean function for each element and drops those for which the function returns true. (KStream details, KTable details)
KStream<String, Long> stream = ...;
// An inverse filter that discards any negative numbers or zero
KStream<String, Long> onlyPositives = stream.filterNot((key, value) -> value <= 0);
FlatMap
- KStream -> KStream
| Takes one record and produces zero, one, or more records. You can modify the record keys and values, including their types. (details) Marks the stream for data re-partitioning: Applying a grouping or a join after flatMap
will result in re-partitioning of the records. If possible use flatMapValues
instead, which will not cause data re-partitioning.
KStream<Long, String> stream = ...;
KStream<String, Integer> transformed = stream.flatMap(
// Here, we generate two output records for each input record.
// We also change the key and value types.
// Example: (345L, "Hello") -> ("HELLO", 1000), ("hello", 9000)
(key, value) -> {
List<KeyValue<String, Integer>> result = new LinkedList<>();
result.add(KeyValue.pair(value.toUpperCase(), 1000));
result.add(KeyValue.pair(value.toLowerCase(), 9000));
return result;
}
);
FlatMapValues
- KStream -> KStream
| Takes one record and produces zero, one, or more records, while retaining the key of the original record. You can modify the record values and the value type. (details) flatMapValues
is preferable to flatMap
because it will not cause data re-partitioning. However, you cannot modify the key or key type like flatMap
does.
// Split a sentence into words.
KStream<byte[], String> sentences = ...;
KStream<byte[], String> words = sentences.flatMapValues(value -> Arrays.asList(value.split("\s+")));
Foreach
- KStream -> void
- KStream -> void
- KTable -> void
| Terminal operation. Performs a stateless action on each record. (details) You would use foreach
to cause side effects based on the input data (similar to peek
) and then stop further processing of the input data (unlike peek
, which is not a terminal operation). Note on processing guarantees: Any side effects of an action (such as writing to external systems) are not trackable by Kafka, which means they will typically not benefit from Kafka’s processing guarantees.
KStream<String, Long> stream = ...;
// Print the contents of the KStream to the local console.
stream.foreach((key, value) -> System.out.println(key + " => " + value));
GroupByKey
- KStream -> KGroupedStream
| Groups the records by the existing key. (details) Grouping is a prerequisite for aggregating a stream or a table and ensures that data is properly partitioned (“keyed”) for subsequent operations. When to set explicit Serdes: Variants of groupByKey
exist to override the configured default Serdes of your application, which you must do if the key and/or value types of the resulting KGroupedStream
do not match the configured default Serdes. Note Grouping vs. Windowing: A related operation is windowing, which lets you control how to “sub-group” the grouped records of the same key into so-called windows for stateful operations such as windowed aggregations or windowed joins. Causes data re-partitioning if and only if the stream was marked for re-partitioning. groupByKey
is preferable to groupBy
because it re-partitions data only if the stream was already marked for re-partitioning. However, groupByKey
does not allow you to modify the key or key type like groupBy
does.
KStream<byte[], String> stream = ...;
// Group by the existing key, using the application's configured
// default serdes for keys and values.
KGroupedStream<byte[], String> groupedStream = stream.groupByKey();
// When the key and/or value types do not match the configured
// default serdes, we must explicitly specify serdes.
KGroupedStream<byte[], String> groupedStream = stream.groupByKey(
Grouped.with(
Serdes.ByteArray(), /* key */
Serdes.String()) /* value */
);
GroupBy
- KStream -> KGroupedStream
- KTable -> KGroupedTable
| Groups the records by a new key, which may be of a different key type. When grouping a table, you may also specify a new value and value type. groupBy
is a shorthand for selectKey(...).groupByKey()
. (KStream details, KTable details) Grouping is a prerequisite for aggregating a stream or a table and ensures that data is properly partitioned (“keyed”) for subsequent operations. When to set explicit Serdes: Variants of groupBy
exist to override the configured default Serdes of your application, which you must do if the key and/or value types of the resulting KGroupedStream
or KGroupedTable
do not match the configured default Serdes. Note Grouping vs. Windowing: A related operation is windowing, which lets you control how to “sub-group” the grouped records of the same key into so-called windows for stateful operations such as windowed aggregations or windowed joins. Always causes data re-partitioning: groupBy
always causes data re-partitioning. If possible use groupByKey
instead, which will re-partition data only if required.
KStream<byte[], String> stream = ...;
KTable<byte[], String> table = ...;
// Group the stream by a new key and key type
KGroupedStream<String, String> groupedStream = stream.groupBy(
(key, value) -> value,
Grouped.with(
Serdes.String(), /* key (note: type was modified) */
Serdes.String()) /* value */
);
// Group the table by a new key and key type, and also modify the value and value type.
KGroupedTable<String, Integer> groupedTable = table.groupBy(
(key, value) -> KeyValue.pair(value, value.length()),
Grouped.with(
Serdes.String(), /* key (note: type was modified) */
Serdes.Integer()) /* value (note: type was modified) */
);
Cogroup
- KGroupedStream -> CogroupedKStream
- CogroupedKStream -> CogroupedKStream
| Cogrouping allows to aggregate multiple input streams in a single operation. The different (already grouped) input streams must have the same key type and may have different values types. KGroupedStream#cogroup() creates a new cogrouped stream with a single input stream, while CogroupedKStream#cogroup() adds a grouped stream to an existing cogrouped stream. A CogroupedKStream
may be windowed before it is aggregated. Cogroup does not cause a repartition as it has the prerequisite that the input streams are grouped. In the process of creating these groups they will have already been repartitioned if the stream was already marked for repartitioning.
KStream<byte[], String> stream = ...;
KStream<byte[], String> stream2 = ...;
// Group by the existing key, using the application's configured
// default serdes for keys and values.
KGroupedStream<byte[], String> groupedStream = stream.groupByKey();
KGroupedStream<byte[], String> groupedStream2 = stream2.groupByKey();
CogroupedKStream<byte[], String> cogroupedStream = groupedStream.cogroup(aggregator1).cogroup(groupedStream2, aggregator2);
KTable<byte[], String> table = cogroupedStream.aggregate(initializer);
KTable<byte[], String> table2 = cogroupedStream.windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofMillis(500))).aggregate(initializer);
Map
- KStream -> KStream
| Takes one record and produces one record. You can modify the record key and value, including their types. (details) Marks the stream for data re-partitioning: Applying a grouping or a join after map
will result in re-partitioning of the records. If possible use mapValues
instead, which will not cause data re-partitioning.
KStream<byte[], String> stream = ...;
// Note how we change the key and the key type (similar to `selectKey`)
// as well as the value and the value type.
KStream<String, Integer> transformed = stream.map(
(key, value) -> KeyValue.pair(value.toLowerCase(), value.length()));
Map (values only)
- KStream -> KStream
- KTable -> KTable
| Takes one record and produces one record, while retaining the key of the original record. You can modify the record value and the value type. (KStream details, KTable details) mapValues
is preferable to map
because it will not cause data re-partitioning. However, it does not allow you to modify the key or key type like map
does.
KStream<byte[], String> stream = ...;
KStream<byte[], String> uppercased = stream.mapValues(value -> value.toUpperCase());
Merge
- KStream -> KStream
| Merges records of two streams into one larger stream. (details) There is no ordering guarantee between records from different streams in the merged stream. Relative order is preserved within each input stream though (ie, records within the same input stream are processed in order)
KStream<byte[], String> stream1 = ...;
KStream<byte[], String> stream2 = ...;
KStream<byte[], String> merged = stream1.merge(stream2);
Peek
- KStream -> KStream
| Performs a stateless action on each record, and returns an unchanged stream. (details) You would use peek
to cause side effects based on the input data (similar to foreach
) and continue processing the input data (unlike foreach
, which is a terminal operation). peek
returns the input stream as-is; if you need to modify the input stream, use map
or mapValues
instead. peek
is helpful for use cases such as logging or tracking metrics or for debugging and troubleshooting. Note on processing guarantees: Any side effects of an action (such as writing to external systems) are not trackable by Kafka, which means they will typically not benefit from Kafka’s processing guarantees.
KStream<byte[], String> stream = ...;
KStream<byte[], String> unmodifiedStream = stream.peek(
(key, value) -> System.out.println("key=" + key + ", value=" + value));
- KStream -> void
| Terminal operation. Prints the records to System.out
. See Javadocs for serde and toString()
caveats. (details) Calling print()
is the same as calling foreach((key, value) -> System.out.println(key + ", " + value))
print
is mainly for debugging/testing purposes, and it will try to flush on each record print. Hence it should not be used for production usage if performance requirements are concerned.
KStream<byte[], String> stream = ...;
// print to sysout
stream.print();
// print to file with a custom label
stream.print(Printed.toFile("streams.out").withLabel("streams"));
SelectKey
- KStream -> KStream
| Assigns a new key - possibly of a new key type - to each record. (details) Calling selectKey(mapper)
is the same as calling map((key, value) -> mapper(key, value), value)
. Marks the stream for data re-partitioning: Applying a grouping or a join after selectKey
will result in re-partitioning of the records.
KStream<byte[], String> stream = ...;
// Derive a new record key from the record's value. Note how the key type changes, too.
KStream<String, String> rekeyed = stream.selectKey((key, value) -> value.split(" ")[0])
Table to Stream
- KTable -> KStream
| Get the changelog stream of this table. (details)
KTable<byte[], String> table = ...;
// Also, a variant of `toStream` exists that allows you
// to select a new key for the resulting stream.
KStream<byte[], String> stream = table.toStream();
Stream to Table
- KStream -> KTable
| Convert an event stream into a table, or say a changelog stream. (details)
KStream<byte[], String> stream = ...;
KTable<byte[], String> table = stream.toTable();
Repartition
- KStream -> KStream
| Manually trigger repartitioning of the stream with desired number of partitions. (details) repartition()
is similar to through()
however Kafka Streams will manage the topic for you. Generated topic is treated as internal topic, as a result data will be purged automatically as any other internal repartition topic. In addition, you can specify the desired number of partitions, which allows to easily scale in/out downstream sub-topologies. repartition()
operation always triggers repartitioning of the stream, as a result it can be used with embedded Processor API methods (like transform()
et al.) that do not trigger auto repartitioning when key changing operation is performed beforehand.
KStream<byte[], String> stream = ... ;
KStream<byte[], String> repartitionedStream = stream.repartition(Repartitioned.numberOfPartitions(10));
Stateful transformations
Stateful transformations depend on state for processing inputs and producing outputs and require a state store associated with the stream processor. For example, in aggregating operations, a windowing state store is used to collect the latest aggregation results per window. In join operations, a windowing state store is used to collect all of the records received so far within the defined window boundary.
Note: Following store types are used regardless of the possibly specified type (via the parameter materialized
):
- non-windowed aggregations and non-windowed KTables use TimestampedKeyValueStores or VersionedKeyValueStores, depending on whether the parameter
materialized
is versioned - time-windowed aggregations and KStream-KStream joins use TimestampedWindowStores
- session windowed aggregations use SessionStores (there is no timestamped session store as of now)
Note, that state stores are fault-tolerant. In case of failure, Kafka Streams guarantees to fully restore all state stores prior to resuming the processing. See Fault Tolerance for further information.
Available stateful transformations in the DSL include:
- Aggregating
- Joining
- Windowing (as part of aggregations and joins)
- Applying custom processors and transformers, which may be stateful, for Processor API integration
The following diagram shows their relationships:
Stateful transformations in the DSL.
Here is an example of a stateful application: the WordCount algorithm.
WordCount example:
// Assume the record values represent lines of text. For the sake of this example, you can ignore
// whatever may be stored in the record keys.
KStream<String, String> textLines = ...;
KStream<String, Long> wordCounts = textLines
// Split each text line, by whitespace, into words. The text lines are the record
// values, i.e. you can ignore whatever data is in the record keys and thus invoke
// `flatMapValues` instead of the more generic `flatMap`.
.flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\W+")))
// Group the stream by word to ensure the key of the record is the word.
.groupBy((key, word) -> word)
// Count the occurrences of each word (record key).
//
// This will change the stream type from `KGroupedStream<String, String>` to
// `KTable<String, Long>` (word -> count).
.count()
// Convert the `KTable<String, Long>` into a `KStream<String, Long>`.
.toStream();
Aggregating
After records are grouped by key via groupByKey
or groupBy
- and thus represented as either a KGroupedStream
or a KGroupedTable
, they can be aggregated via an operation such as reduce
. Aggregations are key-based operations, which means that they always operate over records (notably record values) of the same key. You can perform aggregations on windowed or non-windowed data.
Transformation | Description |
---|---|
Aggregate |
- KGroupedStream -> KTable
- KGroupedTable -> KTable
| Rolling aggregation. Aggregates the values of (non-windowed) records by the grouped key or cogrouped. Aggregating is a generalization of reduce
and allows, for example, the aggregate value to have a different type than the input values. (KGroupedStream details, KGroupedTable details KGroupedTable details) When aggregating a grouped stream , you must provide an initializer (e.g., aggValue = 0
) and an “adder” aggregator (e.g., aggValue + curValue
). When aggregating a grouped table , you must additionally provide a “subtractor” aggregator (think: aggValue - oldValue
). When aggregating a cogrouped stream , the actual aggregators are provided for each input stream in the prior cogroup()
calls, and thus you only need to provide an initializer (e.g., aggValue = 0
) Several variants of aggregate
exist, see Javadocs for details.
KGroupedStream<byte[], String> groupedStream = ...;
KGroupedTable<byte[], String> groupedTable = ...;
// Aggregating a KGroupedStream (note how the value type changes from String to Long)
KTable<byte[], Long> aggregatedStream = groupedStream.aggregate(
() -> 0L, /* initializer */
(aggKey, newValue, aggValue) -> aggValue + newValue.length(), /* adder */
Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("aggregated-stream-store") /* state store name */
.withValueSerde(Serdes.Long()); /* serde for aggregate value */
// Aggregating a KGroupedTable (note how the value type changes from String to Long)
KTable<byte[], Long> aggregatedTable = groupedTable.aggregate(
() -> 0L, /* initializer */
(aggKey, newValue, aggValue) -> aggValue + newValue.length(), /* adder */
(aggKey, oldValue, aggValue) -> aggValue - oldValue.length(), /* subtractor */
Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("aggregated-table-store") /* state store name */
.withValueSerde(Serdes.Long()) /* serde for aggregate value */
Detailed behavior of KGroupedStream
:
- Input records with
null
keys are ignored. - When a record key is received for the first time, the initializer is called (and called before the adder).
- Whenever a record with a non-
null
value is received, the adder is called.
Detailed behavior of KGroupedTable
:
- Input records with
null
keys are ignored. - When a record key is received for the first time, the initializer is called (and called before the adder and subtractor). Note that, in contrast to
KGroupedStream
, over time the initializer may be called more than once for a key as a result of having received input tombstone records for that key (see below). - When the first non-
null
value is received for a key (e.g., INSERT), then only the adder is called. - When subsequent non-
null
values are received for a key (e.g., UPDATE), then (1) the subtractor is called with the old value as stored in the table and (2) the adder is called with the new value of the input record that was just received. The subtractor is guaranteed to be called before the adder if the extracted grouping key of the old and new value is the same. The detection of this case depends on the correct implementation of the equals() method of the extracted key type. Otherwise, the order of execution for the subtractor and adder is not defined. - When a tombstone record - i.e. a record with a
null
value - is received for a key (e.g., DELETE), then only the subtractor is called. Note that, whenever the subtractor returns anull
value itself, then the corresponding key is removed from the resultingKTable
. If that happens, any next input record for that key will trigger the initializer again.
See the example at the bottom of this section for a visualization of the aggregation semantics.
Aggregate (windowed)
- KGroupedStream -> KTable
| Windowed aggregation. Aggregates the values of records, per window, by the grouped key. Aggregating is a generalization of reduce
and allows, for example, the aggregate value to have a different type than the input values. (TimeWindowedKStream details, SessionWindowedKStream details) You must provide an initializer (e.g., aggValue = 0
), “adder” aggregator (e.g., aggValue + curValue
), and a window. When windowing based on sessions, you must additionally provide a “session merger” aggregator (e.g., mergedAggValue = leftAggValue + rightAggValue
). The windowed aggregate
turns a TimeWindowedKStream<K, V>
or SessionWindowedKStream<K, V>
into a windowed KTable<Windowed<K>, V>
. Several variants of aggregate
exist, see Javadocs for details.
import java.time.Duration;
KGroupedStream<String, Long> groupedStream = ...;
// Aggregating with time-based windowing (here: with 5-minute tumbling windows)
KTable<Windowed<String>, Long> timeWindowedAggregatedStream = groupedStream.windowedBy(Duration.ofMinutes(5))
.aggregate(
() -> 0L, /* initializer */
(aggKey, newValue, aggValue) -> aggValue + newValue, /* adder */
Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("time-windowed-aggregated-stream-store") /* state store name */
.withValueSerde(Serdes.Long())); /* serde for aggregate value */
// Aggregating with time-based windowing (here: with 5-minute sliding windows and 30-minute grace period)
KTable<Windowed<String>, Long> timeWindowedAggregatedStream = groupedStream.windowedBy(SlidingWindows.ofTimeDifferenceAndGrace(Duration.ofMinutes(5), Duration.ofMinutes(30)))
.aggregate(
() -> 0L, /* initializer */
(aggKey, newValue, aggValue) -> aggValue + newValue, /* adder */
Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("time-windowed-aggregated-stream-store") /* state store name */
.withValueSerde(Serdes.Long())); /* serde for aggregate value */
// Aggregating with session-based windowing (here: with an inactivity gap of 5 minutes)
KTable<Windowed<String>, Long> sessionizedAggregatedStream = groupedStream.windowedBy(SessionWindows.ofInactivityGapWithNoGrace(Duration.ofMinutes(5)).
aggregate(
() -> 0L, /* initializer */
(aggKey, newValue, aggValue) -> aggValue + newValue, /* adder */
(aggKey, leftAggValue, rightAggValue) -> leftAggValue + rightAggValue, /* session merger */
Materialized.<String, Long, SessionStore<Bytes, byte[]>>as("sessionized-aggregated-stream-store") /* state store name */
.withValueSerde(Serdes.Long())); /* serde for aggregate value */
Detailed behavior:
- The windowed aggregate behaves similar to the rolling aggregate described above. The additional twist is that the behavior applies per window.
- Input records with
null
keys are ignored in general. - When a record key is received for the first time for a given window, the initializer is called (and called before the adder).
- Whenever a record with a non-
null
value is received for a given window, the adder is called. - When using session windows: the session merger is called whenever two sessions are being merged.
See the example at the bottom of this section for a visualization of the aggregation semantics.
Count
- KGroupedStream -> KTable
- KGroupedTable -> KTable
| Rolling aggregation. Counts the number of records by the grouped key. (KGroupedStream details, KGroupedTable details) Several variants of count
exist, see Javadocs for details.
KGroupedStream<String, Long> groupedStream = ...;
KGroupedTable<String, Long> groupedTable = ...;
// Counting a KGroupedStream
KTable<String, Long> aggregatedStream = groupedStream.count();
// Counting a KGroupedTable
KTable<String, Long> aggregatedTable = groupedTable.count();
Detailed behavior for KGroupedStream
:
- Input records with
null
keys or values are ignored.
Detailed behavior for KGroupedTable
:
- Input records with
null
keys are ignored. Records withnull
values are not ignored but interpreted as “tombstones” for the corresponding key, which indicate the deletion of the key from the table.
Count (windowed)
- KGroupedStream -> KTable
| Windowed aggregation. Counts the number of records, per window, by the grouped key. (TimeWindowedKStream details, SessionWindowedKStream details) The windowed count
turns a TimeWindowedKStream<K, V>
or SessionWindowedKStream<K, V>
into a windowed KTable<Windowed<K>, V>
. Several variants of count
exist, see Javadocs for details.
import java.time.Duration;
KGroupedStream<String, Long> groupedStream = ...;
// Counting a KGroupedStream with time-based windowing (here: with 5-minute tumbling windows)
KTable<Windowed<String>, Long> aggregatedStream = groupedStream.windowedBy(
TimeWindows.ofSizeWithNoGrace(Duration.ofMinutes(5))) /* time-based window */
.count();
// Counting a KGroupedStream with time-based windowing (here: with 5-minute sliding windows and 30-minute grace period)
KTable<Windowed<String>, Long> aggregatedStream = groupedStream.windowedBy(
SlidingWindows.ofTimeDifferenceAndGrace(Duration.ofMinutes(5), Duration.ofMinutes(30))) /* time-based window */
.count();
// Counting a KGroupedStream with session-based windowing (here: with 5-minute inactivity gaps)
KTable<Windowed<String>, Long> aggregatedStream = groupedStream.windowedBy(
SessionWindows.ofInactivityGapWithNoGrace(Duration.ofMinutes(5))) /* session window */
.count();
Detailed behavior:
- Input records with
null
keys or values are ignored.
Reduce
- KGroupedStream -> KTable
- KGroupedTable -> KTable
| Rolling aggregation. Combines the values of (non-windowed) records by the grouped key. The current record value is combined with the last reduced value, and a new reduced value is returned. The result value type cannot be changed, unlike aggregate
. (KGroupedStream details, KGroupedTable details) When reducing a grouped stream , you must provide an “adder” reducer (e.g., aggValue + curValue
). When reducing a grouped table , you must additionally provide a “subtractor” reducer (e.g., aggValue - oldValue
). Several variants of reduce
exist, see Javadocs for details.
KGroupedStream<String, Long> groupedStream = ...;
KGroupedTable<String, Long> groupedTable = ...;
// Reducing a KGroupedStream
KTable<String, Long> aggregatedStream = groupedStream.reduce(
(aggValue, newValue) -> aggValue + newValue /* adder */);
// Reducing a KGroupedTable
KTable<String, Long> aggregatedTable = groupedTable.reduce(
(aggValue, newValue) -> aggValue + newValue, /* adder */
(aggValue, oldValue) -> aggValue - oldValue /* subtractor */);
Detailed behavior for KGroupedStream
:
- Input records with
null
keys are ignored in general. - When a record key is received for the first time, then the value of that record is used as the initial aggregate value.
- Whenever a record with a non-
null
value is received, the adder is called.
Detailed behavior for KGroupedTable
:
- Input records with
null
keys are ignored in general. - When a record key is received for the first time, then the value of that record is used as the initial aggregate value. Note that, in contrast to
KGroupedStream
, over time this initialization step may happen more than once for a key as a result of having received input tombstone records for that key (see below). - When the first non-
null
value is received for a key (e.g., INSERT), then only the adder is called. - When subsequent non-
null
values are received for a key (e.g., UPDATE), then (1) the subtractor is called with the old value as stored in the table and (2) the adder is called with the new value of the input record that was just received. The subtractor is guaranteed be called before the adder if the extracted grouping key of the old and new value is the same. The detection of this case depends on the correct implementation of the equals() method of the extracted key type. Otherwise, the order of execution for the subtractor and adder is not defined. - When a tombstone record - i.e. a record with a
null
value - is received for a key (e.g., DELETE), then only the subtractor is called. Note that, whenever the subtractor returns anull
value itself, then the corresponding key is removed from the resultingKTable
. If that happens, any next input record for that key will re-initialize its aggregate value.
See the example at the bottom of this section for a visualization of the aggregation semantics.
Reduce (windowed)
- KGroupedStream -> KTable
| Windowed aggregation. Combines the values of records, per window, by the grouped key. The current record value is combined with the last reduced value, and a new reduced value is returned. Records with null
key or value are ignored. The result value type cannot be changed, unlike aggregate
. (TimeWindowedKStream details, SessionWindowedKStream details) The windowed reduce
turns a turns a TimeWindowedKStream<K, V>
or a SessionWindowedKStream<K, V>
into a windowed KTable<Windowed<K>, V>
. Several variants of reduce
exist, see Javadocs for details.
import java.time.Duration;
KGroupedStream<String, Long> groupedStream = ...;
// Aggregating with time-based windowing (here: with 5-minute tumbling windows)
KTable<Windowed<String>, Long> timeWindowedAggregatedStream = groupedStream.windowedBy(
TimeWindows.ofSizeWithNoGrace(Duration.ofMinutes(5)) /* time-based window */)
.reduce(
(aggValue, newValue) -> aggValue + newValue /* adder */
);
// Aggregating with time-based windowing (here: with 5-minute sliding windows and 30-minute grace)
KTable<Windowed<String>, Long> timeWindowedAggregatedStream = groupedStream.windowedBy(
SlidingWindows.ofTimeDifferenceAndGrace(Duration.ofMinutes(5), Duration.ofMinutes(30))) /* time-based window */)
.reduce(
(aggValue, newValue) -> aggValue + newValue /* adder */
);
// Aggregating with session-based windowing (here: with an inactivity gap of 5 minutes)
KTable<Windowed<String>, Long> sessionzedAggregatedStream = groupedStream.windowedBy(
SessionWindows.ofInactivityGapWithNoGrace(Duration.ofMinutes(5))) /* session window */
.reduce(
(aggValue, newValue) -> aggValue + newValue /* adder */
);
Detailed behavior:
- The windowed reduce behaves similar to the rolling reduce described above. The additional twist is that the behavior applies per window.
- Input records with
null
keys are ignored in general. - When a record key is received for the first time for a given window, then the value of that record is used as the initial aggregate value.
- Whenever a record with a non-
null
value is received for a given window, the adder is called.
See the example at the bottom of this section for a visualization of the aggregation semantics.
Example of semantics for stream aggregations: A KGroupedStream
-> KTable
example is shown below. The streams and the table are initially empty. Bold font is used in the column for “KTable aggregated
” to highlight changed state. An entry such as (hello, 1)
denotes a record with key hello
and value 1
. To improve the readability of the semantics table you can assume that all records are processed in timestamp order.
// Key: word, value: count
KStream<String, Integer> wordCounts = ...;
KGroupedStream<String, Integer> groupedStream = wordCounts
.groupByKey(Grouped.with(Serdes.String(), Serdes.Integer()));
KTable<String, Integer> aggregated = groupedStream.aggregate(
() -> 0, /* initializer */
(aggKey, newValue, aggValue) -> aggValue + newValue, /* adder */
Materialized.<String, Long, KeyValueStore<Bytes, byte[]>as("aggregated-stream-store" /* state store name */)
.withKeySerde(Serdes.String()) /* key serde */
.withValueSerde(Serdes.Integer()); /* serde for aggregate value */
Note
Impact of record caches : For illustration purposes, the column “KTable aggregated
” below shows the table’s state changes over time in a very granular way. In practice, you would observe state changes in such a granular way only when record caches are disabled (default: enabled). When record caches are enabled, what might happen for example is that the output results of the rows with timestamps 4 and 5 would be compacted, and there would only be a single state update for the key kafka
in the KTable (here: from (kafka 1)
directly to (kafka, 3)
. Typically, you should only disable record caches for testing or debugging purposes - under normal circumstances it is better to leave record caches enabled.
KStream wordCounts | KGroupedStream groupedStream | KTable aggregated | |
---|---|---|---|
Timestamp | Input record | Grouping | Initializer |
1 | (hello, 1) | (hello, 1) | 0 (for hello) |
2 | (kafka, 1) | (kafka, 1) | 0 (for kafka) |
3 | (streams, 1) | (streams, 1) | 0 (for streams) |
4 | (kafka, 1) | (kafka, 1) | |
5 | (kafka, 1) | (kafka, 1) | |
6 | (streams, 1) | (streams, 1) |
Example of semantics for table aggregations: A KGroupedTable
-> KTable
example is shown below. The tables are initially empty. Bold font is used in the column for “KTable aggregated
” to highlight changed state. An entry such as (hello, 1)
denotes a record with key hello
and value 1
. To improve the readability of the semantics table you can assume that all records are processed in timestamp order.
// Key: username, value: user region (abbreviated to "E" for "Europe", "A" for "Asia")
KTable<String, String> userProfiles = ...;
// Re-group `userProfiles`. Don't read too much into what the grouping does:
// its prime purpose in this example is to show the *effects* of the grouping
// in the subsequent aggregation.
KGroupedTable<String, Integer> groupedTable = userProfiles
.groupBy((user, region) -> KeyValue.pair(region, user.length()), Serdes.String(), Serdes.Integer());
KTable<String, Integer> aggregated = groupedTable.aggregate(
() -> 0, /* initializer */
(aggKey, newValue, aggValue) -> aggValue + newValue, /* adder */
(aggKey, oldValue, aggValue) -> aggValue - oldValue, /* subtractor */
Materialized.<String, Long, KeyValueStore<Bytes, byte[]>as("aggregated-table-store" /* state store name */)
.withKeySerde(Serdes.String()) /* key serde */
.withValueSerde(Serdes.Integer()); /* serde for aggregate value */
Note
Impact of record caches : For illustration purposes, the column “KTable aggregated
” below shows the table’s state changes over time in a very granular way. In practice, you would observe state changes in such a granular way only when record caches are disabled (default: enabled). When record caches are enabled, what might happen for example is that the output results of the rows with timestamps 4 and 5 would be compacted, and there would only be a single state update for the key kafka
in the KTable (here: from (kafka 1)
directly to (kafka, 3)
. Typically, you should only disable record caches for testing or debugging purposes - under normal circumstances it is better to leave record caches enabled.
KTable userProfiles | KGroupedTable groupedTable | KTable aggregated | |
---|---|---|---|
Timestamp | Input record | Interpreted as | Grouping |
1 | (alice, E) | INSERT alice | (E, 5) |
2 | (bob, A) | INSERT bob | (A, 3) |
3 | (charlie, A) | INSERT charlie | (A, 7) |
4 | (alice, A) | UPDATE alice | (A, 5) |
5 | (charlie, null) | DELETE charlie | (null, 7) |
6 | (null, E) | ignored | |
7 | (bob, E) | UPDATE bob | (E, 3) |
Joining
Streams and tables can also be joined. Many stream processing applications in practice are coded as streaming joins. For example, applications backing an online shop might need to access multiple, updating database tables (e.g. sales prices, inventory, customer information) in order to enrich a new data record (e.g. customer transaction) with context information. That is, scenarios where you need to perform table lookups at very large scale and with a low processing latency. Here, a popular pattern is to make the information in the databases available in Kafka through so-called change data capture in combination with Kafka’s Connect API, and then implementing applications that leverage the Streams API to perform very fast and efficient local joins of such tables and streams, rather than requiring the application to make a query to a remote database over the network for each record. In this example, the KTable concept in Kafka Streams would enable you to track the latest state (e.g., snapshot) of each table in a local state store, thus greatly reducing the processing latency as well as reducing the load of the remote databases when doing such streaming joins.
The following join operations are supported, see also the diagram in the overview section of Stateful Transformations. Depending on the operands, joins are either windowed joins or non-windowed joins.
Join operands | Type | (INNER) JOIN | LEFT JOIN | OUTER JOIN |
---|---|---|---|---|
KStream-to-KStream | Windowed | Supported | Supported | Supported |
KTable-to-KTable | Non-windowed | Supported | Supported | Supported |
KTable-to-KTable Foreign-Key Join | Non-windowed | Supported | Supported | Not Supported |
KStream-to-KTable | Non-windowed | Supported | Supported | Not Supported |
KStream-to-GlobalKTable | Non-windowed | Supported | Supported | Not Supported |
KTable-to-GlobalKTable | N/A | Not Supported | Not Supported | Not Supported |
Each case is explained in more detail in the subsequent sections.
Join co-partitioning requirements
For equi-joins, input data must be co-partitioned when joining. This ensures that input records with the same key from both sides of the join, are delivered to the same stream task during processing. It is your responsibility to ensure data co-partitioning when joining. Co-partitioning is not required when performing KTable-KTable Foreign-Key joins and Global KTable joins.
The requirements for data co-partitioning are:
- The input topics of the join (left side and right side) must have the same number of partitions.
- All applications that write to the input topics must have the same partitioning strategy so that records with the same key are delivered to same partition number. In other words, the keyspace of the input data must be distributed across partitions in the same manner. This means that, for example, applications that use Kafka’s Java Producer API must use the same partitioner (cf. the producer setting
"partitioner.class"
akaProducerConfig.PARTITIONER_CLASS_CONFIG
), and applications that use the Kafka’s Streams API must use the sameStreamPartitioner
for operations such asKStream#to()
. The good news is that, if you happen to use the default partitioner-related settings across all applications, you do not need to worry about the partitioning strategy.
Why is data co-partitioning required? Because KStream-KStream, KTable-KTable, and KStream-KTable joins are performed based on the keys of records (e.g., leftRecord.key == rightRecord.key
), it is required that the input streams/tables of a join are co-partitioned by key.
There are two exceptions where co-partitioning is not required. For KStream-GlobalKTable joins joins, co-partitioning is not required because all partitions of the GlobalKTable
’s underlying changelog stream are made available to each KafkaStreams
instance. That is, each instance has a full copy of the changelog stream. Further, a KeyValueMapper
allows for non-key based joins from the KStream
to the GlobalKTable
. KTable-KTable Foreign-Key joins also do not require co-partitioning. Kafka Streams internally ensures co-partitioning for Foreign-Key joins.
Note
Kafka Streams partly verifies the co-partitioning requirement: During the partition assignment step, i.e. at runtime, Kafka Streams verifies whether the number of partitions for both sides of a join are the same. If they are not, a TopologyBuilderException
(runtime exception) is being thrown. Note that Kafka Streams cannot verify whether the partitioning strategy matches between the input streams/tables of a join - it is up to the user to ensure that this is the case.
Ensuring data co-partitioning: If the inputs of a join are not co-partitioned yet, you must ensure this manually. You may follow a procedure such as outlined below. It is recommended to repartition the topic with fewer partitions to match the larger partition number of avoid bottlenecks. Technically it would also be possible to repartition the topic with more partitions to the smaller partition number. For stream-table joins, it’s recommended to repartition the KStream because repartitioning a KTable might result in a second state store. For table-table joins, you might also consider to size of the KTables and repartition the smaller KTable.
Identify the input KStream/KTable in the join whose underlying Kafka topic has the smaller number of partitions. Let’s call this stream/table “SMALLER”, and the other side of the join “LARGER”. To learn about the number of partitions of a Kafka topic you can use, for example, the CLI tool
bin/kafka-topics
with the--describe
option.Within your application, re-partition the data of “SMALLER”. You must ensure that, when repartitioning the data with
repartition
, the same partitioner is used as for “LARGER”.
* If "SMALLER" is a KStream: `KStream#repartition(Repartitioned.numberOfPartitions(...))`. * If "SMALLER" is a KTable: `KTable#toStream#repartition(Repartitioned.numberOfPartitions(...).toTable())`.
- Within your application, perform the join between “LARGER” and the new stream/table.
KStream-KStream Join
KStream-KStream joins are always windowed joins, because otherwise the size of the internal state store used to perform the join - e.g., a sliding window or “buffer” - would grow indefinitely. For stream-stream joins it’s important to highlight that a new input record on one side will produce a join output for each matching record on the other side, and there can be multiple such matching records in a given join window (cf. the row with timestamp 15 in the join semantics table below, for example).
Join output records are effectively created as follows, leveraging the user-supplied ValueJoiner
:
KeyValue<K, LV> leftRecord = ...;
KeyValue<K, RV> rightRecord = ...;
ValueJoiner<LV, RV, JV> joiner = ...;
KeyValue<K, JV> joinOutputRecord = KeyValue.pair(
leftRecord.key, /* by definition, leftRecord.key == rightRecord.key */
joiner.apply(leftRecord.value, rightRecord.value)
);
Transformation | Description |
---|---|
Inner Join (windowed) |
- (KStream, KStream) -> KStream
| Performs an INNER JOIN of this stream with another stream. Even though this operation is windowed, the joined stream will be of type KStream<K, ...>
rather than KStream<Windowed<K>, ...>
. (details) Data must be co-partitioned : The input data for both sides must be co-partitioned. Causes data re-partitioning of a stream if and only if the stream was marked for re-partitioning (if both are marked, both are re-partitioned). Several variants of join
exists, see the Javadocs for details.
import java.time.Duration;
KStream<String, Long> left = ...;
KStream<String, Double> right = ...;
KStream<String, String> joined = left.join(right,
(leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */
JoinWindows.ofTimeDifferenceWithNoGrace(Duration.ofMinutes(5)),
Joined.with(
Serdes.String(), /* key */
Serdes.Long(), /* left value */
Serdes.Double()) /* right value */
);
Detailed behavior:
- The join is key-based , i.e. with the join predicate
leftRecord.key == rightRecord.key
, and window-based , i.e. two input records are joined if and only if their timestamps are “close” to each other as defined by the user-suppliedJoinWindows
, i.e. the window defines an additional join predicate over the record timestamps. - The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied
ValueJoiner
will be called to produce join output records.
* Input records with a `null` key or a `null` value are ignored and do not trigger the join.
See the semantics overview at the bottom of this section for a detailed description.
Left Join (windowed)
- (KStream, KStream) -> KStream
| Performs a LEFT JOIN of this stream with another stream. Even though this operation is windowed, the joined stream will be of type KStream<K, ...>
rather than KStream<Windowed<K>, ...>
. (details) Data must be co-partitioned : The input data for both sides must be co-partitioned. Causes data re-partitioning of a stream if and only if the stream was marked for re-partitioning (if both are marked, both are re-partitioned). Several variants of leftJoin
exists, see the Javadocs for details.
import java.time.Duration;
KStream<String, Long> left = ...;
KStream<String, Double> right = ...;
KStream<String, String> joined = left.leftJoin(right,
(leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */
JoinWindows.ofTimeDifferenceWithNoGrace(Duration.ofMinutes(5)),
Joined.with(
Serdes.String(), /* key */
Serdes.Long(), /* left value */
Serdes.Double()) /* right value */
);
Detailed behavior:
- The join is key-based , i.e. with the join predicate
leftRecord.key == rightRecord.key
, and window-based , i.e. two input records are joined if and only if their timestamps are “close” to each other as defined by the user-suppliedJoinWindows
, i.e. the window defines an additional join predicate over the record timestamps. - The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied
ValueJoiner
will be called to produce join output records.
* Input records with a `null` value are ignored and do not trigger the join.
- For each input record on the left side that does not have any match on the right side, the
ValueJoiner
will be called withValueJoiner#apply(leftRecord.value, null)
; this explains the row with timestamp=60 and timestampe=80 in the table below, which lists[E, null]
and[F, null]
in the LEFT JOIN column. Note that these left results are emitted after the specified grace period passed. Caution: using the deprecatedJoinWindows.of(...).grace(...)
API might result in eagerly emitted spurious left results.
See the semantics overview at the bottom of this section for a detailed description.
Outer Join (windowed)
- (KStream, KStream) -> KStream
| Performs an OUTER JOIN of this stream with another stream. Even though this operation is windowed, the joined stream will be of type KStream<K, ...>
rather than KStream<Windowed<K>, ...>
. (details) Data must be co-partitioned : The input data for both sides must be co-partitioned. Causes data re-partitioning of a stream if and only if the stream was marked for re-partitioning (if both are marked, both are re-partitioned). Several variants of outerJoin
exists, see the Javadocs for details.
import java.time.Duration;
KStream<String, Long> left = ...;
KStream<String, Double> right = ...;
KStream<String, String> joined = left.outerJoin(right,
(leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */
JoinWindows.ofTimeDifferenceWithNoGrace(Duration.ofMinutes(5)),
Joined.with(
Serdes.String(), /* key */
Serdes.Long(), /* left value */
Serdes.Double()) /* right value */
);
Detailed behavior:
- The join is key-based , i.e. with the join predicate
leftRecord.key == rightRecord.key
, and window-based , i.e. two input records are joined if and only if their timestamps are “close” to each other as defined by the user-suppliedJoinWindows
, i.e. the window defines an additional join predicate over the record timestamps. - The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied
ValueJoiner
will be called to produce join output records.
* Input records with a `null` value are ignored and do not trigger the join.
- For each input record on one side that does not have any match on the other side, the
ValueJoiner
will be called withValueJoiner#apply(leftRecord.value, null)
orValueJoiner#apply(null, rightRecord.value)
, respectively; this explains the row with timestamp=60, timestamp=80, and timestamp=100 in the table below, which lists[E, null]
,[F, null]
, and[null, f]
in the OUTER JOIN column. Note that these left and right results are emitted after the specified grace period passed. Caution: using the deprecatedJoinWindows.of(...).grace(...)
API might result in eagerly emitted spurious left/right results.
See the semantics overview at the bottom of this section for a detailed description.
Semantics of stream-stream joins: The semantics of the various stream-stream join variants are explained below. To improve the readability of the table, assume that (1) all records have the same key (and thus the key in the table is omitted), and (2) all records are processed in timestamp order. We assume a join window size of 15 seconds with a grace period of 5 seconds.
Note: If you use the old and now deprecated API to specify the grace period, i.e., JoinWindows.of(...).grace(...)
, left/outer join results are emitted eagerly, and the observed result might differ from the result shown below.
The columns INNER JOIN, LEFT JOIN, and OUTER JOIN denote what is passed as arguments to the user-supplied ValueJoiner for the join
, leftJoin
, and outerJoin
methods, respectively, whenever a new input record is received on either side of the join. An empty table cell denotes that the ValueJoiner
is not called at all.
Timestamp | Left (KStream) | Right (KStream) | (INNER) JOIN | LEFT JOIN | OUTER JOIN |
---|---|---|---|---|---|
1 | null | ||||
2 | null | ||||
3 | A | ||||
4 | a | [A, a] | [A, a] | [A, a] | |
5 | B | [B, a] | [B, a] | [B, a] | |
6 | b | [A, b], [B, b] | [A, b], [B, b] | [A, b], [B, b] | |
7 | null | ||||
8 | null | ||||
9 | C | [C, a], [C, b] | [C, a], [C, b] | [C, a], [C, b] | |
10 | c | [A, c], [B, c], [C, c] | [A, c], [B, c], [C, c] | [A, c], [B, c], [C, c] | |
11 | null | ||||
12 | null | ||||
13 | null | ||||
14 | d | [A, d], [B, d], [C, d] | [A, d], [B, d], [C, d] | [A, d], [B, d], [C, d] | |
15 | D | [D, a], [D, b], [D, c], [D, d] | [D, a], [D, b], [D, c], [D, d] | [D, a], [D, b], [D, c], [D, d] | |
… | |||||
40 | E | ||||
… | |||||
60 | F | [E, null] | [E, null] | ||
… | |||||
80 | f | [F, null] | [F, null] | ||
… | |||||
100 | G | [null, f] |
KTable-KTable Equi-Join
KTable-KTable equi-joins are always non-windowed joins. They are designed to be consistent with their counterparts in relational databases. The changelog streams of both KTables are materialized into local state stores to represent the latest snapshot of their table duals. The join result is a new KTable that represents the changelog stream of the join operation.
Join output records are effectively created as follows, leveraging the user-supplied ValueJoiner
:
KeyValue<K, LV> leftRecord = ...;
KeyValue<K, RV> rightRecord = ...;
ValueJoiner<LV, RV, JV> joiner = ...;
KeyValue<K, JV> joinOutputRecord = KeyValue.pair(
leftRecord.key, /* by definition, leftRecord.key == rightRecord.key */
joiner.apply(leftRecord.value, rightRecord.value)
);
Transformation | Description |
---|---|
Inner Join |
- (KTable, KTable) -> KTable
| Performs an INNER JOIN of this table with another table. The result is an ever-updating KTable that represents the “current” result of the join. (details) Data must be co-partitioned : The input data for both sides must be co-partitioned.
KTable<String, Long> left = ...;
KTable<String, Double> right = ...;
KTable<String, String> joined = left.join(right,
(leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */
);
Detailed behavior:
- The join is key-based , i.e. with the join predicate
leftRecord.key == rightRecord.key
. - The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied
ValueJoiner
will be called to produce join output records.
* Input records with a `null` key are ignored and do not trigger the join. * Input records with a `null` value are interpreted as _tombstones_ for the corresponding key, which indicate the deletion of the key from the table. Tombstones do not trigger the join. When an input tombstone is received, then an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding key actually exists already in the join result KTable). * When joining versioned tables, out-of-order input records, i.e., those for which another record from the same table, with the same key and a larger timestamp, has already been processed, are ignored and do not trigger the join.
See the semantics overview at the bottom of this section for a detailed description.
Left Join
- (KTable, KTable) -> KTable
| Performs a LEFT JOIN of this table with another table. (details) Data must be co-partitioned : The input data for both sides must be co-partitioned.
KTable<String, Long> left = ...;
KTable<String, Double> right = ...;
KTable<String, String> joined = left.leftJoin(right,
(leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */
);
Detailed behavior:
- The join is key-based , i.e. with the join predicate
leftRecord.key == rightRecord.key
. - The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied
ValueJoiner
will be called to produce join output records.
* Input records with a `null` key are ignored and do not trigger the join. * Input records with a `null` value are interpreted as _tombstones_ for the corresponding key, which indicate the deletion of the key from the table. Right-tombstones trigger the join, but left-tombstones don't: when an input tombstone is received, an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding key actually exists already in the join result KTable). * When joining versioned tables, out-of-order input records, i.e., those for which another record from the same table, with the same key and a larger timestamp, has already been processed, are ignored and do not trigger the join.
- For each input record on the left side that does not have any match on the right side, the
ValueJoiner
will be called withValueJoiner#apply(leftRecord.value, null)
; this explains the row with timestamp=3 in the table below, which lists[A, null]
in the LEFT JOIN column.
See the semantics overview at the bottom of this section for a detailed description.
Outer Join
- (KTable, KTable) -> KTable
| Performs an OUTER JOIN of this table with another table. (details) Data must be co-partitioned : The input data for both sides must be co-partitioned.
KTable<String, Long> left = ...;
KTable<String, Double> right = ...;
KTable<String, String> joined = left.outerJoin(right,
(leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */
);
Detailed behavior:
- The join is key-based , i.e. with the join predicate
leftRecord.key == rightRecord.key
. - The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied
ValueJoiner
will be called to produce join output records.
* Input records with a `null` key are ignored and do not trigger the join. * Input records with a `null` value are interpreted as _tombstones_ for the corresponding key, which indicate the deletion of the key from the table. Tombstones may trigger joins, depending on the content in the left and right tables. When an input tombstone is received, an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding key actually exists already in the join result KTable). * When joining versioned tables, out-of-order input records, i.e., those for which another record from the same table, with the same key and a larger timestamp, has already been processed, are ignored and do not trigger the join.
- For each input record on one side that does not have any match on the other side, the
ValueJoiner
will be called withValueJoiner#apply(leftRecord.value, null)
orValueJoiner#apply(null, rightRecord.value)
, respectively; this explains the rows with timestamp=3 and timestamp=7 in the table below, which list[A, null]
and[null, b]
, respectively, in the OUTER JOIN column.
See the semantics overview at the bottom of this section for a detailed description.
Semantics of table-table equi-joins: The semantics of the various table-table equi-join variants are explained below. To improve the readability of the table, you can assume that (1) all records have the same key (and thus the key in the table is omitted) and that (2) all records are processed in timestamp order. The columns INNER JOIN, LEFT JOIN, and OUTER JOIN denote what is passed as arguments to the user-supplied ValueJoiner for the join
, leftJoin
, and outerJoin
methods, respectively, whenever a new input record is received on either side of the join. An empty table cell denotes that the ValueJoiner
is not called at all.
Timestamp | Left (KTable) | Right (KTable) | (INNER) JOIN | LEFT JOIN | OUTER JOIN |
---|---|---|---|---|---|
1 | null | ||||
2 | null | ||||
3 | A | [A, null] | [A, null] | ||
4 | a | [A, a] | [A, a] | [A, a] | |
5 | B | [B, a] | [B, a] | [B, a] | |
6 | b | [B, b] | [B, b] | [B, b] | |
7 | null | null | null | [null, b] | |
8 | null | null | |||
9 | C | [C, null] | [C, null] | ||
10 | c | [C, c] | [C, c] | [C, c] | |
11 | null | null | [C, null] | [C, null] | |
12 | null | null | null | ||
13 | null | ||||
14 | d | [null, d] | |||
15 | D | [D, d] | [D, d] | [D, d] |
KTable-KTable Foreign-Key Join
KTable-KTable foreign-key joins are always non-windowed joins. Foreign-key joins are analogous to joins in SQL. As a rough example:
SELECT ... FROM {this KTable} JOIN {other KTable} ON {other.key} = {result of foreignKeyExtractor(this.value)} ...
The output of the operation is a new KTable containing the join result.
The changelog streams of both KTables are materialized into local state stores to represent the latest snapshot of their table duals. A foreign-key extractor function is applied to the left record, with a new intermediate record created and is used to lookup and join with the corresponding primary key on the right hand side table. The result is a new KTable that represents the changelog stream of the join operation.
The left KTable can have multiple records which map to the same key on the right KTable. An update to a single left KTable entry may result in a single output event, provided the corresponding key exists in the right KTable. Consequently, a single update to a right KTable entry will result in an update for each record in the left KTable that has the same foreign key.
Transformation | Description |
---|---|
Inner Join |
- (KTable, KTable) -> KTable
| Performs a foreign-key INNER JOIN of this table with another table. The result is an ever-updating KTable that represents the “current” result of the join. (details)
KTable<String, Long> left = ...;
KTable<Long, Double> right = ...;
//This foreignKeyExtractor simply uses the left-value to map to the right-key.
Function<Long, Long> foreignKeyExtractor = (v) -> v;
//Alternative: with access to left table key
BiFunction<String, Long, Long> foreignKeyExtractor = (k, v) -> v;
KTable<String, String> joined = left.join(right, foreignKeyExtractor,
(leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */
);
Detailed behavior:
The join is key-based , i.e. with the join predicate:
foreignKeyExtractor.apply(leftRecord.value) == rightRecord.key
The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied
ValueJoiner
will be called to produce join output records.
* Records for which the `foreignKeyExtractor` produces `null` are ignored and do not trigger a join. If you want to join with `null` foreign keys, use a suitable sentinel value to do so (i.e. `"NULL"` for a String field, or `-1` for an auto-incrementing integer field). * Input records with a `null` value are interpreted as _tombstones_ for the corresponding key, which indicate the deletion of the key from the table. Tombstones do not trigger the join. When an input tombstone is received, then an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding key actually exists already in the join result KTable). * When joining versioned tables, out-of-order input records, i.e., those for which another record from the same table, with the same key and a larger timestamp, has already been processed, are ignored and do not trigger the join.
See the semantics overview at the bottom of this section for a detailed description.
Left Join
- (KTable, KTable) -> KTable
| Performs a foreign-key LEFT JOIN of this table with another table. (details)
KTable<String, Long> left = ...;
KTable<Long, Double> right = ...;
//This foreignKeyExtractor simply uses the left-value to map to the right-key.
Function<Long, Long> foreignKeyExtractor = (v) -> v;
//Alternative: with access to left table key
BiFunction<String, Long, Long> foreignKeyExtractor = (k, v) -> v;
KTable<String, String> joined = left.leftJoin(right, foreignKeyExtractor,
(leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */
);
Detailed behavior:
The join is key-based , i.e. with the join predicate:
foreignKeyExtractor.apply(leftRecord.value) == rightRecord.key
The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied
ValueJoiner
will be called to produce join output records.
* Input records with a `null` value are interpreted as _tombstones_ for the corresponding key, which indicate the deletion of the key from the table. Right-tombstones trigger the join, but left-tombstones don't: when an input tombstone is received, then an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding key actually exists already in the join result KTable). * When joining versioned tables, out-of-order input records, i.e., those for which another record from the same table, with the same key and a larger timestamp, has already been processed, are ignored and do not trigger the join.
- For each input record on the left side that does not have any match on the right side, the
ValueJoiner
will be called withValueJoiner#apply(leftRecord.value, null)
; this explains the row with timestamp=7 & 8 in the table below, which lists(q,10,null) and (r,10,null)
in the LEFT JOIN column.
See the semantics overview at the bottom of this section for a detailed description.
Semantics of table-table foreign-key joins: The semantics of the table-table foreign-key INNER and LEFT JOIN variants are demonstrated below. The key is shown alongside the value for each record. Records are processed in incrementing offset order. The columns INNER JOIN and LEFT JOIN denote what is passed as arguments to the user-supplied ValueJoiner for the join
and leftJoin
methods, respectively, whenever a new input record is received on either side of the join. An empty table cell denotes that the ValueJoiner
is not called at all. For the purpose of this example, Function foreignKeyExtractor
simply uses the left-value as the output.
Record Offset | Left KTable (K, extracted-FK) | Right KTable (FK, VR) | (INNER) JOIN | LEFT JOIN |
---|---|---|---|---|
1 | (k,1) | (1,foo) | (k,1,foo) | |
(k,1,foo) | ||||
2 | (k,2) | |||
(k,null) | (k,2,null) |
3 | (k,3)
| | (k,null) | (k,3,null)
4 | | (3,bar)
| (k,3,bar)
| (k,3,bar)
5 | (k,null)
| | (k,null)
| (k,null,null)
6 | (k,1) |
| (k,1,foo)
| (k,1,foo)
7 | (q,10)
| |
| (q,10,null)
8 | (r,10) |
| | (r,10,null)
9 |
| (10,baz) | (q,10,baz), (r,10,baz) | (q,10,baz), (r,10,baz)
KStream-KTable Join
KStream-KTable joins are always non-windowed joins. They allow you to perform table lookups against a KTable (changelog stream) upon receiving a new record from the KStream (record stream). An example use case would be to enrich a stream of user activities (KStream) with the latest user profile information (KTable).
Join output records are effectively created as follows, leveraging the user-supplied ValueJoiner
:
KeyValue<K, LV> leftRecord = ...;
KeyValue<K, RV> rightRecord = ...;
ValueJoiner<LV, RV, JV> joiner = ...;
KeyValue<K, JV> joinOutputRecord = KeyValue.pair(
leftRecord.key, /* by definition, leftRecord.key == rightRecord.key */
joiner.apply(leftRecord.value, rightRecord.value)
);
Transformation | Description |
---|---|
Inner Join |
- (KStream, KTable) -> KStream
| Performs an INNER JOIN of this stream with the table, effectively doing a table lookup. (details) Data must be co-partitioned : The input data for both sides must be co-partitioned. Causes data re-partitioning of the stream if and only if the stream was marked for re-partitioning. Several variants of join
exists, see the Javadocs for details.
KStream<String, Long> left = ...;
KTable<String, Double> right = ...;
KStream<String, String> joined = left.join(right,
(leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */
Joined.keySerde(Serdes.String()) /* key */
.withValueSerde(Serdes.Long()) /* left value */
.withGracePeriod(Duration.ZERO) /* grace period */
);
Detailed behavior:
- The join is key-based , i.e. with the join predicate
leftRecord.key == rightRecord.key
. - The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied
ValueJoiner
will be called to produce join output records.
* Only input records for the left side (stream) trigger the join. Input records for the right side (table) update only the internal right-side join state. * Input records for the stream with a `null` key or a `null` value are ignored and do not trigger the join. * Input records for the table with a `null` value are interpreted as _tombstones_ for the corresponding key, which indicate the deletion of the key from the table. Tombstones do not trigger the join.
- When the table is versioned, the table record to join with is determined by performing a timestamped lookup, i.e., the table record which is joined will be the latest-by-timestamp record with timestamp less than or equal to the stream record timestamp. If the stream record timestamp is older than the table’s history retention, then the record is dropped.
- To use the grace period, the table needs to be versioned. This will cause the stream to buffer for the specified grace period before trying to find a matching record with the right timestamp in the table. The case where the grace period would be used for is if a record in the table has a timestamp less than or equal to the stream record timestamp but arrives after the stream record. If the table record arrives within the grace period the join will still occur. If the table record does not arrive before the grace period the join will continue as normal.
See the semantics overview at the bottom of this section for a detailed description.
Left Join
- (KStream, KTable) -> KStream
| Performs a LEFT JOIN of this stream with the table, effectively doing a table lookup. (details) Data must be co-partitioned : The input data for both sides must be co-partitioned. Causes data re-partitioning of the stream if and only if the stream was marked for re-partitioning. Several variants of leftJoin
exists, see the Javadocs for details.
KStream<String, Long> left = ...;
KTable<String, Double> right = ...;
KStream<String, String> joined = left.leftJoin(right,
(leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue, /* ValueJoiner */
Joined.keySerde(Serdes.String()) /* key */
.withValueSerde(Serdes.Long()) /* left value */
.withGracePeriod(Duration.ZERO) /* grace period */
);
Detailed behavior:
- The join is key-based , i.e. with the join predicate
leftRecord.key == rightRecord.key
. - The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied
ValueJoiner
will be called to produce join output records.
* Only input records for the left side (stream) trigger the join. Input records for the right side (table) update only the internal right-side join state. * Input records for the stream with a `null` value are ignored and do not trigger the join. * Input records for the table with a `null` value are interpreted as _tombstones_ for the corresponding key, which indicate the deletion of the key from the table. Tombstones do not trigger the join.
- For each input record on the left side that does not have any match on the right side, the
ValueJoiner
will be called withValueJoiner#apply(leftRecord.value, null)
; this explains the row with timestamp=3 in the table below, which lists[A, null]
in the LEFT JOIN column. - When the table is versioned, the table record to join with is determined by performing a timestamped lookup, i.e., the table record which is joined will be the latest-by-timestamp record with timestamp less than or equal to the stream record timestamp. If the stream record timestamp is older than the table’s history retention, then the record that is joined will be
null
. - To use the grace period, the table needs to be versioned. This will cause the stream to buffer for the specified grace period before trying to find a matching record with the right timestamp in the table. The case where the grace period would be used for is if a record in the table has a timestamp less than or equal to the stream record timestamp but arrives after the stream record. If the table record arrives within the grace period the join will still occur. If the table record does not arrive before the grace period the join will continue as normal.
See the semantics overview at the bottom of this section for a detailed description.
Semantics of stream-table joins: The semantics of the various stream-table join variants are explained below. To improve the readability of the table we assume that (1) all records have the same key (and thus we omit the key in the table) and that (2) all records are processed in timestamp order. The columns INNER JOIN and LEFT JOIN denote what is passed as arguments to the user-supplied ValueJoiner for the join
and leftJoin
methods, respectively, whenever a new input record is received on either side of the join. An empty table cell denotes that the ValueJoiner
is not called at all.
Timestamp | Left (KStream) | Right (KTable) | (INNER) JOIN | LEFT JOIN |
---|---|---|---|---|
1 | null | |||
2 | null | |||
3 | A | [A, null] | ||
4 | a | |||
5 | B | [B, a] | [B, a] | |
6 | b | |||
7 | null | |||
8 | null | |||
9 | C | [C, null] | ||
10 | c | |||
11 | null | |||
12 | null | |||
13 | null | |||
14 | d | |||
15 | D | [D, d] | [D, d] |
KStream-GlobalKTable Join
KStream-GlobalKTable joins are always non-windowed joins. They allow you to perform table lookups against a GlobalKTable (entire changelog stream) upon receiving a new record from the KStream (record stream). An example use case would be “star queries” or “star joins”, where you would enrich a stream of user activities (KStream) with the latest user profile information (GlobalKTable) and further context information (further GlobalKTables). However, because GlobalKTables have no notion of time, a KStream-GlobalKTable join is not a temporal join, and there is no event-time synchronization between updates to a GlobalKTable and processing of KStream records.
At a high-level, KStream-GlobalKTable joins are very similar to KStream-KTable joins. However, global tables provide you with much more flexibility at the some expense when compared to partitioned tables:
- They do not require data co-partitioning.
- They allow for efficient “star joins”; i.e., joining a large-scale “facts” stream against “dimension” tables
- They allow for joining against foreign keys; i.e., you can lookup data in the table not just by the keys of records in the stream, but also by data in the record values.
- They make many use cases feasible where you must work on heavily skewed data and thus suffer from hot partitions.
- They are often more efficient than their partitioned KTable counterpart when you need to perform multiple joins in succession.
Join output records are effectively created as follows, leveraging the user-supplied ValueJoiner
:
KeyValue<K, LV> leftRecord = ...;
KeyValue<K, RV> rightRecord = ...;
ValueJoiner<LV, RV, JV> joiner = ...;
KeyValue<K, JV> joinOutputRecord = KeyValue.pair(
leftRecord.key, /* by definition, leftRecord.key == rightRecord.key */
joiner.apply(leftRecord.value, rightRecord.value)
);
Transformation | Description |
---|---|
Inner Join |
- (KStream, GlobalKTable) -> KStream
| Performs an INNER JOIN of this stream with the global table, effectively doing a table lookup. (details) The GlobalKTable
is fully bootstrapped upon (re)start of a KafkaStreams
instance, which means the table is fully populated with all the data in the underlying topic that is available at the time of the startup. The actual data processing begins only once the bootstrapping has completed. Causes data re-partitioning of the stream if and only if the stream was marked for re-partitioning.
KStream<String, Long> left = ...;
GlobalKTable<Integer, Double> right = ...;
KStream<String, String> joined = left.join(right,
(leftKey, leftValue) -> leftKey.length(), /* derive a (potentially) new key by which to lookup against the table */
(leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */
);
Detailed behavior:
- The join is indirectly key-based , i.e. with the join predicate
KeyValueMapper#apply(leftRecord.key, leftRecord.value) == rightRecord.key
. - The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied
ValueJoiner
will be called to produce join output records.
* Only input records for the left side (stream) trigger the join. Input records for the right side (table) update only the internal right-side join state. * Input records for the stream with a `null` key or a `null` value are ignored and do not trigger the join. * Input records for the table with a `null` value are interpreted as _tombstones_ , which indicate the deletion of a record key from the table. Tombstones do not trigger the join.
Left Join
- (KStream, GlobalKTable) -> KStream
| Performs a LEFT JOIN of this stream with the global table, effectively doing a table lookup. (details) The GlobalKTable
is fully bootstrapped upon (re)start of a KafkaStreams
instance, which means the table is fully populated with all the data in the underlying topic that is available at the time of the startup. The actual data processing begins only once the bootstrapping has completed. Causes data re-partitioning of the stream if and only if the stream was marked for re-partitioning.
KStream<String, Long> left = ...;
GlobalKTable<Integer, Double> right = ...;
KStream<String, String> joined = left.leftJoin(right,
(leftKey, leftValue) -> leftKey.length(), /* derive a (potentially) new key by which to lookup against the table */
(leftValue, rightValue) -> "left=" + leftValue + ", right=" + rightValue /* ValueJoiner */
);
Detailed behavior:
- The join is indirectly key-based , i.e. with the join predicate
KeyValueMapper#apply(leftRecord.key, leftRecord.value) == rightRecord.key
. - The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied
ValueJoiner
will be called to produce join output records.
* Only input records for the left side (stream) trigger the join. Input records for the right side (table) update only the internal right-side join state. * Input records for the stream with a `null` value are ignored and do not trigger the join. * Input records for the table with a `null` value are interpreted as _tombstones_ , which indicate the deletion of a record key from the table. Tombstones do not trigger the join.
- For each input record on the left side that does not have any match on the right side, the
ValueJoiner
will be called withValueJoiner#apply(leftRecord.value, null)
.
Semantics of stream-global-table joins: The join semantics are different to KStream-KTable joins because it’s not a temporal join. Another difference is that, for KStream-GlobalKTable joins, the left input record is first “mapped” with a user-supplied KeyValueMapper
into the table’s keyspace prior to the table lookup.
Windowing
Windowing lets you control how to group records that have the same key for stateful operations such as aggregations or joins into so-called windows. Windows are tracked per record key.
Note
A related operation is grouping, which groups all records that have the same key to ensure that data is properly partitioned (“keyed”) for subsequent operations. Once grouped, windowing allows you to further sub-group the records of a key.
For example, in join operations, a windowing state store is used to store all the records received so far within the defined window boundary. In aggregating operations, a windowing state store is used to store the latest aggregation results per window. Old records in the state store are purged after the specified window retention period. Kafka Streams guarantees to keep a window for at least this specified time; the default value is one day and can be changed via Materialized#withRetention()
.
The DSL supports the following types of windows:
Window name | Behavior | Short description |
---|---|---|
Hopping time window | Time-based | Fixed-size, overlapping windows |
Tumbling time window | Time-based | Fixed-size, non-overlapping, gap-less windows |
Sliding time window | Time-based | Fixed-size, overlapping windows that work on differences between record timestamps |
Session window | Session-based | Dynamically-sized, non-overlapping, data-driven windows |
Hopping time windows
Hopping time windows are windows based on time intervals. They model fixed-sized, (possibly) overlapping windows. A hopping window is defined by two properties: the window’s size and its advance interval (aka “hop”). The advance interval specifies by how much a window moves forward relative to the previous one. For example, you can configure a hopping window with a size 5 minutes and an advance interval of 1 minute. Since hopping windows can overlap - and in general they do - a data record may belong to more than one such windows.
Note
Hopping windows vs. sliding windows: Hopping windows are sometimes called “sliding windows” in other stream processing tools. Kafka Streams follows the terminology in academic literature, where the semantics of sliding windows are different to those of hopping windows.
The following code defines a hopping window with a size of 5 minutes and an advance interval of 1 minute:
import java.time.Duration;
import org.apache.kafka.streams.kstream.TimeWindows;
// A hopping time window with a size of 5 minutes and an advance interval of 1 minute.
// The window's name -- the string parameter -- is used to e.g. name the backing state store.
Duration windowSize = Duration.ofMinutes(5);
Duration advance = Duration.ofMinutes(1);
TimeWindows.ofSizeWithNoGrace(windowSize).advanceBy(advance);
This diagram shows windowing a stream of data records with hopping windows. In this diagram the time numbers represent minutes; e.g. t=5 means “at the five-minute mark”. In reality, the unit of time in Kafka Streams is milliseconds, which means the time numbers would need to be multiplied with 60 * 1,000 to convert from minutes to milliseconds (e.g. t=5 would become t=300,000).
Hopping time windows are aligned to the epoch , with the lower interval bound being inclusive and the upper bound being exclusive. “Aligned to the epoch” means that the first window starts at timestamp zero. For example, hopping windows with a size of 5000ms and an advance interval (“hop”) of 3000ms have predictable window boundaries [0;5000),[3000;8000),...
– and not [1000;6000),[4000;9000),...
or even something “random” like [1452;6452),[4452;9452),...
.
Unlike non-windowed aggregates that we have seen previously, windowed aggregates return a windowed KTable whose keys type is Windowed<K>
. This is to differentiate aggregate values with the same key from different windows. The corresponding window instance and the embedded key can be retrieved as Windowed#window()
and Windowed#key()
, respectively.
Tumbling time windows
Tumbling time windows are a special case of hopping time windows and, like the latter, are windows based on time intervals. They model fixed-size, non-overlapping, gap-less windows. A tumbling window is defined by a single property: the window’s size. A tumbling window is a hopping window whose window size is equal to its advance interval. Since tumbling windows never overlap, a data record will belong to one and only one window.
This diagram shows windowing a stream of data records with tumbling windows. Windows do not overlap because, by definition, the advance interval is identical to the window size. In this diagram the time numbers represent minutes; e.g. t=5 means “at the five-minute mark”. In reality, the unit of time in Kafka Streams is milliseconds, which means the time numbers would need to be multiplied with 60 * 1,000 to convert from minutes to milliseconds (e.g. t=5 would become t=300,000).
Tumbling time windows are aligned to the epoch , with the lower interval bound being inclusive and the upper bound being exclusive. “Aligned to the epoch” means that the first window starts at timestamp zero. For example, tumbling windows with a size of 5000ms have predictable window boundaries [0;5000),[5000;10000),...
– and not [1000;6000),[6000;11000),...
or even something “random” like [1452;6452),[6452;11452),...
.
The following code defines a tumbling window with a size of 5 minutes:
import java.time.Duration;
import org.apache.kafka.streams.kstream.TimeWindows;
// A tumbling time window with a size of 5 minutes (and, by definition, an implicit
// advance interval of 5 minutes), and grace period of 1 minute.
Duration windowSize = Duration.ofMinutes(5);
Duration gracePeriod = Duration.ofMinutes(1);
TimeWindows.ofSizeAndGrace(windowSize, gracePeriod);
// The above is equivalent to the following code:
TimeWindows.ofSizeAndGrace(windowSize, gracePeriod).advanceBy(windowSize);
Sliding time windows
Sliding windows are actually quite different from hopping and tumbling windows. In Kafka Streams, sliding windows are used for join operations, specified by using the JoinWindows
class, and windowed aggregations, specified by using the SlidingWindows
class.
A sliding window models a fixed-size window that slides continuously over the time axis. In this model, two data records are said to be included in the same window if (in the case of symmetric windows) the difference of their timestamps is within the window size. As a sliding window moves along the time axis, records may fall into multiple snapshots of the sliding window, but each unique combination of records appears only in one sliding window snapshot.
The following code defines a sliding window with a time difference of 10 minutes and a grace period of 30 minutes:
import org.apache.kafka.streams.kstream.SlidingWindows;
// A sliding time window with a time difference of 10 minutes and grace period of 30 minutes
Duration timeDifference = Duration.ofMinutes(10);
Duration gracePeriod = Duration.ofMinutes(30);
SlidingWindows.ofTimeDifferenceAndGrace(timeDifference, gracePeriod);
This diagram shows windowing a stream of data records with sliding windows. The overlap of the sliding window snapshots varies depending on the record times. In this diagram, the time numbers represent milliseconds. For example, t=5 means “at the five millisecond mark”.
Sliding windows are aligned to the data record timestamps, not to the epoch. In contrast to hopping and tumbling windows, the lower and upper window time interval bounds of sliding windows are both inclusive.
Session Windows
Session windows are used to aggregate key-based events into so-called sessions , the process of which is referred to as sessionization. Sessions represent a period of activity separated by a defined gap of inactivity (or “idleness”). Any events processed that fall within the inactivity gap of any existing sessions are merged into the existing sessions. If an event falls outside of the session gap, then a new session will be created.
Session windows are different from the other window types in that:
- all windows are tracked independently across keys - e.g. windows of different keys typically have different start and end times
- their window sizes sizes vary - even windows for the same key typically have different sizes
The prime area of application for session windows is user behavior analysis. Session-based analyses can range from simple metrics (e.g. count of user visits on a news website or social platform) to more complex metrics (e.g. customer conversion funnel and event flows).
The following code defines a session window with an inactivity gap of 5 minutes:
import java.time.Duration;
import org.apache.kafka.streams.kstream.SessionWindows;
// A session window with an inactivity gap of 5 minutes.
SessionWindows.ofInactivityGapWithNoGrace(Duration.ofMinutes(5));
Given the previous session window example, here’s what would happen on an input stream of six records. When the first three records arrive (upper part of in the diagram below), we’d have three sessions (see lower part) after having processed those records: two for the green record key, with one session starting and ending at the 0-minute mark (only due to the illustration it looks as if the session goes from 0 to 1), and another starting and ending at the 6-minute mark; and one session for the blue record key, starting and ending at the 2-minute mark.
Detected sessions after having received three input records: two records for the green record key at t=0 and t=6, and one record for the blue record key at t=2. In this diagram the time numbers represent minutes; e.g. t=5 means “at the five-minute mark”. In reality, the unit of time in Kafka Streams is milliseconds, which means the time numbers would need to be multiplied with 60 * 1,000 to convert from minutes to milliseconds (e.g. t=5 would become t=300,000).
If we then receive three additional records (including two out-of-order records), what would happen is that the two existing sessions for the green record key will be merged into a single session starting at time 0 and ending at time 6, consisting of a total of three records. The existing session for the blue record key will be extended to end at time 5, consisting of a total of two records. And, finally, there will be a new session for the blue key starting and ending at time 11.
Detected sessions after having received six input records. Note the two out-of-order data records at t=4 (green) and t=5 (blue), which lead to a merge of sessions and an extension of a session, respectively.
Window Final Results
In Kafka Streams, windowed computations update their results continuously. As new data arrives for a window, freshly computed results are emitted downstream. For many applications, this is ideal, since fresh results are always available. and Kafka Streams is designed to make programming continuous computations seamless. However, some applications need to take action only on the final result of a windowed computation. Common examples of this are sending alerts or delivering results to a system that doesn’t support updates.
Suppose that you have an hourly windowed count of events per user. If you want to send an alert when a user has less than three events in an hour, you have a real challenge. All users would match this condition at first, until they accrue enough events, so you cannot simply send an alert when someone matches the condition; you have to wait until you know you won’t see any more events for a particular window and then send the alert.
Kafka Streams offers a clean way to define this logic: after defining your windowed computation, you can suppress the intermediate results, emitting the final count for each user when the window is closed.
For example:
KGroupedStream<UserId, Event> grouped = ...;
grouped
.windowedBy(TimeWindows.ofSizeAndGrace(Duration.ofHours(1), Duration.ofMinutes(10)))
.count()
.suppress(Suppressed.untilWindowCloses(unbounded()))
.filter((windowedUserId, count) -> count < 3)
.toStream()
.foreach((windowedUserId, count) -> sendAlert(windowedUserId.window(), windowedUserId.key(), count));
The key parts of this program are:
ofSizeAndGrace(Duration.ofHours(1), Duration.ofMinutes(10))
The specified grace period of 10 minutes (i.e., the Duration.ofMinutes(10)
argument) allows us to bound the lateness of events the window will accept. For example, the 09:00 to 10:00 window will accept out-of-order records until 10:10, at which point, the window is closed.
.suppress(Suppressed.untilWindowCloses(...))
This configures the suppression operator to emit nothing for a window until it closes, and then emit the final result. For example, if user U
gets 10 events between 09:00 and 10:10, the filter
downstream of the suppression will get no events for the windowed key U@09:00-10:00
until 10:10, and then it will get exactly one with the value 10
. This is the final result of the windowed count.
unbounded()
This configures the buffer used for storing events until their windows close. Production code is able to put a cap on the amount of memory to use for the buffer, but this simple example creates a buffer with no upper bound.
One thing to note is that suppression is just like any other Kafka Streams operator, so you can build a topology with two branches emerging from the count
, one suppressed, and one not, or even multiple differently configured suppressions. This allows you to apply suppressions where they are needed and otherwise rely on the default continuous update behavior.
For more detailed information, see the JavaDoc on the Suppressed
config object and KIP-328.
Applying processors (Processor API integration)
Beyond the aforementioned stateless and stateful transformations, you may also leverage the Processor API from the DSL. There are a number of scenarios where this may be helpful:
- Customization: You need to implement special, customized logic that is not or not yet available in the DSL.
- Combining ease-of-use with full flexibility where it’s needed: Even though you generally prefer to use the expressiveness of the DSL, there are certain steps in your processing that require more flexibility and tinkering than the DSL provides. For example, only the Processor API provides access to a record’s metadata such as its topic, partition, and offset information. However, you don’t want to switch completely to the Processor API just because of that; and
- Migrating from other tools: You are migrating from other stream processing technologies that provide an imperative API, and migrating some of your legacy code to the Processor API was faster and/or easier than to migrate completely to the DSL right away.
Operations and concepts
KStream#process
: Process all records in a stream, one record at a time, by applying aProcessor
(provided by a givenProcessorSupplier
);KStream#processValues
: Process all records in a stream, one record at a time, by applying aFixedKeyProcessor
(provided by a givenFixedKeyProcessorSupplier
);Processor
: A processor of key-value pair records;ContextualProcessor
: An abstract implementation ofProcessor
that manages theProcessorContext
instance.FixedKeyProcessor
: A processor of key-value pair records where keys are immutable;ContextualFixedKeyProcessor
: An abstract implementation ofFixedKeyProcessor
that manages theFixedKeyProcessorContext
instance.ProcessorSupplier
: A processor supplier that can create one or moreProcessor
instances; andFixedKeyProcessorSupplier
: A processor supplier that can create one or moreFixedKeyProcessor
instances.
Examples
Follow the examples below to learn how to apply process
and processValues
to your KStream
.
Example | Operation | State Type |
---|---|---|
Categorizing Logs by Severity | process | Stateless |
Replacing Slang in Text Messages | processValues | Stateless |
Cumulative Discounts for a Loyalty Program | process | Stateful |
Traffic Radar Monitoring Car Count | processValues | Stateful |
Categorizing Logs by Severity
Idea: You have a stream of log messages. Each message contains a severity level (e.g., INFO, WARN, ERROR) in the value. The processor filters messages, routing ERROR messages to a dedicated topic and discarding INFO messages. The rest (WARN) are forwarded to a dedicated topic too.
Real-World Context: In a production monitoring system, categorizing logs by severity ensures ERROR logs are sent to a critical incident management system, WARN logs are analyzed for potential risks, and INFO logs are stored for basic reporting purposes.
public class CategorizingLogsBySeverityExample { private static final String ERROR_LOGS_TOPIC = “error-logs-topic”; private static final String INPUT_LOGS_TOPIC = “input-logs-topic”; private static final String UNKNOWN_LOGS_TOPIC = “unknown-logs-topic”; private static final String WARN_LOGS_TOPIC = “warn-logs-topic”;
public static void categorizeWithProcess(final StreamsBuilder builder) { final KStream<String, String> logStream = builder.stream(INPUT_LOGS_TOPIC); logStream.process(LogSeverityProcessor::new) .to((key, value, recordContext) -> { // Determine the target topic dynamically if ("ERROR".equals(key)) return ERROR_LOGS_TOPIC; if ("WARN".equals(key)) return WARN_LOGS_TOPIC; return UNKNOWN_LOGS_TOPIC; }); } private static class LogSeverityProcessor extends ContextualProcessor<String, String, String, String> { @Override public void process(final Record<String, String> record) { if (record.value() == null) { return; // Skip null values } // Assume the severity is the first word in the log message // For example: "ERROR: Disk not found" -> "ERROR" final int colonIndex = record.value().indexOf(':'); final String severity = colonIndex > 0 ? record.value().substring(0, colonIndex).trim() : "UNKNOWN"; // Route logs based on severity switch (severity) { case "ERROR": context().forward(record.withKey(ERROR_LOGS_TOPIC)); break; case "WARN": context().forward(record.withKey(WARN_LOGS_TOPIC)); break; case "INFO": // INFO logs are ignored break; default: // Forward to an "unknown" topic for logs with unrecognized severities context().forward(record.withKey(UNKNOWN_LOGS_TOPIC)); } } }
}
Replacing Slang in Text Messages
Idea: A messaging stream contains user-generated content, and you want to replace slang words with their formal equivalents (e.g., “u” becomes “you”, “brb” becomes “be right back”). The operation only modifies the message value and keeps the key intact.
Real-World Context: In customer support chat systems, normalizing text by replacing slang with formal equivalents ensures that automated sentiment analysis tools work accurately and provide reliable insights.
public class ReplacingSlangTextInMessagesExample { private static final Map<String, String> SLANG_DICTIONARY = Map.of( “u”, “you”, “brb”, “be right back”, “omg”, “oh my god”, “btw”, “by the way” ); private static final String INPUT_MESSAGES_TOPIC = “input-messages-topic”; private static final String OUTPUT_MESSAGES_TOPIC = “output-messages-topic”;
public static void replaceWithProcessValues(final StreamsBuilder builder) { KStream<String, String> messageStream = builder.stream(INPUT_MESSAGES_TOPIC); messageStream.processValues(SlangReplacementProcessor::new).to(OUTPUT_MESSAGES_TOPIC); } private static class SlangReplacementProcessor extends ContextualFixedKeyProcessor<String, String, String> { @Override public void process(final FixedKeyRecord<String, String> record) { if (record.value() == null) { return; // Skip null values } // Replace slang words in the message final String[] words = record.value().split("\s+"); for (final String word : words) { String replacedWord = SLANG_DICTIONARY.getOrDefault(word, word); context().forward(record.withValue(replacedWord)); } } }
}
Cumulative Discounts for a Loyalty Program
Idea: A stream of purchase events contains user IDs and transaction amounts. Use a state store to accumulate the total spending of each user. When their total crosses a threshold, apply a discount on their next transaction and update their accumulated total.
Real-World Context: In a retail loyalty program, tracking cumulative customer spending enables dynamic rewards, such as issuing a discount when a customer’s total purchases exceed a predefined limit.
public class CumulativeDiscountsForALoyaltyProgramExample { private static final double DISCOUNT_THRESHOLD = 100.0; private static final String CUSTOMER_SPENDING_STORE = “customer-spending-store”; private static final String DISCOUNT_NOTIFICATION_MESSAGE = “Discount applied! You have received a reward for your purchases.”; private static final String DISCOUNT_NOTIFICATIONS_TOPIC = “discount-notifications-topic”; private static final String PURCHASE_EVENTS_TOPIC = “purchase-events-topic”;
public static void applyDiscountWithProcess(final StreamsBuilder builder) { // Define the state store for tracking cumulative spending builder.addStateStore( Stores.keyValueStoreBuilder( Stores.inMemoryKeyValueStore(CUSTOMER_SPENDING_STORE), Serdes.String(), Serdes.Double() ) ); final KStream<String, Double> purchaseStream = builder.stream(PURCHASE_EVENTS_TOPIC); // Apply the Processor with the state store final KStream<String, String> notificationStream = purchaseStream.process(CumulativeDiscountProcessor::new, CUSTOMER_SPENDING_STORE); // Send the notifications to the output topic notificationStream.to(DISCOUNT_NOTIFICATIONS_TOPIC); } private static class CumulativeDiscountProcessor implements Processor<String, Double, String, String> { private KeyValueStore<String, Double> spendingStore; private ProcessorContext<String, String> context; @Override public void init(final ProcessorContext<String, String> context) { this.context = context; // Retrieve the state store for cumulative spending spendingStore = context.getStateStore(CUSTOMER_SPENDING_STORE); } @Override public void process(final Record<String, Double> record) { if (record.value() == null) { return; // Skip null purchase amounts } // Get the current spending total for the customer Double currentSpending = spendingStore.get(record.key()); if (currentSpending == null) { currentSpending = 0.0; } // Update the cumulative spending currentSpending += record.value(); spendingStore.put(record.key(), currentSpending); // Check if the customer qualifies for a discount if (currentSpending >= DISCOUNT_THRESHOLD) { // Reset the spending after applying the discount spendingStore.put(record.key(), currentSpending - DISCOUNT_THRESHOLD); // Send a discount notification context.forward(record.withValue(DISCOUNT_NOTIFICATION_MESSAGE)); } } }
}
Traffic Radar Monitoring Car Count
Idea: A radar monitors cars passing along a road stretch. A system counts the cars for each day, maintaining a cumulative total for the current day in a state store. At the end of the day, the count is emitted and the state is cleared for the next day.
Real-World Context: A car counting system can be useful for determining measures for widening or controlling traffic depending on the number of cars passing through the monitored stretch.
public class TrafficRadarMonitoringCarCountExample { private static final String DAILY_COUNT_STORE = “price-state-store”; private static final String DAILY_COUNT_TOPIC = “price-state-topic”; private static final String RADAR_COUNT_TOPIC = “car-radar-topic”;
public static void countWithProcessValues(final StreamsBuilder builder) { // Define a state store for tracking daily car counts builder.addStateStore( Stores.keyValueStoreBuilder( Stores.inMemoryKeyValueStore(DAILY_COUNT_STORE), Serdes.String(), Serdes.Long() ) ); final KStream<Void, String> radarStream = builder.stream(RADAR_COUNT_TOPIC); // Apply the FixedKeyProcessor with the state store radarStream.processValues(DailyCarCountProcessor::new, DAILY_COUNT_STORE) .to(DAILY_COUNT_TOPIC); } private static class DailyCarCountProcessor implements FixedKeyProcessor<Void, String, String> { private FixedKeyProcessorContext<Void, String> context; private KeyValueStore<String, Long> stateStore; private static final DateTimeFormatter DATE_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd").withZone(ZoneId.systemDefault()); @Override public void init(final FixedKeyProcessorContext<Void, String> context) { this.context = context; stateStore = context.getStateStore(DAILY_COUNT_STORE); } @Override public void process(final FixedKeyRecord<Void, String> record) { if (record.value() == null) { return; // Skip null events } // Derive the current day from the event timestamp final long timestamp = System.currentTimeMillis(); // Use system time for simplicity final String currentDay = DATE_FORMATTER.format(Instant.ofEpochMilli(timestamp)); // Retrieve the current count for the day Long dailyCount = stateStore.get(currentDay); if (dailyCount == null) { dailyCount = 0L; } // Increment the count dailyCount++; stateStore.put(currentDay, dailyCount); // Emit the current day's count context.forward(record.withValue(String.format("Day: %s, Car Count: %s", currentDay, dailyCount))); } }
}
Keynotes
- Type Safety and Flexibility: The process and processValues APIs utilize
ProcessorContext
andRecord
orFixedKeyRecord
objects for better type safety and flexibility of custom processing logic. - Clear State and Logic Management: Implementations for
Processor
orFixedKeyProcessor
should manage state and logic clearly. Usecontext().forward()
for emitting records downstream. - Unified API: Consolidates multiple methods into a single, versatile API.
- Future-Proof: Ensures compatibility with the latest Kafka Streams releases.
Transformers removal and migration to processors
As of Kafka 4.0, several deprecated methods in the Kafka Streams API, such as transform
, flatTransform
, transformValues
, flatTransformValues
, and process
have been removed. These methods have been replaced with the more versatile Processor API. This guide provides detailed steps for migrating existing code to use the new Processor API and explains the benefits of the changes.
The following deprecated methods are no longer available in Kafka Streams:
KStream#transform
KStream#flatTransform
KStream#transformValues
KStream#flatTransformValues
KStream#process
The Processor API now serves as a unified replacement for all these methods. It simplifies the API surface while maintaining support for both stateless and stateful operations.
Migration Examples
To migrate from the deprecated transform
, transformValues
, flatTransform
, and flatTransformValues
methods to the Processor API (PAPI) in Kafka Streams, let’s resume the previouss examples. The new process
and processValues
methods enable a more flexible and reusable approach by requiring implementations of the Processor
or FixedKeyProcessor
interfaces.
Example | Migrating from | Migrating to | State Type |
---|---|---|---|
Categorizing Logs by Severity | flatTransform | process | Stateless |
Replacing Slang in Text Messages | flatTransformValues | processValues | Stateless |
Cumulative Discounts for a Loyalty Program | transform | process | Stateful |
Traffic Radar Monitoring Car Count | transformValues | processValues | Stateful |
Categorizing Logs by Severity
Below, methods categorizeWithFlatTransform
and categorizeWithProcess
show how you can migrate from flatTransform
to process
.
public class CategorizingLogsBySeverityExample {
private static final String ERROR_LOGS_TOPIC = "error-logs-topic";
private static final String INPUT_LOGS_TOPIC = "input-logs-topic";
private static final String UNKNOWN_LOGS_TOPIC = "unknown-logs-topic";
private static final String WARN_LOGS_TOPIC = "warn-logs-topic";
public static void categorizeWithFlatTransform(final StreamsBuilder builder) {
final KStream<String, String> logStream = builder.stream(INPUT_LOGS_TOPIC);
logStream.flatTransform(LogSeverityTransformer::new)
.to((key, value, recordContext) -> {
// Determine the target topic dynamically
if ("ERROR".equals(key)) return ERROR_LOGS_TOPIC;
if ("WARN".equals(key)) return WARN_LOGS_TOPIC;
return UNKNOWN_LOGS_TOPIC;
});
}
public static void categorizeWithProcess(final StreamsBuilder builder) {
final KStream<String, String> logStream = builder.stream(INPUT_LOGS_TOPIC);
logStream.process(LogSeverityProcessor::new)
.to((key, value, recordContext) -> {
// Determine the target topic dynamically
if ("ERROR".equals(key)) return ERROR_LOGS_TOPIC;
if ("WARN".equals(key)) return WARN_LOGS_TOPIC;
return UNKNOWN_LOGS_TOPIC;
});
}
private static class LogSeverityTransformer implements Transformer<String, String, Iterable<KeyValue<String, String>>> {
@Override
public void init(org.apache.kafka.streams.processor.ProcessorContext context) {
}
@Override
public Iterable<KeyValue<String, String>> transform(String key, String value) {
if (value == null) {
return Collections.emptyList(); // Skip null values
}
// Assume the severity is the first word in the log message
// For example: "ERROR: Disk not found" -> "ERROR"
int colonIndex = value.indexOf(':');
String severity = colonIndex > 0 ? value.substring(0, colonIndex).trim() : "UNKNOWN";
// Create appropriate KeyValue pair based on severity
return switch (severity) {
case "ERROR" -> List.of(new KeyValue<>("ERROR", value));
case "WARN" -> List.of(new KeyValue<>("WARN", value));
case "INFO" -> Collections.emptyList(); // INFO logs are ignored
default -> List.of(new KeyValue<>("UNKNOWN", value));
};
}
@Override
public void close() {
}
}
private static class LogSeverityProcessor extends ContextualProcessor<String, String, String, String> {
@Override
public void process(final Record<String, String> record) {
if (record.value() == null) {
return; // Skip null values
}
// Assume the severity is the first word in the log message
// For example: "ERROR: Disk not found" -> "ERROR"
final int colonIndex = record.value().indexOf(':');
final String severity = colonIndex > 0 ? record.value().substring(0, colonIndex).trim() : "UNKNOWN";
// Route logs based on severity
switch (severity) {
case "ERROR":
context().forward(record.withKey(ERROR_LOGS_TOPIC));
break;
case "WARN":
context().forward(record.withKey(WARN_LOGS_TOPIC));
break;
case "INFO":
// INFO logs are ignored
break;
default:
// Forward to an "unknown" topic for logs with unrecognized severities
context().forward(record.withKey(UNKNOWN_LOGS_TOPIC));
}
}
}
}
Replacing Slang in Text Messages
Below, methods replaceWithFlatTransformValues
and replaceWithProcessValues
show how you can migrate from flatTransformValues
to processValues
.
public class ReplacingSlangTextInMessagesExample {
private static final Map<String, String> SLANG_DICTIONARY = Map.of(
"u", "you",
"brb", "be right back",
"omg", "oh my god",
"btw", "by the way"
);
private static final String INPUT_MESSAGES_TOPIC = "input-messages-topic";
private static final String OUTPUT_MESSAGES_TOPIC = "output-messages-topic";
public static void replaceWithFlatTransformValues(final StreamsBuilder builder) {
KStream<String, String> messageStream = builder.stream(INPUT_MESSAGES_TOPIC);
messageStream.flatTransformValues(SlangReplacementTransformer::new).to(OUTPUT_MESSAGES_TOPIC);
}
public static void replaceWithProcessValues(final StreamsBuilder builder) {
KStream<String, String> messageStream = builder.stream(INPUT_MESSAGES_TOPIC);
messageStream.processValues(SlangReplacementProcessor::new).to(OUTPUT_MESSAGES_TOPIC);
}
private static class SlangReplacementTransformer implements ValueTransformer<String, Iterable<String>> {
@Override
public void init(final org.apache.kafka.streams.processor.ProcessorContext context) {
}
@Override
public Iterable<String> transform(final String value) {
if (value == null) {
return Collections.emptyList(); // Skip null values
}
// Replace slang words in the message
final String[] words = value.split("\s+");
return Arrays.asList(
Arrays.stream(words)
.map(word -> SLANG_DICTIONARY.getOrDefault(word, word))
.toArray(String[]::new)
);
}
@Override
public void close() {
}
}
private static class SlangReplacementProcessor extends ContextualFixedKeyProcessor<String, String, String> {
@Override
public void process(final FixedKeyRecord<String, String> record) {
if (record.value() == null) {
return; // Skip null values
}
// Replace slang words in the message
final String[] words = record.value().split("\s+");
for (final String word : words) {
String replacedWord = SLANG_DICTIONARY.getOrDefault(word, word);
context().forward(record.withValue(replacedWord));
}
}
}
}
Cumulative Discounts for a Loyalty Program
public class CumulativeDiscountsForALoyaltyProgramExample {
private static final double DISCOUNT_THRESHOLD = 100.0;
private static final String CUSTOMER_SPENDING_STORE = "customer-spending-store";
private static final String DISCOUNT_NOTIFICATION_MESSAGE =
"Discount applied! You have received a reward for your purchases.";
private static final String DISCOUNT_NOTIFICATIONS_TOPIC = "discount-notifications-topic";
private static final String PURCHASE_EVENTS_TOPIC = "purchase-events-topic";
public static void applyDiscountWithTransform(final StreamsBuilder builder) {
// Define the state store for tracking cumulative spending
builder.addStateStore(
Stores.keyValueStoreBuilder(
Stores.inMemoryKeyValueStore(CUSTOMER_SPENDING_STORE),
Serdes.String(),
Serdes.Double()
)
);
final KStream<String, Double> purchaseStream = builder.stream(PURCHASE_EVENTS_TOPIC);
// Apply the Transformer with the state store
final KStream<String, String> notificationStream =
purchaseStream.transform(CumulativeDiscountTransformer::new, CUSTOMER_SPENDING_STORE);
// Send the notifications to the output topic
notificationStream.to(DISCOUNT_NOTIFICATIONS_TOPIC);
}
public static void applyDiscountWithProcess(final StreamsBuilder builder) {
// Define the state store for tracking cumulative spending
builder.addStateStore(
Stores.keyValueStoreBuilder(
Stores.inMemoryKeyValueStore(CUSTOMER_SPENDING_STORE),
org.apache.kafka.common.serialization.Serdes.String(),
org.apache.kafka.common.serialization.Serdes.Double()
)
);
final KStream<String, Double> purchaseStream = builder.stream(PURCHASE_EVENTS_TOPIC);
// Apply the Processor with the state store
final KStream<String, String> notificationStream =
purchaseStream.process(CumulativeDiscountProcessor::new, CUSTOMER_SPENDING_STORE);
// Send the notifications to the output topic
notificationStream.to(DISCOUNT_NOTIFICATIONS_TOPIC);
}
private static class CumulativeDiscountTransformer implements Transformer<String, Double, KeyValue<String, String>> {
private KeyValueStore<String, Double> spendingStore;
@Override
public void init(final org.apache.kafka.streams.processor.ProcessorContext context) {
// Retrieve the state store for cumulative spending
spendingStore = context.getStateStore(CUSTOMER_SPENDING_STORE);
}
@Override
public KeyValue<String, String> transform(final String key, final Double value) {
if (value == null) {
return null; // Skip null purchase amounts
}
// Get the current spending total for the customer
Double currentSpending = spendingStore.get(key);
if (currentSpending == null) {
currentSpending = 0.0;
}
// Update the cumulative spending
currentSpending += value;
spendingStore.put(key, currentSpending);
// Check if the customer qualifies for a discount
if (currentSpending >= DISCOUNT_THRESHOLD) {
// Reset the spending after applying the discount
spendingStore.put(key, currentSpending - DISCOUNT_THRESHOLD);
// Return a notification message
return new KeyValue<>(key, DISCOUNT_NOTIFICATION_MESSAGE);
}
return null; // No discount, so no output for this record
}
@Override
public void close() {
}
}
private static class CumulativeDiscountProcessor implements Processor<String, Double, String, String> {
private KeyValueStore<String, Double> spendingStore;
private ProcessorContext<String, String> context;
@Override
public void init(final ProcessorContext<String, String> context) {
this.context = context;
// Retrieve the state store for cumulative spending
spendingStore = context.getStateStore(CUSTOMER_SPENDING_STORE);
}
@Override
public void process(final Record<String, Double> record) {
if (record.value() == null) {
return; // Skip null purchase amounts
}
// Get the current spending total for the customer
Double currentSpending = spendingStore.get(record.key());
if (currentSpending == null) {
currentSpending = 0.0;
}
// Update the cumulative spending
currentSpending += record.value();
spendingStore.put(record.key(), currentSpending);
// Check if the customer qualifies for a discount
if (currentSpending >= DISCOUNT_THRESHOLD) {
// Reset the spending after applying the discount
spendingStore.put(record.key(), currentSpending - DISCOUNT_THRESHOLD);
// Send a discount notification
context.forward(record.withValue(DISCOUNT_NOTIFICATION_MESSAGE));
}
}
}
}
Traffic Radar Monitoring Car Count
Below, methods countWithTransformValues
and countWithProcessValues
show how you can migrate from transformValues
to processValues
.
public class TrafficRadarMonitoringCarCountExample {
private static final String DAILY_COUNT_STORE = "price-state-store";
private static final String DAILY_COUNT_TOPIC = "price-state-topic";
private static final String RADAR_COUNT_TOPIC = "car-radar-topic";
public static void countWithTransformValues(final StreamsBuilder builder) {
// Define a state store for tracking daily car counts
builder.addStateStore(
Stores.keyValueStoreBuilder(
Stores.inMemoryKeyValueStore(DAILY_COUNT_STORE),
org.apache.kafka.common.serialization.Serdes.String(),
org.apache.kafka.common.serialization.Serdes.Long()
)
);
final KStream<Void, String> radarStream = builder.stream(RADAR_COUNT_TOPIC);
// Apply the ValueTransformer with the state store
radarStream.transformValues(DailyCarCountTransformer::new, DAILY_COUNT_STORE)
.to(DAILY_COUNT_TOPIC);
}
public static void countWithProcessValues(final StreamsBuilder builder) {
// Define a state store for tracking daily car counts
builder.addStateStore(
Stores.keyValueStoreBuilder(
Stores.inMemoryKeyValueStore(DAILY_COUNT_STORE),
org.apache.kafka.common.serialization.Serdes.String(),
org.apache.kafka.common.serialization.Serdes.Long()
)
);
final KStream<Void, String> radarStream = builder.stream(RADAR_COUNT_TOPIC);
// Apply the FixedKeyProcessor with the state store
radarStream.processValues(DailyCarCountProcessor::new, DAILY_COUNT_STORE)
.to(DAILY_COUNT_TOPIC);
}
private static class DailyCarCountTransformer implements ValueTransformerWithKey<Void, String, String> {
private KeyValueStore<String, Long> stateStore;
private static final DateTimeFormatter DATE_FORMATTER =
DateTimeFormatter.ofPattern("yyyy-MM-dd").withZone(ZoneId.systemDefault());
@Override
public void init(final org.apache.kafka.streams.processor.ProcessorContext context) {
// Access the state store
stateStore = context.getStateStore(DAILY_COUNT_STORE);
}
@Override
public String transform(Void readOnlyKey, String value) {
if (value == null) {
return null; // Skip null events
}
// Derive the current day from the event timestamp
final long timestamp = System.currentTimeMillis(); // Use system time for simplicity
final String currentDay = DATE_FORMATTER.format(Instant.ofEpochMilli(timestamp));
// Retrieve the current count for the day
Long dailyCount = stateStore.get(currentDay);
if (dailyCount == null) {
dailyCount = 0L;
}
// Increment the count
dailyCount++;
stateStore.put(currentDay, dailyCount);
// Return the current day's count
return String.format("Day: %s, Car Count: %s", currentDay, dailyCount);
}
@Override
public void close() {
}
}
private static class DailyCarCountProcessor implements FixedKeyProcessor<Void, String, String> {
private FixedKeyProcessorContext<Void, String> context;
private KeyValueStore<String, Long> stateStore;
private static final DateTimeFormatter DATE_FORMATTER =
DateTimeFormatter.ofPattern("yyyy-MM-dd").withZone(ZoneId.systemDefault());
@Override
public void init(final FixedKeyProcessorContext<Void, String> context) {
this.context = context;
stateStore = context.getStateStore(DAILY_COUNT_STORE);
}
@Override
public void process(final FixedKeyRecord<Void, String> record) {
if (record.value() == null) {
return; // Skip null events
}
// Derive the current day from the event timestamp
final long timestamp = System.currentTimeMillis(); // Use system time for simplicity
final String currentDay = DATE_FORMATTER.format(Instant.ofEpochMilli(timestamp));
// Retrieve the current count for the day
Long dailyCount = stateStore.get(currentDay);
if (dailyCount == null) {
dailyCount = 0L;
}
// Increment the count
dailyCount++;
stateStore.put(currentDay, dailyCount);
// Emit the current day's count
context.forward(record.withValue(String.format("Day: %s, Car Count: %s", currentDay, dailyCount)));
}
}
}
Keynotes
- Type Safety and Flexibility: The process and processValues APIs utilize
ProcessorContext
andRecord
orFixedKeyRecord
objects for better type safety and flexibility of custom processing logic. - Clear State and Logic Management: Implementations for
Processor
orFixedKeyProcessor
should manage state and logic clearly. Usecontext().forward()
for emitting records downstream. - Unified API: Consolidates multiple methods into a single, versatile API.
- Future-Proof: Ensures compatibility with the latest Kafka Streams releases.
Removal of Old process
Method
It is worth mentioning that, in addition to the methods mentioned above, the process
method, which integrated the ‘old’ Processor API (i.e., Processor
as opposed to the new api.Processor
) into the DSL, has also been removed. The following example shows how to migrate to the new process
.
Example
Idea: The system monitors page views for a website in real-time. When a page reaches a predefined popularity threshold (e.g., 1000 views), the system automatically sends an email alert to the site administrator or marketing team to notify them of the page’s success. This helps teams quickly identify high-performing content and act on it, such as promoting the page further or analyzing the traffic source.
Real-World Context: In a content management system (CMS) for a news or blogging platform, it’s crucial to track the popularity of articles or posts. For example:
- Marketing Teams: Use the notification to highlight trending content on social media or email newsletters.
- Operations Teams: Use the alert to ensure the site can handle increased traffic for popular pages.
- Ad Managers: Identify pages where additional ad placements might maximize revenue.
By automating the detection of popular pages, the system eliminates the need for manual monitoring and ensures timely actions to capitalize on the content’s performance.
public class PopularPageEmailAlertExample {
private static final String ALERTS_EMAIL = "alerts@yourcompany.com";
private static final String PAGE_VIEWS_TOPIC = "page-views-topic";
public static void alertWithOldProcess(StreamsBuilder builder) {
KStream<String, Long> pageViews = builder.stream(PAGE_VIEWS_TOPIC);
// Filter pages with exactly 1000 views and process them using the old API
pageViews.filter((pageId, viewCount) -> viewCount == 1000)
.process(PopularPageEmailAlertOld::new);
}
public static void alertWithNewProcess(StreamsBuilder builder) {
KStream<String, Long> pageViews = builder.stream(PAGE_VIEWS_TOPIC);
// Filter pages with exactly 1000 views and process them using the new API
pageViews.filter((pageId, viewCount) -> viewCount == 1000)
.process(PopularPageEmailAlertNew::new);
}
private static class PopularPageEmailAlertOld extends AbstractProcessor<String, Long> {
@Override
public void init(org.apache.kafka.streams.processor.ProcessorContext context) {
super.init(context);
System.out.println("Initialized email client for: " + ALERTS_EMAIL);
}
@Override
public void process(String key, Long value) {
if (value == null) return;
if (value == 1000) {
// Send an email alert
System.out.printf("ALERT (Old API): Page %s has reached 1000 views. Sending email to %s%n", key, ALERTS_EMAIL);
}
}
@Override
public void close() {
System.out.println("Tearing down email client for: " + ALERTS_EMAIL);
}
}
private static class PopularPageEmailAlertNew implements Processor<String, Long, Void, Void> {
@Override
public void init(ProcessorContext<Void, Void> context) {
System.out.println("Initialized email client for: " + ALERTS_EMAIL);
}
@Override
public void process(Record<String, Long> record) {
if (record.value() == null) return;
if (record.value() == 1000) {
// Send an email alert
System.out.printf("ALERT (New API): Page %s has reached 1000 views. Sending email to %s%n", record.key(), ALERTS_EMAIL);
}
}
@Override
public void close() {
System.out.println("Tearing down email client for: " + ALERTS_EMAIL);
}
}
}
Naming Operators in a Streams DSL application Kafka Streams allows you to name processors created via the Streams DSL
Controlling KTable emit rate
A KTable is logically a continuously updated table. These updates make their way to downstream operators whenever new data is available, ensuring that the whole computation is as fresh as possible. Logically speaking, most programs describe a series of transformations, and the update rate is not a factor in the program behavior. In these cases, the rate of update is more of a performance concern. Operators are able to optimize both the network traffic (to the Kafka brokers) and the disk traffic (to the local state stores) by adjusting commit interval and batch size configurations.
However, some applications need to take other actions, such as calling out to external systems, and therefore need to exercise some control over the rate of invocations, for example of KStream#foreach
.
Rather than achieving this as a side-effect of the KTable record cache, you can directly impose a rate limit via the KTable#suppress
operator.
For example:
KGroupedTable<String, String> groupedTable = ...;
groupedTable
.count()
.suppress(untilTimeLimit(ofMinutes(5), maxBytes(1_000_000L).emitEarlyWhenFull()))
.toStream()
.foreach((key, count) -> updateCountsDatabase(key, count));
This configuration ensures that updateCountsDatabase
gets events for each key
no more than once every 5 minutes. Note that the latest state for each key has to be buffered in memory for that 5-minute period. You have the option to control the maximum amount of memory to use for this buffer (in this case, 1MB). There is also an option to impose a limit in terms of number of records (or to leave both limits unspecified).
Additionally, it is possible to choose what happens if the buffer fills up. This example takes a relaxed approach and just emits the oldest records before their 5-minute time limit to bring the buffer back down to size. Alternatively, you can choose to stop processing and shut the application down. This may seem extreme, but it gives you a guarantee that the 5-minute time limit will be absolutely enforced. After the application shuts down, you could allocate more memory for the buffer and resume processing. Emitting early is preferable for most applications.
For more detailed information, see the JavaDoc on the Suppressed
config object and KIP-328.
Using timestamp-based semantics for table processors
By default, tables in Kafka Streams use offset-based semantics. When multiple records arrive for the same key, the one with the largest record offset is considered the latest record for the key, and is the record that appears in aggregation and join results computed on the table. This is true even in the event of out-of-order data. The record with the largest offset is considered to be the latest record for the key, even if this record does not have the largest timestamp.
An alternative to offset-based semantics is timestamp-based semantics. With timestamp-based semantics, the record with the largest timestamp is considered the latest record, even if there is another record with a larger offset (and smaller timestamp). If there is no out-of-order data (per key), then offset-based semantics and timestamp-based semantics are equivalent; the difference only appears when there is out-of-order data.
Starting with Kafka Streams 3.5, Kafka Streams supports timestamp-based semantics through the use of versioned state stores. When a table is materialized with a versioned state store, it is a versioned table and will result in different processor semantics in the presence of out-of-order data.
- When performing a stream-table join, stream-side records will join with the latest-by-timestamp table record which has a timestamp less than or equal to the stream record’s timestamp. This is in contrast to joining a stream to an unversioned table, in which case the latest-by-offset table record will be joined, even if the stream-side record is out-of-order and has a lower timestamp.
- Aggregations computed on the table will include the latest-by-timestamp record for each key, instead of the latest-by-offset record. Out-of-order updates (per key) will not trigger a new aggregation result. This is true for
count
andreduce
operations as well, in addition toaggregate
operations. - Table joins will use the latest-by-timestamp record for each key, instead of the latest-by-offset record. Out-of-order updates (per key) will not trigger a new join result. This is true for both primary-key table-table joins and also foreign-key table-table joins. If a versioned table is joined with an unversioned table, the result will be the join of the latest-by-timestamp record from the versioned table with the latest-by-offset record from the unversioned table.
- Table filter operations will no longer suppress consecutive tombstones, so users may observe more
null
records downstream of the filter than compared to when filtering an unversioned table. This is done in order to preserve a complete version history downstream, in the event of out-of-order data. suppress
operations are not allowed on versioned tables, as this would collapse the version history and lead to undefined behavior.
Once a table is materialized with a versioned store, downstream tables are also considered versioned until any of the following occurs:
- A downstream table is explicitly materialized, either with an unversioned store supplier or with no store supplier (all stores are unversioned by default, including the default store supplier)
- Any stateful transformation occurs, including aggregations and joins
- A table is converted to a stream and back.
The results of certain processors should not be materialized with versioned stores, as these processors do not produce a complete older version history, and therefore materialization as a versioned table would lead to unpredictable results:
- Aggregate processors, for both table and stream aggregations. This includes
aggregate
,count
andreduce
operations. - Table-table join processors, including both primary-key and foreign-key joins.
For more on versioned stores and how to start using them in your application, see here.
Writing streams back to Kafka
Any streams and tables may be (continuously) written back to a Kafka topic. As we will describe in more detail below, the output data might be re-partitioned on its way to Kafka, depending on the situation.
Writing to Kafka | Description |
---|---|
To |
- KStream -> void
| Terminal operation. Write the records to Kafka topic(s). (KStream details) When to provide serdes explicitly:
- If you do not specify Serdes explicitly, the default Serdes from the configuration are used.
- You must specify Serdes explicitly via the
Produced
class if the key and/or value types of theKStream
do not match the configured default Serdes. - See Data Types and Serialization for information about configuring default Serdes, available Serdes, and implementing your own custom Serdes.
A variant of to
exists that enables you to specify how the data is produced by using a Produced
instance to specify, for example, a StreamPartitioner
that gives you control over how output records are distributed across the partitions of the output topic. Another variant of to
exists that enables you to dynamically choose which topic to send to for each record via a TopicNameExtractor
instance.
KStream<String, Long> stream = ...;
// Write the stream to the output topic, using the configured default key
// and value serdes.
stream.to("my-stream-output-topic");
// Write the stream to the output topic, using explicit key and value serdes,
// (thus overriding the defaults in the config properties).
stream.to("my-stream-output-topic", Produced.with(Serdes.String(), Serdes.Long());
Causes data re-partitioning if any of the following conditions is true:
- If the output topic has a different number of partitions than the stream/table.
- If the
KStream
was marked for re-partitioning. - If you provide a custom
StreamPartitioner
to explicitly control how to distribute the output records across the partitions of the output topic. - If the key of an output record is
null
.
Note
When you want to write to systems other than Kafka: Besides writing the data back to Kafka, you can also apply a custom processor as a stream sink at the end of the processing to, for example, write to external databases. First, doing so is not a recommended pattern - we strongly suggest to use the Kafka Connect API instead. However, if you do use such a sink processor, please be aware that it is now your responsibility to guarantee message delivery semantics when talking to such external systems (e.g., to retry on delivery failure or to prevent message duplication).
Testing a Streams application
Kafka Streams comes with a test-utils
module to help you test your application here.
Kafka Streams DSL for Scala
The Kafka Streams DSL Java APIs are based on the Builder design pattern, which allows users to incrementally build the target functionality using lower level compositional fluent APIs. These APIs can be called from Scala, but there are several issues:
- Additional type annotations - The Java APIs use Java generics in a way that are not fully compatible with the type inferencer of the Scala compiler. Hence the user has to add type annotations to the Scala code, which seems rather non-idiomatic in Scala.
- Verbosity - In some cases the Java APIs appear too verbose compared to idiomatic Scala.
- Type Unsafety - The Java APIs offer some options where the compile time type safety is sometimes subverted and can result in runtime errors. This stems from the fact that the Serdes defined as part of config are not type checked during compile time. Hence any missing Serdes can result in runtime errors.
The Kafka Streams DSL for Scala library is a wrapper over the existing Java APIs for Kafka Streams DSL that addresses the concerns raised above. It does not attempt to provide idiomatic Scala APIs that one would implement in a Scala library developed from scratch. The intention is to make the Java APIs more usable in Scala through better type inferencing, enhanced expressiveness, and lesser boilerplates.
The library wraps Java Stream DSL APIs in Scala thereby providing:
- Better type inference in Scala.
- Less boilerplate in application code.
- The usual builder-style composition that developers get with the original Java API.
- Implicit serializers and de-serializers leading to better abstraction and less verbosity.
- Better type safety during compile time.
All functionality provided by Kafka Streams DSL for Scala are under the root package name of org.apache.kafka.streams.scala
.
Many of the public facing types from the Java API are wrapped. The following Scala abstractions are available to the user:
org.apache.kafka.streams.scala.StreamsBuilder
org.apache.kafka.streams.scala.kstream.KStream
org.apache.kafka.streams.scala.kstream.KTable
org.apache.kafka.streams.scala.kstream.KGroupedStream
org.apache.kafka.streams.scala.kstream.KGroupedTable
org.apache.kafka.streams.scala.kstream.SessionWindowedKStream
org.apache.kafka.streams.scala.kstream.TimeWindowedKStream
The library also has several utility abstractions and modules that the user needs to use for proper semantics.
org.apache.kafka.streams.scala.ImplicitConversions
: Module that brings into scope the implicit conversions between the Scala and Java classes.org.apache.kafka.streams.scala.serialization.Serdes
: Module that contains all primitive Serdes that can be imported as implicits and a helper to create custom Serdes.
The library is cross-built with Scala 2.12 and 2.13. To reference the library compiled against Scala 2.13 include the following in your maven pom.xml
add the following:
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-streams-scala_2.13</artifactId>
<version>4.0.0</version>
</dependency>
To use the library compiled against Scala 2.12 replace the artifactId
with kafka-streams-scala_2.12
.
When using SBT then you can reference the correct library using the following:
libraryDependencies += "org.apache.kafka" %% "kafka-streams-scala" % "4.0.0"
Sample Usage
The library works by wrapping the original Java abstractions of Kafka Streams within a Scala wrapper object and then using implicit conversions between them. All the Scala abstractions are named identically as the corresponding Java abstraction, but they reside in a different package of the library e.g. the Scala class org.apache.kafka.streams.scala.StreamsBuilder
is a wrapper around org.apache.kafka.streams.StreamsBuilder
, org.apache.kafka.streams.scala.kstream.KStream
is a wrapper around org.apache.kafka.streams.kstream.KStream
, and so on.
Here’s an example of the classic WordCount program that uses the Scala StreamsBuilder
that builds an instance of KStream
which is a wrapper around Java KStream
. Then we reify to a table and get a KTable
, which, again is a wrapper around Java KTable
.
The net result is that the following code is structured just like using the Java API, but with Scala and with far fewer type annotations compared to using the Java API directly from Scala. The difference in type annotation usage is more obvious when given an example. Below is an example WordCount implementation that will be used to demonstrate the differences between the Scala and Java API.
import java.time.Duration
import java.util.Properties
import org.apache.kafka.streams.kstream.Materialized
import org.apache.kafka.streams.scala.ImplicitConversions._
import org.apache.kafka.streams.scala._
import org.apache.kafka.streams.scala.kstream._
import org.apache.kafka.streams.{KafkaStreams, StreamsConfig}
object WordCountApplication extends App {
import Serdes._
val props: Properties = {
val p = new Properties()
p.put(StreamsConfig.APPLICATION_ID_CONFIG, "wordcount-application")
p.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092")
p
}
val builder: StreamsBuilder = new StreamsBuilder
val textLines: KStream[String, String] = builder.stream[String, String]("TextLinesTopic")
val wordCounts: KTable[String, Long] = textLines
.flatMapValues(textLine => textLine.toLowerCase.split("\W+"))
.groupBy((_, word) => word)
.count(Materialized.as("counts-store"))
wordCounts.toStream.to("WordsWithCountsTopic")
val streams: KafkaStreams = new KafkaStreams(builder.build(), props)
streams.start()
sys.ShutdownHookThread {
streams.close(Duration.ofSeconds(10))
}
}
In the above code snippet, we don’t have to provide any Serdes, Grouped
, Produced
, Consumed
or Joined
explicitly. They will also not be dependent on any Serdes specified in the config. In fact all Serdes specified in the config will be ignored by the Scala APIs. All Serdes and Grouped
, Produced
, Consumed
or Joined
will be handled through implicit Serdes as discussed later in the Implicit Serdes section. The complete independence from configuration based Serdes is what makes this library completely typesafe. Any missing instances of Serdes, Grouped
, Produced
, Consumed
or Joined
will be flagged as a compile time error.
Implicit Serdes
One of the common complaints of Scala users with the Java API has been the repetitive usage of the Serdes in API invocations. Many of the APIs need to take the Serdes through abstractions like Grouped
, Produced
, Repartitioned
, Consumed
or Joined
. And the user has to supply them every time through the with function of these classes.
The library uses the power of Scala implicit parameters to alleviate this concern. As a user you can provide implicit Serdes or implicit values of Grouped
, Produced
, Repartitioned
, Consumed
or Joined
once and make your code less verbose. In fact you can just have the implicit Serdes in scope and the library will make the instances of Grouped
, Produced
, Consumed
or Joined
available in scope.
The library also bundles all implicit Serdes of the commonly used primitive types in a Scala module - so just import the module vals and have all Serdes in scope. A similar strategy of modular implicits can be adopted for any user-defined Serdes as well (User-defined Serdes are discussed in the next section).
Here’s an example:
// DefaultSerdes brings into scope implicit Serdes (mostly for primitives)
// that will set up all Grouped, Produced, Consumed and Joined instances.
// So all APIs below that accept Grouped, Produced, Consumed or Joined will
// get these instances automatically
import Serdes._
val builder = new StreamsBuilder()
val userClicksStream: KStream[String, Long] = builder.stream(userClicksTopic)
val userRegionsTable: KTable[String, String] = builder.table(userRegionsTopic)
// The following code fragment does not have a single instance of Grouped,
// Produced, Consumed or Joined supplied explicitly.
// All of them are taken care of by the implicit Serdes imported by DefaultSerdes
val clicksPerRegion: KTable[String, Long] =
userClicksStream
.leftJoin(userRegionsTable)((clicks, region) => (if (region == null) "UNKNOWN" else region, clicks))
.map((_, regionWithClicks) => regionWithClicks)
.groupByKey
.reduce(_ + _)
clicksPerRegion.toStream.to(outputTopic)
Quite a few things are going on in the above code snippet that may warrant a few lines of elaboration:
- The code snippet does not depend on any config defined Serdes. In fact any Serdes defined as part of the config will be ignored.
- All Serdes are picked up from the implicits in scope. And
import Serdes._
brings all necessary Serdes in scope. - This is an example of compile time type safety that we don’t have in the Java APIs.
- The code looks less verbose and more focused towards the actual transformation that it does on the data stream.
User-Defined Serdes
When the default primitive Serdes are not enough and we need to define custom Serdes, the usage is exactly the same as above. Just define the implicit Serdes and start building the stream transformation. Here’s an example with AvroSerde
:
// domain object as a case class
case class UserClicks(clicks: Long)
// An implicit Serde implementation for the values we want to
// serialize as avro
implicit val userClicksSerde: Serde[UserClicks] = new AvroSerde
// Primitive Serdes
import Serdes._
// And then business as usual ..
val userClicksStream: KStream[String, UserClicks] = builder.stream(userClicksTopic)
val userRegionsTable: KTable[String, String] = builder.table(userRegionsTopic)
// Compute the total per region by summing the individual click counts per region.
val clicksPerRegion: KTable[String, Long] =
userClicksStream
// Join the stream against the table.
.leftJoin(userRegionsTable)((clicks, region) => (if (region == null) "UNKNOWN" else region, clicks.clicks))
// Change the stream from <user> -> <region, clicks> to <region> -> <clicks>
.map((_, regionWithClicks) => regionWithClicks)
// Compute the total per region by summing the individual click counts per region.
.groupByKey
.reduce(_ + _)
// Write the (continuously updating) results to the output topic.
clicksPerRegion.toStream.to(outputTopic)
A complete example of user-defined Serdes can be found in a test class within the library.
7.4 - Processor API
Processor API
The Processor API allows developers to define and connect custom processors and to interact with state stores. With the Processor API, you can define arbitrary stream processors that process one received record at a time, and connect these processors with their associated state stores to compose the processor topology that represents a customized processing logic.
Table of Contents
- Overview
- Defining a Stream Processor
- Unit Testing Processors
- State Stores
- Defining and creating a State Store
- Fault-tolerant State Stores
- Enable or Disable Fault Tolerance of State Stores (Store Changelogs)
- Timestamped State Stores
- Versioned Key-Value State Stores
- Readonly State Stores
- Implementing Custom State Stores
- Connecting Processors and State Stores
- Accessing Processor Context
Overview
The Processor API can be used to implement both stateless as well as stateful operations, where the latter is achieved through the use of state stores.
Tip
Combining the DSL and the Processor API: You can combine the convenience of the DSL with the power and flexibility of the Processor API as described in the section Applying processors (Processor API integration).
For a complete list of available API functionality, see the Streams API docs.
Defining a Stream Processor
A stream processor is a node in the processor topology that represents a single processing step. With the Processor API, you can define arbitrary stream processors that processes one received record at a time, and connect these processors with their associated state stores to compose the processor topology.
You can define a customized stream processor by implementing the Processor
interface, which provides the process()
API method. The process()
method is called on each of the received records.
The Processor
interface also has an init()
method, which is called by the Kafka Streams library during task construction phase. Processor instances should perform any required initialization in this method. The init()
method passes in a ProcessorContext
instance, which provides access to the metadata of the currently processed record, including its source Kafka topic and partition, its corresponding message offset, and further such information. You can also use this context instance to schedule a punctuation function (via ProcessorContext#schedule()
), to forward a new record to the downstream processors (via ProcessorContext#forward()
), and to request a commit of the current processing progress (via ProcessorContext#commit()
). Any resources you set up in init()
can be cleaned up in the close()
method. Note that Kafka Streams may re-use a single Processor
object by calling init()
on it again after close()
.
The Processor
interface takes four generic parameters: KIn, VIn, KOut, VOut
. These define the input and output types that the processor implementation can handle. KIn
and VIn
define the key and value types of the Record
that will be passed to process()
. Likewise, KOut
and VOut
define the forwarded key and value types for the result Record
that ProcessorContext#forward()
will accept. If your processor does not forward any records at all (or if it only forwards null
keys or values), a best practice is to set the output generic type argument to Void
. If it needs to forward multiple types that don’t share a common superclass, you will have to set the output generic type argument to Object
.
Both the Processor#process()
and the ProcessorContext#forward()
methods handle records in the form of the Record<K, V>
data class. This class gives you access to the main components of a Kafka record: the key, value, timestamp and headers. When forwarding records, you can use the constructor to create a new Record
from scratch, or you can use the convenience builder methods to replace one of the Record
’s properties and copy over the rest. For example, inputRecord.withValue(newValue)
would copy the key, timestamp, and headers from inputRecord
while setting the output record’s value to newValue
. Note that this does not mutate inputRecord
, but instead creates a shallow copy. Beware that this is only a shallow copy, so if you plan to mutate the key, value, or headers elsewhere in the program, you will want to create a deep copy of those fields yourself.
In addition to handling incoming records via Processor#process()
, you have the option to schedule periodic invocation (called “punctuation”) in your processor’s init()
method by calling ProcessorContext#schedule()
and passing it a Punctuator
. The PunctuationType
determines what notion of time is used for the punctuation scheduling: either stream-time or wall-clock-time (by default, stream-time is configured to represent event-time via TimestampExtractor
). When stream-time is used, punctuate()
is triggered purely by data because stream-time is determined (and advanced forward) by the timestamps derived from the input data. When there is no new input data arriving, stream-time is not advanced and thus punctuate()
is not called.
For example, if you schedule a Punctuator
function every 10 seconds based on PunctuationType.STREAM_TIME
and if you process a stream of 60 records with consecutive timestamps from 1 (first record) to 60 seconds (last record), then punctuate()
would be called 6 times. This happens regardless of the time required to actually process those records. punctuate()
would be called 6 times regardless of whether processing these 60 records takes a second, a minute, or an hour.
When wall-clock-time (i.e. PunctuationType.WALL_CLOCK_TIME
) is used, punctuate()
is triggered purely by the wall-clock time. Reusing the example above, if the Punctuator
function is scheduled based on PunctuationType.WALL_CLOCK_TIME
, and if these 60 records were processed within 20 seconds, punctuate()
is called 2 times (one time every 10 seconds). If these 60 records were processed within 5 seconds, then no punctuate()
is called at all. Note that you can schedule multiple Punctuator
callbacks with different PunctuationType
types within the same processor by calling ProcessorContext#schedule()
multiple times inside init()
method.
Attention
Stream-time is only advanced when Streams processes records. If there are no records to process, or if Streams is waiting for new records due to the Task Idling configuration, then the stream time will not advance and punctuate()
will not be triggered if PunctuationType.STREAM_TIME
was specified. This behavior is independent of the configured timestamp extractor, i.e., using WallclockTimestampExtractor
does not enable wall-clock triggering of punctuate()
.
Example
The following example Processor
defines a simple word-count algorithm and the following actions are performed:
In the
init()
method, schedule the punctuation every 1000 time units (the time unit is normally milliseconds, which in this example would translate to punctuation every 1 second) and retrieve the local state store by its name “Counts”.In the
process()
method, upon each received record, split the value string into words, and update their counts into the state store (we will talk about this later in this section).In the
punctuate()
method, iterate the local state store and send the aggregated counts to the downstream processor (we will talk about downstream processors later in this section), and commit the current stream state.public class WordCountProcessor implements Processor<String, String, String, String> { private KeyValueStore<String, Integer> kvStore;
@Override public void init(final ProcessorContext<String, String> context) { context.schedule(Duration.ofSeconds(1), PunctuationType.STREAM_TIME, timestamp -> { try (final KeyValueIterator<String, Integer> iter = kvStore.all()) { while (iter.hasNext()) { final KeyValue<String, Integer> entry = iter.next(); context.forward(new Record<>(entry.key, entry.value.toString(), timestamp)); } } }); kvStore = context.getStateStore("Counts"); } @Override public void process(final Record<String, String> record) { final String[] words = record.value().toLowerCase(Locale.getDefault()).split("\W+"); for (final String word : words) { final Integer oldValue = kvStore.get(word); if (oldValue == null) { kvStore.put(word, 1); } else { kvStore.put(word, oldValue + 1); } } } @Override public void close() { // close any resources managed by this processor // Note: Do not close any StateStores as these are managed by the library }
}
Note
Stateful processing with state stores: The WordCountProcessor
defined above can access the currently received record in its process()
method, and it can leverage state stores to maintain processing states to, for example, remember recently arrived records for stateful processing needs like aggregations and joins. For more information, see the state stores documentation.
Unit Testing Processors
Kafka Streams comes with a test-utils
module to help you write unit tests for your processors here.
State Stores
To implement a stateful Processor
, you must provide one or more state stores to the processor (stateless processors do not need state stores). State stores can be used to remember recently received input records, to track rolling aggregates, to de-duplicate input records, and more. Another feature of state stores is that they can be interactively queried from other applications, such as a NodeJS-based dashboard or a microservice implemented in Scala or Go.
The available state store types in Kafka Streams have fault tolerance enabled by default.
Defining and creating a State Store
You can either use one of the available store types or implement your own custom store type. It’s common practice to leverage an existing store type via the Stores
factory.
Note that, when using Kafka Streams, you normally don’t create or instantiate state stores directly in your code. Rather, you define state stores indirectly by creating a so-called StoreBuilder
. This builder is used by Kafka Streams as a factory to instantiate the actual state stores locally in application instances when and where needed.
The following store types are available out of the box.
Store Type | Storage Engine | Fault-tolerant? | Description |
---|---|---|---|
Persistent KeyValueStore<K, V> | RocksDB | Yes (enabled by default) |
The recommended store type for most use cases.
Stores its data on local disk.
Storage capacity: managed local state can be larger than the memory (heap space) of an application instance, but must fit into the available local disk space.
RocksDB settings can be fine-tuned, see RocksDB configuration.
Available store variants: timestamped key-value store, versioned key-value store, time window key-value store, session window key-value store.
Use persistentTimestampedKeyValueStore when you need a persistent key-(value/timestamp) store that supports put/get/delete and range queries.
Use persistentVersionedKeyValueStore when you need a persistent, versioned key-(value/timestamp) store that supports put/get/delete and timestamped get operations.
Use persistentWindowStore or persistentTimestampedWindowStore when you need a persistent timeWindowedKey-value or timeWindowedKey-(value/timestamp) store, respectively.
Use persistentSessionStore when you need a persistent sessionWindowedKey-value store.
// Creating a persistent key-value store: // here, we create a
KeyValueStore<String, Long>
named “persistent-counts”. import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores;// Using a
KeyValueStoreBuilder
to build aKeyValueStore
. StoreBuilder<KeyValueStore<String, Long» countStoreSupplier = Stores.keyValueStoreBuilder( Stores.persistentKeyValueStore(“persistent-counts”), Serdes.String(), Serdes.Long()); KeyValueStore<String, Long> countStore = countStoreSupplier.build();
In-memory KeyValueStore<K, V>
| - | Yes (enabled by default) |
Stores its data in memory.
Storage capacity: managed local state must fit into memory (heap space) of an application instance.
Useful when application instances run in an environment where local disk space is either not available or local disk space is wiped in-between app instance restarts.
Available store variants: time window key-value store, session window key-value store.
Use TimestampedKeyValueStore when you need a key-(value/timestamp) store that supports put/get/delete and range queries.
Use TimestampedWindowStore when you need to store windowedKey-(value/timestamp) pairs.
There is no built-in in-memory, versioned key-value store at this time.
// Creating an in-memory key-value store: // here, we create a
KeyValueStore<String, Long>
named “inmemory-counts”. import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores;// Using a
KeyValueStoreBuilder
to build aKeyValueStore
. StoreBuilder<KeyValueStore<String, Long» countStoreSupplier = Stores.keyValueStoreBuilder( Stores.inMemoryKeyValueStore(“inmemory-counts”), Serdes.String(), Serdes.Long()); KeyValueStore<String, Long> countStore = countStoreSupplier.build();
Fault-tolerant State Stores
To make state stores fault-tolerant and to allow for state store migration without data loss, a state store can be continuously backed up to a Kafka topic behind the scenes. For example, to migrate a stateful stream task from one machine to another when elastically adding or removing capacity from your application. This topic is sometimes referred to as the state store’s associated changelog topic , or its changelog. For example, if you experience machine failure, the state store and the application’s state can be fully restored from its changelog. You can enable or disable this backup feature for a state store.
Fault-tolerant state stores are backed by a compacted changelog topic. The purpose of compacting this topic is to prevent the topic from growing indefinitely, to reduce the storage consumed in the associated Kafka cluster, and to minimize recovery time if a state store needs to be restored from its changelog topic.
Fault-tolerant windowed state stores are backed by a topic that uses both compaction and deletion. Because of the structure of the message keys that are being sent to the changelog topics, this combination of deletion and compaction is required for the changelog topics of window stores. For window stores, the message keys are composite keys that include the “normal” key and window timestamps. For these types of composite keys it would not be sufficient to only enable compaction to prevent a changelog topic from growing out of bounds. With deletion enabled, old windows that have expired will be cleaned up by Kafka’s log cleaner as the log segments expire. The default retention setting is Windows#maintainMs()
+ 1 day. You can override this setting by specifying StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG
in the StreamsConfig
.
When you open an Iterator
from a state store you must call close()
on the iterator when you are done working with it to reclaim resources; or you can use the iterator from within a try-with-resources statement. If you do not close an iterator, you may encounter an OOM error.
Enable or Disable Fault Tolerance of State Stores (Store Changelogs)
You can enable or disable fault tolerance for a state store by enabling or disabling the change logging of the store through enableLogging()
and disableLogging()
. You can also fine-tune the associated topic’s configuration if needed.
Example for disabling fault-tolerance:
import org.apache.kafka.streams.state.StoreBuilder;
import org.apache.kafka.streams.state.Stores;
StoreBuilder<KeyValueStore<String, Long>> countStoreSupplier = Stores.keyValueStoreBuilder(
Stores.persistentKeyValueStore("Counts"),
Serdes.String(),
Serdes.Long())
.withLoggingDisabled(); // disable backing up the store to a changelog topic
Attention
If the changelog is disabled then the attached state store is no longer fault tolerant and it can’t have any standby replicas.
Here is an example for enabling fault tolerance, with additional changelog-topic configuration: You can add any log config from kafka.log.LogConfig. Unrecognized configs will be ignored.
import org.apache.kafka.streams.state.StoreBuilder;
import org.apache.kafka.streams.state.Stores;
Map<String, String> changelogConfig = new HashMap();
// override min.insync.replicas
changelogConfig.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "1")
StoreBuilder<KeyValueStore<String, Long>> countStoreSupplier = Stores.keyValueStoreBuilder(
Stores.persistentKeyValueStore("Counts"),
Serdes.String(),
Serdes.Long())
.withLoggingEnabled(changelogConfig); // enable changelogging, with custom changelog settings
Timestamped State Stores
KTables always store timestamps by default. A timestamped state store improves stream processing semantics and enables handling out-of-order data in source KTables, detecting out-of-order joins and aggregations, and getting the timestamp of the latest update in an Interactive Query.
You can query timestamped state stores both with and without a timestamp.
Upgrade note: All users upgrade with a single rolling bounce per instance.
- For Processor API users, nothing changes in existing applications, and you have the option of using the timestamped stores.
- For DSL operators, store data is upgraded lazily in the background.
- No upgrade happens if you provide a custom XxxBytesStoreSupplier, but you can opt-in by implementing the TimestampedBytesStore interface. In this case, the old format is retained, and Streams uses a proxy store that removes/adds timestamps on read/write.
Versioned Key-Value State Stores
Versioned key-value state stores are available since Kafka Streams 3.5. Rather than storing a single record version (value and timestamp) per key, versioned state stores may store multiple record versions per key. This allows versioned state stores to support timestamped retrieval operations to return the latest record (per key) as of a specified timestamp.
You can create a persistent, versioned state store by passing a VersionedBytesStoreSupplier to the versionedKeyValueStoreBuilder, or by implementing your own VersionedKeyValueStore.
Each versioned store has an associated, fixed-duration history retention parameter which specifies long old record versions should be kept for. In particular, a versioned store guarantees to return accurate results for timestamped retrieval operations where the timestamp being queried is within history retention of the current observed stream time.
History retention also doubles as its grace period , which determines how far back in time out-of-order writes to the store will be accepted. A versioned store will not accept writes (inserts, updates, or deletions) if the timestamp associated with the write is older than the current observed stream time by more than the grace period. Stream time in this context is tracked per-partition, rather than per-key, which means it’s important that grace period (i.e., history retention) be set high enough to accommodate a record with one key arriving out-of-order relative to a record for another key.
Because the memory footprint of versioned key-value stores is higher than that of non-versioned key-value stores, you may want to adjust your RocksDB memory settings accordingly. Benchmarking your application with versioned stores is also advised as performance is expected to be worse than when using non-versioned stores.
Versioned stores do not support caching or interactive queries at this time. Also, window stores and global tables may not be versioned.
Upgrade note: Versioned state stores are opt-in only; no automatic upgrades from non-versioned to versioned stores will take place.
Upgrades are supported from persistent, non-versioned key-value stores to persistent, versioned key-value stores as long as the original store has the same changelog topic format as the versioned store being upgraded to. Both persistent key-value stores and timestamped key-value stores share the same changelog topic format as persistent versioned key-value stores, and therefore both are eligible for upgrades.
If you wish to upgrade an application using persistent, non-versioned key-value stores to use persistent, versioned key-value stores instead, you can perform the following procedure:
- Stop all application instances, and clear any local state directories for the store(s) being upgraded.
- Update your application code to use versioned stores where desired.
- Update your changelog topic configs, for the relevant state stores, to set the value of
min.compaction.lag.ms
to be at least your desired history retention. History retention plus one day is recommended as buffer for the use of broker wall clock time during compaction. - Restart your application instances and allow time for the versioned stores to rebuild state from changelog.
ReadOnly State Stores
A read-only state store materialized the data from its input topic. It also uses the input topic for fault-tolerance, and thus does not have an additional changelog topic (the input topic is re-used as changelog). Thus, the input topic should be configured with log compaction. Note that no other processor should modify the content of the state store, and the only writer should be the associated “state update processor”; other processors may read the content of the read-only store.
note: beware of the partitioning requirements when using read-only state stores for lookups during processing. You might want to make sure the original changelog topic is co-partitioned with the processors reading the read-only statestore.
Implementing Custom State Stores
You can use the built-in state store types or implement your own. The primary interface to implement for the store is org.apache.kafka.streams.processor.StateStore
. Kafka Streams also has a few extended interfaces such as KeyValueStore
and VersionedKeyValueStore
.
Note that your customized org.apache.kafka.streams.processor.StateStore
implementation also needs to provide the logic on how to restore the state via the org.apache.kafka.streams.processor.StateRestoreCallback
or org.apache.kafka.streams.processor.BatchingStateRestoreCallback
interface. Details on how to instantiate these interfaces can be found in the javadocs.
You also need to provide a “builder” for the store by implementing the org.apache.kafka.streams.state.StoreBuilder
interface, which Kafka Streams uses to create instances of your store.
Accessing Processor Context
As we have mentioned in the Defining a Stream Processor section, a ProcessorContext
control the processing workflow, such as scheduling a punctuation function, and committing the current processed state.
This object can also be used to access the metadata related with the application like applicationId
, taskId
, and stateDir
, and also RecordMetadata
such as topic
, partition
, and offset
.
Connecting Processors and State Stores
Now that a processor (WordCountProcessor) and the state stores have been defined, you can construct the processor topology by connecting these processors and state stores together by using the Topology
instance. In addition, you can add source processors with the specified Kafka topics to generate input data streams into the topology, and sink processors with the specified Kafka topics to generate output data streams out of the topology.
Here is an example implementation:
Topology builder = new Topology();
// add the source processor node that takes Kafka topic "source-topic" as input
builder.addSource("Source", "source-topic")
// add the WordCountProcessor node which takes the source processor as its upstream processor
.addProcessor("Process", () -> new WordCountProcessor(), "Source")
// add the count store associated with the WordCountProcessor processor
.addStateStore(countStoreBuilder, "Process")
// add the sink processor node that takes Kafka topic "sink-topic" as output
// and the WordCountProcessor node as its upstream processor
.addSink("Sink", "sink-topic", "Process");
Here is a quick explanation of this example:
- A source processor node named
"Source"
is added to the topology using theaddSource
method, with one Kafka topic"source-topic"
fed to it. - A processor node named
"Process"
with the pre-definedWordCountProcessor
logic is then added as the downstream processor of the"Source"
node using theaddProcessor
method. - A predefined persistent key-value state store is created and associated with the
"Process"
node, usingcountStoreBuilder
. - A sink processor node is then added to complete the topology using the
addSink
method, taking the"Process"
node as its upstream processor and writing to a separate"sink-topic"
Kafka topic (note that users can also use another overloaded variant ofaddSink
to dynamically determine the Kafka topic to write to for each received record from the upstream processor).
In some cases, it may be more convenient to add and connect a state store at the same time as you add the processor to the topology. This can be done by implementing ConnectedStoreProvider#stores()
on the ProcessorSupplier
instead of calling Topology#addStateStore()
, like this:
Topology builder = new Topology();
// add the source processor node that takes Kafka "source-topic" as input
builder.addSource("Source", "source-topic")
// add the WordCountProcessor node which takes the source processor as its upstream processor.
// the ProcessorSupplier provides the count store associated with the WordCountProcessor
.addProcessor("Process", new ProcessorSupplier<String, String, String, String>() {
public Processor<String, String, String, String> get() {
return new WordCountProcessor();
}
public Set<StoreBuilder<?>> stores() {
final StoreBuilder<KeyValueStore<String, Long>> countsStoreBuilder =
Stores
.keyValueStoreBuilder(
Stores.persistentKeyValueStore("Counts"),
Serdes.String(),
Serdes.Long()
);
return Collections.singleton(countsStoreBuilder);
}
}, "Source")
// add the sink processor node that takes Kafka topic "sink-topic" as output
// and the WordCountProcessor node as its upstream processor
.addSink("Sink", "sink-topic", "Process");
This allows for a processor to “own” state stores, effectively encapsulating their usage from the user wiring the topology. Multiple processors that share a state store may provide the same store with this technique, as long as the StoreBuilder
is the same instance
.
In these topologies, the "Process"
stream processor node is considered a downstream processor of the "Source"
node, and an upstream processor of the "Sink"
node. As a result, whenever the "Source"
node forwards a newly fetched record from Kafka to its downstream "Process"
node, the WordCountProcessor#process()
method is triggered to process the record and update the associated state store. Whenever context#forward()
is called in the WordCountProcessor#punctuate()
method, the aggregate records will be sent via the "Sink"
processor node to the Kafka topic "sink-topic"
. Note that in the WordCountProcessor
implementation, you must refer to the same store name "Counts"
when accessing the key-value store, otherwise an exception will be thrown at runtime, indicating that the state store cannot be found. If the state store is not associated with the processor in the Topology
code, accessing it in the processor’s init()
method will also throw an exception at runtime, indicating the state store is not accessible from this processor.
Note that the Topology#addProcessor
function takes a ProcessorSupplier
as argument, and that the supplier pattern requires that a new Processor
instance is returned each time ProcessorSupplier#get()
is called. Creating a single Processor
object and returning the same object reference in ProcessorSupplier#get()
would be a violation of the supplier pattern and leads to runtime exceptions. So remember not to provide a singleton Processor
instance to Topology
. The ProcessorSupplier
should always generate a new instance each time ProcessorSupplier#get()
gets called.
Now that you have fully defined your processor topology in your application, you can proceed to running the Kafka Streams application.
7.5 - Naming Operators in a Streams DSL application
Developer Guide for Kafka Streams
Naming Operators in a Kafka Streams DSL Application
You now can give names to processors when using the Kafka Streams DSL. In the PAPI there are Processors
and State Stores
and you are required to explicitly name each one.
At the DSL layer, there are operators. A single DSL operator may compile down to multiple Processors
and State Stores
, and if required repartition topics
. But with the Kafka Streams DSL, all these names are generated for you. There is a relationship between the generated processor name state store names (hence changelog topic names) and repartition topic names. Note, that the names of state stores and changelog/repartition topics are “stateful” while processor names are “stateless”.
This distinction of stateful vs. stateless names has important implications when updating your topology. While the internal naming makes creating a topology with the DSL much more straightforward, there are a couple of trade-offs. The first trade-off is what we could consider a readability issue. The other more severe trade-off is the shifting of names due to the relationship between the DSL operator and the generated Processors
, State Stores
changelog topics and repartition topics.
Readability Issues
By saying there is a readability trade-off, we are referring to viewing a description of the topology. When you render the string description of your topology via the Topology#describe()
method, you can see what the processor is, but you don’t have any context for its business purpose. For example, consider the following simple topology:
KStream<String,String> stream = builder.stream("input");
stream.filter((k,v) -> !v.equals("invalid_txn"))
.mapValues((v) -> v.substring(0,5))
.to("output");
Running Topology#describe()
yields this string:
Topologies:
Sub-topology: 0
Source: KSTREAM-SOURCE-0000000000 (topics: [input])
--> KSTREAM-FILTER-0000000001
Processor: KSTREAM-FILTER-0000000001 (stores: [])
--> KSTREAM-MAPVALUES-0000000002
<-- KSTREAM-SOURCE-0000000000
Processor: KSTREAM-MAPVALUES-0000000002 (stores: [])
--> KSTREAM-SINK-0000000003
<-- KSTREAM-FILTER-0000000001
Sink: KSTREAM-SINK-0000000003 (topic: output)
<-- KSTREAM-MAPVALUES-0000000002
From this report, you can see what the different operators are, but what is the broader context here? For example, consider KSTREAM-FILTER-0000000001
, we can see that it’s a filter operation, which means that records are dropped that don’t match the given predicate. But what is the meaning of the predicate? Additionally, you can see the topic names of the source and sink nodes, but what if the topics aren’t named in a meaningful way? Then you’re left to guess the business purpose behind these topics.
Also notice the numbering here: the source node is suffixed with 0000000000
indicating it’s the first processor in the topology. The filter is suffixed with 0000000001
, indicating it’s the second processor in the topology. In Kafka Streams, there are now overloaded methods for both KStream
and KTable
that accept a new parameter Named
. By using the Named
class DSL users can provide meaningful names to the processors in their topology.
Now let’s take a look at your topology with all the processors named:
KStream<String,String> stream =
builder.stream("input", Consumed.as("Customer_transactions_input_topic"));
stream.filter((k,v) -> !v.equals("invalid_txn"), Named.as("filter_out_invalid_txns"))
.mapValues((v) -> v.substring(0,5), Named.as("Map_values_to_first_6_characters"))
.to("output", Produced.as("Mapped_transactions_output_topic"));
Topologies:
Sub-topology: 0
Source: Customer_transactions_input_topic (topics: [input])
--> filter_out_invalid_txns
Processor: filter_out_invalid_txns (stores: [])
--> Map_values_to_first_6_characters
<-- Customer_transactions_input_topic
Processor: Map_values_to_first_6_characters (stores: [])
--> Mapped_transactions_output_topic
<-- filter_out_invalid_txns
Sink: Mapped_transactions_output_topic (topic: output)
<-- Map_values_to_first_6_characters
Now you can look at the topology description and easily understand what role each processor plays in the topology. But there’s another reason for naming your processor nodes when you have stateful operators that remain between restarts of your Kafka Streams applications, state stores, changelog topics, and repartition topics.
Changing Names
Generated names are numbered where they are built in the topology. The name generation strategy is KSTREAM|KTABLE->operator name<->number suffix<
. The number is a globally incrementing number that represents the operator’s order in the topology. The generated number is prefixed with a varying number of “0"s to create a string that is consistently 10 characters long. This means that if you add/remove or shift the order of operations, the position of the processor shifts, which shifts the name of the processor. Since most processors exist in memory only, this name shifting presents no issue for many topologies. But the name shifting does have implications for topologies with stateful operators or repartition topics. Here’s a different topology with some state:
KStream<String,String> stream = builder.stream("input");
stream.groupByKey()
.count()
.toStream()
.to("output");
This topology description yields the following:
Topologies:
Sub-topology: 0
Source: KSTREAM-SOURCE-0000000000 (topics: [input])
--> KSTREAM-AGGREGATE-0000000002
Processor: KSTREAM-AGGREGATE-0000000002 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000001])
--> KTABLE-TOSTREAM-0000000003
<-- KSTREAM-SOURCE-0000000000
Processor: KTABLE-TOSTREAM-0000000003 (stores: [])
--> KSTREAM-SINK-0000000004
<-- KSTREAM-AGGREGATE-0000000002
Sink: KSTREAM-SINK-0000000004 (topic: output)
<-- KTABLE-TOSTREAM-0000000003
You can see from the topology description above that the state store is named KSTREAM-AGGREGATE-STATE-STORE-0000000002
. Here’s what happens when you add a filter to keep some of the records out of the aggregation:
KStream<String,String> stream = builder.stream("input");
stream.filter((k,v)-> v !=null && v.length() >= 6 )
.groupByKey()
.count()
.toStream()
.to("output");
And the corresponding topology:
Topologies:
Sub-topology: 0
Source: KSTREAM-SOURCE-0000000000 (topics: [input])
--> KSTREAM-FILTER-0000000001
Processor: KSTREAM-FILTER-0000000001 (stores: [])
--> KSTREAM-AGGREGATE-0000000003
<-- KSTREAM-SOURCE-0000000000
Processor: KSTREAM-AGGREGATE-0000000003 (stores: [KSTREAM-AGGREGATE-STATE-STORE-0000000002])
--> KTABLE-TOSTREAM-0000000004
<-- KSTREAM-FILTER-0000000001
Processor: KTABLE-TOSTREAM-0000000004 (stores: [])
--> KSTREAM-SINK-0000000005
<-- KSTREAM-AGGREGATE-0000000003
Sink: KSTREAM-SINK-0000000005 (topic: output)
<-- KTABLE-TOSTREAM-0000000004
Notice that since you’ve added an operation before the count
operation, the state store (and the changelog topic) names have changed. This name change means you can’t do a rolling re-deployment of your updated topology. Also, you must use the Streams Reset Tool to re-calculate the aggregations, because the changelog topic has changed on start-up and the new changelog topic contains no data. Fortunately, there’s an easy solution to remedy this situation. Give the state store a user-defined name instead of relying on the generated one, so you don’t have to worry about topology changes shifting the name of the state store. You’ve had the ability to name repartition topics with the Joined
, StreamJoined
, andGrouped
classes, and name state store and changelog topics with Materialized
. But it’s worth reiterating the importance of naming these DSL topology operations again. Here’s how your DSL code looks now giving a specific name to your state store:
KStream<String,String> stream = builder.stream("input");
stream.filter((k, v) -> v != null && v.length() >= 6)
.groupByKey()
.count(Materialized.as("Purchase_count_store"))
.toStream()
.to("output");
And here’s the topology
Topologies:
Sub-topology: 0
Source: KSTREAM-SOURCE-0000000000 (topics: [input])
--> KSTREAM-FILTER-0000000001
Processor: KSTREAM-FILTER-0000000001 (stores: [])
--> KSTREAM-AGGREGATE-0000000002
<-- KSTREAM-SOURCE-0000000000
Processor: KSTREAM-AGGREGATE-0000000002 (stores: [Purchase_count_store])
--> KTABLE-TOSTREAM-0000000003
<-- KSTREAM-FILTER-0000000001
Processor: KTABLE-TOSTREAM-0000000003 (stores: [])
--> KSTREAM-SINK-0000000004
<-- KSTREAM-AGGREGATE-0000000002
Sink: KSTREAM-SINK-0000000004 (topic: output)
<-- KTABLE-TOSTREAM-0000000003
Now, even though you’ve added processors before your state store, the store name and its changelog topic names don’t change. This makes your topology more robust and resilient to changes made by adding or removing processors.
Conclusion
It’s a good practice to name your processing nodes when using the DSL, and it’s even more important to do this when you have “stateful” processors your application such as repartition topics and state stores (and the accompanying changelog topics).
Here are a couple of points to remember when naming your DSL topology:
- If you have an existing topology and you haven’t named your state stores (and changelog topics) and repartition topics, we recommended that you do so. But this will be a topology breaking change, so you’ll need to shut down all application instances, make the changes, and run the Streams Reset Tool. Although this may be inconvenient at first, it’s worth the effort to protect your application from unexpected errors due to topology changes.
- If you have a new topology , make sure you name the persistent parts of your topology: state stores (changelog topics) and repartition topics. This way, when you deploy your application, you’re protected from topology changes that otherwise would break your Kafka Streams application. If you don’t want to add names to stateless processors at first, that’s fine as you can always go back and add the names later.
Here’s a quick reference on naming the critical parts of your Kafka Streams application to prevent topology name changes from breaking your application: Operation | Naming Class |
---|---|
Aggregation repartition topics | Grouped |
KStream-KStream Join repartition topics | StreamJoined |
KStream-KTable Join repartition topic | Joined |
KStream-KStream Join state stores | StreamJoined |
State Stores (for aggregations and KTable-KTable joins) | Materialized |
Stream/Table non-stateful operations | Named |
7.6 - Data Types and Serialization
Data Types and Serialization
Every Kafka Streams application must provide Serdes (Serializer/Deserializer) for the data types of record keys and record values (e.g. java.lang.String
) to materialize the data when necessary. Operations that require such Serdes information include: stream()
, table()
, to()
, repartition()
, groupByKey()
, groupBy()
.
You can provide Serdes by using either of these methods, but you must use at least one:
- By setting default Serdes in the
java.util.Properties
config instance. - By specifying explicit Serdes when calling the appropriate API methods, thus overriding the defaults.
Table of Contents
- Configuring Serdes
- Overriding default Serdes
- Available Serdes
- Primitive and basic types
- JSON
- Implementing custom serdes
- Kafka Streams DSL for Scala Implicit Serdes
Configuring Serdes
Serdes specified in the Streams configuration are used as the default in your Kafka Streams application. Because this config’s default is null, you must either set a default Serde by using this configuration or pass in Serdes explicitly, as described below.
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.StreamsConfig;
Properties settings = new Properties();
// Default serde for keys of data records (here: built-in serde for String type)
settings.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
// Default serde for values of data records (here: built-in serde for Long type)
settings.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass().getName());
Overriding default Serdes
You can also specify Serdes explicitly by passing them to the appropriate API methods, which overrides the default serde settings:
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serdes;
final Serde<String> stringSerde = Serdes.String();
final Serde<Long> longSerde = Serdes.Long();
// The stream userCountByRegion has type `String` for record keys (for region)
// and type `Long` for record values (for user counts).
KStream<String, Long> userCountByRegion = ...;
userCountByRegion.to("RegionCountsTopic", Produced.with(stringSerde, longSerde));
If you want to override serdes selectively, i.e., keep the defaults for some fields, then don’t specify the serde whenever you want to leverage the default settings:
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serdes;
// Use the default serializer for record keys (here: region as String) by not specifying the key serde,
// but override the default serializer for record values (here: userCount as Long).
final Serde<Long> longSerde = Serdes.Long();
KStream<String, Long> userCountByRegion = ...;
userCountByRegion.to("RegionCountsTopic", Produced.valueSerde(Serdes.Long()));
If some of your incoming records are corrupted or ill-formatted, they will cause the deserializer class to report an error. Since 1.0.x we have introduced an DeserializationExceptionHandler
interface which allows you to customize how to handle such records. The customized implementation of the interface can be specified via the StreamsConfig
. For more details, please feel free to read the Configuring a Streams Application section.
Available Serdes
Primitive and basic types
Apache Kafka includes several built-in serde implementations for Java primitives and basic types such as byte[]
in its kafka-clients
Maven artifact:
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-clients</artifactId>
<version>2.8.0</version>
</dependency>
This artifact provides the following serde implementations under the package org.apache.kafka.common.serialization, which you can leverage when e.g., defining default serializers in your Streams configuration.
Data type | Serde |
---|---|
byte[] | Serdes.ByteArray() , Serdes.Bytes() (see tip below) |
ByteBuffer | Serdes.ByteBuffer() |
Double | Serdes.Double() |
Integer | Serdes.Integer() |
Long | Serdes.Long() |
String | Serdes.String() |
UUID | Serdes.UUID() |
Void | Serdes.Void() |
List | Serdes.ListSerde() |
Boolean | Serdes.Boolean() |
Tip
Bytes is a wrapper for Java’s byte[]
(byte array) that supports proper equality and ordering semantics. You may want to consider using Bytes
instead of byte[]
in your applications.
JSON
You can use JsonSerializer
and JsonDeserializer
from Kafka Connect to construct JSON compatible serializers and deserializers using Serdes.serdeFrom(<serializerInstance>, <deserializerInstance>)
. Note, that Kafka Connect’s Json (de)serializer requires Java 17.
Implementing custom Serdes
If you need to implement custom Serdes, your best starting point is to take a look at the source code references of existing Serdes (see previous section). Typically, your workflow will be similar to:
- Write a serializer for your data type
T
by implementing org.apache.kafka.common.serialization.Serializer. - Write a deserializer for
T
by implementing org.apache.kafka.common.serialization.Deserializer. - Write a serde for
T
by implementing org.apache.kafka.common.serialization.Serde, which you either do manually (see existing Serdes in the previous section) or by leveraging helper functions in Serdes such asSerdes.serdeFrom(Serializer<T>, Deserializer<T>)
. Note that you will need to implement your own class (that has no generic types) if you want to use your custom serde in the configuration provided toKafkaStreams
. If your serde class has generic types or you useSerdes.serdeFrom(Serializer<T>, Deserializer<T>)
, you can pass your serde only via methods calls (for examplebuilder.stream("topicName", Consumed.with(...))
).
Kafka Streams DSL for Scala Implicit Serdes
When using the Kafka Streams DSL for Scala you’re not required to configure a default Serdes. In fact, it’s not supported. Serdes are instead provided implicitly by default implementations for common primitive datatypes. See the Implicit Serdes and User-Defined Serdes sections in the DSL API documentation for details
7.7 - Testing a Streams Application
Testing Kafka Streams
Table of Contents
- Importing the test utilities
- Testing Streams applications
- Unit testing Processors
Importing the test utilities
To test a Kafka Streams application, Kafka provides a test-utils artifact that can be added as regular dependency to your test code base. Example pom.xml
snippet when using Maven:
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-streams-test-utils</artifactId>
<version>4.0.0</version>
<scope>test</scope>
</dependency>
Testing a Streams application
The test-utils package provides a TopologyTestDriver
that can be used pipe data through a Topology
that is either assembled manually using Processor API or via the DSL using StreamsBuilder
. The test driver simulates the library runtime that continuously fetches records from input topics and processes them by traversing the topology. You can use the test driver to verify that your specified processor topology computes the correct result with the manually piped in data records. The test driver captures the results records and allows to query its embedded state stores.
// Processor API
Topology topology = new Topology();
topology.addSource("sourceProcessor", "input-topic");
topology.addProcessor("processor", ..., "sourceProcessor");
topology.addSink("sinkProcessor", "output-topic", "processor");
// or
// using DSL
StreamsBuilder builder = new StreamsBuilder();
builder.stream("input-topic").filter(...).to("output-topic");
Topology topology = builder.build();
// create test driver
TopologyTestDriver testDriver = new TopologyTestDriver(topology);
With the test driver you can create TestInputTopic
giving topic name and the corresponding serializers. TestInputTopic
provides various methods to pipe new message values, keys and values, or list of KeyValue objects.
TestInputTopic<String, Long> inputTopic = testDriver.createInputTopic("input-topic", stringSerde.serializer(), longSerde.serializer());
inputTopic.pipeInput("key", 42L);
To verify the output, you can use TestOutputTopic
where you configure the topic and the corresponding deserializers during initialization. It offers helper methods to read only certain parts of the result records or the collection of records. For example, you can validate returned KeyValue
with standard assertions if you only care about the key and value, but not the timestamp of the result record.
TestOutputTopic<String, Long> outputTopic = testDriver.createOutputTopic("output-topic", stringSerde.deserializer(), longSerde.deserializer());
assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("key", 42L)));
TopologyTestDriver
supports punctuations, too. Event-time punctuations are triggered automatically based on the processed records’ timestamps. Wall-clock-time punctuations can also be triggered by advancing the test driver’s wall-clock-time (the driver mocks wall-clock-time internally to give users control over it).
testDriver.advanceWallClockTime(Duration.ofSeconds(20));
Additionally, you can access state stores via the test driver before or after a test. Accessing stores before a test is useful to pre-populate a store with some initial values. After data was processed, expected updates to the store can be verified.
KeyValueStore store = testDriver.getKeyValueStore("store-name");
Note, that you should always close the test driver at the end to make sure all resources are release properly.
testDriver.close();
Example
The following example demonstrates how to use the test driver and helper classes. The example creates a topology that computes the maximum value per key using a key-value-store. While processing, no output is generated, but only the store is updated. Output is only sent downstream based on event-time and wall-clock punctuations.
private TopologyTestDriver testDriver;
private TestInputTopic<String, Long> inputTopic;
private TestOutputTopic<String, Long> outputTopic;
private KeyValueStore<String, Long> store;
private Serde<String> stringSerde = new Serdes.StringSerde();
private Serde<Long> longSerde = new Serdes.LongSerde();
@Before
public void setup() {
Topology topology = new Topology();
topology.addSource("sourceProcessor", "input-topic");
topology.addProcessor("aggregator", new CustomMaxAggregatorSupplier(), "sourceProcessor");
topology.addStateStore(
Stores.keyValueStoreBuilder(
Stores.inMemoryKeyValueStore("aggStore"),
Serdes.String(),
Serdes.Long()).withLoggingDisabled(), // need to disable logging to allow store pre-populating
"aggregator");
topology.addSink("sinkProcessor", "result-topic", "aggregator");
// setup test driver
Properties props = new Properties();
props.setProperty(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
props.setProperty(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass().getName());
testDriver = new TopologyTestDriver(topology, props);
// setup test topics
inputTopic = testDriver.createInputTopic("input-topic", stringSerde.serializer(), longSerde.serializer());
outputTopic = testDriver.createOutputTopic("result-topic", stringSerde.deserializer(), longSerde.deserializer());
// pre-populate store
store = testDriver.getKeyValueStore("aggStore");
store.put("a", 21L);
}
@After
public void tearDown() {
testDriver.close();
}
@Test
public void shouldFlushStoreForFirstInput() {
inputTopic.pipeInput("a", 1L);
assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("a", 21L)));
assertThat(outputTopic.isEmpty(), is(true));
}
@Test
public void shouldNotUpdateStoreForSmallerValue() {
inputTopic.pipeInput("a", 1L);
assertThat(store.get("a"), equalTo(21L));
assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("a", 21L)));
assertThat(outputTopic.isEmpty(), is(true));
}
@Test
public void shouldNotUpdateStoreForLargerValue() {
inputTopic.pipeInput("a", 42L);
assertThat(store.get("a"), equalTo(42L));
assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("a", 42L)));
assertThat(outputTopic.isEmpty(), is(true));
}
@Test
public void shouldUpdateStoreForNewKey() {
inputTopic.pipeInput("b", 21L);
assertThat(store.get("b"), equalTo(21L));
assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("a", 21L)));
assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("b", 21L)));
assertThat(outputTopic.isEmpty(), is(true));
}
@Test
public void shouldPunctuateIfEvenTimeAdvances() {
final Instant recordTime = Instant.now();
inputTopic.pipeInput("a", 1L, recordTime);
assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("a", 21L)));
inputTopic.pipeInput("a", 1L, recordTime);
assertThat(outputTopic.isEmpty(), is(true));
inputTopic.pipeInput("a", 1L, recordTime.plusSeconds(10L));
assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("a", 21L)));
assertThat(outputTopic.isEmpty(), is(true));
}
@Test
public void shouldPunctuateIfWallClockTimeAdvances() {
testDriver.advanceWallClockTime(Duration.ofSeconds(60));
assertThat(outputTopic.readKeyValue(), equalTo(new KeyValue<>("a", 21L)));
assertThat(outputTopic.isEmpty(), is(true));
}
public class CustomMaxAggregatorSupplier implements ProcessorSupplier<String, Long> {
@Override
public Processor<String, Long> get() {
return new CustomMaxAggregator();
}
}
public class CustomMaxAggregator implements Processor<String, Long> {
ProcessorContext context;
private KeyValueStore<String, Long> store;
@SuppressWarnings("unchecked")
@Override
public void init(ProcessorContext context) {
this.context = context;
context.schedule(Duration.ofSeconds(60), PunctuationType.WALL_CLOCK_TIME, time -> flushStore());
context.schedule(Duration.ofSeconds(10), PunctuationType.STREAM_TIME, time -> flushStore());
store = (KeyValueStore<String, Long>) context.getStateStore("aggStore");
}
@Override
public void process(String key, Long value) {
Long oldValue = store.get(key);
if (oldValue == null || value > oldValue) {
store.put(key, value);
}
}
private void flushStore() {
KeyValueIterator<String, Long> it = store.all();
while (it.hasNext()) {
KeyValue<String, Long> next = it.next();
context.forward(next.key, next.value);
}
}
@Override
public void close() {}
}
Unit Testing Processors
If you write a Processor, you will want to test it.
Because the Processor
forwards its results to the context rather than returning them, Unit testing requires a mocked context capable of capturing forwarded data for inspection. For this reason, we provide a MockProcessorContext
in test-utils
.
Construction
To begin with, instantiate your processor and initialize it with the mock context:
final Processor processorUnderTest = ...;
final MockProcessorContext<String, Long> context = new MockProcessorContext<>();
processorUnderTest.init(context);
If you need to pass configuration to your processor or set the default serdes, you can create the mock with config:
final Properties props = new Properties();
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass());
props.put("some.other.config", "some config value");
final MockProcessorContext<String, Long> context = new MockProcessorContext<>(props);
Captured data
The mock will capture any values that your processor forwards. You can make assertions on them:
processorUnderTest.process("key", "value");
final Iterator<CapturedForward<? extends String, ? extends Long>> forwarded = context.forwarded().iterator();
assertEquals(forwarded.next().record(), new Record<>(..., ...));
assertFalse(forwarded.hasNext());
// you can reset forwards to clear the captured data. This may be helpful in constructing longer scenarios.
context.resetForwards();
assertEquals(context.forwarded().size(), 0);
If your processor forwards to specific child processors, you can query the context for captured data by child name:
final List<CapturedForward<? extends String, ? extends Long>> captures = context.forwarded("childProcessorName");
The mock also captures whether your processor has called commit()
on the context:
assertTrue(context.committed());
// commit captures can also be reset.
context.resetCommit();
assertFalse(context.committed());
Setting record metadata
In case your processor logic depends on the record metadata (topic, partition, offset), you can set them on the context:
context.setRecordMetadata("topicName", /*partition*/ 0, /*offset*/ 0L);
Once these are set, the context will continue returning the same values, until you set new ones.
State stores
In case your punctuator is stateful, the mock context allows you to register state stores. You’re encouraged to use a simple in-memory store of the appropriate type (KeyValue, Windowed, or Session), since the mock context does not manage changelogs, state directories, etc.
final KeyValueStore<String, Integer> store =
Stores.keyValueStoreBuilder(
Stores.inMemoryKeyValueStore("myStore"),
Serdes.String(),
Serdes.Integer()
)
.withLoggingDisabled() // Changelog is not supported by MockProcessorContext.
.build();
store.init(context, store);
context.register(store, /*deprecated parameter*/ false, /*parameter unused in mock*/ null);
Verifying punctuators
Processors can schedule punctuators to handle periodic tasks. The mock context does not automatically execute punctuators, but it does capture them to allow you to unit test them as well:
final MockProcessorContext.CapturedPunctuator capturedPunctuator = context.scheduledPunctuators().get(0);
final long interval = capturedPunctuator.getIntervalMs();
final PunctuationType type = capturedPunctuator.getType();
final boolean cancelled = capturedPunctuator.cancelled();
final Punctuator punctuator = capturedPunctuator.getPunctuator();
punctuator.punctuate(/*timestamp*/ 0L);
If you need to write tests involving automatic firing of scheduled punctuators, we recommend creating a simple topology with your processor and using the TopologyTestDriver
.
7.8 - Interactive Queries
Interactive Queries
Interactive queries allow you to leverage the state of your application from outside your application. The Kafka Streams enables your applications to be queryable.
Table of Contents
- Querying local state stores for an app instance
- Querying local key-value stores
- Querying local window stores
- Querying local custom state stores
- Querying remote state stores for the entire app
- Adding an RPC layer to your application
- Exposing the RPC endpoints of your application
- Discovering and accessing application instances and their local state stores
- Demo applications
The full state of your application is typically split across many distributed instances of your application, and across many state stores that are managed locally by these application instances.
There are local and remote components to interactively querying the state of your application.
Local state An application instance can query the locally managed portion of the state and directly query its own local state stores. You can use the corresponding local data in other parts of your application code, as long as it doesn’t require calling the Kafka Streams API. Querying state stores is always read-only to guarantee that the underlying state stores will never be mutated out-of-band (e.g., you cannot add new entries). State stores should only be mutated by the corresponding processor topology and the input data it operates on. For more information, see Querying local state stores for an app instance. Remote state
To query the full state of your application, you must connect the various fragments of the state, including:
- query local state stores
- discover all running instances of your application in the network and their state stores
- communicate with these instances over the network (e.g., an RPC layer)
Connecting these fragments enables communication between instances of the same app and communication from other applications for interactive queries. For more information, see Querying remote state stores for the entire app.
Kafka Streams natively provides all of the required functionality for interactively querying the state of your application, except if you want to expose the full state of your application via interactive queries. To allow application instances to communicate over the network, you must add a Remote Procedure Call (RPC) layer to your application (e.g., REST API).
This table shows the Kafka Streams native communication support for various procedures.
Procedure | Application instance | Entire application |
---|---|---|
Query local state stores of an app instance | Supported | Supported |
Make an app instance discoverable to others | Supported | Supported |
Discover all running app instances and their state stores | Supported | Supported |
Communicate with app instances over the network (RPC) | Supported | Not supported (you must configure) |
Querying local state stores for an app instance
A Kafka Streams application typically runs on multiple instances. The state that is locally available on any given instance is only a subset of the application’s entire state. Querying the local stores on an instance will only return data locally available on that particular instance.
The method KafkaStreams#store(...)
finds an application instance’s local state stores by name and type. Note that interactive queries are not supported for versioned state stores at this time.
Every application instance can directly query any of its local state stores.
The name of a state store is defined when you create the store. You can create the store explicitly by using the Processor API or implicitly by using stateful operations in the DSL.
The type of a state store is defined by QueryableStoreType
. You can access the built-in types via the class QueryableStoreTypes
. Kafka Streams currently has two built-in types:
- A key-value store
QueryableStoreTypes#keyValueStore()
, see Querying local key-value stores. - A window store
QueryableStoreTypes#windowStore()
, see Querying local window stores.
You can also implement your own QueryableStoreType as described in section Querying local custom state stores.
Note
Kafka Streams materializes one state store per stream partition. This means your application will potentially manage many underlying state stores. The API enables you to query all of the underlying stores without having to know which partition the data is in.
Querying local key-value stores
To query a local key-value store, you must first create a topology with a key-value store. This example creates a key-value store named “CountsKeyValueStore”. This store will hold the latest count for any word that is found on the topic “word-count-input”.
Properties props = ...;
StreamsBuilder builder = ...;
KStream<String, String> textLines = ...;
// Define the processing topology (here: WordCount)
KGroupedStream<String, String> groupedByWord = textLines
.flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\W+")))
.groupBy((key, word) -> word, Grouped.with(stringSerde, stringSerde));
// Create a key-value store named "CountsKeyValueStore" for the all-time word counts
groupedByWord.count(Materialized.<String, String, KeyValueStore<Bytes, byte[]>as("CountsKeyValueStore"));
// Start an instance of the topology
KafkaStreams streams = new KafkaStreams(builder, props);
streams.start();
After the application has started, you can get access to “CountsKeyValueStore” and then query it via the ReadOnlyKeyValueStore API:
// Get the key-value store CountsKeyValueStore
ReadOnlyKeyValueStore<String, Long> keyValueStore =
streams.store("CountsKeyValueStore", QueryableStoreTypes.keyValueStore());
// Get value by key
System.out.println("count for hello:" + keyValueStore.get("hello"));
// Get the values for a range of keys available in this application instance
KeyValueIterator<String, Long> range = keyValueStore.range("all", "streams");
while (range.hasNext()) {
KeyValue<String, Long> next = range.next();
System.out.println("count for " + next.key + ": " + next.value);
}
// Get the values for all of the keys available in this application instance
KeyValueIterator<String, Long> range = keyValueStore.all();
while (range.hasNext()) {
KeyValue<String, Long> next = range.next();
System.out.println("count for " + next.key + ": " + next.value);
}
You can also materialize the results of stateless operators by using the overloaded methods that take a queryableStoreName
as shown in the example below:
StreamsBuilder builder = ...;
KTable<String, Integer> regionCounts = ...;
// materialize the result of filtering corresponding to odd numbers
// the "queryableStoreName" can be subsequently queried.
KTable<String, Integer> oddCounts = numberLines.filter((region, count) -> (count % 2 != 0),
Materialized.<String, Integer, KeyValueStore<Bytes, byte[]>as("queryableStoreName"));
// do not materialize the result of filtering corresponding to even numbers
// this means that these results will not be materialized and cannot be queried.
KTable<String, Integer> oddCounts = numberLines.filter((region, count) -> (count % 2 == 0));
Querying local window stores
A window store will potentially have many results for any given key because the key can be present in multiple windows. However, there is only one result per window for a given key.
To query a local window store, you must first create a topology with a window store. This example creates a window store named “CountsWindowStore” that contains the counts for words in 1-minute windows.
StreamsBuilder builder = ...;
KStream<String, String> textLines = ...;
// Define the processing topology (here: WordCount)
KGroupedStream<String, String> groupedByWord = textLines
.flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\W+")))
.groupBy((key, word) -> word, Grouped.with(stringSerde, stringSerde));
// Create a window state store named "CountsWindowStore" that contains the word counts for every minute
groupedByWord.windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofSeconds(60)))
.count(Materialized.<String, Long, WindowStore<Bytes, byte[]>as("CountsWindowStore"));
After the application has started, you can get access to “CountsWindowStore” and then query it via the ReadOnlyWindowStore API:
// Get the window store named "CountsWindowStore"
ReadOnlyWindowStore<String, Long> windowStore =
streams.store("CountsWindowStore", QueryableStoreTypes.windowStore());
// Fetch values for the key "world" for all of the windows available in this application instance.
// To get *all* available windows we fetch windows from the beginning of time until now.
Instant timeFrom = Instant.ofEpochMilli(0); // beginning of time = oldest available
Instant timeTo = Instant.now(); // now (in processing-time)
WindowStoreIterator<Long> iterator = windowStore.fetch("world", timeFrom, timeTo);
while (iterator.hasNext()) {
KeyValue<Long, Long> next = iterator.next();
long windowTimestamp = next.key;
System.out.println("Count of 'world' @ time " + windowTimestamp + " is " + next.value);
}
Querying local custom state stores
Note
Only the Processor API supports custom state stores.
Before querying the custom state stores you must implement these interfaces:
- Your custom state store must implement
StateStore
. - You must have an interface to represent the operations available on the store.
- You must provide an implementation of
StoreBuilder
for creating instances of your store. - It is recommended that you provide an interface that restricts access to read-only operations. This prevents users of this API from mutating the state of your running Kafka Streams application out-of-band.
The class/interface hierarchy for your custom store might look something like:
public class MyCustomStore<K,V> implements StateStore, MyWriteableCustomStore<K,V> {
// implementation of the actual store
}
// Read-write interface for MyCustomStore
public interface MyWriteableCustomStore<K,V> extends MyReadableCustomStore<K,V> {
void write(K Key, V value);
}
// Read-only interface for MyCustomStore
public interface MyReadableCustomStore<K,V> {
V read(K key);
}
public class MyCustomStoreBuilder implements StoreBuilder {
// implementation of the supplier for MyCustomStore
}
To make this store queryable you must:
- Provide an implementation of QueryableStoreType.
- Provide a wrapper class that has access to all of the underlying instances of the store and is used for querying.
Here is how to implement QueryableStoreType
:
public class MyCustomStoreType<K,V> implements QueryableStoreType<MyReadableCustomStore<K,V>> {
// Only accept StateStores that are of type MyCustomStore
public boolean accepts(final StateStore stateStore) {
return stateStore instanceOf MyCustomStore;
}
public MyReadableCustomStore<K,V> create(final StateStoreProvider storeProvider, final String storeName) {
return new MyCustomStoreTypeWrapper(storeProvider, storeName, this);
}
}
A wrapper class is required because each instance of a Kafka Streams application may run multiple stream tasks and manage multiple local instances of a particular state store. The wrapper class hides this complexity and lets you query a “logical” state store by name without having to know about all of the underlying local instances of that state store.
When implementing your wrapper class you must use the StateStoreProvider interface to get access to the underlying instances of your store. StateStoreProvider#stores(String storeName, QueryableStoreType<T> queryableStoreType)
returns a List
of state stores with the given storeName and of the type as defined by queryableStoreType
.
Here is an example implementation of the wrapper:
// We strongly recommended implementing a read-only interface
// to restrict usage of the store to safe read operations!
public class MyCustomStoreTypeWrapper<K,V> implements MyReadableCustomStore<K,V> {
private final QueryableStoreType<MyReadableCustomStore<K, V>> customStoreType;
private final String storeName;
private final StateStoreProvider provider;
public CustomStoreTypeWrapper(final StateStoreProvider provider,
final String storeName,
final QueryableStoreType<MyReadableCustomStore<K, V>> customStoreType) {
// ... assign fields ...
}
// Implement a safe read method
@Override
public V read(final K key) {
// Get all the stores with storeName and of customStoreType
final List<MyReadableCustomStore<K, V>> stores = provider.getStores(storeName, customStoreType);
// Try and find the value for the given key
final Optional<V> value = stores.stream().filter(store -> store.read(key) != null).findFirst();
// Return the value if it exists
return value.orElse(null);
}
}
You can now find and query your custom store:
Topology topology = ...;
ProcessorSupplier processorSuppler = ...;
// Create CustomStoreSupplier for store name the-custom-store
MyCustomStoreBuilder customStoreBuilder = new MyCustomStoreBuilder("the-custom-store") //...;
// Add the source topic
topology.addSource("input", "inputTopic");
// Add a custom processor that reads from the source topic
topology.addProcessor("the-processor", processorSupplier, "input");
// Connect your custom state store to the custom processor above
topology.addStateStore(customStoreBuilder, "the-processor");
KafkaStreams streams = new KafkaStreams(topology, config);
streams.start();
// Get access to the custom store
MyReadableCustomStore<String,String> store = streams.store("the-custom-store", new MyCustomStoreType<String,String>());
// Query the store
String value = store.read("key");
Querying remote state stores for the entire app
To query remote states for the entire app, you must expose the application’s full state to other applications, including applications that are running on different machines.
For example, you have a Kafka Streams application that processes user events in a multi-player video game, and you want to retrieve the latest status of each user directly and display it in a mobile app. Here are the required steps to make the full state of your application queryable:
- Add an RPC layer to your application so that the instances of your application can be interacted with via the network (e.g., a REST API, Thrift, a custom protocol, and so on). The instances must respond to interactive queries. You can follow the reference examples provided to get started.
- Expose the RPC endpoints of your application’s instances via the
application.server
configuration setting of Kafka Streams. Because RPC endpoints must be unique within a network, each instance has its own value for this configuration setting. This makes an application instance discoverable by other instances. - In the RPC layer, discover remote application instances and their state stores and query locally available state stores to make the full state of your application queryable. The remote application instances can forward queries to other app instances if a particular instance lacks the local data to respond to a query. The locally available state stores can directly respond to queries.
Discover any running instances of the same application as well as the respective RPC endpoints they expose for interactive queries
Adding an RPC layer to your application
There are many ways to add an RPC layer. The only requirements are that the RPC layer is embedded within the Kafka Streams application and that it exposes an endpoint that other application instances and applications can connect to.
Exposing the RPC endpoints of your application
To enable remote state store discovery in a distributed Kafka Streams application, you must set the configuration property in the config properties. The application.server
property defines a unique host:port
pair that points to the RPC endpoint of the respective instance of a Kafka Streams application. The value of this configuration property will vary across the instances of your application. When this property is set, Kafka Streams will keep track of the RPC endpoint information for every instance of an application, its state stores, and assigned stream partitions through instances of StreamsMetadata.
Tip
Consider leveraging the exposed RPC endpoints of your application for further functionality, such as piggybacking additional inter-application communication that goes beyond interactive queries.
This example shows how to configure and run a Kafka Streams application that supports the discovery of its state stores.
Properties props = new Properties();
// Set the unique RPC endpoint of this application instance through which it
// can be interactively queried. In a real application, the value would most
// probably not be hardcoded but derived dynamically.
String rpcEndpoint = "host1:4460";
props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, rpcEndpoint);
// ... further settings may follow here ...
StreamsBuilder builder = new StreamsBuilder();
KStream<String, String> textLines = builder.stream(stringSerde, stringSerde, "word-count-input");
final KGroupedStream<String, String> groupedByWord = textLines
.flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\W+")))
.groupBy((key, word) -> word, Grouped.with(stringSerde, stringSerde));
// This call to `count()` creates a state store named "word-count".
// The state store is discoverable and can be queried interactively.
groupedByWord.count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>as("word-count"));
// Start an instance of the topology
KafkaStreams streams = new KafkaStreams(builder, props);
streams.start();
// Then, create and start the actual RPC service for remote access to this
// application instance's local state stores.
//
// This service should be started on the same host and port as defined above by
// the property `StreamsConfig.APPLICATION_SERVER_CONFIG`. The example below is
// fictitious, but we provide end-to-end demo applications (such as KafkaMusicExample)
// that showcase how to implement such a service to get you started.
MyRPCService rpcService = ...;
rpcService.listenAt(rpcEndpoint);
Discovering and accessing application instances and their local state stores
The following methods return StreamsMetadata objects, which provide meta-information about application instances such as their RPC endpoint and locally available state stores.
KafkaStreams#allMetadata()
: find all instances of this applicationKafkaStreams#allMetadataForStore(String storeName)
: find those applications instances that manage local instances of the state store “storeName”KafkaStreams#metadataForKey(String storeName, K key, Serializer<K> keySerializer)
: using the default stream partitioning strategy, find the one application instance that holds the data for the given key in the given state storeKafkaStreams#metadataForKey(String storeName, K key, StreamPartitioner<K, ?> partitioner)
: usingpartitioner
, find the one application instance that holds the data for the given key in the given state store
Attention
If application.server
is not configured for an application instance, then the above methods will not find any StreamsMetadata for it.
For example, we can now find the StreamsMetadata
for the state store named “word-count” that we defined in the code example shown in the previous section:
KafkaStreams streams = ...;
// Find all the locations of local instances of the state store named "word-count"
Collection<StreamsMetadata> wordCountHosts = streams.allMetadataForStore("word-count");
// For illustrative purposes, we assume using an HTTP client to talk to remote app instances.
HttpClient http = ...;
// Get the word count for word (aka key) 'alice': Approach 1
//
// We first find the one app instance that manages the count for 'alice' in its local state stores.
StreamsMetadata metadata = streams.metadataForKey("word-count", "alice", Serdes.String().serializer());
// Then, we query only that single app instance for the latest count of 'alice'.
// Note: The RPC URL shown below is fictitious and only serves to illustrate the idea. Ultimately,
// the URL (or, in general, the method of communication) will depend on the RPC layer you opted to
// implement. Again, we provide end-to-end demo applications (such as KafkaMusicExample) that showcase
// how to implement such an RPC layer.
Long result = http.getLong("http://" + metadata.host() + ":" + metadata.port() + "/word-count/alice");
// Get the word count for word (aka key) 'alice': Approach 2
//
// Alternatively, we could also choose (say) a brute-force approach where we query every app instance
// until we find the one that happens to know about 'alice'.
Optional<Long> result = streams.allMetadataForStore("word-count")
.stream()
.map(streamsMetadata -> {
// Construct the (fictituous) full endpoint URL to query the current remote application instance
String url = "http://" + streamsMetadata.host() + ":" + streamsMetadata.port() + "/word-count/alice";
// Read and return the count for 'alice', if any.
return http.getLong(url);
})
.filter(s -> s != null)
.findFirst();
At this point the full state of the application is interactively queryable:
- You can discover the running instances of the application and the state stores they manage locally.
- Through the RPC layer that was added to the application, you can communicate with these application instances over the network and query them for locally available state.
- The application instances are able to serve such queries because they can directly query their own local state stores and respond via the RPC layer.
- Collectively, this allows us to query the full state of the entire application.
To see an end-to-end application with interactive queries, review the demo applications.
7.9 - Memory Management
Memory Management
You can specify the total memory (RAM) size used for internal caching and compacting of records. This caching happens before the records are written to state stores or forwarded downstream to other nodes.
The record caches are implemented slightly different in the DSL and Processor API.
Table of Contents
- Record caches in the DSL
- Record caches in the Processor API
- RocksDB
- Other memory usage
Record caches in the DSL
You can specify the total memory (RAM) size of the record cache for an instance of the processing topology. It is leveraged by the following KTable
instances:
- Source
KTable
:KTable
instances that are created viaStreamsBuilder#table()
orStreamsBuilder#globalTable()
. - Aggregation
KTable
: instances ofKTable
that are created as a result of aggregations.
For such KTable
instances, the record cache is used for:
- Internal caching and compacting of output records before they are written by the underlying stateful processor node to its internal state stores.
- Internal caching and compacting of output records before they are forwarded from the underlying stateful processor node to any of its downstream processor nodes.
Use the following example to understand the behaviors with and without record caching. In this example, the input is a KStream<String, Integer>
with the records <K,V>: <A, 1>, <D, 5>, <A, 20>, <A, 300>
. The focus in this example is on the records with key == A
.
- An aggregation computes the sum of record values, grouped by key, for the input and returns a
KTable<String, Integer>
.
* **Without caching** : a sequence of output records is emitted for key `A` that represent changes in the resulting aggregation table. The parentheses (`()`) denote changes, the left number is the new aggregate value and the right number is the old aggregate value: `<A, (1, null)>, <A, (21, 1)>, <A, (321, 21)>`. * **With caching** : a single output record is emitted for key `A` that would likely be compacted in the cache, leading to a single output record of `<A, (321, null)>`. This record is written to the aggregation's internal state store and forwarded to any downstream operations.
The cache size is specified through the cache.max.bytes.buffering
parameter, which is a global setting per processing topology:
// Enable record cache of size 10 MB.
Properties props = new Properties();
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10 * 1024 * 1024L);
This parameter controls the number of bytes allocated for caching. Specifically, for a processor topology instance with T
threads and C
bytes allocated for caching, each thread will have an even C/T
bytes to construct its own cache and use as it sees fit among its tasks. This means that there are as many caches as there are threads, but no sharing of caches across threads happens.
The basic API for the cache is made of put()
and get()
calls. Records are evicted using a simple LRU scheme after the cache size is reached. The first time a keyed record R1 = <K1, V1>
finishes processing at a node, it is marked as dirty in the cache. Any other keyed record R2 = <K1, V2>
with the same key K1
that is processed on that node during that time will overwrite <K1, V1>
, this is referred to as “being compacted”. This has the same effect as Kafka’s log compaction, but happens earlier, while the records are still in memory, and within your client-side application, rather than on the server-side (i.e. the Kafka broker). After flushing, R2
is forwarded to the next processing node and then written to the local state store.
The semantics of caching is that data is flushed to the state store and forwarded to the next downstream processor node whenever the earliest of commit.interval.ms
or cache.max.bytes.buffering
(cache pressure) hits. Both commit.interval.ms
and cache.max.bytes.buffering
are global parameters. As such, it is not possible to specify different parameters for individual nodes.
Here are example settings for both parameters based on desired scenarios.
To turn off caching the cache size can be set to zero:
// Disable record cache
Properties props = new Properties(); props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
To enable caching but still have an upper bound on how long records will be cached, you can set the commit interval. In this example, it is set to 1000 milliseconds:
Properties props = new Properties();
// Enable record cache of size 10 MB. props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10 * 1024 * 1024L); // Set commit interval to 1 second. props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
The effect of these two configurations is described in the figure below. The records are shown using 4 keys: blue, red, yellow, and green. Assume the cache has space for only 3 keys.
When the cache is disabled (a), all of the input records will be output.
When the cache is enabled (b):
* Most records are output at the end of commit intervals (e.g., at `t1` a single blue record is output, which is the final over-write of the blue key up to that time). * Some records are output because of cache pressure (i.e. before the end of a commit interval). For example, see the red record before `t2`. With smaller cache sizes we expect cache pressure to be the primary factor that dictates when records are output. With large cache sizes, the commit interval will be the primary factor. * The total number of records output has been reduced from 15 to 8.
Record caches in the Processor API
You can specify the total memory (RAM) size of the record cache for an instance of the processing topology. It is used for internal caching and compacting of output records before they are written from a stateful processor node to its state stores.
The record cache in the Processor API does not cache or compact any output records that are being forwarded downstream. This means that all downstream processor nodes can see all records, whereas the state stores see a reduced number of records. This does not impact correctness of the system, but is a performance optimization for the state stores. For example, with the Processor API you can store a record in a state store while forwarding a different value downstream.
Following from the example first shown in section State Stores, to disable caching, you can add the withCachingDisabled
call (note that caches are enabled by default, however there is an explicit withCachingEnabled
call).
StoreBuilder countStoreBuilder =
Stores.keyValueStoreBuilder(
Stores.persistentKeyValueStore("Counts"),
Serdes.String(),
Serdes.Long())
.withCachingEnabled();
Record caches are not supported for versioned state stores.
To avoid reading stale data, you can flush()
the store before creating the iterator. Note, that flushing too often can lead to performance degration if RocksDB is used, so we advice to avoid flushing manually in general.
RocksDB
Each instance of RocksDB allocates off-heap memory for a block cache, index and filter blocks, and memtable (write buffer). Critical configs (for RocksDB version 4.1.0) include block_cache_size
, write_buffer_size
and max_write_buffer_number
. These can be specified through the rocksdb.config.setter
configuration.
Also, we recommend changing RocksDB’s default memory allocator, because the default allocator may lead to increased memory consumption. To change the memory allocator to jemalloc
, you need to set the environment variable LD_PRELOAD
before you start your Kafka Streams application:
# example: install jemalloc (on Debian)
$ apt install -y libjemalloc-dev
# set LD_PRELOAD before you start your Kafka Streams application
$ export LD_PRELOAD="/usr/lib/x86_64-linux-gnu/libjemalloc.so"
As of 2.3.0 the memory usage across all instances can be bounded, limiting the total off-heap memory of your Kafka Streams application. To do so you must configure RocksDB to cache the index and filter blocks in the block cache, limit the memtable memory through a shared WriteBufferManager and count its memory against the block cache, and then pass the same Cache object to each instance. See RocksDB Memory Usage for details. An example RocksDBConfigSetter implementing this is shown below:
public static class BoundedMemoryRocksDBConfig implements RocksDBConfigSetter {
private static org.rocksdb.Cache cache = new org.rocksdb.LRUCache(TOTAL_OFF_HEAP_MEMORY, -1, false, INDEX_FILTER_BLOCK_RATIO);1
private static org.rocksdb.WriteBufferManager writeBufferManager = new org.rocksdb.WriteBufferManager(TOTAL_MEMTABLE_MEMORY, cache);
@Override
public void setConfig(final String storeName, final Options options, final Map<String, Object> configs) {
BlockBasedTableConfig tableConfig = (BlockBasedTableConfig) options.tableFormatConfig();
// These three options in combination will limit the memory used by RocksDB to the size passed to the block cache (TOTAL_OFF_HEAP_MEMORY)
tableConfig.setBlockCache(cache);
tableConfig.setCacheIndexAndFilterBlocks(true);
options.setWriteBufferManager(writeBufferManager);
// These options are recommended to be set when bounding the total memory
tableConfig.setCacheIndexAndFilterBlocksWithHighPriority(true);2
tableConfig.setPinTopLevelIndexAndFilter(true);
tableConfig.setBlockSize(BLOCK_SIZE);3
options.setMaxWriteBufferNumber(N_MEMTABLES);
options.setWriteBufferSize(MEMTABLE_SIZE);
options.setTableFormatConfig(tableConfig);
}
@Override
public void close(final String storeName, final Options options) {
// Cache and WriteBufferManager should not be closed here, as the same objects are shared by every store instance.
}
}
1. INDEX_FILTER_BLOCK_RATIO can be used to set a fraction of the block cache to set aside for “high priority” (aka index and filter) blocks, preventing them from being evicted by data blocks. The boolean parameter in the cache constructor lets you control whether the cache should enforce a strict memory limit by failing the read or iteration in the rare cases where it might go larger than its capacity. See the full signature of the LRUCache constructor here.
2. This must be set in order for INDEX_FILTER_BLOCK_RATIO to take effect (see footnote 1) as described in the RocksDB docs
3. You may want to modify the default block size per these instructions from the RocksDB docs. A larger block size means index blocks will be smaller, but the cached data blocks may contain more cold data that would otherwise be evicted.
Note: While we recommend setting at least the above configs, the specific options that yield the best performance are workload dependent and you should consider experimenting with these to determine the best choices for your specific use case. Keep in mind that the optimal configs for one app may not apply to one with a different topology or input topic. In addition to the recommended configs above, you may want to consider using partitioned index filters as described by the RocksDB docs.
Other memory usage
There are other modules inside Apache Kafka that allocate memory during runtime. They include the following:
- Producer buffering, managed by the producer config
buffer.memory
. - Consumer buffering, currently not strictly managed, but can be indirectly controlled by fetch size, i.e.,
fetch.max.bytes
andfetch.max.wait.ms
. - Both producer and consumer also have separate TCP send / receive buffers that are not counted as the buffering memory. These are controlled by the
send.buffer.bytes
/receive.buffer.bytes
configs. - Deserialized objects buffering: after
consumer.poll()
returns records, they will be deserialized to extract timestamp and buffered in the streams space. Currently this is only indirectly controlled bybuffered.records.per.partition
.
Tip
Iterators should be closed explicitly to release resources: Store iterators (e.g., KeyValueIterator
and WindowStoreIterator
) must be closed explicitly upon completeness to release resources such as open file handlers and in-memory read buffers, or use try-with-resources statement (available since JDK7) for this Closeable class.
Otherwise, stream application’s memory usage keeps increasing when running until it hits an OOM.
7.10 - Running Streams Applications
Running Streams Applications
You can run Java applications that use the Kafka Streams library without any additional configuration or requirements. Kafka Streams also provides the ability to receive notification of the various states of the application. The ability to monitor the runtime status is discussed in the monitoring guide.
Table of Contents
- Starting a Kafka Streams application
- Elastic scaling of your application
- Adding capacity to your application
- Removing capacity from your application
- State restoration during workload rebalance
- Determining how many application instances to run
Starting a Kafka Streams application
You can package your Java application as a fat JAR file and then start the application like this:
# Start the application in class `com.example.MyStreamsApp`
# from the fat JAR named `path-to-app-fatjar.jar`.
$ java -cp path-to-app-fatjar.jar com.example.MyStreamsApp
When you start your application you are launching a Kafka Streams instance of your application. You can run multiple instances of your application. A common scenario is that there are multiple instances of your application running in parallel. For more information, see Parallelism Model.
When the application instance starts running, the defined processor topology will be initialized as one or more stream tasks. If the processor topology defines any state stores, these are also constructed during the initialization period. For more information, see the State restoration during workload rebalance section).
Elastic scaling of your application
Kafka Streams makes your stream processing applications elastic and scalable. You can add and remove processing capacity dynamically during application runtime without any downtime or data loss. This makes your applications resilient in the face of failures and for allows you to perform maintenance as needed (e.g. rolling upgrades).
For more information about this elasticity, see the Parallelism Model section. Kafka Streams leverages the Kafka group management functionality, which is built right into the Kafka wire protocol. It is the foundation that enables the elasticity of Kafka Streams applications: members of a group coordinate and collaborate jointly on the consumption and processing of data in Kafka. Additionally, Kafka Streams provides stateful processing and allows for fault-tolerant state in environments where application instances may come and go at any time.
Adding capacity to your application
If you need more processing capacity for your stream processing application, you can simply start another instance of your stream processing application, e.g. on another machine, in order to scale out. The instances of your application will become aware of each other and automatically begin to share the processing work. More specifically, what will be handed over from the existing instances to the new instances is (some of) the stream tasks that have been run by the existing instances. Moving stream tasks from one instance to another results in moving the processing work plus any internal state of these stream tasks (the state of a stream task will be re-created in the target instance by restoring the state from its corresponding changelog topic).
The various instances of your application each run in their own JVM process, which means that each instance can leverage all the processing capacity that is available to their respective JVM process (minus the capacity that any non-Kafka-Streams part of your application may be using). This explains why running additional instances will grant your application additional processing capacity. The exact capacity you will be adding by running a new instance depends of course on the environment in which the new instance runs: available CPU cores, available main memory and Java heap space, local storage, network bandwidth, and so on. Similarly, if you stop any of the running instances of your application, then you are removing and freeing up the respective processing capacity.
Before adding capacity: only a single instance of your Kafka Streams application is running. At this point the corresponding Kafka consumer group of your application contains only a single member (this instance). All data is being read and processed by this single instance.
After adding capacity: now two additional instances of your Kafka Streams application are running, and they have automatically joined the application’s Kafka consumer group for a total of three current members. These three instances are automatically splitting the processing work between each other. The splitting is based on the Kafka topic partitions from which data is being read.
Removing capacity from your application
To remove processing capacity, you can stop running stream processing application instances (e.g., shut down two of the four instances), it will automatically leave the application’s consumer group, and the remaining instances of your application will automatically take over the processing work. The remaining instances take over the stream tasks that were run by the stopped instances. Moving stream tasks from one instance to another results in moving the processing work plus any internal state of these stream tasks. The state of a stream task is recreated in the target instance from its changelog topic.
State restoration during workload rebalance
When a task is migrated, the task processing state is fully restored before the application instance resumes processing. This guarantees the correct processing results. In Kafka Streams, state restoration is usually done by replaying the corresponding changelog topic to reconstruct the state store. To minimize changelog-based restoration latency by using replicated local state stores, you can specify num.standby.replicas
. When a stream task is initialized or re-initialized on the application instance, its state store is restored like this:
- If no local state store exists, the changelog is replayed from the earliest to the current offset. This reconstructs the local state store to the most recent snapshot.
- If a local state store exists, the changelog is replayed from the previously checkpointed offset. The changes are applied and the state is restored to the most recent snapshot. This method takes less time because it is applying a smaller portion of the changelog.
For more information, see Standby Replicas.
As of version 2.6, Streams will now do most of a task’s restoration in the background through warmup replicas. These will be assigned to instances that need to restore a lot of state for a task. A stateful active task will only be assigned to an instance once its state is within the configured acceptable.recovery.lag
, if one exists. This means that most of the time, a task migration will not result in downtime for that task. It will remain active on the instance that’s already caught up, while the instance that it’s being migrated to works on restoring the state. Streams will regularly probe for warmup tasks that have finished restoring and transition them to active tasks when ready.
Note, the one exception to this task availability is if none of the instances have a caught up version of that task. In that case, we have no choice but to assign the active task to an instance that is not caught up and will have to block further processing on restoration of the task’s state from the changelog. If high availability is important for your application, you are highly recommended to enable standbys.
Determining how many application instances to run
The parallelism of a Kafka Streams application is primarily determined by how many partitions the input topics have. For example, if your application reads from a single topic that has ten partitions, then you can run up to ten instances of your applications. You can run further instances, but these will be idle.
The number of topic partitions is the upper limit for the parallelism of your Kafka Streams application and for the number of running instances of your application.
To achieve balanced workload processing across application instances and to prevent processing hotpots, you should distribute data and processing workloads:
- Data should be equally distributed across topic partitions. For example, if two topic partitions each have 1 million messages, this is better than a single partition with 2 million messages and none in the other.
- Processing workload should be equally distributed across topic partitions. For example, if the time to process messages varies widely, then it is better to spread the processing-intensive messages across partitions rather than storing these messages within the same partition.
7.11 - Managing Streams Application Topics
Managing Streams Application Topics
A Kafka Streams application continuously reads from Kafka topics, processes the read data, and then writes the processing results back into Kafka topics. The application may also auto-create other Kafka topics in the Kafka brokers, for example state store changelogs topics. This section describes the differences these topic types and how to manage the topics and your applications.
Kafka Streams distinguishes between user topics and internal topics.
User topics
User topics exist externally to an application and are read from or written to by the application, including:
Input topics
Topics that are specified via source processors in the application’s topology; e.g. via StreamsBuilder#stream()
, StreamsBuilder#table()
and Topology#addSource()
.
Output topics
Topics that are specified via sink processors in the application’s topology; e.g. via KStream#to()
, KTable.to()
and Topology#addSink()
.
User topics must be created and manually managed ahead of time (e.g., via the topic tools). If user topics are shared among multiple applications for reading and writing, the application users must coordinate topic management. If user topics are centrally managed, then application users then would not need to manage topics themselves but simply obtain access to them.
Note
You should not use the auto-create topic feature on the brokers to create user topics, because:
- Auto-creation of topics may be disabled in your Kafka cluster.
- Auto-creation automatically applies the default topic settings such as the replicaton factor. These default settings might not be what you want for certain output topics (e.g.,
auto.create.topics.enable=true
in the Kafka broker configuration).
Internal topics
Internal topics are used internally by the Kafka Streams application while executing, for example the changelog topics for state stores. These topics are created by the application and are only used by that stream application.
If security is enabled on the Kafka brokers, you must grant the underlying clients admin permissions so that they can create internal topics set. For more information, see Streams Security.
Note
The internal topics follow the naming convention <application.id>-<operatorName>-<suffix>
, but this convention is not guaranteed for future releases.
The following settings apply to the default configuration for internal topics:
- For all internal topics,
message.timestamp.type
is set toCreateTime
. - For internal repartition topics, the compaction policy is
delete
and the retention time is-1
(infinite). - For internal changelog topics for key-value stores, the compaction policy is
compact
. - For internal changelog topics for windowed key-value stores, the compaction policy is
delete,compact
. The retention time is set to 24 hours plus your setting for the windowed store. - For internal changelog topics for versioned state stores, the cleanup policy is
compact
, andmin.compaction.lag.ms
is set to 24 hours plus the store’s historyRetentionMs` value.
7.12 - Streams Security
Streams Security
Table of Contents
- Required ACL setting for secure Kafka clusters
- Security example
Kafka Streams natively integrates with the Kafka’s security features and supports all of the client-side security features in Kafka. Streams leverages the Java Producer and Consumer API.
To secure your Stream processing applications, configure the security settings in the corresponding Kafka producer and consumer clients, and then specify the corresponding configuration settings in your Kafka Streams application.
Kafka supports cluster encryption and authentication, including a mix of authenticated and unauthenticated, and encrypted and non-encrypted clients. Using security is optional.
Here a few relevant client-side security features:
Encrypt data-in-transit between your applications and Kafka brokers You can enable the encryption of the client-server communication between your applications and the Kafka brokers. For example, you can configure your applications to always use encryption when reading and writing data to and from Kafka. This is critical when reading and writing data across security domains such as internal network, public internet, and partner networks. Client authentication You can enable client authentication for connections from your application to Kafka brokers. For example, you can define that only specific applications are allowed to connect to your Kafka cluster. Client authorization You can enable client authorization of read and write operations by your applications. For example, you can define that only specific applications are allowed to read from a Kafka topic. You can also restrict write access to Kafka topics to prevent data pollution or fraudulent activities.
For more information about the security features in Apache Kafka, see Kafka Security.
Required ACL setting for secure Kafka clusters
Kafka clusters can use ACLs to control access to resources (like the ability to create topics), and for such clusters each client, including Kafka Streams, is required to authenticate as a particular user in order to be authorized with appropriate access. In particular, when Streams applications are run against a secured Kafka cluster, the principal running the application must have the ACL set so that the application has the permissions to create, read and write internal topics.
To avoid providing this permission to your application, you can create the required internal topics manually. If the internal topics exist, Kafka Streams will not try to recreate them. Note, that the internal repartition and changelog topics must be created with the correct number of partitions–otherwise, Kafka Streams will fail on startup. The topics must be created with the same number of partitions as your input topic, or if there are multiple topics, the maximum number of partitions across all input topics. Additionally, changelog topics must be created with log compaction enabled–otherwise, your application might lose data. For changelog topics for windowed KTables, apply “delete,compact” and set the retention time based on the corresponding store retention time. To avoid premature deletion, add a delta to the store retention time. By default, Kafka Streams adds 24 hours to the store retention time. You can find out more about the names of the required internal topics via Topology#describe()
. All internal topics follow the naming pattern <application.id>-<operatorName>-<suffix>
where the suffix
is either repartition
or changelog
. Note, that there is no guarantee about this naming pattern in future releases–it’s not part of the public API.
Since all internal topics as well as the embedded consumer group name are prefixed with the application id, it is recommended to use ACLs on prefixed resource pattern to configure control lists to allow client to manage all topics and consumer groups started with this prefix as --resource-pattern-type prefixed --topic your.application.id --operation All
(see KIP-277 and KIP-290 for details).
Security example
The purpose is to configure a Kafka Streams application to enable client authentication and encrypt data-in-transit when communicating with its Kafka cluster.
This example assumes that the Kafka brokers in the cluster already have their security setup and that the necessary SSL certificates are available to the application in the local filesystem locations. For example, if you are using Docker then you must also include these SSL certificates in the correct locations within the Docker image.
The snippet below shows the settings to enable client authentication and SSL encryption for data-in-transit between your Kafka Streams application and the Kafka cluster it is reading and writing from:
# Essential security settings to enable client authentication and SSL encryption
bootstrap.servers=kafka.example.com:9093
security.protocol=SSL
ssl.truststore.location=/etc/security/tls/kafka.client.truststore.jks
ssl.truststore.password=test1234
ssl.keystore.location=/etc/security/tls/kafka.client.keystore.jks
ssl.keystore.password=test1234
ssl.key.password=test1234
Configure these settings in the application for your Properties
instance. These settings will encrypt any data-in-transit that is being read from or written to Kafka, and your application will authenticate itself against the Kafka brokers that it is communicating with. Note that this example does not cover client authorization.
// Code of your Java application that uses the Kafka Streams library
Properties settings = new Properties();
settings.put(StreamsConfig.APPLICATION_ID_CONFIG, "secure-kafka-streams-app");
// Where to find secure Kafka brokers. Here, it's on port 9093.
settings.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka.example.com:9093");
//
// ...further non-security related settings may follow here...
//
// Security settings.
// 1. These settings must match the security settings of the secure Kafka cluster.
// 2. The SSL trust store and key store files must be locally accessible to the application.
settings.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL");
settings.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, "/etc/security/tls/kafka.client.truststore.jks");
settings.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "test1234");
settings.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, "/etc/security/tls/kafka.client.keystore.jks");
settings.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, "test1234");
settings.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, "test1234");
If you incorrectly configure a security setting in your application, it will fail at runtime, typically right after you start it. For example, if you enter an incorrect password for the ssl.keystore.password
setting, an error message similar to this would be logged and then the application would terminate:
# Misconfigured ssl.keystore.password
Exception in thread "main" org.apache.kafka.common.KafkaException: Failed to construct kafka producer
[...snip...]
Caused by: org.apache.kafka.common.KafkaException: org.apache.kafka.common.KafkaException:
java.io.IOException: Keystore was tampered with, or password was incorrect
[...snip...]
Caused by: java.security.UnrecoverableKeyException: Password verification failed
Monitor your Kafka Streams application log files for such error messages to spot any misconfigured applications quickly.
7.13 - Application Reset Tool
Application Reset Tool
You can reset an application and force it to reprocess its data from scratch by using the application reset tool. This can be useful for development and testing, or when fixing bugs.
The application reset tool handles the Kafka Streams user topics (input, and output) and internal topics differently when resetting the application.
Here’s what the application reset tool does for each topic type:
- Input topics: Reset offsets to specified position (by default to the beginning of the topic).
- Internal topics: Delete the internal topic (this automatically deletes any committed offsets).
The application reset tool does not:
- Reset output topics of an application. If any output topics are consumed by downstream applications, it is your responsibility to adjust those downstream applications as appropriate when you reset the upstream application.
- Reset the local environment of your application instances. It is your responsibility to delete the local state on any machine on which an application instance was run. See the instructions in section Step 2: Reset the local environments of your application instances on how to do this.
Prerequisites
All instances of your application must be stopped. Otherwise, the application may enter an invalid state, crash, or produce incorrect results. You can verify whether the consumer group with ID
application.id
is still active by usingbin/kafka-consumer-groups
. When long session timeout has been configured, active members could take longer to get expired on the broker thus blocking the reset job to complete. Use the--force
option could remove those left-over members immediately. Make sure to shut down all stream applications when this option is specified to avoid unexpected rebalances.Use this tool with care and double-check its parameters: If you provide wrong parameter values (e.g., typos in
application.id
) or specify parameters inconsistently (e.g., specify the wrong input topics for the application), this tool might invalidate the application’s state or even impact other applications, consumer groups, or your Kafka topics.
Step 1: Run the application reset tool
Invoke the application reset tool from the command line
Warning! This tool makes irreversible changes to your application. It is strongly recommended that you run this once with --dry-run
to preview your changes before making them.
$ bin/kafka-streams-application-reset
The tool accepts the following parameters:
Option (* = required) Description
--------------------- -----------
* --application-id <String: id> The Kafka Streams application ID
(application.id).
--bootstrap-server <String: server to REQUIRED unless --bootstrap-servers
connect to> (deprecated) is specified. The server
(s) to connect to. The broker list
string in the form HOST1:PORT1,HOST2:
PORT2.
--by-duration <String: urls> Reset offsets to offset by duration from
current timestamp. Format: 'PnDTnHnMnS'
--config-file <String: file name> Property file containing configs to be
passed to admin clients and embedded
consumer.
--dry-run Display the actions that would be
performed without executing the reset
commands.
--from-file <String: urls> Reset offsets to values defined in CSV
file.
--input-topics <String: list> Comma-separated list of user input
topics. For these topics, the tool will
reset the offset to the earliest
available offset.
--internal-topics <String: list> Comma-separated list of internal topics
to delete. Must be a subset of the
internal topics marked for deletion by
the default behaviour (do a dry-run without
this option to view these topics).
--shift-by <Long: number-of-offsets> Reset offsets shifting current offset by
'n', where 'n' can be positive or
negative
--to-datetime <String> Reset offsets to offset from datetime.
Format: 'YYYY-MM-DDTHH:mm:SS.sss'
--to-earliest Reset offsets to earliest offset.
--to-latest Reset offsets to latest offset.
--to-offset <Long> Reset offsets to a specific offset.
--force Force removing members of the consumer group
(intended to remove left-over members if
long session timeout was configured).
Consider the following as reset-offset scenarios for input-topics
:
- by-duration
- from-file
- shift-by
- to-datetime
- to-earliest
- to-latest
- to-offset
Only one of these scenarios can be defined. If not, to-earliest
will be executed by default
All the other parameters can be combined as needed. For example, if you want to restart an application from an empty internal state, but not reprocess previous data, simply omit the parameter --input-topics
.
Step 2: Reset the local environments of your application instances
For a complete application reset, you must delete the application’s local state directory on any machines where the application instance was run. You must do this before restarting an application instance on the same machine. You can use either of these methods:
- The API method
KafkaStreams#cleanUp()
in your application code. - Manually delete the corresponding local state directory (default location:
/${java.io.tmpdir}/kafka-streams/<application.id>
). For more information, see Streams javadocs.