This page collects common usage pattern on how to use Kafka Streams Processor API or DSL.
Feel free to add your own code snippets and/or improve existing examples.
How to compute an (windowed) average?
Kafka Streams DSL aggregation natively support so-called "incremental" or "cumulative" aggregation functions (cf, https://en.wikipedia.org/wiki/Associative_property – also called "distributive functions" by Jim Gray in "Data Cube: A Relational Aggregation Operator Generalizing Group-By, Cross-Tab, and Sub-Totals") like count, sum, min, max. However, average function (a so-called "algebraic" function, cf. "Data Cube: A Relational Aggregation Operator Generalizing Group-By, Cross-Tab, and Sub-Totals") cannot be computed like this, but it can be composed of distributive functions, namely count and sum. Thus, it can be implemented in a two step approach:
class Tuple2<T1, T2> { public T1 value1; public T2 value2; Tuple2(T1 v1, T2 v2) { value1 = v1; value2 = v2; } } final KStreamBuilder builder = new KStreamBuilder(); // first step: compute count and sum in a single aggregation step and emit 2-tuples <count,sum> as aggregation result values final KTable<String,Tuple2<Long,Long>> countAndSum = builder.stream("someInputTopic") .groupByKey() .aggregate( new Initializer<Tuple2<Long, Long>>() { @Override public Tuple2<Long, Long> apply() { return new Tuple2<>(0L, 0L); } }, new Aggregator<String, String, Tuple2<Long, Long>>() { @Override public Tuple2<Long, Long> apply(final String key, final Long value, final Tuple2<Long, Long> aggregate) { ++aggregate.value1; aggregate.value2 += value; return aggregate; } }, new Tuple2Serde()); // omitted for brevity // second step: compute average for each 2-tuple final KTable<String,Double> average = countAndSum.mapValues( new ValueMapper<Tuple2<Long, Long>, Double>() { @Override public Double apply(Tuple2<Long, Long> value) { return value.value2 / (double) value.value1; } });
This pattern can also be applied to compute a windowed average or to compose other algebraic functions.
How to compute windowed aggregations over successively increasing timed windows?
Status: Draft TODO: to mitigate infinite state store growth (until re-balance rebuilds it from the changelog) you can implement the Windowed key serde to store the timestamp(s) before the actual record key and periodically do a ranged query on each of the state stores to find and delete all data older than x (using punctuate() inside a Processor). TBC...KTable<Windowed<Key>, Value> oneMinuteWindowed = // where Key and Value stand for your actual key and value types
yourKStream
.groupByKey()
.reduce(/*your adder*/, TimeWindows.of(60*1000, 60*1000), "store1m");
//where your adder can be as simple as (val, agg) -> agg + val
//for primitive types or as complex as you need
KTable<Windowed<Key>, Value> fiveMinuteWindowed =
oneMinuteWindowed
.groupBy( (windowedKey, value) ->
new KeyValue<>(
new Windowed<>(
windowedKey.key(),
new Window<>(
windowedKey.window().start() /1000/60/5 *1000*60*5,
windowedKey.window().start() /1000/60/5 *1000*60*5 + 1000*60*5
// the above rounds time down to a timestamp divisible by 5 minutes
)
),
value
),
/* your key serde */,
/* your value serde */
)
.reduce(/*your adder*/, /*your subtractor*/, "store5m");
// where your subtractor can be as simple as (val, agg) -> agg - val for primitive types
// or as complex as you need,
// just make sure you get the parameter order right, subtraction is not commutative!
KTable<Windowed<Key>, Value> fifteenMinuteWindowed =
fiveMinuteWindowed
.groupBy( (windowedKey, value) ->
new KeyValue<>(
new Windowed<>(
windowedKey.key(),
new Window<>(
windowedKey.window().start() /1000/60/15 *1000*60*15,
windowedKey.window().start() /1000/60/15 *1000*60*15 + 1000*60*15
// the above rounds time down to a timestamp divisible by 15 minutes
)
),
value
),
/* your key serde */,
/* your value serde */
)
.reduce(/*your adder*/, /*your subtractor*/, "store15m");
KTable<Windowed<Key>, Value> sixtyMinuteWindowed =
fifteeenMinuteWindowed
.groupBy( (windowedKey, value) ->
new KeyValue<>(
new Windowed<>(
windowedKey.key(),
new Window<>(
windowedKey.window().start() /1000/60/60 *1000*60*60,
windowedKey.window().start() /1000/60/60 *1000*60*60 + 1000*60*60
// the above rounds time down to a timestamp divisible by 60 minutes
)
),
value
),
/* your key serde */,
/* your value serde */
)
.reduce(/*your adder*/, /*your subtractor*/, "store60m");
How to aggregate data from all currently active sessions?
The idea behind the % PARTITION_COUNT_FOR_TOTALS bit is that I want to first do summation with max parallelism and minimize the work needed downstream. So I calculate a per-partition sum first to limit the updates that the totals topic will receive and the summing work done by the interactive queries on the global store. builder
.stream(/*key serde*/, /*transaction serde*/, "transaciton-topic")
.groupByKey(/*key serde*/, /*transaction serde*/)
.aggregate(
() -> /*empty aggregate*/,
aggregator(),
merger(),
SessionWindows.with(SESSION_TIMEOUT_MS).until(SESSION_TIMEOUT_MS*2),
/* aggregate serde */,
txPerCustomerSumStore() // this store can be queried for per customer session data )
.toStream()
.filter(((key, value) -> value != null)) // tombstones only come when a session is merged into a bigger session, so ignore them
// the below map/groupByKey/reduce operations are to only propagate updates to the latest session per customer to downstream
.map((windowedCustomerId, agg) -> // this moves timestamp from the windowed key into the value
// so that we can group by customerId only and reduce to the later value
new KeyValue<>(
windowedCustomerId.key(), // just customerId
new WindowedAggsImpl( // this is just like a tuple2 but with nicely named accessors: timestamp() and aggs()
windowedCustomerId.window().end(),
agg
)
)
)
.groupByKey( /*key serde*/, /*windowed aggs serde*/ ) // key is just customerId
.reduce( // take later session value and ignore any older - downstream only cares about current sessions
(val, agg) -> val.timestamp() > agg.timestamp() ? val : agg,
TimeWindows.of(SESSION_TIMEOUT_MS).advanceBy(SESSION_TIMOUT_DELAY_TOLERANCE_MS),
"latest-session-windowed"
)
.groupBy((windowedCustomerId, timeAndAggs) -> // calculate totals with maximum granularity, which is per-partition
new KeyValue<>(
new Windowed<>(
windowedCustomerId.key().hashCode() % PARTITION_COUNT_FOR_TOTALS, // KIP-159 would come in handy here, to access partition number instead
windowedCustomerId.window() // will use this in the interactive queries to pick the oldest not-yet-expired window
),
timeAndAggs.aggs()
),
new SessionKeySerde<>(Serdes.Integer()),
/* aggregate serde */
)
.reduce(
(val, agg) -> agg.add(val),
(val, agg) -> agg.subtract(val),
txTotalsStore() // this store can be queried to get totals per partition for all active sessions
);
builder.globalTable(
new SessionKeySerde<>(Serdes.Integer()),
/* aggregate serde */,
changelogTopicForStore(TRANSACTION_TOTALS), "totals");
// this global table puts per partition totals on every node, so that they can be easily summed for global totals, picking the oldest not-yet-expired window
TODO: put in StreamParitioners (with KTable.through variants added in KAFKA-5045) to avoid re-partitioning where I know it's unnecessary.
How to manage the size of state stores using tombstones?
An application that uses aggregations can make better use of it's resources by removing records it no longer needs from its state stores. Kafka Streams makes this possible through the usage of tombstone records, which are records that contain a non-null key, and a null value. When Kafka Streams sees a tombstone record, it deletes the corresponding key from the state store, thus freeing up space.
The usage of tombstone records will become apparent in the example below, but it's important to note that any record with a null key will be internally dropped, and will not be seen by your aggregation. Therefore, it is necessary that your aggregation include the logic for recognizing when a record can be dropped from the state store, and by returning null when this condition is met.
Consider the following example. An airline wants to track the various stages of a customer's flight. For this example, a customer can be in one of 4 stages: booked, boarded, landed, and post-flight survey completed. Once the customer has completed the post-flight survey, the airline no longer needs to track the customer. Until then, the airline would like to know what stage the customer is in, and perform various aggregations on the customer's data. This can be accomplished using the following topology.
// customer flight statuses final String BOOKED = "booked"; final String BOARDED = "boarded"; final String LANDED = "landed"; final String COMPLETED_FLIGHT_SURVEY = "survey"; // topics final String SOURCE_TOPIC = "someInputTopic"; final String STORE_NAME = "someStateStore"; // topology KStreamBuilder builder = new KStreamBuilder(); KStream<String, String> stream = builder.stream(SOURCE_TOPIC); KTable<String, String> customerFlightStatus = stream .groupByKey() .reduce( new Reducer<String>() { @Override public String apply(String value1, String value2) { if (value2.equals(COMPLETED_FLIGHT_SURVEY)) { // we no longer need to keep track of this customer since // they completed the flight survey. Create a tombstone return null; } // keeping this simple for brevity return value2; } }, STORE_NAME);
Returning null in the Reducer, but only when the customer has completed their post-flight survey, allows us to perform aggregations until we no longer are interested in tracking this key. A tombstone is created by returning null, and the record is deleted from the state store immediately. We can verify this with the following tests:
// Simple test final String customerFlightNumber = "customer123_dl90210"; File stateDir = createStateDir(); // implementation omitted KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, Serdes.String(), Serdes.String()); driver.setTime(0L); // get a reference to the state store KeyValueStore<String, String> store = (KeyValueStore<String, String>) driver.context().getStateStore(STORE_NAME); // the customer has booked their flight driver.process(SOURCE_TOPIC, customerFlightNumber, BOOKED); store.flush(); assertEquals(BOOKED, store.get(customerFlightNumber)); // the customer has boarded their flight driver.process(SOURCE_TOPIC, customerFlightNumber, BOARDED); store.flush(); assertEquals(BOARDED, store.get(customerFlightNumber)); // the customer's flight has landed driver.process(SOURCE_TOPIC, customerFlightNumber, LANDED); store.flush(); assertEquals(LANDED, store.get(customerFlightNumber)); // the customer has filled out the post-flight survey, so we no longer need to track them // in the state store. make sure the key was deleted driver.process(SOURCE_TOPIC, customerFlightNumber, COMPLETED_FLIGHT_SURVEY); store.flush(); assertEquals(null, store.get(customerFlightNumber));
Finally, it's important to note how tombstones are forwarded downstream. Whether or a not a tombstone is visible to additional sub-topologies depends on which abstraction (e.g. KTable or KStream) a sub-topology uses for streaming it's input data. The following code snippets highlight these differences. Tombstones are visible in record streams, and it is common to filter them out before performing additional transformations (see below). However, in changelog streams, the tombstones are forwarded directly, and not visible when using operators like filter, mapValues, etc.
// tombstones are visible in a KStream and must be filtered out KStream<String, String> subtopologyStream = customerFlightStatus .toStream() .filter((key, value) -> { if (value == null) { // tombstone! skip this return false; } // other filtering conditions... return true; }); // tombstone forwarding is different in KTables. The filter below is not evaluated for a tombstone KTable<String, String> subtopologyKtable = customerFlightStatus .filter((key, value) -> { // the tombstone never makes it here. no need to check for null // other filtering conditions... return true; });
How to purge data from KTables based on age
I'm not certain I would recommend this in general, but I've been asked to recommend a pattern for effectively implementing a TTL in a KTable. In principle, this could be done straightforwardly with a custom state store, but it raises questions about the integrity of the data provenance and the soundness of the application.
If the data in question is truly not needed anymore after 24 hours (or whatever other criteria), I think a better approach is to emit tombstones into the topic that populates the KTable in question. This circumvents a lot of tricky distributed systems questions.
Even better than this would be to purge the data from the source system and let those deletes naturally propagate into the topic and then into the KTable, but there are some special cases in which this isn't practical.
For example, I have seen one application that populates a keyed topic from a daily feed rather than a database's changelog. The feed only contains records that exist, records that have been deleted from the prior feed are simply not mentioned. Thus, there's no opportunity for the ingest to emit tombstones into the topic. One approach would be to effectively diff the current and prior feeds to identify records that have been deleted. But depending on the size and complexity of the feed, this might not be so simple.
In contrast, we can separate the concern of purging old data into a the following Streams application, intended to be run independently for each topic that needs purging. It simply watches the topic for records that have not been updated in a configured threshold of time and purges them from the topic by writing a tombstone back to it. Thus, the ingest job can just naively reflect the latest feed into the topic and all consumers can just consume the topic naively as well, and "forgotten" records will be purged from the topic by this job.
A refinement on this approach would be to use some identifying characteristic of the feed itself as a "generation" number and then tombstoning records that are not of the current generation, rather than using the record timestamp and age as the determining factor.
So, here you go: the comments should explain everything: