Developer Guide

Table of Contents

Code examples

Before we begin the deep-dive into Kafka Streams in the subsequent sections, you might want to take a look at a few examples first.

Application examples for Kafka Streams in Apache Kafka

The Apache Kafka project includes a few Kafka Streams code examples, which demonstrate the use of the Kafka Streams DSL and the low-level Processor API; and a juxtaposition of typed vs. untyped examples.

Application examples for Kafka Streams provided by Confluent

The Confluent examples repository contains several Kafka Streams examples, which demonstrate the use of Java 8 lambda expressions (which simplify the code significantly), how to read/write Avro data, and how to implement end-to-end integration tests using embedded Kafka clusters.

Security examples

Interactive Queries examples

As of Kafka 0.10.1.0 it is possible to query state stores created via the Kafka Streams DSL and the Processor API. Please refer to Interactive Queries for further information.

End-to-end demo applications

These demo applications use embedded instances of Kafka, ZooKeeper, and/or Confluent Schema Registry. They are implemented as integration tests.

Configuring a Kafka Streams application

Overview

The configuration of Kafka Streams is done by specifying parameters in a StreamsConfig instance.

Typically, you create a java.util.Properties instance, set the necessary parameters, and construct a StreamsConfig instance from the Properties instance. How this StreamsConfig instance is used further (and passed along to other Streams classes) will be explained in the subsequent sections.

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");
settings.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "zookeeper1:2181");
// Any further settings
settings.put(... , ...);

// Create an instance of StreamsConfig from the Properties instance
StreamsConfig config = new StreamsConfig(settings);

Required configuration parameters

The table below is a list of required configuration parameters.

Parameter Name Description Default Value
application.id An identifier for the stream processing application. Must be unique within the Kafka cluster. <none>
bootstrap.servers A list of host/port pairs to use for establishing the initial connection to the Kafka cluster <none>
zookeeper.connect Zookeeper connect string for Kafka topic management the empty string

Application Id (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 sub-directory in the state directory (cf. state.dir)
  • As the prefix of internal Kafka topic names

Attention

When an application is updated, it is recommended to change application.id unless it is safe to let the updated application re-use the existing data in internal topics and state stores. One pattern could be to embed version information within application.id, e.g., my-app-v1.0.0 vs. my-app-v1.0.2.

Kafka Bootstrap Servers (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".

Note

One Kafka cluster only: Currently Kafka Streams applications can only talk to a single Kafka cluster specified by this config value. In the future Kafka Streams will be able to support connecting to different Kafka clusters for reading input streams and/or writing output streams.

ZooKeeper connection (zookeeper.connect): Currently Kafka Streams needs to access ZooKeeper directly for creating its internal topics. Internal topics are created when a state store is used, or when a stream is repartitioned for aggregation. This setting must point to the same ZooKeeper ensemble that your Kafka cluster uses (cf. bootstrap.servers). Example: "zookeeper1:2181".

Note

ZooKeeper dependency of Kafka Streams and zookeeper.connect: This configuration option is temporary and will be removed post 0.10.1.1 release.

Optional configuration parameters

The table below is a list of optional configuration parameters. However, users should consider setting the following parameters consciously.

Parameter Name Description Default Value
application.server 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 The maximum number of records to buffer per partition 1000
client.id 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 The frequency with which to save the position (offsets in source topics) of tasks 30000 (millisecs)
key.serde Default serializer/deserializer class for record keys, implements the Serde interface (see also value.serde) Serdes.ByteArraySerde.class.getName()
metric.reporters A list of classes to use as metrics reporters the empty list
metrics.num.samples The number of samples maintained to compute metrics. 2
metrics.sample.window.ms The window of time a metrics sample is computed over. 30000 (millisecs)
num.standby.replicas The number of standby replicas for each task 0
num.stream.threads The number of threads to execute stream processing 1
partition.grouper Partition grouper class that implements the PartitionGrouper interface see Partition Grouper
poll.ms The amount of time in milliseconds to block waiting for input 100 (millisecs)
replication.factor The replication factor for changelog topics and repartition topics created by the application 1
state.cleanup.delay.ms The amount of time in milliseconds to wait before deleting state when a partition has migrated 60000 (millisecs)
state.dir Directory location for state stores /var/lib/kafka-streams
timestamp.extractor Timestamp extractor class that implements the TimestampExtractor interface see Timestamp Extractor
value.serde Default serializer/deserializer class for record values, implements the Serde interface (see also key.serde) Serdes.ByteArraySerde.class.getName()
cache.max.bytes.buffering Maximum number of memory bytes to be used for record caches across all threads 10485760 (bytes)

Ser-/Deserialization (key.serde, value.serde): Serialization and deserialization in Kafka Streams happens whenever data needs to be materialized, i.e.,:

  • Whenever data is read from or written to a Kafka topic (e.g., via the KStreamBuilder#stream() and KStream#to() methods).
  • Whenever data is read from or written to a state store.

We will discuss this in more details later in Data types and serialization.

Number of Standby Replicas (num.standby.replicas): This specifies the number of standby replicas. Standby replicas are shadow copies of local state stores. Kafka Streams attempts to create the specified number of replicas 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.

Number of Stream Threads (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 threads. Details about Kafka Streams threading model can be found in section Threading Model.

Replication Factor of Internal Topics (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.

State Directory (state.dir): Kafka Streams persists local states under the state directory. Each application has a subdirectory on its hosting machine, whose name is the application id, directly under the state directory. The state stores associated with the application are created under this subdirectory.

Timestamp Extractor (timestamp.extractor): A timestamp extractor extracts a timestamp from an instance of ConsumerRecord. Timestamps are used to control the progress of streams.

The default extractor is ConsumerRecordTimestampExtractor. This extractor retrieves built-in timestamps that are automatically embedded into Kafka messages by the Kafka producer client (introduced in Kafka 0.10.0.0, see KIP-32: Add timestamps to Kafka message). Depending on the setting of Kafka’s log.message.timestamp.type parameter, this extractor will provide you with:

  • event-time processing semantics if log.message.timestamp.type is set to CreateTime aka “producer time” (which is the default). This represents the time when the Kafka producer sent the original message.
  • ingestion-time processing semantics if log.message.timestamp.type is set to LogAppendTime aka “broker time”. This represents the time when the Kafka broker received the original message.

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, 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. Here is an example of a custom TimestampExtractor 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(ConsumerRecord<Object, Object> record) {
    // `Foo` is your own custom class, which we assume has a method that returns
    // the embedded timestamp (in milliseconds).
    Foo myPojo = (Foo) record.value();
    if (myPojo != null) {
      return myPojo.getTimestampInMillis();
    }
    else {
      // Kafka allows `null` as message value.  How to handle such message values
      // depends on your use case.  In this example, we decide to fallback to
      // wall-clock time (= processing-time).
      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 settings = new Properties();
settings.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MyEventTimeExtractor.class.getName());

Partition Grouper (partition.grouper): A partition grouper is used to create a list of stream tasks given the partitions of source topics, where each created task is assigned with a group of source topic partitions. The default implementation provided by Kafka Streams is DefaultPartitionGrouper, which assigns each task with at most one partition for each of the source topic partitions; therefore, the generated number of tasks is equal to the largest number of partitions among the input topics. Usually an application does not need to customize the partition grouper.

Non-Streams configuration parameters

Apart from Kafka Streams’ own configuration parameters (see previous sections) you can also specify parameters for the Kafka consumers and producers that are used internally, depending on the needs of your application. Similar to the Streams settings you define any such consumer and/or producer settings via StreamsConfig:

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);
StreamsConfig config = new StreamsConfig(streamsSettings);

Some consumer and producer configuration parameters do use the same parameter name. For example, send.buffer.bytes or receive.buffer.bytes which are used to configure TCP buffers; request.timeout.ms and retry.backoff.ms which control retries for client request (and some more). If you want to set different values for consumer and producer for such a parameter, you can prefix the parameter name with consumer. or producer.:

Properties streamsSettings = new Properties();
// same value for consumer and producer
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");
// alternatively, you can use
streamsSettings.put(StreamsConfig.consumerPrefix("PARAMETER_NAME"), "consumer-value");
streamsSettings.put(StremasConfig.producerConfig("PARAMETER_NAME"), "producer-value");

In addition, Kafka Streams uses different default values for some of the underlying client configs, which are summarized below. For detailed descriptions of these configs, please read Producer Configs and Consumer Configs in Apache Kafka web documentation respectively.

RocksDB Configuration: Kafka Streams uses RocksDB as the default storage engine for persistent stores. In order to change the default configuration values for RocksDB, you need to implement RocksDBConfigSetter and provide your custom class via rocksdb.config.setter.

public class CustomRocksDBConfig implements RocksDBConfigSetter {
  // put your code here
}

Properties streamsSettings = new Properties();
streamsConfig.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, CustomRocksDBConfig.class);
Parameter Name Corresponding Client Default Value
linger.ms Producer 100
max.poll.records Consumer 1000
auto.offset.reset Consumer earliest
enable.auto.commit Consumer false, see Consumer Auto Commit

Consumer Auto Commit (enable.auto.commit): To guarantee at-least-once processing semantics, Kafka Streams will always override this consumer config value to false in order to turn off auto committing. Instead, consumers will only commit explicitly via commitSync calls when Kafka Streams library or users decide to commit the current processing state.

Note

A future version of Kafka Streams will allow developers to set their own app-specific configuration settings through StreamsConfig as well, which can then be accessed through ProcessorContext.

Writing a Kafka Streams application

Overview

Any Java 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).

Currently Kafka Streams provides two sets of APIs to define the processor topology:

  1. A low-level Processor API that lets you add and connect processors as well as interact directly with state stores.
  2. A high-level Kafka Streams DSL that provides common data transformation operations in a functional programming style such as map and filter operations. The DSL is the recommended starting point for developers new to Kafka Streams, and should cover many use cases and stream processing needs.

We describe both of these APIs in the subsequent sections.

Libraries and maven artifacts

This section lists the Kafka Streams related libraries that are available for writing your Kafka Streams applications.

The corresponding maven artifacts of these libraries are available in Confluent’s maven repository:

<!-- Example pom.xml snippet when using maven to build your Java applications. -->
<repositories>
    <repository>
        <id>confluent</id>
        <url>http://packages.confluent.io/maven/</url>
    </repository>
</repositories>

Depending on your use case you will need to define dependencies on the following libraries for your Kafka Streams applications.

Group Id Artifact Id Version Description / why needed
org.apache.kafka kafka-streams 0.10.1.1-cp1 Base library for Kafka Streams. Required.
org.apache.kafka kafka-clients 0.10.1.1-cp1 Kafka client library. Contains built-in serializers/deserializers. Required.
org.apache.avro avro 1.7.7 Apache Avro library. Optional (needed only when using Avro).
io.confluent kafka-avro-serializer 3.1.2 Confluent’s Avro serializer/deserializer. Optional (needed only when using Avro).

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>0.10.1.1-cp1</version>
</dependency>
<dependency>
    <groupId>org.apache.kafka</groupId>
    <artifactId>kafka-clients</artifactId>
    <version>0.10.1.1-cp1</version>
</dependency>

<!-- Dependencies below are required/recommended only when using Apache Avro. -->
<dependency>
    <groupId>io.confluent</groupId>
    <artifactId>kafka-avro-serializer</artifactId>
    <version>3.1.2</version>
</dependency>
<dependency>
    <groupId>org.apache.avro</groupId>
    <artifactId>avro</artifactId>
    <version>1.7.7</version>
</dependency>
<dependency>
    <groupId>org.apache.avro</groupId>
    <artifactId>avro-maven-plugin</artifactId>
    <version>1.7.7</version>
</dependency>

See the Kafka Streams examples in the Confluent examples repository for a full maven/pom setup.

Using Kafka Streams within your application code

You can call Kafka Streams from anywhere in your application code. Very commonly though you would do so within the main() method of your application, or some variant thereof. The most important 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 builder (either KStreamBuilder for the Kafka Streams DSL, or TopologyBuilder for the Processor API) that is used to define a topology.
  • The second argument is an instance of StreamsConfig, which defines the configuration for this specific topology.
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.kstream.KStreamBuilder;
import org.apache.kafka.streams.processor.TopologyBuilder;

// 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.

KStreamBuilder builder = ...;  // when using the Kafka Streams DSL
//
// OR
//
TopologyBuilder builder = ...; // 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.
StreamsConfig config = ...;

KafkaStreams streams = new KafkaStreams(builder, config);

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 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. See Stream Partitions and Tasks and Threading Model for details.

To catch any unexpected exceptions, you may 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(new Thread.UncaughtExceptionHandler() {
    public uncaughtException(Thread t, throwable e) {
        // here you should examine the exception and perform an appropriate action!
    }
);

To stop the application instance call the close() method:

// Stop the Kafka Streams threads
streams.close();

So that your application can gracefully shutdown in response to SIGTERM, it is recommended that you add a shutdown hook and call KafkaStreams#close.

Shutdown hook example In Java 8+

// add shutdown hook to stop the Kafka Streams threads
Runtime.getRuntime().addShutdownHook(new Thread(streams::close));

Shutdown hook example In Java 7

// add shutdown hook
Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
  @Override
  public void run() {
      // Stop the Kafka Streams threads
      streams.close();
  }
}));

After a particular instance of the application was stopped, Kafka Streams migrates any tasks that had been running in this instance to other running instances (assuming there are any such instances remaining).

In the following sections we describe the two APIs of Kafka Streams in more detail, which is what you will use in combination with KStreamBuilder (Kafka Streams DSL) or TopologyBuilder (Processor API) to define the actual processing topology.

Processor API

Note

See also the Kafka Streams Javadocs for a complete list of available API functionality.

Overview

As mentioned in the Concepts section, a stream processor is a node in the processor topology that represents a single processing step. With the Processor API users 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.

Defining a Stream Processor

Users can define their customized stream processor by implementing the Processor interface, which provides two main API methods: process() and punctuate().

  • process() is called on each of the received record.
  • punctuate() is called periodically based on elapsed stream-time (by default, stream-time is configured to represent event-time). Thus, punctuate() is purely data-driven and not related to wall-clock time (even if you use WallclockTimestampExtractor). For example, let’s assume you registered a punctuate() schedule of 10 seconds. If you were to 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 – regardless of the time required to actually process those records; i.e., punctuate() would be called 6 times no matter whether processing these 60 records would take a second, a minute, or an hour.

Attention

Stream-time is only advanced if all input partitions over all input topics have new data (with newer timestamps) available. If at least one partition does not have any new data available, stream-time will not be advanced and thus punctuate() will not be triggered. This behavior is independent of the configured timestamp extractor, i.e., using WallclockTimestampExtractor does not enable wall-clock triggering of punctuate().

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 . This context instance can also be used to schedule the punctuation period (via ProcessorContext#schedule()) for punctuate(), to forward a new record as a key-value pair to the downstream processors (via ProcessorContext#forward()), and to commit the current processing progress (via ProcessorContext#commit()).

The following example Processor implementation defines a simple word-count algorithm:

public class WordCountProcessor implements Processor<String, String> {

  private ProcessorContext context;
  private KeyValueStore<String, Long> kvStore;

  @Override
  @SuppressWarnings("unchecked")
  public void init(ProcessorContext context) {
      // keep the processor context locally because we need it in punctuate() and commit()
      this.context = context;

      // call this processor's punctuate() method every 1000 time units.
      this.context.schedule(1000);

      // retrieve the key-value store named "Counts"
      kvStore = (KeyValueStore) context.getStateStore("Counts");
  }

  @Override
  public void process(String dummy, String line) {
      String[] words = line.toLowerCase().split(" ");

      for (String word : words) {
          Long oldValue = kvStore.get(word);
          if (oldValue == null) {
              kvStore.put(word, 1L);
          } else {
              kvStore.put(word, oldValue + 1L);
          }
      }
  }

  @Override
  public void punctuate(long timestamp) {
      KeyValueIterator<String, Long> iter = this.kvStore.all();
      while (iter.hasNext()) {
          KeyValue<String, Long> entry = iter.next();
          context.forward(entry.key, entry.value.toString());
      }
      iter.close();
      // commit the current processing progress
      context.commit();
  }

  @Override
  public void close() {
      // close the key-value store
      kvStore.close();
  }

}

In the above implementation, 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.

Defining a State Store

Note that the WordCountProcessor defined above can not only access the currently received record in the process() method, but also maintain processing states to keep recently arrived records for stateful processing needs such as aggregations and joins. To take advantage of these states, users can define a state store by implementing the StateStore interface (the Kafka Streams library also has a few extended interfaces such as KeyValueStore); in practice, though, users usually do not need to customize such a state store from scratch but can simply use the Stores factory to define a state store by specifying whether it should be persistent, log-backed, etc.

StateStoreSupplier countStore = Stores.create("Counts")
              .withKeys(Serdes.String())
              .withValues(Serdes.Long())
              .persistent()
              .build();

In the above example, a persistent key-value store named “Counts” with key type String and value type long is is created.

Enable / Disable State Store Changelogs

By default, persistent key-value stores are backed by a compacted topic, which we sometimes refer to as the state store’s associated changelog topic or simply changelog. The purpose of compacting this topic is to prevent the topic from growing out of bounds, to reduce the storage consumed in the associated Kafka cluster, and to minimize recovery time if a state store needs to be reconstructed from its changelog topic. Similarly, persistent window stores are backed by a topic that uses both compaction and deletion. Using deletion in addition to compaction is required for the changelog topics of window stores because of the structure of the message keys that are being sent to the changelog topics: for window stores, the message keys are composite keys that include not only the “normal” key but also window timestamps. For such composite keys it would not be sufficient to enable just compaction in order to prevent a changelog topic from growing out of bounds. With deletion enabled old windows that have expired will be cleaned up by the log cleaner as the log segments expire. The default retention setting is Windows#maintainMs() + 1 day. This setting can be overriden by setting StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG in the StreamsConfig.

Disable logging example:

Attention

If the changelog is disabled then the attached state store is no longer fault tolerant and it can’t have any standby replicas.

StateStoreSupplier countStore = Stores.create("Counts")
            .withKeys(Serdes.String())
            .withValues(Serdes.Long())
            .persistent()
            .disableLogging()
            .build();

Enable logging with configuration example:

You can add any log config from kafka.log.LogConfig. Unrecognized configs will be ignored.

Map<String, String> logConfig = new HashMap();
// override min.insync.replicas
logConfig.put("min.insyc.replicas", "1")

StateStoreSupplier countStore = Stores.create("Counts")
            .withKeys(Serdes.String())
            .withValues(Serdes.Long())
            .persistent()
            .enableLogging(logConfig)
            .build();

Connecting Processors and Stores

Now that we have defined the processor and the state stores, we can now construct the processor topology by connecting these processors and state stores together by using the TopologyBuilder instance. In addition, users 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.

TopologyBuilder builder = new TopologyBuilder();

// 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")

    // create the countStore associated with the WordCountProcessor processor
    .addStateStore(countStore, "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");

There are several steps in the above implementation to build the topology, and here is a quick walk through:

  • A source processor node named “Source” is added to the topology using the addSource method, with one Kafka topic “source-topic” fed to it.
  • A processor node named “Process” with the pre-defined WordCountProcessor logic is then added as the downstream processor of the “Source” node using the addProcessor method.
  • A predefined persistent key-value state store countStore is created and associated to the “Process” node.
  • 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.

In this defined topology, 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 forward a newly fetched record from Kafka to its downstream “Process” node, WordCountProcessor#process() method is triggered to process the record and update the associated state store; and whenever context#forward() is called in the WordCountProcessor#punctuate() method, the aggregate key-value pair will be sent via the “Sink” processor node to the Kafka topic “sink-topic”. Note that in the WordCountProcessor implementation, users need to refer with 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; also if the state store itself is not associated with the processor in the TopologyBuilder 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.

With the defined processor topology, users can now start running a Kafka Streams application instance with the defined processing state stores. Please read how to run a Kafka Streams application for details.

Kafka Streams DSL

Note

See also the Kafka Streams Javadocs for a complete list of available API functionality.

Overview

As mentioned in the Concepts section, a stream is an unbounded, continuously updating data set. With the Kafka Streams DSL users can define the processor topology by concatenating multiple transformation operations where each operation transforming one stream into other stream(s); the resulted topology then takes the input streams from source Kafka topics and generates the final output streams throughout its concatenated transformations. However, different streams may have different semantics in their data records:

  • In some streams, each record represents a new immutable datum in their unbounded data set; we call these record streams.
  • In other streams, each record represents a revision (or update) of their unbounded data set in chronological order; we call these changelog streams.

Both of these two types of streams can be stored as Kafka topics. However, their computational semantics can be quite different. Take the example of an aggregation operation that counts the number of records for the given key. For record streams, each record is a keyed message from a Kafka topic (e.g., a page view stream keyed by user ids):

# Example: a record stream for page view events
# Notation is <record key> => <record value>
1 => {"time":1440557383335, "user_id":1, "url":"/home?user=1"}
5 => {"time":1440557383345, "user_id":5, "url":"/home?user=5"}
2 => {"time":1440557383456, "user_id":2, "url":"/profile?user=2"}
1 => {"time":1440557385365, "user_id":1, "url":"/profile?user=1"}

The counting operation for record streams is trivial to implement: you can maintain a local state store that tracks the latest count for each key, and, upon receiving a new record, update the corresponding key by incrementing its count by one.

For changelog streams, on the other hand, each record is an update of the unbounded data set (e.g., a changelog stream for a user profile table, where the user id serves as both the primary key for the table and as the record key for the stream; here, a new record represents a changed row of the table). In practice you would usually store such streams in Kafka topics where log compaction is enabled.

# Example: a changelog stream for a user profile table
1 => {"last_modified_time":1440557383335, "user_id":1, "email":"[email protected]"}
5 => {"last_modified_time":1440557383345, "user_id":5, "email":"[email protected]"}
2 => {"last_modified_time":1440557383456, "user_id":2, "email":"[email protected]"}
1 => {"last_modified_time":1440557385365, "user_id":1, "email":"[email protected]"}
2 => {"last_modified_time":1440557385395, "user_id":2, "email":null}  <-- user has been deleted

As a result the counting operation for changelog streams is no longer monotonically incrementing: you need to also decrement the counts when a delete update record is received on some given key as well. In addition, even for counting aggregations on a record stream, the resulting aggregate is no longer a record stream but a relation / table, which can then be represented as a changelog stream of updates on the table.

Note

The counting operation of a user profile changelog stream is peculiar because it will generate, for a given key, a count of either 0 (meaning the key does not exist or does not exist anymore) or 1 (the key exists) only. Multiple records for the same key are considered duplicate, old information of the most recent record, and thus will not contribute to the count.

For changelog streams developers usually prefer counting a non-primary-key field. We use the example above just for the sake of illustration.

One of the key design principles of the Kafka Streams DSL is to understand and distinguish between record streams and changelog streams and to provide operators with the correct semantics for these two different types of streams. More concretely, in the Kafka Streams DSL we use the KStream interface to represent record streams, and we use a separate KTable interface to represent changelog streams. The Kafka Streams DSL is therefore the recommended API to implement a Kafka Streams application. Compared to the lower-level Processor API, its benefits are:

  • More concise and expressive code, particularly when using Java 8+ with lambda expressions.
  • Easier to implement stateful transformations such as joins and aggregations.
  • Understands the semantic differences of record streams and changelog streams, so that transformations such as aggregations work as expected depending on which type of stream they operate against.

In the subsequent sections we provide a step-by-step guide for writing a stream processing application using the Kafka Streams DSL.

Creating source streams from Kafka

Both KStream or KTable objects can be created as a source stream from one or more Kafka topics via KStreamBuilder, an extended class of TopologyBuilder used in the lower-level Processor API (for KTable you can only create the source stream from a single topic).

Interface How to instantiate
KStream<K, V> KStreamBuilder#stream(...)
KTable<K, V> KStreamBuilder#table(...)

When creating an instance, you may override the default serdes for record keys (K) and record values (V) used for reading the data from Kafka topics (see Data types and serdes for more details); otherwise the default serdes specified through StreamsConfig will be used.

import org.apache.kafka.streams.kstream.KStreamBuilder;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.KTable;

KStreamBuilder builder = new KStreamBuilder();

// In this example we assume that the default serdes for keys and values are
// the String serde and the generic Avro serde, respectively.

// Create a stream of page view events from the PageViews topic, where the key of
// a record is assumed to be the user id (String) and the value an Avro GenericRecord
// that represents the full details of the page view event.
KStream<String, GenericRecord> pageViews = builder.stream("PageViews");

// Create a changelog stream for user profiles from the UserProfiles topic,
// where the key of a record is assumed to be the user id (String) and its value
// an Avro GenericRecord.
KTable<String, GenericRecord> userProfiles = builder.table("UserProfiles");

Transform a stream

KStream and KTable 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 these transformation operations are defined as generics functions where users could specify the input and output data types.

Some KStream transformations may generate one or more KStream objects (e.g., filter and map on KStream generate another KStream, while branch on KStream can generate multiple KStream) while some others may generate a KTable object (e.g., aggregation) interpreted as the changelog stream to the resulted relation. This allows Kafka Streams to continuously update the computed value upon arrivals of late records after it has already been produced to the downstream transformation operators. As for KTable, all its transformation operations can only generate another KTable (though the Kafka Streams DSL does provide a special function to convert a KTable representation into a KStream, which we will describe later). Nevertheless, all these transformation methods can be chained together to compose a complex processor topology.

We describe these transformation operations in the following subsections, categorizing them as two categories: stateless and stateful transformations.

Stateless transformations

Stateless transformations include filter, filterNot, foreach, map, mapValues, selectKey, flatMap, flatMapValues, branch. Most of them can be applied to both KStream and KTable, where users usually pass a customized function to these functions as a parameter; e.g. a Predicate for filter, a KeyValueMapper for map, and so on. Stateless transformations, by definition, do not depend on any state for processing, and hence implementation-wise they do not require a state store associated with the stream processor.

Example of mapValues in Java 8+, using lambda expressions.

KStream<Long, String> uppercased =
    nicknameByUserId.mapValues(nickname -> nickname.toUpperCase());

Example of mapValues in Java 7:

KStream<Long, String> uppercased =
    nicknameByUserId.mapValues(
        new ValueMapper<String>() {
            @Override
            public String apply(String nickname) {
                return nickname.toUpperCase();
            }
        }
    );

The function is applied to each record, and its result will trigger the creation a new record.

Stateful transformations

Available stateful transformations include:

  • joins (KStream/KTable): join, leftJoin, outerJoin
  • aggregations (KStream/KTable): groupBy, groupByKey (KStream) plus count, reduce, aggregate (via KGroupedStream and KGroupedTable)
  • general transformations (KStream): process, transform, transformValues

Stateful transformations are transformations where the processing logic requires accessing an associated state for processing and producing outputs. For example, in join and aggregation operations, a windowing state is usually used to store all the records received so far within the defined window boundary. The operators can then access accumulated records in the store and compute based on them (see Windowing a Stream for details).

WordCount example in Java 8+, using lambda expressions (see WordCountLambdaIntegrationTest for the full code):

// We assume 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 = ...;

KStream<String, Long> wordCounts = textLines
    // Split each text line, by whitespace, into words.  The text lines are the message
    // values, i.e. we can ignore whatever data is in the message 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).  We must provide a name for
    // the resulting KTable, which will be used to name e.g. its associated
    // state store and changelog topic.
    .count("Counts")
    // Convert the `KTable<String, Long>` into a `KStream<String, Long>`.
    .toStream();

WordCount example in Java 7:

// Code below is equivalent to the previous Java 8+ example above.
KStream<String, String> textLines = ...;

KStream<String, Long> wordCounts = textLines
    .flatMapValues(new ValueMapper<String, Iterable<String>>() {
        @Override
        public Iterable<String> apply(String value) {
            return Arrays.asList(value.toLowerCase().split("\\W+"));
        }
    })
    .groupBy(new KeyValueMapper<String, String, String>>() {
        @Override
        public String apply(String key, String word) {
            return word;
        }
    })
    .count("Counts")
    .toStream();

Tip

A KTable object can be converted back into a KStream via the KTable#toStream() function.

Windowing a Stream

Windowing is a common prerequisite for stateful transformations which group records in a stream, for example, by their timestamps. A local state store is usually needed for a windowing operation to store recently received records based on the window interval, while old records in the 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 Windows#until().

Kafka Streams currently defines the following types of windows:

Window name Behavior Short description
Tumbling time window Time-based Fixed-size, non-overlapping, gap-less windows
Hopping time window Time-based Fixed-size, overlapping windows
Sliding time window Time-based Fixed-size, overlapping windows that work on differences between record timestamps

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.

// A tumbling time window with a size 60 seconds (and, by definition, an implicit
// advance interval of 60 seconds).
// The window's name -- the string parameter -- is used to e.g. name the backing state store.
long windowSizeMs = 60 * 1000L;
TimeWindows.of("tumbling-window-example", windowSizeMs);

// The above is equivalent to the following code:
TimeWindows.of("tumbling-window-example", windowSizeMs).advanceBy(windowSizeMs);

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.

// 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.
long windowSizeMs = 5 * 60 * 1000L;
long advanceMs =    1 * 60 * 1000L;
TimeWindows.of("hopping-window-example", windowSizeMs).advanceBy(advanceMs);

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.

Pay attention, that tumbling and hopping time windows are aligned to the epoch and that the lower window time interval bound is inclusive, while the upper bound is exclusive.

Aligned to the epoch means, that the first window starts at timestamp zero. For example, hopping windows with size of 5000ms and advance of 3000ms, have window boundaries [0;5000),[3000;8000),...— and not [1000;6000),[4000;9000),... or even something “random” like [1452;6452),[4452;9452),..., which might be the case if windows get initialized depending on system/application start-up time, introducing non-determinism.

Windowed counting example:

KStream<String, GenericRecord> viewsByUser = ...;

KTable<Windowed<String>, Long> userCounts =
    // count users, using hopping windows of size 5 minutes that advance every 1 minute
    viewsByUser.countByKey(TimeWindows.of("GeoPageViewsWindow", 5 * 60 * 1000L).advanceBy(60 * 1000L));

Unlike non-windowed aggregates that we have seen previously, windowed aggregates return a windowed KTable whose key 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.

Sliding windows are actually quite different from hopping and tumbling windows. A sliding window models a fixed-size window that slides continuously over the time axis; here, two data records are said to be included in the same window if the difference of their timestamps is within the window size. Thus, sliding windows are not aligned to the epoch, but on the data record timestamps. Pay attention, that in contrast to hopping and tumbling windows, lower and upper window time interval bounds are both inclusive. In Kafka Streams, sliding windows are used only for join operations, and can be specified through the JoinWindows class.

Joining Streams

Many stream processing applications can be coded as stream join operations. For example, applications backing an online shop might need to access multiple, updating database tables (e.g. sales prices, inventory, customer information) when processing a new record. These applications can be implemented such that they work on the tables’ changelog streams directly, i.e. without requiring to make a database query over the network for each record. In this example, the KTable concept in Kafka Streams would enable you to track the latest state (think: snapshot) of each table in a local key-value store, thus greatly reducing the processing latency as well as reducing the load of the upstream databases.

In Kafka Streams, you may perform the following join operations:

  • Join a KStream with another KStream or KTable.
  • Join a KTable with another KTable only.

We explain each case in more detail below.

  • KStream-to-KStream Joins are always windowed joins, since otherwise the join result size might grow infinitely in size. Here, a newly received record from one of the streams is joined with the other stream’s records within the specified window interval to produce one result for each matching pair based on user-provided ValueJoiner. A new KStream instance representing the result stream of the join is returned from this operator.
  • KTable-to-KTable Joins are join operations designed to be consistent with the ones in relational databases. Here, both changelog streams are materialized into local state stores to represent the latest snapshot of their data table duals. When a new record is received from one of the streams, it is joined with the other stream’s materialized state stores to produce one result for each matching pair based on user-provided ValueJoiner. A new KTable instance representing the result stream of the join, which is also a changelog stream of the represented table, is returned from this operator.
  • KStream-to-KTable Joins allow you to perform table lookups against a changelog stream (KTable) upon receiving a new record from another record stream (KStream). An example use case would be to enrich a stream of user activities (KStream) with the latest user profile information (KTable). Only records received from the record stream will trigger the join and produce results via ValueJoiner, not vice versa (i.e., records received from the changelog stream will be used only to update the materialized state store). A new KStream instance representing the result stream of the join is returned from this operator.

Depending on the operands the following join operations are supported:

Join operands (INNER) JOIN OUTER JOIN LEFT JOIN
KStream-to-KStream Supported Supported Supported
KTable-to-KTable Supported Supported Supported
KStream-to-KTable N/A N/A Supported

The join semantics are similar to the corresponding operators in relational databases:

  • Inner join produces new joined records when the join operator finds some records with the same key in the other stream / materialized store.
  • Outer join works like inner join if some records are found in the windowed stream / materialized store. The difference is that outer join still produces a record even when no records are found. It uses null as the value of missing record.
  • Left join is like outer join except that for KStream-KStream join it is always driven by record arriving from the primary stream; while for KTable-KTable join it is driven by both streams to make the result consistent with the left join of databases while only permits missing records in the secondary stream. In a KStream-KTable left join, a KStream record will only join a KTable record if the KTable record arrived before the KStream record (and is in the KTable). Otherwise, the join result will be null.

Note

Since stream joins are performed over the keys of records, it is required that joining streams are co-partitioned by key, i.e., their corresponding Kafka topics must have the same number of partitions and partitioned on the same key so that records with the same keys are delivered to the same processing thread. This is validated by Kafka Streams library at runtime (we talked about the threading model and data parallelism with more details in the Architecture section).

Join example in Java 8+, using lambda expressions.

// Key is user, value is number of clicks by that user
KStream<String, Long> userClicksStream =  ...;
// Key is user, value is the geo-region of that user
KTable<String, String> userRegionsTable = ...;

// KStream-KTable join
KStream<String, RegionWithClicks> userClicksWithRegion = userClicksStream
    // Null values possible: In general, null values are possible for region (i.e. the value of
    // the KTable we are joining against) so we must guard against that (here: by setting the
    // fallback region "UNKNOWN").
    //
    // Also, we need to return a tuple of (region, clicks) for each user.  But because Java does
    // not support tuples out-of-the-box, we must use a custom class `RegionWithClicks` to
    // achieve the same effect.  This class two fields -- the region (String) and the number of
    // clicks (Long) for that region -- as well as a matching constructor, which we use here.
    .leftJoin(userRegionsTable,
      (clicks, region) -> new RegionWithClicks(region == null ? "UNKNOWN" : region, clicks));

Join example in Java 7+:

// Key is user, value is number of clicks by that user
KStream<String, Long> userClicksStream =  ...;
// Key is user, value is the geo-region of that user
KTable<String, String> userRegionsTable = ...;

// KStream-KTable join
KStream<String, RegionWithClicks> userClicksWithRegion = userClicksStream
    // Null values possible: In general, null values are possible for region (i.e. the value of
    // the KTable we are joining against) so we must guard against that (here: by setting the
    // fallback region "UNKNOWN").
    //
    // Also, we need to return a tuple of (region, clicks) for each user.  But because Java does
    // not support tuples out-of-the-box, we must use a custom class `RegionWithClicks` to
    // achieve the same effect.  This class two fields -- the region (String) and the number of
    // clicks (Long) for that region -- as well as a matching constructor, which we use here.
    .leftJoin(userRegionsTable, new ValueJoiner<Long, String, RegionWithClicks>() {
      @Override
      public RegionWithClicks apply(Long clicks, String region) {
        return new RegionWithClicks(region == null ? "UNKNOWN" : region, clicks);
      }
    });
Applying a custom processor

Tip

See also the documentation of the low-level Processor API.

Beyond the provided transformation operators, users can also specify any customized processing logic on their stream data via the KStream#process() method, which takes an implementation of the ProcessorSupplier interface as its parameter. This is essentially equivalent to the addProcessor() method in the Processor API.

The following example shows how to leverage, via the process() method, a custom processor that sends an email notification whenever a page view count reaches a predefined threshold.

In Java 8+, using lambda expressions:

// Send an email notification when the view count of a page reaches one thousand.
pageViews.groupByKey()
         .count("PageViewCounts")
         .filter((PageId pageId, Long viewCount) -> viewCount == 1000)
         // PopularPageEmailAlert is your custom processor that implements the
         // `Processor` interface, see further down below.
         .process(() -> new PopularPageEmailAlert("[email protected]"));

In Java 7:

// Send an email notification when the view count of a page reaches one thousand.
pageViews.groupByKey().
         .count("PageViewCounts")
         .filter(
            new Predicate<PageId, Long>() {
              public boolean test(PageId pageId, Long viewCount) {
                return viewCount == 1000;
              }
            })
         .process(
           new ProcessorSupplier<PageId, Long>() {
             public Processor<PageId, Long> get() {
               // PopularPageEmailAlert is your custom processor that implements
               // the `Processor` interface, see further down below.
               return new PopularPageEmailAlert("[email protected]");
             }
           });

In the above examples, PopularPageEmailAlert is a custom stream processor that implements the Processor interface:

// A processor that sends an alert message about a popular page to a configurable email address
public class PopularPageEmailAlert implements Processor<PageId, Long> {

  private final String emailAddress;
  private ProcessorContext;

  public PopularPageEmailAlert(String emailAddress) {
    this.emailAddress = emailAddress;
  }

  @Override
  public void init(ProcessorContext context) {
    this.context = context;

    // Here you would perform any additional initializations
    // such as setting up an email client.
  }

  @Override
  void process(PageId pageId, Long count) {
    // Here would format and send the alert email.
    //
    // In this specific example, you would be able to include information
    // about the page's ID and its view count (because the class implements
    // `Processor<PageId, Long>`).
  }

  @Override
  void punctuate(long timestamp) {
    // Stays empty.  In this use case there would be no need for a periodical
    // action of this processor.
  }

  @Override
  void close() {
    // Any code for clean up would go here.
    // This processor instance will not be used again after this call.
  }

}

As mentioned before, a stream processor can access any available state stores by calling ProcessorContext#getStateStore(). Only such state stores are available that have been named in the corresponding KStream#process() method call (note that this is a different method than Processor#process()).

Writing streams back to Kafka

Any streams may be (continuously) written back to a Kafka topic via KStream#to() and KTable#to().

// Write the stream userCountByRegion to the output topic 'RegionCountsTopic'
userCountByRegion.to("RegionCountsTopic");

If your application needs to continue reading and processing the records after they have been written to a topic via to() above, one option is to construct a new stream that reads from the output topic:

// Write to a Kafka topic.
userCountByRegion.to("RegionCountsTopic");

// Read from the same Kafka topic by constructing a new stream from the
// topic RegionCountsTopic, and then begin processing it (here: via `map`)
builder.stream("RegionCountsTopic").map(...)...;

Kafka Streams provides a convenience method called through() that is equivalent to the code above:

// `through` combines write-to-Kafka-topic and read-from-same-Kafka-topic operations
userCountByRegion.through("RegionCountsTopic").map(...)...;

Whenever data is read from or written to a Kafka topic, Streams must know the serdes to be used for the respective data records. By default the to() and through() methods use the default serdes defined in the Streams configuration. You can override these default serdes by passing explicit serdes to the to() and through() methods.

Tip

Besides writing the data back to Kafka, you can also apply a custom processor as a stream sink at the end of the processing (cf. above) to write to external databases and systems. If you do, 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).

Interactive Queries

Overview

Introduced in Kafka 0.10.1, interactive queries let you get more from streaming than just the processing of data. This feature allows you to treat the stream processing layer as a lightweight embedded database and, more concretely, to directly query the latest state of your stream processing application, without needing to materialize that state to external databases or external storage first.

As a result, interactive queries simplify the architecture of many use cases and lead to more application-centric architectures. For example, you often no longer need to operate and interface with a separate database cluster – or a separate infrastructure team in your company that runs that cluster – to share data between a Kafka Streams application (say, an event-driven microservice) and downstream applications, regardless of whether these applications use Kafka Streams or not; they may even be applications that do not run on the JVM, e.g. implemented in Python, C/C++, or JavaScript.

The following diagrams juxtapose two architectures: the first does not use interactive queries whereas the second architecture does. It depends on the concrete use case to determine which of these architectures is a better fit – the important takeaway is that Kafka Streams and interactive queries give you the flexibility to pick and to compose the right one, rather than limiting you to just a single way.

Tip

Best of both worlds: Of course you also have the option to run hybrid architectures where, for example, your application may be queried interactively but at the same time also shares some of its results with external systems (e.g. via Kafka Connect).

../_images/streams-interactive-queries-01.png

Without interactive queries: increased complexity and heavier footprint of architecture

../_images/streams-interactive-queries-02.png

With interactive queries: simplified, more application-centric architecture

Here are some use case examples for applications that benefit from interactive queries:

  • Real-time monitoring: A front-end dashboard that provides threat intelligence (e.g., web servers currently under attack by cyber criminals) can directly query a Kafka Streams application that continuously generates the relevant information by processing network telemetry data in real-time.
  • Video gaming: A Kafka Streams application continuously tracks location updates from players in the gaming universe. A mobile companion app can then directly query the Kafka Streams application to show the current location of a player to friends and family, and invite them to come along. Similarly, the game vendor can use the data to identify unusual hotspots of players, which may indicate a bug or an operational issue.
  • Risk and fraud: A Kafka Streams application continuously analyzes user transactions for anomalies and suspicious behavior. An online banking application can directly query the Kafka Streams application when a user logs in to deny access to those users that have been flagged as suspicious.
  • Trend detection: A Kafka Streams application continuously computes the latest top charts across music genres based on user listening behavior that is collected in real-time. Mobile or desktop applications of a music store can then interactively query for the latest charts while users are browsing the store.

Demo applications

Before we explain interactive queries in detail, let us point out that we provide end-to-end demo applications to get you started:

  • KafkaMusicExample: This application continuously computes the latest Top 5 music charts based on song play events collected in real-time in a Kafka topic. This charts data is maintained in a continuously updated state store that can be queried interactively via a REST API.
  • WordCountInteractiveQueriesExample: This application continuously counts the occurrences of words based on text data that is consumed in real-time from a Kafka topic. The word counts are maintained in a continuously updated state store that can be queried interactively via a REST API.

Once you have familiarized yourself with the concept of interactive queries by reading the following sections, you may want to get back to the examples above and use them as a starting point for your own applications.

Your application and interactive queries

Interactive queries allow you to tap into the state of your application, and notably to do that from outside your application. However, an application is not interactively queryable out of the box: you make it queryable by leveraging the API of Kafka Streams.

It is important to understand that the state of your application – to be extra clear, we might call it “the full state of the entire application” – is typically split across many distributed instances of your application, and thus across many state stores that are managed locally by these application instances.

../_images/streams-interactive-queries-03.png

Accordingly, the API to let you interactively query your application’s state has two parts, a local and a remote one:

  1. Querying local state stores (for an application instance): You can query that (part of the full) state that is managed locally by an instance of your application. Here, an application instance can directly query its own local state stores. You can thus use the corresponding (local) data in other parts of your application code that are not related to 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 ever be mutated by the corresponding processor topology and the input data it operates on.
  2. Querying remote state stores (for the entire application): To query the full state of your entire application we must be able to piece together the various local fragments of the state. In addition to being able to (a) query local state stores as described in the previous bullet point, we also need to (b) discover all the running instances of your application in the network, including their respective state stores and (c) have a way to communicate with these instances over the network, i.e. an RPC layer. Collectively, these building blocks enable intra-app communcation (between instances of the same app) as well as inter-app communication (from other applications) for interactive queries.
Required to access the state of ... an app instance (local state) the entire application (full state)
Query local state stores of an app instance Yes (provided) Yes (provided)
Make an app instance discoverable to others - Yes (provided)
Discover all running app instances and their state stores - Yes (provided)
Communicate with app instances over the network (RPC) - Yes, user is responsible

Kafka Streams provides all the required functionality for interactively querying your application’s state out of the box, with but one exception: if you want to expose your application’s full state via interactive queries, then – for reasons we explain further down below – it is your responsibility to add an appropriate RPC layer to your application that allows application instances to communicate over the network. If, however, you only need to let your application instances access their own local state, then you do not need to add such an RPC layer at all.

Querying local state stores (for an application instance)

Important

A Kafka Streams application is typically running on many 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, by definition, only return data locally available on that particular instance. We explain how to access data in state stores that are not locally available in section Querying remote state stores (for the entire application).

The method KafkaStreams#store(...) finds an application instance’s local state stores by name and by type.

../_images/streams-interactive-queries-api-01.png

Every application instance can directly query any of its local state stores.

The name of a state store is defined when you are creating the store, either when creating the store explicitly (e.g. when using the Processor API) or when creating the store implicitly (e.g. when using stateful operations in the DSL). We show examples of how to name a state store further down below.

The type of a state store is defined by QueryableStoreType, and you can access the built-in types via the class QueryableStoreTypes. Kafka Streams currently has two built-in types:

Both store types return read-only versions of the underlying state stores. This read-only constraint is important to guarantee that the underlying state stores will never be mutated (e.g. new entries added) out-of-band, i.e. only the corresponding processing topology of Kafka Streams is allowed to mutate and update the state stores in order to ensure data consistency.

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, which means your application will potentially manage many underlying state stores. The API to query local state stores enables you to query all of the underlying stores without having to know which partition the data is in. The objects returned from KafkaStreams#store(...) are therefore wrapping potentially many underlying state stores.

Querying local key-value stores

To query a local key-value store, you must first create a topology with a key-value store:

StreamsConfig config = ...;
KStreamBuilder 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, stringSerde, stringSerde);

// Create a key-value store named "CountsKeyValueStore" for the all-time word counts
groupedByWord.count("CountsKeyValueStore");

// Start an instance of the topology
KafkaStreams streams = new KafkaStreams(builder, config);
streams.start();

Above we created 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”. Once the application has started we 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 + ": " + 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 + ": " + value);
}

Querying local window stores

A window store differs from a key-value store in that you will potentially have many results for any given key because the key can be present in multiple windows. However, there will ever be at most one result per window for a given key.

To query a local window store, you must first create a topology with a window store:

StreamsConfig config = ...;
KStreamBuilder 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, stringSerde, stringSerde);

// Create a window state store named "CountsWindowStore" that contains the word counts for every minute
groupedByWord.count(TimeWindows.of(60000), "CountsWindowStore");

Above we created a window store named “CountsWindowStore” that contains the counts for words in 1-minute windows. Once the application has started we 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.
long timeFrom = 0; // beginning of time = oldest available
long timeTo = System.currentTimeMillis(); // 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

Custom state stores can only be used through the Processor API. They are not currently supported by the DSL.

Any custom state stores you use in your Kafka Streams applications can also be queried. However there are some interfaces that will need to be implemented first:

  1. Your custom state store must implement StateStore.
  2. You should have an interface to represent the operations available on the store.
  3. It is recommended that you also provide an interface that restricts access to read-only operations so users of this API can’t mutate the state of your running Kafka Streams application out-of-band.
  4. You also need to provide an implementation of StateStoreSupplier for creating instances of your store.

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 MyCustomStoreSupplier implements StateStoreSupplier {
  // implementation of the supplier for MyCustomStore
}

To make this store queryable you need to:

  • Provide an implementation of QueryableStoreType.
  • Provide a wrapper class that will have access to all of the underlying instances of the store and will be used for querying.

Implementing QueryableStoreType is straight forward:

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 even a single instance of a Kafka Streams application may run multiple stream tasks and, by doing so, manage multiple local instances of a particular state store. The wrapper class hides this complexity and lets you query a “logical” state store with a particular name without having to know about all of the underlying local instances of that state store.

When implementing your wrapper class you will need to make use of 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.

An example implemention of the wrapper follows (Java 8+):

// 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);
  }

}

Putting it all together you can now find and query your custom store:

StreamsConfig config = ...;
TopologyBuilder builder = ...;
ProcessorSupplier processorSuppler = ...;

// Create CustomStoreSupplier for store name the-custom-store
MyCustomStoreSuppler customStoreSupplier = new MyCustomStoreSupplier("the-custom-store");
// Add the source topic
builder.addSource("input", "inputTopic");
// Add a custom processor that reads from the source topic
builder.addProcessor("the-processor", processorSupplier, "input");
// Connect your custom state store to the custom processor above
builder.addStateStore(customStoreSupplier, "the-processor");

KafkaStreams streams = new KafkaStreams(builder, 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 application)

Typically, the ultimate goal for interactive queries is not to just query locally available state stores from within an instance of a Kafka Streams application as described in the previous section. Rather, you want to expose the application’s full state (i.e. the state across all its instances) to other applications that might be running on different machines. For example, you might have a Kafka Streams application that processes the user events in a multi-player video game, and you want to retrieve the latest status of each user directly from this application so that you can display it in a mobile companion app.

Three steps are needed to make the full state of your application queryable:

  1. You must add an RPC layer to your application so that the instances of your application may be interacted with via the network – notably to respond to interactive queries. By design Kafka Streams does not provide any such RPC functionality out of the box so that you can freely pick your favorite approach: a REST API, Thrift, a custom protocol, and so on. You can follow the reference examples we provide to get started with this (details further down below).
  2. You need to expose the respective 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 will have its own value for this configuration setting. This makes an application instance discoverable by other instances.
  3. In the RPC layer, you can then discover remote application instances and their respective state stores (e.g. for forwarding queries to other app instances if an instance lacks the local data to respond to a query) as well as query locally available state stores (in order to directly respond to queries) in order to make the full state of your application queryable.
../_images/streams-interactive-queries-api-02.png

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

As Kafka Streams doesn’t provide an RPC layer you are free to choose your favorite approach. There are many ways of doing this, and it will depend on the technologies you have chosen to use. 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.

The KafkaMusicExample and WordCountInteractiveQueriesExample are end-to-end demo applications for interactive queries that showcase the implementation of an RPC layer through a REST API.

Exposing the RPC endpoints of your application

To enable the remote discovery of state stores running within a (typically distributed) Kafka Streams application you need to set the application.server configuration property in StreamsConfig. 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. It’s important to understand that the value of this configuration property varies across the instances of your application. When this property is set, then, for every instance of an application, Kafka Streams will keep track of the instance’s RPC endpoint information, its state stores, and assigned stream partitions through instances of StreamsMetadata.

Tip

You may also consider leveraging the exposed RPC endpoints of your application for further functionality, such as piggybacking additional inter-application communication that go beyond interactive queries.

Below is an example of configuring and running 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 ...

StreamsConfig config = new StreamsConfig(props);
KStreamBuilder builder = new KStreamBuilder();

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, stringSerde, stringSerde);

// This call to `count()` creates a state store named "word-count".
groupedByWord.count("word-count");

// ... definition of the processing topology follows here ...

// Start an instance of the topology
KafkaStreams streams = new KafkaStreams(builder, streamsConfiguration);
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 respective local state stores

With the application.server property set, we can now find the locations of remote app instances and their 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 application
  • KafkaStreams#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 store
  • KafkaStreams#metadataForKey(String storeName, K key, StreamPartitioner<K, ?> partitioner): using partitioner, 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:

  • We can discover the running instances of the application as well as the state stores they manage locally.
  • Through the RPC layer that was added to the application, we 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.

Now, if you are interested in seeing how an end-to-end application with interactive queries looks like, then we recommend to take a look at the demo applications we provide.

Memory management

Record caches in the DSL

Developers of a Kafka Streams application using the DSL have the option to specify, for an instance of a processing topology, the total memory (RAM) size of the record cache that is used for:

  1. Internal caching and compacting of output records before they are written from a processor node to its state stores, if any.
  2. Internal caching and compacting of output records before they are forwarded from a processor node to downstream processor nodes, if any.

Here is a motivating example:

  • The input is a sequence of records <K,V>: <1, 1>, <88, 5>, <1, 20>, <1, 300> (Note: The focus in this example is on the records with key == 1.)
  • A processor node computes the sum of values, grouped by key, for the input above. Without caching, what is emitted for key 1 is a sequence of output records: <1, null>, <1, 1>, <1, 21>, <1, 321>.
  • With caching, all three records for key 1 would likely be compacted in cache, leading to a single output record <1, 321> that is written to the state store and being forwarded to any downstream processor nodes.

The cache size is specified through the cache.max.bytes.buffering parameter:

...
// Enable record cache of size 10 MB.
config.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. I.e., 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 once 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>, which we also refer to as “being compacted”. Note that this has the same effect as Kafka’s log compaction, but happens earlier while the records are still in memory. Upon flushing R2 is 1) forwarded to the next processing node and 2) 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 pressure hits. Both commit.interval.ms and cache.max.bytes.buffering are global parameters. They apply to all processor nodes in the topology, i.e., it is not possible to specify different parameters for each node. Below we provide some example settings for both parameters based on desired scenarios.

To turn off caching the cache size can be set to zero:

...
// Disable record cache
config.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);

Turning off caching might result in high write traffic for the underlying RocksDB store. With default settings caching is enabled within Kafka Streams but RocksDB caching is disabled. Thus, to avoid high write traffic it is recommended to enable RocksDB caching if Kafka Streams caching is turned off.

For example, RocksDB Block Cache could be set to 100MB and Write Buffer size to 32 MB. See RocksDB config.

To enable caching but still have an upper bound on how long records will be cached, the commit interval can be set appropriately (in this example, it is set to 1000 milliseconds):

...
// Enable record cache of size 10 MB.
config.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 10 * 1024 * 1024L);
// Set commit interval to 1 second.
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);

The illustration below shows the effect of these two configurations visually. For simplicity we have records with 4 keys: blue, red, yellow and green. Without loss of generality, let’s assume the cache has space for only 3 keys. When the cache is disabled, we observer that all the input records will be output. With the cache enabled, we make the following observations. First, most records are output at the end of a commit intervals (e.g., at t1 one blue records is output, which is the final over-write of the blue key up to that time). Second, some records are output because of cache pressure, i.e. before the end of a commit interval (cf. the red record right 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. Third, the number of records output has been reduced (here: from 15 to 8).

../_images/streams-cache-and-commit-interval.png

State store caches in the Processor API

Developers of a Kafka Streams application using the Processor API have the option to specify, for an instance of a processing topology, the total memory (RAM) size of the state store cache that is used for:

  1. Internal caching and compacting of output records before they are written from a processor node to its state stores, if any.
  2. Internal caching of output records before they are forwarded from a processor node to downstream processor nodes, if any. Note that, unlike record caches in the DSL, the state store cache in the Processor API will not compact any output records that are being forwarded downstream. In other words, downstream nodes see all records, whereas state stores see a reduced number of records. It is important to note that this does not impact correctness of the system but is merely a performance optimization for the state stores.

A note on terminology: we use the narrower term state store caches when we refer to the Processor API and the broader term record caches when we are writing about the DSL. We made a conscious choice to not expose the more general record caches to the Processor API so that we keep it simple and flexible. For example, developers of the Processor API might chose to store a record in a state store while forwarding a different value downstream, i.e., they might not want to use the unified record cache for both state store and forwarding downstream.

Following from the example first shown in section Defining a State Store, to enable caching, you can add the enableCaching call (note that caches are disabled by default and there is no explicit disableCaching call) :

StateStoreSupplier countStore = Stores.create("Counts")
              .withKeys(Serdes.String())
              .withValues(Serdes.Long())
              .persistent()
              .enableCaching()
              .build();

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 and fetch.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 by buffered.records.per.partition.
  • RocksDB’s own memory usage, both on-heap and off-heap; 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.

Running a Kafka Streams application

In this section we describe how you can launch your stream processing application, and how you can elasticly add capacity to and remove capacity from your application during runtime.

Starting a Kafka Streams application

A Java application that uses the Kafka Streams library can be run just like any other Java application – there is no special magic or requirement on the side of Kafka Streams.

For example, you can package your Java application as a fat jar file and then start the application via:

# 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

The Kafka Streams examples under https://github.com/confluentinc/examples demonstrate how you can package and start your application in this way. See the README and pom.xml for details.

It is important to understand that, when starting your application as described above, you are actually launching what Kafka Streams considers to be one instance of your application. More than one instance of your application may be running at a time, and in fact the common scenario is that there are indeed multiple instances of your application running in parallel. See Parallelism Model for further information.

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 state stores will also be (re-)constructed, if possible, during the initialization period of their associated stream tasks (for more details read 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 the runtime of your application, and you can do so without any downtime or data loss. This means that, unlike other stream processing technologies, with Kafka Streams you do not have to completely stop your application, recompile/reconfigure, and then restart. This is great not just for intentionally adding or removing processing capacity, but also for being resilient in the face of failures (e.g. machine crashes, network outages) and for allowing maintenance work (e.g. rolling upgrades).

If you are wondering how this elasticity is actually achieved behind the scenes, you may want to read the Architecture chapter, notably the Parallelism Model section. In a nutshell, Kafka Streams leverages existing functionality in Kafka, notably its group management functionality. This group management, which is built right into the Kafka wire protocol, is the foundation that enables the elasticity of Kafka Streams applications: members of a group will coordinate and collaborate jointly on the consumption and processing of data in Kafka. On top of this foundation Kafka Streams provides some additional functionality, e.g. to enable stateful processing and to allow for fault-tolerant state in environments where application instances may come and go at any time.

Adding capacity to your application (“expand”)

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.

../_images/streams-elastic-scaling-1.png

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.

../_images/streams-elastic-scaling-2.png

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 (“shrink”)

If you need less processing capacity for your stream processing application, you can simply stop one or more running instances of your stream processing application, e.g. shut down 2 of 4 running instances. The remaining instances of your application will become aware that other instances were stopped and automatically take over the processing work of the stopped instances. More specifically, what will be handed over from the stopped instances to the remaining instances is 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 will be re-created in the target instance by restoring the state from its corresponding changelog topic).

../_images/streams-elastic-scaling-3.png

If one of the application instances is stopped (e.g. intentional reduction of capacity, maintenance, machine failure), it will automatically leave the application’s consumer group, which causes the remaining instances to automatically take over the stopped instance’s processing work.

State restoration during workload rebalance

As mentioned above, when the processing workload is rebalanced among the existing application instances either due to scaling changes (e.g. adding capacity to the application) or due to unexpected failures, some stream tasks will be migrated from one instance to another. And when a task is migrated, the processing state of this task will be fully restored before the application instance can resume processing in order to guarantee correct processing results. In Kafka Streams, state restoration is usually done by replaying the corresponding changelog topic to reconstruct the state store; additionally, users can also specify num.standby.replicas to minimize changelog-based restoration latency with replicated local state stores (see Standby Replicas for more details). As a result, when the stream task is (re-)initialized on the application instance, its state store is restored in the following way:

  • If no local state store exists then replay the changelog from the earliest to the current offset. In doing so, the local state store is reconstructed to the most recent snapshot.
  • If a local state store exists then replay the changelog from the previously checkpointed offset. Apply the changes to restore the state state to the most recent snapshot. This will take less time as it is applying a smaller portion of the changelog.

How many application instances to run?

How many instances can or should you run for your application? Is there an upper limit for the number of instances and, similarly, for the parallelism of your application? In a nutshell, the parallelism of a Kafka Streams application – similar to the parallelism of Kafka – is primarily determined by the number of partitions of the input topic(s) from which your application is reading. For example, if your application reads from a single topic that has 10 partitions, then you can run up to 10 instances of your applications (note that 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 thus for the number of running instances of your application.

Tip

How to achieve a balanced processing workload across application instances to prevent processing hotpots: The balance of the processing work between application instances depends on factors such as how well data messages are balanced between partitions (think: if you have 2 topic partitions, having 1 million messages in each partition is better than having 2 million messages in the first partition and no messages in the second) and how much processing capacity is required to process the messages (think: if the time to process messages varies heavily, then it is better to spread the processing-intensive messages across partitions rather than storing these messages within the same partition).

If your data happens to be heavily skewed in the way described above, some application instances may become processing hotspots (say, when most messages would end up being stored in only 1 of 10 partitions, then the application instance that is processing this one partition would be performing most of the work while other instances might be idle). You can minimize the likelihood of such hotspots by ensuring better data balancing across partitions (i.e. minimizing data skew at the point in time when data is being written to the input topics in Kafka) and by over-partitioning the input topics (think: use 50 or 100 partitions instead of just 10), which lowers the probability that a small subset of partitions will end up storing most of the topic’s data.

Managing topics of a Kafka Streams application

A Kafka Streams application executes by continuously reading from some Kafka topics, processing the read data, and then writing process results back into Kafka topics. In addition, the application itself may also auto-create some other Kafka topics in the Kafka brokers such as state store changelogs topics. Therefore it is important for users to be able to understand the difference between these topics and understand how they could be managed along with Kafka Streams applications.

In Kafka Streams, we distinguish between user topics and internal topics. Both kinds are normal Kafka topics but they are used differently and, in the case of internal topics, follow a specific naming convention.

User topics: Topics that exist externally to a certain application that will be read or written by the application, including:

  • Input topics : topics that are specified via source processors in the application’s topology (e.g. KStreamBuilder#stream(), KStreamBuilder#table() and TopologyBuilder#addSource() methods).
  • Output topics : topics that are specified via sink processors in the application’s topology (e.g. KStream#to(), KTable.to() and TopologyBuilder#addSink() methods).
  • Intermediate topics : topics that are both input and output topics of the application’s topology (e.g., via the KStream#through() methods).

In practice, all user topics must be created and managed manually ahead of time (e.g., via the topic tools). Note that in some cases these topics may be shared among multiple applications to read from or write to, in which case application users need to coordinate on managing such topics; in some other cases these topics are managed in a centralized way (e.g., by the team who operates the Kafka broker clusters) and application users then would not need to manage topics themselves but simply obtain access to them.

Note

Auto-creation of topics is strongly discouraged: It is strongly recommended to NOT rely on the broker-side topic auto-creation feature to create user topics. First, auto-creation of topics may be disabled in your Kafka cluster. Second, auto-creation will always apply the default topic settings such as the replicaton factor, and these default settings might not be what you want for certain output topics (cf. auto.create.topics.enable=true in the Kafka broker configuration).

Internal topics: Topics that are used internally by the Kafka Streams application while executing, for example, the changelog topics for state stores. Such topics are created by the application under the hood, and exclusively used by that stream application only. If security is enabled on the Kafka brokers, users need to set the corresponding security configs to authorize the underlying clients with corresponding admin functionality to be able to create such topics (details can be found in section Security). Internal topics currently follow the naming convention <application.id>-<operatorName>-<suffix>, but this convention is not guaranteed for future releases.

Data types and serialization

Overview

Every Kafka Streams application must provide serializers and deserializers – abbreviated as serdes – for the data types of record keys and record values (e.g. java.lang.String or Avro objects) to materialize the data when necessary. Operations that require such serde information include: stream(), table(), to(), through(), groupByKey(), groupBy().

There are two ways to provide these serdes:

  1. By setting default serdes via a StreamsConfig instance.
  2. By specifying explicit serdes when calling the appropriate API methods, thus overriding the defaults.

Configuring default serializers/deserializers (serdes)

Serdes specified in the Streams configuration via StreamsConfig are used as the default in your Kafka Streams application.

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.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.VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass().getName());

StreamsConfig config = new StreamsConfig(settings);

Overriding default serializers/deserializers (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(stringSerde, longSerde, "RegionCountsTopic");

If you want to override serdes selectively, i.e., keep the defaults for some fields, then pass null whenever you want to leverage the default serde 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 passing `null`,
// but override the default serializer for record values (here: userCount as Long).
final Serde<Long> longSerde = Serdes.Long();
KStream<String, Long> userCountByRegion = ...;
userCountByRegion.to(null, longSerde, "RegionCountsTopic");

Available serializers/deserializers (serdes)

Apache Kafka includes several built-in serde implementations in its kafka-clients maven artifact:

<dependency>
    <groupId>org.apache.kafka</groupId>
    <artifactId>kafka-clients</artifactId>
    <version>0.10.1.1-cp1</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()

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.

You would use the built-in serdes as follows, using the example of the String serde:

// When configuring the default serdes of StreamConfig
Properties streamsConfiguration = new Properties();
streamsConfiguration.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG,   Serdes.String().getClass().getName());
streamsConfiguration.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());

// When you want to override serdes explicitly/selectively
final Serde<String> stringSerde = Serdes.String();
KStreamBuilder builder = new KStreamBuilder();
KStream<String, String> textLines = builder.stream(stringSerde, stringSerde, "TextLinesTopic");

The code examples of Kafka Streams also include a basic serde implementation for JSON:

Lastly, the Confluent examples repository includes basic serde implementations for Apache Avro:

As well as templated serde implementations:

Implementing custom serializers/deserializers (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:

  1. Write a serializer for your data type T by implementing org.apache.kafka.common.serialization.Serializer.
  2. Write a deserializer for T by implementing org.apache.kafka.common.serialization.Deserializer.
  3. 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 as Serdes.serdeFrom(Serializer<T>, Deserializer<T>).

Security

Overview

Kafka Streams can help you make your stream processing applications secure, and it achieves this through its native integration with Kafka’s security functionality. In general, Kafka Streams supports all the client-side security features in Kafka. The most important aspect to understand is that Kafka Streams leverages the standard Kafka producer and consumer clients behind the scenes. Hence what you need to do to secure your stream processing applications on the side of Kafka Streams is to configure the appropriate security settings of the corresponding Kafka producer/consumer clients. Once you know which client-side security features you want to use, you simply need to include the corresponding settings in the configuration of your Kafka Streams application.

In this section we will not cover these client-side security features in full detail, so we recommend reading the Kafka Security chapter (see also the Confluent blog post Apache Kafka Security 101) to familiarize yourself with the security features that are currently available in Apache Kafka. That said, let us highlight a couple of important client-side security features:

  1. Encrypting data-in-transit between Kafka brokers and Kafka clients: You can enable the encryption of the client-server communication between the Kafka brokers and Kafka clients. Kafka clients include stream processing applications built using the Kafka Streams library.
  • Example: You can configure your Kafka Streams applications to always use encryption when reading data from Kafka and when writing data to Kafka; this is very important when reading/writing data across security domains (e.g. internal network vs. public Internet or partner network).
  1. Client authentication: You can enable client authentication for connections from Kafka clients (including Kafka Streams) to Kafka brokers.
  • Example: You can define that only some specific Kafka Streams applications are allowed to connect to your production Kafka cluster.
  1. Client authorization: You can enable client authorization of read/write operations by Kafka clients.
  • Example: You can define that only some specific Kafka Streams applications are allowed to read from a Kafka topic that stores sensitive data. Similarly, you can restrict write access to certain Kafka topics to only a few stream processing applications to prevent e.g. data pollution or fraudulent activities.

Note

The aforementioned security features in Apache Kafka are optional, and it is up to you to decide whether to enable or disable any of them. And you can mix and match these security features as needed: both secured and non-secured Kafka clusters are supported, as well as a mix of authenticated, unauthenticated, encrypted and non-encrypted clients. This flexibility allows you to model the security functionality in Kafka to match your specific needs, and to make effective cost vs. benefit (read: security vs. convenience/agility) tradeoffs: tighter security requirements in places where security matters (e.g. production), and relaxed requirements in other situations (e.g. development, testing).

Security example

The following example is based on the Confluent blog post Apache Kafka Security 101. What we want to do is to configure a Kafka Streams application to A. enable client authentication and B. encrypt data-in-transit when communicating with its Kafka cluster.

Tip

A complete demo application including step-by-step instructions is available at SecureKafkaStreamsExample.java under https://github.com/confluentinc/examples.

Because we want to focus this example on the side of Kafka Streams, we assume that 1. the security setup of the Kafka brokers in the cluster is already completed, and 2. the necessary SSL certificates are available to the Kafka Streams application in the local filesystem locations specified below (the aforementioned blog post walks you through the steps to generate them); for example, if you are using Docker to containerize your Kafka Streams applications, then you must also include these SSL certificates in the right locations within the Docker image.

Once these two assumptions are met, you must only configure the corresponding settings for the Kafka clients in your Kafka Streams application. The configuration snippet below shows the settings to A. enable client authentication and B. enable SSL encryption for data-in-transit between your Kafka Streams application and the Kafka cluster it is reading from and writing to:

# 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

Within a Kafka Streams application, you’d use code such as the following to configure these settings in your StreamsConfig instance (see section Configuring a Kafka Streams application):

// 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");
StreamsConfig streamsConfiguration = new StreamsConfig(settings);

With these settings in place your Kafka Streams application will encrypt any data-in-transit that is being read from or written to Kafka, and it will also authenticate itself against the Kafka brokers that it is talking to. (Note that this simple example does not cover client authorization.)

When things go wrong (e.g. wrong password): What would happen if you misconfigured the security settings in your Kafka Streams application? In this case, the application would fail at runtime, typically right after you started it. For example, if you entered an incorrect password for the ssl.keystore.password setting, then error messages similar to the following would be logged, and after that 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

Tip

You can monitor the application log files of your Kafka Streams applications for such error messages to spot any misconfigured applications quickly.

Application Reset Tool

Overview

The Application Reset Tool allows you to quickly reset an application in order to reprocess its data from scratch – think: an application “reset” button.

Scenarios when would you like to reset an application include:

  • Development and testing
  • Addressing bugs in production
  • Demos

However, resetting an application manually is a bit involved. Kafka Streams is designed to hide many details about operator state, fault-tolerance, and internal topic management from the user (especially when using Kafka Streams DSL). In general this hiding of details is very desirable but it also makes manually resetting an application more difficult. The Application Reset Tool fills this gap and allows you to quickly reset an application.

Scope

As mentioned in section Managing topics of a Kafka Streams application, there are two different categories of topics in Kafka Streams: user topics (input, output, and intermediate), and internal topics. The reset tool treats these topics differently when resetting the application.

What the application reset tool does:

  • For any specified input topics:
    • Reset to the beginning of the topic, i.e., set the application’s committed consumer offsets for all partitions to each partition’s earliest offset (for consumer group application.id).
  • For any specified intermediate topics:
    • Skip to the end of the topic, i.e., set the application’s committed consumer offsets for all partitions to each partition’s logSize (for consumer group application.id).
  • For any internal topics:
    • Reset the application’s committed consumer offsets to zero for all partitions (for consumer group application.id).
    • Delete the internal topic.

What the application reset tool does not:

  • Does not reset output topics of an application. If any output (or intermediate) topics are consumed by downstream applications, it is your responsibility to adjust those downstream applications as appropriate when you reset the upstream application.
  • Does not 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.

Usage

Step 1: Run the application reset tool

Attention

Use this tool only if the application is fully stopped: You must use this script only if no instances of your application are running. Otherwise the application may enter an invalid state, crash, or produce incorrect results. You can check if the consumer group with ID application.id is still active using bin/kafka-consumer-groups.

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. specifying the wrong input topics for the application), this tool might invalidate the application’s state or even impact other applications, consumer groups, or Kafka topics of your Kafka cluster.

You can call the application reset tool on the command line via bin/kafka-streams-application-reset.

The tool accepts the following parameters:

Option (* = required)         Description
---------------------         -----------
* --application-id <id>       The Kafka Streams application ID (application.id)
--bootstrap-servers <urls>    Comma-separated list of broker urls with format: HOST1:PORT1,HOST2:PORT2
                                (default: localhost:9092)
--intermediate-topics <list>  Comma-separated list of intermediate user topics
--input-topics <list>         Comma-separated list of user input topics
--zookeeper <url>             Format: HOST:POST
                                (default: localhost:2181)

You can combine the parameters of the script as needed. For example, if an application should only restart from an empty internal state but not reprocess previous data, simply omit the parameters --input-topics and --intermediate-topics.

Tip

On intermediate topics: In general, we recommend to manually delete and re-create any intermediate topics before running the application reset tool. This allows to free disk space in Kafka brokers early on. It is important to first delete and re-create intermediate topics before running the application reset tool.

Not deleting intermediate topics and only using the application reset tool is preferable:

  • when there are external downstream consumers for the application’s intermediate topics
  • during development, where manually deleting and re-creating intermediate topics might be cumbersome and often unnecessary

Step 2: Reset the local environments of your application instances

Running the application reset tool (step 1) ensures that your application’s state – as tracked globally in the application’s configured Kafka cluster – is reset. However, by design the reset tool does not modify or reset the local environment of your application instances, which includes the application’s local state directory.

For a complete application reset you must also delete the application’s local state directory on any machines on which an application instance was run prior to restarting an application instance on the same machines. You can either use the API method KafkaStreams#cleanUp() in your application code or manually delete the corresponding local state directory (default location: /var/lib/kafka-streams/<application.id>, cf. state.dir configuration parameter).

Example

Let’s imagine you are developing and testing an application locally and want to iteratively improve your application via run-reset-modify cycles. You might have code such as the following:

package io.confluent.examples.streams;

import ...;

public class ResetDemo {

  public static void main(String[] args) throws Exception {
    // Kafka Streams configuration
    Properties streamsConfiguration = new Properties();
    streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "my-streams-app");
    // ...and so on...

    // Define the processing topology
    KStreamBuilder builder = new KStreamBuilder();
    builder.stream("my-input-topic")
        .selectKey(...)
        .through("rekeyed-topic")
        .countByKey("global-count")
        .to("my-output-topic");

    KafkaStreams app = new KafkaStreams(builder, streamsConfiguration);

    // Delete the application's local state.
    // Note: In real application you'd call `cleanUp()` only under
    // certain conditions.  See tip on `cleanUp()` below.
    app.cleanUp();

    app.start();

    // Note: In real applications you would register a shutdown hook
    // that would trigger the call to `app.close()` rather than
    // using the sleep-then-close example we show here.
    Thread.sleep(30 * 1000L);
    app.close();
  }

}

Tip

Calling cleanUp() is safe but do so judiciously: It is always safe to call KafkaStreams#cleanUp() because the local state of an application instance can be recovered from the underlying internal changelog topic(s). However, to avoid the corresponding recovery overhead it is recommended to not call cleanUp() unconditionally and every time an application instance is restarted/resumed. A production application would therefore use e.g. command line arguments to enable/disable the cleanUp() call as needed.

You can then perform run-reset-modify cycles as follows:

# Run your application
$ bin/kafka-run-class io.confluent.examples.streams.ResetDemo

# After stopping all application instances, reset the application
$ bin/kafka-streams-application-reset --application-id my-streams-app \
                                      --input-topics my-input-topic \
                                      --intermediate-topics rekeyed-topic

# Now you can modify/recompile as needed and then re-run the application again.
# You can also experiment, for example, with different input data without
# modifying the application.