You can not select more than 25 topics
Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
3488 lines
383 KiB
3488 lines
383 KiB
<!-- |
|
Licensed to the Apache Software Foundation (ASF) under one or more |
|
contributor license agreements. See the NOTICE file distributed with |
|
this work for additional information regarding copyright ownership. |
|
The ASF licenses this file to You under the Apache License, Version 2.0 |
|
(the "License"); you may not use this file except in compliance with |
|
the License. You may obtain a copy of the License at |
|
|
|
http://www.apache.org/licenses/LICENSE-2.0 |
|
|
|
Unless required by applicable law or agreed to in writing, software |
|
distributed under the License is distributed on an "AS IS" BASIS, |
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
See the License for the specific language governing permissions and |
|
limitations under the License. |
|
--> |
|
|
|
<script><!--#include virtual="../../js/templateData.js" --></script> |
|
|
|
<script id="content-template" type="text/x-handlebars-template"> |
|
<!-- h1>Developer Guide for Kafka Streams</h1 --> |
|
<!-- div class="sub-nav-sticky"> |
|
<div class="sticky-top"> |
|
<div style="height:35px"> |
|
<a href="/{{version}}/documentation/streams/">Introduction</a> |
|
<a class="active-menu-item" href="/{{version}}/documentation/streams/developer-guide">Developer Guide</a> |
|
<a href="/{{version}}/documentation/streams/core-concepts">Concepts</a> |
|
<a href="/{{version}}/documentation/streams/quickstart">Run Demo App</a> |
|
<a href="/{{version}}/documentation/streams/tutorial">Tutorial: Write App</a> |
|
</div> |
|
</div --> |
|
</div> |
|
|
|
|
|
<div class="section" id="streams-dsl"> |
|
<span id="streams-developer-guide-dsl"></span><h1>Streams DSL<a class="headerlink" href="#streams-dsl" title="Permalink to this headline"></a></h1> |
|
<p>The Kafka Streams DSL (Domain Specific Language) is built on top of the Streams Processor API. It is the recommended for |
|
most users, especially beginners. Most data processing operations can be expressed in just a few lines of DSL code.</p> |
|
<div class="contents local topic" id="table-of-contents"> |
|
<p class="topic-title first"><b>Table of Contents</b></p> |
|
<ul class="simple"> |
|
<li><a class="reference internal" href="#overview" id="id7">Overview</a></li> |
|
<li><a class="reference internal" href="#creating-source-streams-from-kafka" id="id8">Creating source streams from Kafka</a></li> |
|
<li><a class="reference internal" href="#transform-a-stream" id="id9">Transform a stream</a><ul> |
|
<li><a class="reference internal" href="#stateless-transformations" id="id10">Stateless transformations</a></li> |
|
<li><a class="reference internal" href="#stateful-transformations" id="id11">Stateful transformations</a><ul> |
|
<li><a class="reference internal" href="#aggregating" id="id12">Aggregating</a></li> |
|
<li><a class="reference internal" href="#joining" id="id13">Joining</a><ul> |
|
<li><a class="reference internal" href="#join-co-partitioning-requirements" id="id14">Join co-partitioning requirements</a></li> |
|
<li><a class="reference internal" href="#kstream-kstream-join" id="id15">KStream-KStream Join</a></li> |
|
<li><a class="reference internal" href="#ktable-ktable-join" id="id16">KTable-KTable Join</a></li> |
|
<li><a class="reference internal" href="#kstream-ktable-join" id="id17">KStream-KTable Join</a></li> |
|
<li><a class="reference internal" href="#kstream-globalktable-join" id="id18">KStream-GlobalKTable Join</a></li> |
|
</ul> |
|
</li> |
|
<li><a class="reference internal" href="#windowing" id="id19">Windowing</a><ul> |
|
<li><a class="reference internal" href="#tumbling-time-windows" id="id20">Tumbling time windows</a></li> |
|
<li><a class="reference internal" href="#hopping-time-windows" id="id21">Hopping time windows</a></li> |
|
<li><a class="reference internal" href="#sliding-time-windows" id="id22">Sliding time windows</a></li> |
|
<li><a class="reference internal" href="#session-windows" id="id23">Session Windows</a></li> |
|
</ul> |
|
</li> |
|
</ul> |
|
</li> |
|
<li><a class="reference internal" href="#applying-processors-and-transformers-processor-api-integration" id="id24">Applying processors and transformers (Processor API integration)</a></li> |
|
</ul> |
|
</li> |
|
<li><a class="reference internal" href="#writing-streams-back-to-kafka" id="id25">Writing streams back to Kafka</a></li> |
|
<li><a class="reference internal" href="#testing-a-streams-app" id="id26">Testing a Streams application</a></li> |
|
<li><a class="reference internal" href="#scala-dsl" id="id27">Kafka Streams DSL for Scala</a></li> |
|
<ul> |
|
<li><a class="reference internal" href="#scala-dsl-sample-usage" id="id28">Sample Usage</a></li> |
|
<li><a class="reference internal" href="#scala-dsl-implicit-serdes" id="id29">Implicit SerDes</a></li> |
|
<li><a class="reference internal" href="#scala-dsl-user-defined-serdes" id="id30">User-Defined SerDes</a></li> |
|
</ul> |
|
</ul> |
|
</div> |
|
<div class="section" id="overview"> |
|
<h2><a class="toc-backref" href="#id7">Overview</a><a class="headerlink" href="#overview" title="Permalink to this headline"></a></h2> |
|
<p>In comparison to the <a class="reference internal" href="processor-api.html#streams-developer-guide-processor-api"><span class="std std-ref">Processor API</span></a>, only the DSL supports:</p> |
|
<ul class="simple"> |
|
<li>Built-in abstractions for <a class="reference internal" href="../core-concepts.html#streams_concepts_duality"><span class="std std-ref">streams and tables</span></a> in the form of |
|
<a class="reference internal" href="#streams_concepts_kstream"><span class="std std-ref">KStream</span></a>, <a class="reference internal" href="#streams_concepts_ktable"><span class="std std-ref">KTable</span></a>, and |
|
<a class="reference internal" href="#streams_concepts_globalktable"><span class="std std-ref">GlobalKTable</span></a>. Having first-class support for streams and tables is crucial |
|
because, in practice, most use cases require not just either streams or databases/tables, but a combination of both. |
|
For example, if your use case is to create a customer 360-degree view that is updated in real-time, what your |
|
application will be doing is transforming many input <em>streams</em> of customer-related events into an output <em>table</em> |
|
that contains a continuously updated 360-degree view of your customers.</li> |
|
<li>Declarative, functional programming style with |
|
<a class="reference internal" href="#streams-developer-guide-dsl-transformations-stateless"><span class="std std-ref">stateless transformations</span></a> (e.g. <code class="docutils literal"><span class="pre">map</span></code> and <code class="docutils literal"><span class="pre">filter</span></code>) |
|
as well as <a class="reference internal" href="#streams-developer-guide-dsl-transformations-stateful"><span class="std std-ref">stateful transformations</span></a> |
|
such as <a class="reference internal" href="#streams-developer-guide-dsl-aggregating"><span class="std std-ref">aggregations</span></a> (e.g. <code class="docutils literal"><span class="pre">count</span></code> and <code class="docutils literal"><span class="pre">reduce</span></code>), |
|
<a class="reference internal" href="#streams-developer-guide-dsl-joins"><span class="std std-ref">joins</span></a> (e.g. <code class="docutils literal"><span class="pre">leftJoin</span></code>), and |
|
<a class="reference internal" href="#streams-developer-guide-dsl-windowing"><span class="std std-ref">windowing</span></a> (e.g. <a class="reference internal" href="#windowing-session"><span class="std std-ref">session windows</span></a>).</li> |
|
</ul> |
|
<p>With the DSL, you can define <a class="reference internal" href="../core-concepts.html#streams_topology"><span class="std std-ref">processor topologies</span></a> (i.e., the logical |
|
processing plan) in your application. The steps to accomplish this are:</p> |
|
<ol class="arabic simple"> |
|
<li>Specify <a class="reference internal" href="#streams-developer-guide-dsl-sources"><span class="std std-ref">one or more input streams that are read from Kafka topics</span></a>.</li> |
|
<li>Compose <a class="reference internal" href="#streams-developer-guide-dsl-transformations"><span class="std std-ref">transformations</span></a> on these streams.</li> |
|
<li>Write the <a class="reference internal" href="#streams-developer-guide-dsl-destinations"><span class="std std-ref">resulting output streams back to Kafka topics</span></a>, or expose the processing results of your application directly to other applications through <a class="reference internal" href="interactive-queries.html#streams-developer-guide-interactive-queries"><span class="std std-ref">interactive queries</span></a> (e.g., via a REST API).</li> |
|
</ol> |
|
<p>After the application is run, the defined processor topologies are continuously executed (i.e., the processing plan is put into |
|
action). A step-by-step guide for writing a stream processing application using the DSL is provided below.</p> |
|
<p>For a complete list of available API functionality, see also the <a href="../../../javadoc/org/apache/kafka/streams/package-summary.html">Streams</a> API docs.</p> |
|
</div> |
|
|
|
<div class="section" id="dsl-core-constructs-overview"> |
|
<h4><a id="streams_concepts_kstream" href="#streams_concepts_kstream">KStream</a></h4> |
|
|
|
<p> |
|
Only the <strong>Kafka Streams DSL</strong> has the notion of a <code>KStream</code>. |
|
</p> |
|
|
|
<p> |
|
A <strong>KStream</strong> is an abstraction of a <strong>record stream</strong>, where each data record represents a self-contained datum in the unbounded data set. Using the table analogy, data records in a record stream are always interpreted as an "INSERT" -- think: adding more entries to an append-only ledger -- because no record replaces an existing row with the same key. Examples are a credit card transaction, a page view event, or a server log entry. |
|
</p> |
|
|
|
<p> |
|
To illustrate, let's imagine the following two data records are being sent to the stream: |
|
</p> |
|
|
|
<div class="sourcecode"> |
|
<p>("alice", 1) --> ("alice", 3)</p> |
|
</div> |
|
|
|
<p> |
|
If your stream processing application were to sum the values per user, it would return <code>4</code> for <code>alice</code>. Why? Because the second data record would not be considered an update of the previous record. Compare this behavior of KStream to <code>KTable</code> below, |
|
which would return <code>3</code> for <code>alice</code>. |
|
</p> |
|
|
|
<h4><a id="streams_concepts_ktable" href="#streams_concepts_ktable">KTable</a></h4> |
|
|
|
<p> |
|
Only the <strong>Kafka Streams DSL</strong> has the notion of a <code>KTable</code>. |
|
</p> |
|
|
|
<p> |
|
A <strong>KTable</strong> is an abstraction of a <strong>changelog stream</strong>, where each data record represents an update. More precisely, the value in a data record is interpreted as an "UPDATE" of the last value for the same record key, if any (if a corresponding key doesn't exist yet, the update will be considered an INSERT). Using the table analogy, a data record in a changelog stream is interpreted as an UPSERT aka INSERT/UPDATE because any existing row with the same key is overwritten. Also, <code>null</code> values are interpreted in a special way: a record with a <code>null</code> value represents a "DELETE" or tombstone for the record's key. |
|
</p> |
|
<p> |
|
To illustrate, let's imagine the following two data records are being sent to the stream: |
|
</p> |
|
|
|
<div class="sourcecode"> |
|
<p> |
|
("alice", 1) --> ("alice", 3) |
|
</p> |
|
</div> |
|
|
|
<p> |
|
If your stream processing application were to sum the values per user, it would return <code>3</code> for <code>alice</code>. Why? Because the second data record would be considered an update of the previous record. |
|
</p> |
|
|
|
<p> |
|
<strong>Effects of Kafka's log compaction:</strong> Another way of thinking about KStream and KTable is as follows: If you were to store a KTable into a Kafka topic, you'd probably want to enable Kafka's <a href="http://kafka.apache.org/documentation.html#compaction">log compaction</a> feature, e.g. to save storage space. |
|
</p> |
|
|
|
<p> |
|
However, it would not be safe to enable log compaction in the case of a KStream because, as soon as log compaction would begin purging older data records of the same key, it would break the semantics of the data. To pick up the illustration example again, you'd suddenly get a <code>3</code> for <code>alice</code> instead of a <code>4</code> because log compaction would have removed the <code>("alice", 1)</code> data record. Hence log compaction is perfectly safe for a KTable (changelog stream) but it is a mistake for a KStream (record stream). |
|
</p> |
|
|
|
<p> |
|
We have already seen an example of a changelog stream in the section <strong>streams_concepts_duality</strong>. Another example are change data capture (CDC) records in the changelog of a relational database, representing which row in a database table was inserted, updated, or deleted. |
|
</p> |
|
|
|
<p> |
|
KTable also provides an ability to look up <em>current</em> values of data records by keys. This table-lookup functionality is available through <strong>join operations</strong> (see also <strong>Joining</strong> in the Developer Guide) as well as through <strong>Interactive Queries</strong>. |
|
</p> |
|
|
|
<h4><a id="streams_concepts_globalktable" href="#streams_concepts_globalktable">GlobalKTable</a></h4> |
|
|
|
<p>Only the <strong>Kafka Streams DSL</strong> has the notion of a <strong>GlobalKTable</strong>.</p> |
|
|
|
<p> |
|
Like a <strong>KTable</strong>, a <strong>GlobalKTable</strong> is an abstraction of a <strong>changelog stream</strong>, where each data record represents an update. |
|
</p> |
|
|
|
<p> |
|
A GlobalKTable differs from a KTable in the data that they are being populated with, i.e. which data from the underlying Kafka topic is being read into the respective table. Slightly simplified, imagine you have an input topic with 5 partitions. In your application, you want to read this topic into a table. Also, you want to run your application across 5 application instances for <strong>maximum parallelism</strong>. |
|
</p> |
|
|
|
<ul> |
|
<li> |
|
If you read the input topic into a <strong>KTable</strong>, then the "local" KTable instance of each application instance will be populated with data <strong>from only 1 partition</strong> of the topic's 5 partitions. |
|
</li> |
|
|
|
<li> |
|
If you read the input topic into a <strong>GlobalKTable</strong>, then the local GlobalKTable instance of each application instance will be populated with data <strong>from all partitions of the topic</strong>. |
|
</li> |
|
</ul> |
|
|
|
<p> |
|
GlobalKTable provides the ability to look up <em>current</em> values of data records by keys. This table-lookup functionality is available through <code class="interpreted-text">join operations</code>. |
|
</p> |
|
<p>Benefits of global tables:</p> |
|
|
|
<ul> |
|
<li> |
|
More convenient and/or efficient <strong>joins</strong>: Notably, global tables allow you to perform star joins, they support "foreign-key" lookups (i.e., you can lookup data in the table not just by record key, but also by data in the record values), and they are more efficient when chaining multiple joins. Also, when joining against a global table, the input data does not need to be <strong>co-partitioned</strong>. |
|
</li> |
|
<li> |
|
Can be used to "broadcast" information to all the running instances of your application. |
|
</li> |
|
</ul> |
|
|
|
<p>Downsides of global tables:</p> |
|
<ul> |
|
<li>Increased local storage consumption compared to the (partitioned) KTable because the entire topic is tracked.</li> |
|
<li>Increased network and Kafka broker load compared to the (partitioned) KTable because the entire topic is read.</li> |
|
</ul> |
|
|
|
</div> |
|
<div class="section" id="creating-source-streams-from-kafka"> |
|
<span id="streams-developer-guide-dsl-sources"></span><h2><a class="toc-backref" href="#id8">Creating source streams from Kafka</a><a class="headerlink" href="#creating-source-streams-from-kafka" title="Permalink to this headline"></a></h2> |
|
<p>You can easily read data from Kafka topics into your application. The following operations are supported.</p> |
|
<table border="1" class="non-scrolling-table width-100-percent docutils"> |
|
<colgroup> |
|
<col width="22%" /> |
|
<col width="78%" /> |
|
</colgroup> |
|
<thead valign="bottom"> |
|
<tr class="row-odd"><th class="head">Reading from Kafka</th> |
|
<th class="head">Description</th> |
|
</tr> |
|
</thead> |
|
<tbody valign="top"> |
|
<tr class="row-even"><td><p class="first"><strong>Stream</strong></p> |
|
<ul class="last simple"> |
|
<li><em>input topics</em> → KStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Creates a <a class="reference internal" href="#streams_concepts_kstream"><span class="std std-ref">KStream</span></a> from the specified Kafka input topics and interprets the data |
|
as a <a class="reference internal" href="#streams_concepts_kstream"><span class="std std-ref">record stream</span></a>. |
|
A <code class="docutils literal"><span class="pre">KStream</span></code> represents a <em>partitioned</em> record stream. |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/StreamsBuilder.html#stream(java.lang.String)">(details)</a></p> |
|
<p>In the case of a KStream, the local KStream instance of every application instance will |
|
be populated with data from only <strong>a subset</strong> of the partitions of the input topic. Collectively, across |
|
all application instances, all input topic partitions are read and processed.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">org.apache.kafka.common.serialization.Serdes</span><span class="o">;</span> |
|
<span class="kn">import</span> <span class="nn">org.apache.kafka.streams.StreamsBuilder</span><span class="o">;</span> |
|
<span class="kn">import</span> <span class="nn">org.apache.kafka.streams.kstream.KStream</span><span class="o">;</span> |
|
|
|
<span class="n">StreamsBuilder</span> <span class="n">builder</span> <span class="o">=</span> <span class="k">new</span> <span class="n">StreamsBuilder</span><span class="o">();</span> |
|
|
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">wordCounts</span> <span class="o">=</span> <span class="n">builder</span><span class="o">.</span><span class="na">stream</span><span class="o">(</span> |
|
<span class="s">"word-counts-input-topic"</span><span class="o">,</span> <span class="cm">/* input topic */</span> |
|
<span class="n">Consumed</span><span class="o">.</span><span class="na">with</span><span class="o">(</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="cm">/* key serde */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">()</span> <span class="cm">/* value serde */</span> |
|
<span class="o">);</span> |
|
</pre></div> |
|
</div> |
|
<p>If you do not specify SerDes explicitly, the default SerDes from the |
|
<a class="reference internal" href="config-streams.html#streams-developer-guide-configuration"><span class="std std-ref">configuration</span></a> are used.</p> |
|
<p>You <strong>must specify SerDes explicitly</strong> if the key or value types of the records in the Kafka input |
|
topics do not match the configured default SerDes. For information about configuring default SerDes, available |
|
SerDes, and implementing your own custom SerDes see <a class="reference internal" href="datatypes.html#streams-developer-guide-serdes"><span class="std std-ref">Data Types and Serialization</span></a>.</p> |
|
<p class="last">Several variants of <code class="docutils literal"><span class="pre">stream</span></code> exist, for example to specify a regex pattern for input topics to read from).</p> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>Table</strong></p> |
|
<ul class="last simple"> |
|
<li><em>input topic</em> → KTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Reads the specified Kafka input topic into a <a class="reference internal" href="#streams_concepts_ktable"><span class="std std-ref">KTable</span></a>. The topic is |
|
interpreted as a changelog stream, where records with the same key are interpreted as UPSERT aka INSERT/UPDATE |
|
(when the record value is not <code class="docutils literal"><span class="pre">null</span></code>) or as DELETE (when the value is <code class="docutils literal"><span class="pre">null</span></code>) for that key. |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/StreamsBuilder.html#table-java.lang.String(java.lang.String)">(details)</a></p> |
|
<p>In the case of a KTable, the local KTable instance of every application instance will |
|
be populated with data from only <strong>a subset</strong> of the partitions of the input topic. Collectively, across |
|
all application instances, all input topic partitions are read and processed.</p> |
|
<p>You must provide a name for the table (more precisely, for the internal |
|
<a class="reference internal" href="../architecture.html#streams_architecture_state"><span class="std std-ref">state store</span></a> that backs the table). This is required for |
|
supporting <a class="reference internal" href="interactive-queries.html#streams-developer-guide-interactive-queries"><span class="std std-ref">interactive queries</span></a> against the table. When a |
|
name is not provided the table will not queryable and an internal name will be provided for the state store.</p> |
|
<p>If you do not specify SerDes explicitly, the default SerDes from the |
|
<a class="reference internal" href="config-streams.html#streams-developer-guide-configuration"><span class="std std-ref">configuration</span></a> are used.</p> |
|
<p>You <strong>must specify SerDes explicitly</strong> if the key or value types of the records in the Kafka input |
|
topics do not match the configured default SerDes. For information about configuring default SerDes, available |
|
SerDes, and implementing your own custom SerDes see <a class="reference internal" href="datatypes.html#streams-developer-guide-serdes"><span class="std std-ref">Data Types and Serialization</span></a>.</p> |
|
<p class="last">Several variants of <code class="docutils literal"><span class="pre">table</span></code> exist, for example to specify the <code class="docutils literal"><span class="pre">auto.offset.reset</span></code> policy to be used when |
|
reading from the input topic.</p> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td><p class="first"><strong>Global Table</strong></p> |
|
<ul class="last simple"> |
|
<li><em>input topic</em> → GlobalKTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Reads the specified Kafka input topic into a <a class="reference internal" href="#streams_concepts_globalktable"><span class="std std-ref">GlobalKTable</span></a>. The topic is |
|
interpreted as a changelog stream, where records with the same key are interpreted as UPSERT aka INSERT/UPDATE |
|
(when the record value is not <code class="docutils literal"><span class="pre">null</span></code>) or as DELETE (when the value is <code class="docutils literal"><span class="pre">null</span></code>) for that key. |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/StreamsBuilder.html#globalTable-java.lang.String(java.lang.String)">(details)</a></p> |
|
<p>In the case of a GlobalKTable, the local GlobalKTable instance of every application instance will |
|
be populated with data from <strong>all</strong> the partitions of the input topic.</p> |
|
<p>You must provide a name for the table (more precisely, for the internal |
|
<a class="reference internal" href="../architecture.html#streams_architecture_state"><span class="std std-ref">state store</span></a> that backs the table). This is required for |
|
supporting <a class="reference internal" href="interactive-queries.html#streams-developer-guide-interactive-queries"><span class="std std-ref">interactive queries</span></a> against the table. When a |
|
name is not provided the table will not queryable and an internal name will be provided for the state store.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">org.apache.kafka.common.serialization.Serdes</span><span class="o">;</span> |
|
<span class="kn">import</span> <span class="nn">org.apache.kafka.streams.StreamsBuilder</span><span class="o">;</span> |
|
<span class="kn">import</span> <span class="nn">org.apache.kafka.streams.kstream.GlobalKTable</span><span class="o">;</span> |
|
|
|
<span class="n">StreamsBuilder</span> <span class="n">builder</span> <span class="o">=</span> <span class="k">new</span> <span class="n">StreamsBuilder</span><span class="o">();</span> |
|
|
|
<span class="n">GlobalKTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">wordCounts</span> <span class="o">=</span> <span class="n">builder</span><span class="o">.</span><span class="na">globalTable</span><span class="o">(</span> |
|
<span class="s">"word-counts-input-topic"</span><span class="o">,</span> |
|
<span class="n">Materialized</span><span class="o">.<</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">,</span> <span class="n">KeyValueStore</span><span class="o"><</span><span class="n">Bytes</span><span class="o">,</span> <span class="kt">byte</span><span class="o">[]>></span><span class="n">as</span><span class="o">(</span> |
|
<span class="s">"word-counts-global-store"</span> <span class="cm">/* table/store name */</span><span class="o">)</span> |
|
<span class="o">.</span><span class="na">withKeySerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">())</span> <span class="cm">/* key serde */</span> |
|
<span class="o">.</span><span class="na">withValueSerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">())</span> <span class="cm">/* value serde */</span> |
|
<span class="o">);</span> |
|
</pre></div> |
|
</div> |
|
<p>You <strong>must specify SerDes explicitly</strong> if the key or value types of the records in the Kafka input |
|
topics do not match the configured default SerDes. For information about configuring default SerDes, available |
|
SerDes, and implementing your own custom SerDes see <a class="reference internal" href="datatypes.html#streams-developer-guide-serdes"><span class="std std-ref">Data Types and Serialization</span></a>.</p> |
|
<p class="last">Several variants of <code class="docutils literal"><span class="pre">globalTable</span></code> exist to e.g. specify explicit SerDes.</p> |
|
</td> |
|
</tr> |
|
</tbody> |
|
</table> |
|
</div> |
|
<div class="section" id="transform-a-stream"> |
|
<span id="streams-developer-guide-dsl-transformations"></span><h2><a class="toc-backref" href="#id9">Transform a stream</a><a class="headerlink" href="#transform-a-stream" title="Permalink to this headline"></a></h2> |
|
<p>The KStream and KTable interfaces support a variety of transformation operations. |
|
Each of these operations can be translated into one or more connected processors into the underlying processor topology. |
|
Since KStream and KTable are strongly typed, all of these transformation operations are defined as |
|
generic functions where users could specify the input and output data types.</p> |
|
<p>Some KStream transformations may generate one or more KStream objects, for example: |
|
- <code class="docutils literal"><span class="pre">filter</span></code> and <code class="docutils literal"><span class="pre">map</span></code> on a KStream will generate another KStream |
|
- <code class="docutils literal"><span class="pre">branch</span></code> on KStream can generate multiple KStreams</p> |
|
<p>Some others may generate a KTable object, for example an aggregation of a KStream also yields a KTable. This allows Kafka Streams to continuously update the computed value upon arrivals of <a class="reference internal" href="../core-concepts.html#streams_concepts_aggregations"><span class="std std-ref">late records</span></a> after it |
|
has already been produced to the downstream transformation operators.</p> |
|
<p>All KTable transformation operations can only generate another KTable. However, the Kafka Streams DSL does provide a special function |
|
that converts a KTable representation into a KStream. All of these transformation methods can be chained together to compose |
|
a complex processor topology.</p> |
|
<p>These transformation operations are described in the following subsections:</p> |
|
<ul class="simple"> |
|
<li><a class="reference internal" href="#streams-developer-guide-dsl-transformations-stateless"><span class="std std-ref">Stateless transformations</span></a></li> |
|
<li><a class="reference internal" href="#streams-developer-guide-dsl-transformations-stateful"><span class="std std-ref">Stateful transformations</span></a></li> |
|
</ul> |
|
<div class="section" id="stateless-transformations"> |
|
<span id="streams-developer-guide-dsl-transformations-stateless"></span><h3><a class="toc-backref" href="#id10">Stateless transformations</a><a class="headerlink" href="#stateless-transformations" title="Permalink to this headline"></a></h3> |
|
<p>Stateless transformations do not require state for processing and they do not require a state store associated with |
|
the stream processor. Kafka 0.11.0 and later allows you to materialize the result from a stateless <code class="docutils literal"><span class="pre">KTable</span></code> transformation. This allows the result to be queried through <a class="reference internal" href="interactive-queries.html#streams-developer-guide-interactive-queries"><span class="std std-ref">interactive queries</span></a>. To materialize a <code class="docutils literal"><span class="pre">KTable</span></code>, each of the below stateless operations <a class="reference internal" href="interactive-queries.html#streams-developer-guide-interactive-queries-local-key-value-stores"><span class="std std-ref">can be augmented</span></a> with an optional <code class="docutils literal"><span class="pre">queryableStoreName</span></code> argument.</p> |
|
<table border="1" class="non-scrolling-table width-100-percent docutils"> |
|
<colgroup> |
|
<col width="22%" /> |
|
<col width="78%" /> |
|
</colgroup> |
|
<thead valign="bottom"> |
|
<tr class="row-odd"><th class="head">Transformation</th> |
|
<th class="head">Description</th> |
|
</tr> |
|
</thead> |
|
<tbody valign="top"> |
|
<tr class="row-even"><td><p class="first"><strong>Branch</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream → KStream[]</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Branch (or split) a <code class="docutils literal"><span class="pre">KStream</span></code> based on the supplied predicates into one or more <code class="docutils literal"><span class="pre">KStream</span></code> instances. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#branch-org.apache.kafka.streams.kstream.Predicate...-">details</a>)</p> |
|
<p>Predicates are evaluated in order. A record is placed to one and only one output stream on the first match: |
|
if the n-th predicate evaluates to true, the record is placed to n-th stream. If no predicate matches, the |
|
the record is dropped.</p> |
|
<p>Branching is useful, for example, to route records to different downstream topics.</p> |
|
<div class="last highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">>[]</span> <span class="n">branches</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">branch</span><span class="o">(</span> |
|
<span class="o">(</span><span class="n">key</span><span class="o">,</span> <span class="n">value</span><span class="o">)</span> <span class="o">-></span> <span class="n">key</span><span class="o">.</span><span class="na">startsWith</span><span class="o">(</span><span class="s">"A"</span><span class="o">),</span> <span class="cm">/* first predicate */</span> |
|
<span class="o">(</span><span class="n">key</span><span class="o">,</span> <span class="n">value</span><span class="o">)</span> <span class="o">-></span> <span class="n">key</span><span class="o">.</span><span class="na">startsWith</span><span class="o">(</span><span class="s">"B"</span><span class="o">),</span> <span class="cm">/* second predicate */</span> |
|
<span class="o">(</span><span class="n">key</span><span class="o">,</span> <span class="n">value</span><span class="o">)</span> <span class="o">-></span> <span class="kc">true</span> <span class="cm">/* third predicate */</span> |
|
<span class="o">);</span> |
|
|
|
<span class="c1">// KStream branches[0] contains all records whose keys start with "A"</span> |
|
<span class="c1">// KStream branches[1] contains all records whose keys start with "B"</span> |
|
<span class="c1">// KStream branches[2] contains all other records</span> |
|
|
|
<span class="c1">// Java 7 example: cf. `filter` for how to create `Predicate` instances</span> |
|
</pre></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>Filter</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream → KStream</li> |
|
<li>KTable → KTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Evaluates a boolean function for each element and retains those for which the function returns true. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#filter-org.apache.kafka.streams.kstream.Predicate-">KStream details</a>, |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KTable.html#filter-org.apache.kafka.streams.kstream.Predicate-">KTable details</a>)</p> |
|
<div class="last highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// A filter that selects (keeps) only positive numbers</span> |
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">onlyPositives</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">filter</span><span class="o">((</span><span class="n">key</span><span class="o">,</span> <span class="n">value</span><span class="o">)</span> <span class="o">-></span> <span class="n">value</span> <span class="o">></span> <span class="mi">0</span><span class="o">);</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">onlyPositives</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">filter</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">Predicate</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="kt">boolean</span> <span class="nf">test</span><span class="o">(</span><span class="n">String</span> <span class="n">key</span><span class="o">,</span> <span class="n">Long</span> <span class="n">value</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">value</span> <span class="o">></span> <span class="mi">0</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
</pre></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td><p class="first"><strong>Inverse Filter</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream → KStream</li> |
|
<li>KTable → KTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Evaluates a boolean function for each element and drops those for which the function returns true. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#filterNot-org.apache.kafka.streams.kstream.Predicate-">KStream details</a>, |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KTable.html#filterNot-org.apache.kafka.streams.kstream.Predicate-">KTable details</a>)</p> |
|
<div class="last highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// An inverse filter that discards any negative numbers or zero</span> |
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">onlyPositives</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">filterNot</span><span class="o">((</span><span class="n">key</span><span class="o">,</span> <span class="n">value</span><span class="o">)</span> <span class="o">-></span> <span class="n">value</span> <span class="o"><=</span> <span class="mi">0</span><span class="o">);</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">onlyPositives</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">filterNot</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">Predicate</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="kt">boolean</span> <span class="nf">test</span><span class="o">(</span><span class="n">String</span> <span class="n">key</span><span class="o">,</span> <span class="n">Long</span> <span class="n">value</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">value</span> <span class="o"><=</span> <span class="mi">0</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
</pre></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>FlatMap</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream → KStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Takes one record and produces zero, one, or more records. You can modify the record keys and values, including |
|
their types. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#flatMap-org.apache.kafka.streams.kstream.KeyValueMapper-">details</a>)</p> |
|
<p><strong>Marks the stream for data re-partitioning:</strong> |
|
Applying a grouping or a join after <code class="docutils literal"><span class="pre">flatMap</span></code> will result in re-partitioning of the records. |
|
If possible use <code class="docutils literal"><span class="pre">flatMapValues</span></code> instead, which will not cause data re-partitioning.</p> |
|
<div class="last highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="n">Long</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Integer</span><span class="o">></span> <span class="n">transformed</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">flatMap</span><span class="o">(</span> |
|
<span class="c1">// Here, we generate two output records for each input record.</span> |
|
<span class="c1">// We also change the key and value types.</span> |
|
<span class="c1">// Example: (345L, "Hello") -> ("HELLO", 1000), ("hello", 9000)</span> |
|
<span class="o">(</span><span class="n">key</span><span class="o">,</span> <span class="n">value</span><span class="o">)</span> <span class="o">-></span> <span class="o">{</span> |
|
<span class="n">List</span><span class="o"><</span><span class="n">KeyValue</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Integer</span><span class="o">>></span> <span class="n">result</span> <span class="o">=</span> <span class="k">new</span> <span class="n">LinkedList</span><span class="o"><>();</span> |
|
<span class="n">result</span><span class="o">.</span><span class="na">add</span><span class="o">(</span><span class="n">KeyValue</span><span class="o">.</span><span class="na">pair</span><span class="o">(</span><span class="n">value</span><span class="o">.</span><span class="na">toUpperCase</span><span class="o">(),</span> <span class="mi">1000</span><span class="o">));</span> |
|
<span class="n">result</span><span class="o">.</span><span class="na">add</span><span class="o">(</span><span class="n">KeyValue</span><span class="o">.</span><span class="na">pair</span><span class="o">(</span><span class="n">value</span><span class="o">.</span><span class="na">toLowerCase</span><span class="o">(),</span> <span class="mi">9000</span><span class="o">));</span> |
|
<span class="k">return</span> <span class="n">result</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">);</span> |
|
|
|
<span class="c1">// Java 7 example: cf. `map` for how to create `KeyValueMapper` instances</span> |
|
</pre></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td><p class="first"><strong>FlatMap (values only)</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream → KStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Takes one record and produces zero, one, or more records, while retaining the key of the original record. |
|
You can modify the record values and the value type. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#flatMapValues-org.apache.kafka.streams.kstream.ValueMapper-">details</a>)</p> |
|
<p><code class="docutils literal"><span class="pre">flatMapValues</span></code> is preferable to <code class="docutils literal"><span class="pre">flatMap</span></code> because it will not cause data re-partitioning. However, you |
|
cannot modify the key or key type like <code class="docutils literal"><span class="pre">flatMap</span></code> does.</p> |
|
<div class="last highlight-java"><div class="highlight"><pre><span></span><span class="c1">// Split a sentence into words.</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">sentences</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">words</span> <span class="o">=</span> <span class="n">sentences</span><span class="o">.</span><span class="na">flatMapValues</span><span class="o">(</span><span class="n">value</span> <span class="o">-></span> <span class="n">Arrays</span><span class="o">.</span><span class="na">asList</span><span class="o">(</span><span class="n">value</span><span class="o">.</span><span class="na">split</span><span class="o">(</span><span class="s">"\\s+"</span><span class="o">)));</span> |
|
|
|
<span class="c1">// Java 7 example: cf. `mapValues` for how to create `ValueMapper` instances</span> |
|
</pre></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>Foreach</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream → void</li> |
|
<li>KStream → void</li> |
|
<li>KTable → void</li> |
|
</ul> |
|
</td> |
|
<td><p class="first"><strong>Terminal operation.</strong> Performs a stateless action on each record. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#foreach-org.apache.kafka.streams.kstream.ForeachAction-">details</a>)</p> |
|
<p>You would use <code class="docutils literal"><span class="pre">foreach</span></code> to cause <em>side effects</em> based on the input data (similar to <code class="docutils literal"><span class="pre">peek</span></code>) and then <em>stop</em> |
|
<em>further processing</em> of the input data (unlike <code class="docutils literal"><span class="pre">peek</span></code>, which is not a terminal operation).</p> |
|
<p><strong>Note on processing guarantees:</strong> Any side effects of an action (such as writing to external systems) are not |
|
trackable by Kafka, which means they will typically not benefit from Kafka’s processing guarantees.</p> |
|
<div class="last highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Print the contents of the KStream to the local console.</span> |
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="n">stream</span><span class="o">.</span><span class="na">foreach</span><span class="o">((</span><span class="n">key</span><span class="o">,</span> <span class="n">value</span><span class="o">)</span> <span class="o">-></span> <span class="n">System</span><span class="o">.</span><span class="na">out</span><span class="o">.</span><span class="na">println</span><span class="o">(</span><span class="n">key</span> <span class="o">+</span> <span class="s">" => "</span> <span class="o">+</span> <span class="n">value</span><span class="o">));</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">stream</span><span class="o">.</span><span class="na">foreach</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">ForeachAction</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">apply</span><span class="o">(</span><span class="n">String</span> <span class="n">key</span><span class="o">,</span> <span class="n">Long</span> <span class="n">value</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="n">System</span><span class="o">.</span><span class="na">out</span><span class="o">.</span><span class="na">println</span><span class="o">(</span><span class="n">key</span> <span class="o">+</span> <span class="s">" => "</span> <span class="o">+</span> <span class="n">value</span><span class="o">);</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
</pre></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td><p class="first"><strong>GroupByKey</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream → KGroupedStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Groups the records by the existing key. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#groupByKey--">details</a>)</p> |
|
<p>Grouping is a prerequisite for <a class="reference internal" href="#streams-developer-guide-dsl-aggregating"><span class="std std-ref">aggregating a stream or a table</span></a> |
|
and ensures that data is properly partitioned (“keyed”) for subsequent operations.</p> |
|
<p><strong>When to set explicit SerDes:</strong> |
|
Variants of <code class="docutils literal"><span class="pre">groupByKey</span></code> exist to override the configured default SerDes of your application, which <strong>you</strong> |
|
<strong>must do</strong> if the key and/or value types of the resulting <code class="docutils literal"><span class="pre">KGroupedStream</span></code> do not match the configured default |
|
SerDes.</p> |
|
<div class="admonition note"> |
|
<p><b>Note</b></p> |
|
<p class="last"><strong>Grouping vs. Windowing:</strong> |
|
A related operation is <a class="reference internal" href="#streams-developer-guide-dsl-windowing"><span class="std std-ref">windowing</span></a>, which lets you control how to |
|
“sub-group” the grouped records <em>of the same key</em> into so-called <em>windows</em> for stateful operations such as |
|
windowed <a class="reference internal" href="#streams-developer-guide-dsl-aggregating"><span class="std std-ref">aggregations</span></a> or |
|
windowed <a class="reference internal" href="#streams-developer-guide-dsl-joins"><span class="std std-ref">joins</span></a>.</p> |
|
</div> |
|
<p><strong>Causes data re-partitioning if and only if the stream was marked for re-partitioning.</strong> |
|
<code class="docutils literal"><span class="pre">groupByKey</span></code> is preferable to <code class="docutils literal"><span class="pre">groupBy</span></code> because it re-partitions data only if the stream was already marked |
|
for re-partitioning. However, <code class="docutils literal"><span class="pre">groupByKey</span></code> does not allow you to modify the key or key type like <code class="docutils literal"><span class="pre">groupBy</span></code> |
|
does.</p> |
|
<div class="last highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Group by the existing key, using the application's configured</span> |
|
<span class="c1">// default serdes for keys and values.</span> |
|
<span class="n">KGroupedStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">groupedStream</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">groupByKey</span><span class="o">();</span> |
|
|
|
<span class="c1">// When the key and/or value types do not match the configured</span> |
|
<span class="c1">// default serdes, we must explicitly specify serdes.</span> |
|
<span class="n">KGroupedStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">groupedStream</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">groupByKey</span><span class="o">(</span> |
|
<span class="n">Serialized</span><span class="o">.</span><span class="na">with</span><span class="o">(</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">ByteArray</span><span class="o">(),</span> <span class="cm">/* key */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">())</span> <span class="cm">/* value */</span> |
|
<span class="o">);</span> |
|
</pre></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>GroupBy</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream → KGroupedStream</li> |
|
<li>KTable → KGroupedTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Groups the records by a <em>new</em> key, which may be of a different key type. |
|
When grouping a table, you may also specify a new value and value type. |
|
<code class="docutils literal"><span class="pre">groupBy</span></code> is a shorthand for <code class="docutils literal"><span class="pre">selectKey(...).groupByKey()</span></code>. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#groupBy-org.apache.kafka.streams.kstream.KeyValueMapper-">KStream details</a>, |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KTable.html#groupBy-org.apache.kafka.streams.kstream.KeyValueMapper-">KTable details</a>)</p> |
|
<p>Grouping is a prerequisite for <a class="reference internal" href="#streams-developer-guide-dsl-aggregating"><span class="std std-ref">aggregating a stream or a table</span></a> |
|
and ensures that data is properly partitioned (“keyed”) for subsequent operations.</p> |
|
<p><strong>When to set explicit SerDes:</strong> |
|
Variants of <code class="docutils literal"><span class="pre">groupBy</span></code> exist to override the configured default SerDes of your application, which <strong>you must</strong> |
|
<strong>do</strong> if the key and/or value types of the resulting <code class="docutils literal"><span class="pre">KGroupedStream</span></code> or <code class="docutils literal"><span class="pre">KGroupedTable</span></code> do not match the |
|
configured default SerDes.</p> |
|
<div class="admonition note"> |
|
<p><b>Note</b></p> |
|
<p class="last"><strong>Grouping vs. Windowing:</strong> |
|
A related operation is <a class="reference internal" href="#streams-developer-guide-dsl-windowing"><span class="std std-ref">windowing</span></a>, which lets you control how to |
|
“sub-group” the grouped records <em>of the same key</em> into so-called <em>windows</em> for stateful operations such as |
|
windowed <a class="reference internal" href="#streams-developer-guide-dsl-aggregating"><span class="std std-ref">aggregations</span></a> or |
|
windowed <a class="reference internal" href="#streams-developer-guide-dsl-joins"><span class="std std-ref">joins</span></a>.</p> |
|
</div> |
|
<p><strong>Always causes data re-partitioning:</strong> <code class="docutils literal"><span class="pre">groupBy</span></code> always causes data re-partitioning. |
|
If possible use <code class="docutils literal"><span class="pre">groupByKey</span></code> instead, which will re-partition data only if required.</p> |
|
<div class="last highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">table</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ examples, using lambda expressions</span> |
|
|
|
<span class="c1">// Group the stream by a new key and key type</span> |
|
<span class="n">KGroupedStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">groupedStream</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">groupBy</span><span class="o">(</span> |
|
<span class="o">(</span><span class="n">key</span><span class="o">,</span> <span class="n">value</span><span class="o">)</span> <span class="o">-></span> <span class="n">value</span><span class="o">,</span> |
|
<span class="n">Serialized</span><span class="o">.</span><span class="na">with</span><span class="o">(</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="cm">/* key (note: type was modified) */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">())</span> <span class="cm">/* value */</span> |
|
<span class="o">);</span> |
|
|
|
<span class="c1">// Group the table by a new key and key type, and also modify the value and value type.</span> |
|
<span class="n">KGroupedTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Integer</span><span class="o">></span> <span class="n">groupedTable</span> <span class="o">=</span> <span class="n">table</span><span class="o">.</span><span class="na">groupBy</span><span class="o">(</span> |
|
<span class="o">(</span><span class="n">key</span><span class="o">,</span> <span class="n">value</span><span class="o">)</span> <span class="o">-></span> <span class="n">KeyValue</span><span class="o">.</span><span class="na">pair</span><span class="o">(</span><span class="n">value</span><span class="o">,</span> <span class="n">value</span><span class="o">.</span><span class="na">length</span><span class="o">()),</span> |
|
<span class="n">Serialized</span><span class="o">.</span><span class="na">with</span><span class="o">(</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="cm">/* key (note: type was modified) */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">Integer</span><span class="o">())</span> <span class="cm">/* value (note: type was modified) */</span> |
|
<span class="o">);</span> |
|
|
|
|
|
<span class="c1">// Java 7 examples</span> |
|
|
|
<span class="c1">// Group the stream by a new key and key type</span> |
|
<span class="n">KGroupedStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">groupedStream</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">groupBy</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">KeyValueMapper</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">>>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">String</span> <span class="nf">apply</span><span class="o">(</span><span class="kt">byte</span><span class="o">[]</span> <span class="n">key</span><span class="o">,</span> <span class="n">String</span> <span class="n">value</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">value</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="n">Serialized</span><span class="o">.</span><span class="na">with</span><span class="o">(</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="cm">/* key (note: type was modified) */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">())</span> <span class="cm">/* value */</span> |
|
<span class="o">);</span> |
|
|
|
<span class="c1">// Group the table by a new key and key type, and also modify the value and value type.</span> |
|
<span class="n">KGroupedTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Integer</span><span class="o">></span> <span class="n">groupedTable</span> <span class="o">=</span> <span class="n">table</span><span class="o">.</span><span class="na">groupBy</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">KeyValueMapper</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">,</span> <span class="n">KeyValue</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Integer</span><span class="o">>>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">KeyValue</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Integer</span><span class="o">></span> <span class="nf">apply</span><span class="o">(</span><span class="kt">byte</span><span class="o">[]</span> <span class="n">key</span><span class="o">,</span> <span class="n">String</span> <span class="n">value</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">KeyValue</span><span class="o">.</span><span class="na">pair</span><span class="o">(</span><span class="n">value</span><span class="o">,</span> <span class="n">value</span><span class="o">.</span><span class="na">length</span><span class="o">());</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="n">Serialized</span><span class="o">.</span><span class="na">with</span><span class="o">(</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="cm">/* key (note: type was modified) */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">Integer</span><span class="o">())</span> <span class="cm">/* value (note: type was modified) */</span> |
|
<span class="o">);</span> |
|
</pre></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td><p class="first"><strong>Map</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream → KStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Takes one record and produces one record. You can modify the record key and value, including their types. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#map-org.apache.kafka.streams.kstream.KeyValueMapper-">details</a>)</p> |
|
<p><strong>Marks the stream for data re-partitioning:</strong> |
|
Applying a grouping or a join after <code class="docutils literal"><span class="pre">map</span></code> will result in re-partitioning of the records. |
|
If possible use <code class="docutils literal"><span class="pre">mapValues</span></code> instead, which will not cause data re-partitioning.</p> |
|
<div class="last highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="c1">// Note how we change the key and the key type (similar to `selectKey`)</span> |
|
<span class="c1">// as well as the value and the value type.</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Integer</span><span class="o">></span> <span class="n">transformed</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">map</span><span class="o">(</span> |
|
<span class="o">(</span><span class="n">key</span><span class="o">,</span> <span class="n">value</span><span class="o">)</span> <span class="o">-></span> <span class="n">KeyValue</span><span class="o">.</span><span class="na">pair</span><span class="o">(</span><span class="n">value</span><span class="o">.</span><span class="na">toLowerCase</span><span class="o">(),</span> <span class="n">value</span><span class="o">.</span><span class="na">length</span><span class="o">()));</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Integer</span><span class="o">></span> <span class="n">transformed</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">map</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">KeyValueMapper</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">,</span> <span class="n">KeyValue</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Integer</span><span class="o">>>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">KeyValue</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Integer</span><span class="o">></span> <span class="nf">apply</span><span class="o">(</span><span class="kt">byte</span><span class="o">[]</span> <span class="n">key</span><span class="o">,</span> <span class="n">String</span> <span class="n">value</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="k">new</span> <span class="n">KeyValue</span><span class="o"><>(</span><span class="n">value</span><span class="o">.</span><span class="na">toLowerCase</span><span class="o">(),</span> <span class="n">value</span><span class="o">.</span><span class="na">length</span><span class="o">());</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
</pre></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>Map (values only)</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream → KStream</li> |
|
<li>KTable → KTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Takes one record and produces one record, while retaining the key of the original record. |
|
You can modify the record value and the value type. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#mapValues-org.apache.kafka.streams.kstream.ValueMapper-">KStream details</a>, |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KTable.html#mapValues-org.apache.kafka.streams.kstream.ValueMapper-">KTable details</a>)</p> |
|
<p><code class="docutils literal"><span class="pre">mapValues</span></code> is preferable to <code class="docutils literal"><span class="pre">map</span></code> because it will not cause data re-partitioning. However, it does not |
|
allow you to modify the key or key type like <code class="docutils literal"><span class="pre">map</span></code> does.</p> |
|
<div class="last highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">uppercased</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">mapValues</span><span class="o">(</span><span class="n">value</span> <span class="o">-></span> <span class="n">value</span><span class="o">.</span><span class="na">toUpperCase</span><span class="o">());</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">uppercased</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">mapValues</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">ValueMapper</span><span class="o"><</span><span class="n">String</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">String</span> <span class="nf">apply</span><span class="o">(</span><span class="n">String</span> <span class="n">s</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">s</span><span class="o">.</span><span class="na">toUpperCase</span><span class="o">();</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
</pre></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td><p class="first"><strong>Merge</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream → KStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Merges records of two streams into one larger stream. |
|
(<a class="reference external" |
|
href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#merge-org.apache.kafka.streams.kstream.KStream-">details</a>) |
|
<p>There is no ordering guarantee between records |
|
from different streams in the merged stream. Relative order is preserved within each input stream though (ie, records within the same input stream are processed in order)</p> |
|
<div class="last highlight-java"> |
|
<div class="highlight"> |
|
<pre> |
|
<span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">stream1</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">stream2</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">merged</span> <span class="o">=</span> <span class="n">stream1</span><span class="o">.</span><span class="na">merge</span><span class="o">(</span><span class="n">stream2</span><span class="o">);</span> |
|
</pre> |
|
</div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>Peek</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream → KStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Performs a stateless action on each record, and returns an unchanged stream. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#peek-org.apache.kafka.streams.kstream.ForeachAction-">details</a>)</p> |
|
<p>You would use <code class="docutils literal"><span class="pre">peek</span></code> to cause <em>side effects</em> based on the input data (similar to <code class="docutils literal"><span class="pre">foreach</span></code>) and <em>continue</em> |
|
<em>processing</em> the input data (unlike <code class="docutils literal"><span class="pre">foreach</span></code>, which is a terminal operation). <code class="docutils literal"><span class="pre">peek</span></code> returns the input |
|
stream as-is; if you need to modify the input stream, use <code class="docutils literal"><span class="pre">map</span></code> or <code class="docutils literal"><span class="pre">mapValues</span></code> instead.</p> |
|
<p><code class="docutils literal"><span class="pre">peek</span></code> is helpful for use cases such as logging or tracking metrics or for debugging and troubleshooting.</p> |
|
<p><strong>Note on processing guarantees:</strong> Any side effects of an action (such as writing to external systems) are not |
|
trackable by Kafka, which means they will typically not benefit from Kafka’s processing guarantees.</p> |
|
<div class="last highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">unmodifiedStream</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">peek</span><span class="o">(</span> |
|
<span class="o">(</span><span class="n">key</span><span class="o">,</span> <span class="n">value</span><span class="o">)</span> <span class="o">-></span> <span class="n">System</span><span class="o">.</span><span class="na">out</span><span class="o">.</span><span class="na">println</span><span class="o">(</span><span class="s">"key="</span> <span class="o">+</span> <span class="n">key</span> <span class="o">+</span> <span class="s">", value="</span> <span class="o">+</span> <span class="n">value</span><span class="o">));</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">unmodifiedStream</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">peek</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">ForeachAction</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">apply</span><span class="o">(</span><span class="kt">byte</span><span class="o">[]</span> <span class="n">key</span><span class="o">,</span> <span class="n">String</span> <span class="n">value</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="n">System</span><span class="o">.</span><span class="na">out</span><span class="o">.</span><span class="na">println</span><span class="o">(</span><span class="s">"key="</span> <span class="o">+</span> <span class="n">key</span> <span class="o">+</span> <span class="s">", value="</span> <span class="o">+</span> <span class="n">value</span><span class="o">);</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
</pre></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td><p class="first"><strong>Print</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream → void</li> |
|
</ul> |
|
</td> |
|
<td><p class="first"><strong>Terminal operation.</strong> Prints the records to <code class="docutils literal"><span class="pre">System.out</span></code>. See Javadocs for serde and <code class="docutils literal"><span class="pre">toString()</span></code> |
|
caveats. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#print--">details</a>)</p> |
|
<p>Calling <code class="docutils literal"><span class="pre">print()</span></code> is the same as calling <code class="docutils literal"><span class="pre">foreach((key,</span> <span class="pre">value)</span> <span class="pre">-></span> <span class="pre">System.out.println(key</span> <span class="pre">+</span> <span class="pre">",</span> <span class="pre">"</span> <span class="pre">+</span> <span class="pre">value))</span></code></p> |
|
<p><code class="docutils literal"><span class="pre">print</span></code> is mainly for debugging/testing purposes, and it will try to flush on each record print. Hence it <strong>should not</strong> be used for production usage if performance requirements are concerned.</p> |
|
<div class="last highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="c1">// print to sysout</span> |
|
<span class="n">stream</span><span class="o">.</span><span class="na">print</span><span class="o">();</span> |
|
|
|
<span class="c1">// print to file with a custom label</span> |
|
<span class="n">stream</span><span class="o">.</span><span class="na">print</span><span class="o">(</span><span class="n">Printed</span><span class="o">.</span><span class="na">toFile</span><span class="o">(</span><span class="s">"streams.out"</span><span class="o">).</span><span class="na">withLabel</span><span class="o">(</span><span class="s">"streams"</span><span class="o">));</span> |
|
</pre></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>SelectKey</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream → KStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Assigns a new key – possibly of a new key type – to each record. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#selectKey-org.apache.kafka.streams.kstream.KeyValueMapper-">details</a>)</p> |
|
<p>Calling <code class="docutils literal"><span class="pre">selectKey(mapper)</span></code> is the same as calling <code class="docutils literal"><span class="pre">map((key,</span> <span class="pre">value)</span> <span class="pre">-></span> <span class="pre">mapper(key,</span> <span class="pre">value),</span> <span class="pre">value)</span></code>.</p> |
|
<p><strong>Marks the stream for data re-partitioning:</strong> |
|
Applying a grouping or a join after <code class="docutils literal"><span class="pre">selectKey</span></code> will result in re-partitioning of the records.</p> |
|
<div class="last highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Derive a new record key from the record's value. Note how the key type changes, too.</span> |
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">rekeyed</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">selectKey</span><span class="o">((</span><span class="n">key</span><span class="o">,</span> <span class="n">value</span><span class="o">)</span> <span class="o">-></span> <span class="n">value</span><span class="o">.</span><span class="na">split</span><span class="o">(</span><span class="s">" "</span><span class="o">)[</span><span class="mi">0</span><span class="o">])</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">rekeyed</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">selectKey</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">KeyValueMapper</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">String</span> <span class="nf">apply</span><span class="o">(</span><span class="kt">byte</span><span class="o">[]</span> <span class="n">key</span><span class="o">,</span> <span class="n">String</span> <span class="n">value</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">value</span><span class="o">.</span><span class="na">split</span><span class="o">(</span><span class="s">" "</span><span class="o">)[</span><span class="mi">0</span><span class="o">];</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
</pre></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td><p class="first"><strong>Table to Stream</strong></p> |
|
<ul class="last simple"> |
|
<li>KTable → KStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Get the changelog stream of this table. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KTable.html#toStream--">details</a>)</p> |
|
<div class="last highlight-java"><div class="highlight"><pre><span></span><span class="n">KTable</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">table</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Also, a variant of `toStream` exists that allows you</span> |
|
<span class="c1">// to select a new key for the resulting stream.</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span> <span class="n">table</span><span class="o">.</span><span class="na">toStream</span><span class="o">();</span> |
|
</pre></div> |
|
</div> |
|
</td> |
|
</tr> |
|
</tbody> |
|
</table> |
|
</div> |
|
<div class="section" id="stateful-transformations"> |
|
<span id="streams-developer-guide-dsl-transformations-stateful"></span><h3><a class="toc-backref" href="#id11">Stateful transformations</a><a class="headerlink" href="#stateful-transformations" title="Permalink to this headline"></a></h3> |
|
<p id="streams-developer-guide-dsl-transformations-stateful-overview">Stateful transformations depend on state for processing inputs and producing outputs and require a <a class="reference internal" href="../architecture.html#streams_architecture_state"><span class="std std-ref">state store</span></a> associated with the stream processor. For example, in aggregating operations, a windowing state store is used to collect the latest aggregation results per |
|
window. In join operations, a windowing state store is used to collect all of the records received so far within the |
|
defined window boundary.</p> |
|
<p>Note, that state stores are fault-tolerant. |
|
In case of failure, Kafka Streams guarantees to fully restore all state stores prior to resuming the processing. |
|
See <a class="reference internal" href="../architecture.html#streams_architecture_recovery"><span class="std std-ref">Fault Tolerance</span></a> for further information.</p> |
|
<p>Available stateful transformations in the DSL include:</p> |
|
<ul class="simple"> |
|
<li><a class="reference internal" href="#streams-developer-guide-dsl-aggregating"><span class="std std-ref">Aggregating</span></a></li> |
|
<li><a class="reference internal" href="#streams-developer-guide-dsl-joins"><span class="std std-ref">Joining</span></a></li> |
|
<li><a class="reference internal" href="#streams-developer-guide-dsl-windowing"><span class="std std-ref">Windowing</span></a> (as part of aggregations and joins)</li> |
|
<li><a class="reference internal" href="#streams-developer-guide-dsl-process"><span class="std std-ref">Applying custom processors and transformers</span></a>, which may be stateful, for |
|
Processor API integration</li> |
|
</ul> |
|
<p>The following diagram shows their relationships:</p> |
|
<div class="figure align-center" id="id2"> |
|
<img class="centered" src="/{{version}}/images/streams-stateful_operations.png"> |
|
<p class="caption"><span class="caption-text">Stateful transformations in the DSL.</span></p> |
|
</div> |
|
<p>Here is an example of a stateful application: the WordCount algorithm.</p> |
|
<p>WordCount example in Java 8+, using lambda expressions:</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="c1">// Assume the record values represent lines of text. For the sake of this example, you can ignore</span> |
|
<span class="c1">// whatever may be stored in the record keys.</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">textLines</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">wordCounts</span> <span class="o">=</span> <span class="n">textLines</span> |
|
<span class="c1">// Split each text line, by whitespace, into words. The text lines are the record</span> |
|
<span class="c1">// values, i.e. you can ignore whatever data is in the record keys and thus invoke</span> |
|
<span class="c1">// `flatMapValues` instead of the more generic `flatMap`.</span> |
|
<span class="o">.</span><span class="na">flatMapValues</span><span class="o">(</span><span class="n">value</span> <span class="o">-></span> <span class="n">Arrays</span><span class="o">.</span><span class="na">asList</span><span class="o">(</span><span class="n">value</span><span class="o">.</span><span class="na">toLowerCase</span><span class="o">().</span><span class="na">split</span><span class="o">(</span><span class="s">"\\W+"</span><span class="o">)))</span> |
|
<span class="c1">// Group the stream by word to ensure the key of the record is the word.</span> |
|
<span class="o">.</span><span class="na">groupBy</span><span class="o">((</span><span class="n">key</span><span class="o">,</span> <span class="n">word</span><span class="o">)</span> <span class="o">-></span> <span class="n">word</span><span class="o">)</span> |
|
<span class="c1">// Count the occurrences of each word (record key).</span> |
|
<span class="c1">//</span> |
|
<span class="c1">// This will change the stream type from `KGroupedStream<String, String>` to</span> |
|
<span class="c1">// `KTable<String, Long>` (word -> count).</span> |
|
<span class="o">.</span><span class="na">count</span><span class="o">()</span> |
|
<span class="c1">// Convert the `KTable<String, Long>` into a `KStream<String, Long>`.</span> |
|
<span class="o">.</span><span class="na">toStream</span><span class="o">();</span> |
|
</pre></div> |
|
</div> |
|
<p>WordCount example in Java 7:</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="c1">// Code below is equivalent to the previous Java 8+ example above.</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">textLines</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">wordCounts</span> <span class="o">=</span> <span class="n">textLines</span> |
|
<span class="o">.</span><span class="na">flatMapValues</span><span class="o">(</span><span class="k">new</span> <span class="n">ValueMapper</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Iterable</span><span class="o"><</span><span class="n">String</span><span class="o">>>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Iterable</span><span class="o"><</span><span class="n">String</span><span class="o">></span> <span class="nf">apply</span><span class="o">(</span><span class="n">String</span> <span class="n">value</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">Arrays</span><span class="o">.</span><span class="na">asList</span><span class="o">(</span><span class="n">value</span><span class="o">.</span><span class="na">toLowerCase</span><span class="o">().</span><span class="na">split</span><span class="o">(</span><span class="s">"\\W+"</span><span class="o">));</span> |
|
<span class="o">}</span> |
|
<span class="o">})</span> |
|
<span class="o">.</span><span class="na">groupBy</span><span class="o">(</span><span class="k">new</span> <span class="n">KeyValueMapper</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">>>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">String</span> <span class="nf">apply</span><span class="o">(</span><span class="n">String</span> <span class="n">key</span><span class="o">,</span> <span class="n">String</span> <span class="n">word</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">word</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">})</span> |
|
<span class="o">.</span><span class="na">count</span><span class="o">()</span> |
|
<span class="o">.</span><span class="na">toStream</span><span class="o">();</span> |
|
</pre></div> |
|
</div> |
|
<div class="section" id="aggregating"> |
|
<span id="streams-developer-guide-dsl-aggregating"></span><h4><a class="toc-backref" href="#id12">Aggregating</a><a class="headerlink" href="#aggregating" title="Permalink to this headline"></a></h4> |
|
<p>After records are <a class="reference internal" href="#streams-developer-guide-dsl-transformations-stateless"><span class="std std-ref">grouped</span></a> by key via <code class="docutils literal"><span class="pre">groupByKey</span></code> or |
|
<code class="docutils literal"><span class="pre">groupBy</span></code> – and thus represented as either a <code class="docutils literal"><span class="pre">KGroupedStream</span></code> or a <code class="docutils literal"><span class="pre">KGroupedTable</span></code>, they can be aggregated |
|
via an operation such as <code class="docutils literal"><span class="pre">reduce</span></code>. Aggregations are key-based operations, which means that they always operate over records |
|
(notably record values) of the same key. |
|
You can perform aggregations on <a class="reference internal" href="#streams-developer-guide-dsl-windowing"><span class="std std-ref">windowed</span></a> or non-windowed data.</p> |
|
<table border="1" class="non-scrolling-table width-100-percent docutils"> |
|
<colgroup> |
|
<col width="22%" /> |
|
<col width="78%" /> |
|
</colgroup> |
|
<thead valign="bottom"> |
|
<tr class="row-odd"><th class="head">Transformation</th> |
|
<th class="head">Description</th> |
|
</tr> |
|
</thead> |
|
<tbody valign="top"> |
|
<tr class="row-even"><td><p class="first"><strong>Aggregate</strong></p> |
|
<ul class="last simple"> |
|
<li>KGroupedStream → KTable</li> |
|
<li>KGroupedTable → KTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first"><strong>Rolling aggregation.</strong> Aggregates the values of (non-windowed) records by the grouped key. |
|
Aggregating is a generalization of <code class="docutils literal"><span class="pre">reduce</span></code> and allows, for example, the aggregate value to have a different |
|
type than the input values. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KGroupedStream.html">KGroupedStream details</a>, |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KGroupedTable.html">KGroupedTable details</a>)</p> |
|
<p>When aggregating a <em>grouped stream</em>, you must provide an initializer (e.g., <code class="docutils literal"><span class="pre">aggValue</span> <span class="pre">=</span> <span class="pre">0</span></code>) and an “adder” |
|
aggregator (e.g., <code class="docutils literal"><span class="pre">aggValue</span> <span class="pre">+</span> <span class="pre">curValue</span></code>). When aggregating a <em>grouped table</em>, you must provide a |
|
“subtractor” aggregator (think: <code class="docutils literal"><span class="pre">aggValue</span> <span class="pre">-</span> <span class="pre">oldValue</span></code>).</p> |
|
<p>Several variants of <code class="docutils literal"><span class="pre">aggregate</span></code> exist, see Javadocs for details.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">KGroupedStream</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">groupedStream</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KGroupedTable</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">></span> <span class="n">groupedTable</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ examples, using lambda expressions</span> |
|
|
|
<span class="c1">// Aggregating a KGroupedStream (note how the value type changes from String to Long)</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">Long</span><span class="o">></span> <span class="n">aggregatedStream</span> <span class="o">=</span> <span class="n">groupedStream</span><span class="o">.</span><span class="na">aggregate</span><span class="o">(</span> |
|
<span class="o">()</span> <span class="o">-></span> <span class="mi">0</span><span class="n">L</span><span class="o">,</span> <span class="cm">/* initializer */</span> |
|
<span class="o">(</span><span class="n">aggKey</span><span class="o">,</span> <span class="n">newValue</span><span class="o">,</span> <span class="n">aggValue</span><span class="o">)</span> <span class="o">-></span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span><span class="o">.</span><span class="na">length</span><span class="o">(),</span> <span class="cm">/* adder */</span> |
|
<span class="n">Materialized</span><span class="o">.</span><span class="na">as</span><span class="o">(</span><span class="s">"aggregated-stream-store"</span><span class="o">)</span> <span class="cm">/* state store name */</span> |
|
<span class="o">.</span><span class="na">withValueSerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">());</span> <span class="cm">/* serde for aggregate value */</span> |
|
|
|
<span class="c1">// Aggregating a KGroupedTable (note how the value type changes from String to Long)</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">Long</span><span class="o">></span> <span class="n">aggregatedTable</span> <span class="o">=</span> <span class="n">groupedTable</span><span class="o">.</span><span class="na">aggregate</span><span class="o">(</span> |
|
<span class="o">()</span> <span class="o">-></span> <span class="mi">0</span><span class="n">L</span><span class="o">,</span> <span class="cm">/* initializer */</span> |
|
<span class="o">(</span><span class="n">aggKey</span><span class="o">,</span> <span class="n">newValue</span><span class="o">,</span> <span class="n">aggValue</span><span class="o">)</span> <span class="o">-></span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span><span class="o">.</span><span class="na">length</span><span class="o">(),</span> <span class="cm">/* adder */</span> |
|
<span class="o">(</span><span class="n">aggKey</span><span class="o">,</span> <span class="n">oldValue</span><span class="o">,</span> <span class="n">aggValue</span><span class="o">)</span> <span class="o">-></span> <span class="n">aggValue</span> <span class="o">-</span> <span class="n">oldValue</span><span class="o">.</span><span class="na">length</span><span class="o">(),</span> <span class="cm">/* subtractor */</span> |
|
<span class="n">Materialized</span><span class="o">.</span><span class="na">as</span><span class="o">(</span><span class="s">"aggregated-table-store"</span><span class="o">)</span> <span class="cm">/* state store name */</span> |
|
<span class="o">.</span><span class="na">withValueSerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">())</span> <span class="cm">/* serde for aggregate value */</span> |
|
|
|
|
|
<span class="c1">// Java 7 examples</span> |
|
|
|
<span class="c1">// Aggregating a KGroupedStream (note how the value type changes from String to Long)</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">Long</span><span class="o">></span> <span class="n">aggregatedStream</span> <span class="o">=</span> <span class="n">groupedStream</span><span class="o">.</span><span class="na">aggregate</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">Initializer</span><span class="o"><</span><span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* initializer */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Long</span> <span class="nf">apply</span><span class="o">()</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="mi">0</span><span class="n">L</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="k">new</span> <span class="n">Aggregator</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* adder */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Long</span> <span class="nf">apply</span><span class="o">(</span><span class="kt">byte</span><span class="o">[]</span> <span class="n">aggKey</span><span class="o">,</span> <span class="n">String</span> <span class="n">newValue</span><span class="o">,</span> <span class="n">Long</span> <span class="n">aggValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span><span class="o">.</span><span class="na">length</span><span class="o">();</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="n">Materialized</span><span class="o">.</span><span class="na">as</span><span class="o">(</span><span class="s">"aggregated-stream-store"</span><span class="o">)</span> |
|
<span class="o">.</span><span class="na">withValueSerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">());</span> |
|
|
|
<span class="c1">// Aggregating a KGroupedTable (note how the value type changes from String to Long)</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">Long</span><span class="o">></span> <span class="n">aggregatedTable</span> <span class="o">=</span> <span class="n">groupedTable</span><span class="o">.</span><span class="na">aggregate</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">Initializer</span><span class="o"><</span><span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* initializer */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Long</span> <span class="nf">apply</span><span class="o">()</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="mi">0</span><span class="n">L</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="k">new</span> <span class="n">Aggregator</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* adder */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Long</span> <span class="nf">apply</span><span class="o">(</span><span class="kt">byte</span><span class="o">[]</span> <span class="n">aggKey</span><span class="o">,</span> <span class="n">String</span> <span class="n">newValue</span><span class="o">,</span> <span class="n">Long</span> <span class="n">aggValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span><span class="o">.</span><span class="na">length</span><span class="o">();</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="k">new</span> <span class="n">Aggregator</span><span class="o"><</span><span class="kt">byte</span><span class="o">[],</span> <span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* subtractor */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Long</span> <span class="nf">apply</span><span class="o">(</span><span class="kt">byte</span><span class="o">[]</span> <span class="n">aggKey</span><span class="o">,</span> <span class="n">String</span> <span class="n">oldValue</span><span class="o">,</span> <span class="n">Long</span> <span class="n">aggValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">aggValue</span> <span class="o">-</span> <span class="n">oldValue</span><span class="o">.</span><span class="na">length</span><span class="o">();</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="n">Materialized</span><span class="o">.</span><span class="na">as</span><span class="o">(</span><span class="s">"aggregated-stream-store"</span><span class="o">)</span> |
|
<span class="o">.</span><span class="na">withValueSerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">());</span> |
|
</pre></div> |
|
</div> |
|
<p>Detailed behavior of <code class="docutils literal"><span class="pre">KGroupedStream</span></code>:</p> |
|
<ul class="simple"> |
|
<li>Input records with <code class="docutils literal"><span class="pre">null</span></code> keys are ignored.</li> |
|
<li>When a record key is received for the first time, the initializer is called (and called before the adder).</li> |
|
<li>Whenever a record with a non-<code class="docutils literal"><span class="pre">null</span></code> value is received, the adder is called.</li> |
|
</ul> |
|
<p>Detailed behavior of <code class="docutils literal"><span class="pre">KGroupedTable</span></code>:</p> |
|
<ul class="simple"> |
|
<li>Input records with <code class="docutils literal"><span class="pre">null</span></code> keys are ignored.</li> |
|
<li>When a record key is received for the first time, the initializer is called (and called before the adder |
|
and subtractor). Note that, in contrast to <code class="docutils literal"><span class="pre">KGroupedStream</span></code>, over time the initializer may be called |
|
more than once for a key as a result of having received input tombstone records for that key (see below).</li> |
|
<li>When the first non-<code class="docutils literal"><span class="pre">null</span></code> value is received for a key (e.g., INSERT), then only the adder is called.</li> |
|
<li>When subsequent non-<code class="docutils literal"><span class="pre">null</span></code> values are received for a key (e.g., UPDATE), then (1) the subtractor is |
|
called with the old value as stored in the table and (2) the adder is called with the new value of the |
|
input record that was just received. The order of execution for the subtractor and adder is not defined.</li> |
|
<li>When a tombstone record – i.e. a record with a <code class="docutils literal"><span class="pre">null</span></code> value – is received for a key (e.g., DELETE), |
|
then only the subtractor is called. Note that, whenever the subtractor returns a <code class="docutils literal"><span class="pre">null</span></code> value itself, |
|
then the corresponding key is removed from the resulting <code class="docutils literal"><span class="pre">KTable</span></code>. If that happens, any next input |
|
record for that key will trigger the initializer again.</li> |
|
</ul> |
|
<p class="last">See the example at the bottom of this section for a visualization of the aggregation semantics.</p> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>Aggregate (windowed)</strong></p> |
|
<ul class="last simple"> |
|
<li>KGroupedStream → KTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first"><strong>Windowed aggregation.</strong> |
|
Aggregates the values of records, <a class="reference internal" href="#streams-developer-guide-dsl-windowing"><span class="std std-ref">per window</span></a>, by the grouped key. |
|
Aggregating is a generalization of <code class="docutils literal"><span class="pre">reduce</span></code> and allows, for example, the aggregate value to have a different |
|
type than the input values. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/TimeWindowedKStream.html">TimeWindowedKStream details</a>, |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/SessionWindowedKStream.html">SessionWindowedKStream details</a>)</p> |
|
<p>You must provide an initializer (e.g., <code class="docutils literal"><span class="pre">aggValue</span> <span class="pre">=</span> <span class="pre">0</span></code>), “adder” aggregator (e.g., <code class="docutils literal"><span class="pre">aggValue</span> <span class="pre">+</span> <span class="pre">curValue</span></code>), |
|
and a window. When windowing based on sessions, you must additionally provide a “session merger” aggregator |
|
(e.g., <code class="docutils literal"><span class="pre">mergedAggValue</span> <span class="pre">=</span> <span class="pre">leftAggValue</span> <span class="pre">+</span> <span class="pre">rightAggValue</span></code>).</p> |
|
<p>The windowed <code class="docutils literal"><span class="pre">aggregate</span></code> turns a <code class="docutils literal"><span class="pre">TimeWindowedKStream<K,</span> <span class="pre">V></span></code> or <code class="docutils literal"><span class="pre">SessionWindowdKStream<K,</span> <span class="pre">V></span></code> |
|
into a windowed <code class="docutils literal"><span class="pre">KTable<Windowed<K>,</span> <span class="pre">V></span></code>.</p> |
|
<p>Several variants of <code class="docutils literal"><span class="pre">aggregate</span></code> exist, see Javadocs for details.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">java.util.concurrent.TimeUnit</span><span class="o">;</span> |
|
<span class="n">KGroupedStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">groupedStream</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ examples, using lambda expressions</span> |
|
|
|
<span class="c1">// Aggregating with time-based windowing (here: with 5-minute tumbling windows)</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">Windowed</span><span class="o"><</span><span class="n">String</span><span class="o">>,</span> <span class="n">Long</span><span class="o">></span> <span class="n">timeWindowedAggregatedStream</span> <span class="o">=</span> <span class="n">groupedStream</span><span class="o">.</span><span class="na">windowedBy</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">))</span> |
|
<span class="o">.</span><span class="na">aggregate</span><span class="o">(</span> |
|
<span class="o">()</span> <span class="o">-></span> <span class="mi">0</span><span class="n">L</span><span class="o">,</span> <span class="cm">/* initializer */</span> |
|
<span class="o">(</span><span class="n">aggKey</span><span class="o">,</span> <span class="n">newValue</span><span class="o">,</span> <span class="n">aggValue</span><span class="o">)</span> <span class="o">-></span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span><span class="o">,</span> <span class="cm">/* adder */</span> |
|
<span class="n">Materialized</span><span class="o">.<</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">,</span> <span class="n">WindowStore</span><span class="o"><</span><span class="n">Bytes</span><span class="o">,</span> <span class="kt">byte</span><span class="o">[]>></span><span class="n">as</span><span class="o">(</span><span class="s">"time-windowed-aggregated-stream-store"</span><span class="o">)</span> <span class="cm">/* state store name */</span> |
|
<span class="o">.</span><span class="na">withValueSerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">()));</span> <span class="cm">/* serde for aggregate value */</span> |
|
|
|
<span class="c1">// Aggregating with session-based windowing (here: with an inactivity gap of 5 minutes)</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">Windowed</span><span class="o"><</span><span class="n">String</span><span class="o">>,</span> <span class="n">Long</span><span class="o">></span> <span class="n">sessionizedAggregatedStream</span> <span class="o">=</span> <span class="n">groupedStream</span><span class="o">.</span><span class="na">windowedBy</span><span class="o">(</span><span class="n">SessionWindows</span><span class="o">.</span><span class="na">with</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">)).</span> |
|
<span class="n">aggregate</span><span class="o">(</span> |
|
<span class="o">()</span> <span class="o">-></span> <span class="mi">0</span><span class="n">L</span><span class="o">,</span> <span class="cm">/* initializer */</span> |
|
<span class="o">(</span><span class="n">aggKey</span><span class="o">,</span> <span class="n">newValue</span><span class="o">,</span> <span class="n">aggValue</span><span class="o">)</span> <span class="o">-></span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span><span class="o">,</span> <span class="cm">/* adder */</span> |
|
<span class="o">(</span><span class="n">aggKey</span><span class="o">,</span> <span class="n">leftAggValue</span><span class="o">,</span> <span class="n">rightAggValue</span><span class="o">)</span> <span class="o">-></span> <span class="n">leftAggValue</span> <span class="o">+</span> <span class="n">rightAggValue</span><span class="o">,</span> <span class="cm">/* session merger */</span> |
|
<span class="n">Materialized</span><span class="o">.<</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">,</span> <span class="n">SessionStore</span><span class="o"><</span><span class="n">Bytes</span><span class="o">,</span> <span class="kt">byte</span><span class="o">[]>></span><span class="n">as</span><span class="o">(</span><span class="s">"sessionized-aggregated-stream-store"</span><span class="o">)</span> <span class="cm">/* state store name */</span> |
|
<span class="o">.</span><span class="na">withValueSerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">()));</span> <span class="cm">/* serde for aggregate value */</span> |
|
|
|
<span class="c1">// Java 7 examples</span> |
|
|
|
<span class="c1">// Aggregating with time-based windowing (here: with 5-minute tumbling windows)</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">Windowed</span><span class="o"><</span><span class="n">String</span><span class="o">>,</span> <span class="n">Long</span><span class="o">></span> <span class="n">timeWindowedAggregatedStream</span> <span class="o">=</span> <span class="n">groupedStream</span><span class="o">.</span><span class="na">windowedBy</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">))</span> |
|
<span class="o">.</span><span class="na">aggregate</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">Initializer</span><span class="o"><</span><span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* initializer */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Long</span> <span class="nf">apply</span><span class="o">()</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="mi">0</span><span class="n">L</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="k">new</span> <span class="n">Aggregator</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">,</span> <span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* adder */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Long</span> <span class="nf">apply</span><span class="o">(</span><span class="n">String</span> <span class="n">aggKey</span><span class="o">,</span> <span class="n">Long</span> <span class="n">newValue</span><span class="o">,</span> <span class="n">Long</span> <span class="n">aggValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="n">Materialized</span><span class="o">.<</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">,</span> <span class="n">WindowStore</span><span class="o"><</span><span class="n">Bytes</span><span class="o">,</span> <span class="kt">byte</span><span class="o">[]>></span><span class="n">as</span><span class="o">(</span><span class="s">"time-windowed-aggregated-stream-store"</span><span class="o">)</span> |
|
<span class="o">.</span><span class="na">withValueSerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">()));</span> |
|
|
|
<span class="c1">// Aggregating with session-based windowing (here: with an inactivity gap of 5 minutes)</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">Windowed</span><span class="o"><</span><span class="n">String</span><span class="o">>,</span> <span class="n">Long</span><span class="o">></span> <span class="n">sessionizedAggregatedStream</span> <span class="o">=</span> <span class="n">groupedStream</span><span class="o">.</span><span class="na">windowedBy</span><span class="o">(</span><span class="n">SessionWindows</span><span class="o">.</span><span class="na">with</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">)).</span> |
|
<span class="n">aggregate</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">Initializer</span><span class="o"><</span><span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* initializer */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Long</span> <span class="nf">apply</span><span class="o">()</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="mi">0</span><span class="n">L</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="k">new</span> <span class="n">Aggregator</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">,</span> <span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* adder */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Long</span> <span class="nf">apply</span><span class="o">(</span><span class="n">String</span> <span class="n">aggKey</span><span class="o">,</span> <span class="n">Long</span> <span class="n">newValue</span><span class="o">,</span> <span class="n">Long</span> <span class="n">aggValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="k">new</span> <span class="n">Merger</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* session merger */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Long</span> <span class="nf">apply</span><span class="o">(</span><span class="n">String</span> <span class="n">aggKey</span><span class="o">,</span> <span class="n">Long</span> <span class="n">leftAggValue</span><span class="o">,</span> <span class="n">Long</span> <span class="n">rightAggValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">rightAggValue</span> <span class="o">+</span> <span class="n">leftAggValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="n">Materialized</span><span class="o">.<</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">,</span> <span class="n">SessionStore</span><span class="o"><</span><span class="n">Bytes</span><span class="o">,</span> <span class="kt">byte</span><span class="o">[]>></span><span class="n">as</span><span class="o">(</span><span class="s">"sessionized-aggregated-stream-store"</span><span class="o">)</span> |
|
<span class="o">.</span><span class="na">withValueSerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">()));</span> |
|
</pre></div> |
|
</div> |
|
<p>Detailed behavior:</p> |
|
<ul class="simple"> |
|
<li>The windowed aggregate behaves similar to the rolling aggregate described above. The additional twist is that |
|
the behavior applies <em>per window</em>.</li> |
|
<li>Input records with <code class="docutils literal"><span class="pre">null</span></code> keys are ignored in general.</li> |
|
<li>When a record key is received for the first time for a given window, the initializer is called (and called |
|
before the adder).</li> |
|
<li>Whenever a record with a non-<code class="docutils literal"><span class="pre">null</span></code> value is received for a given window, the adder is called.</li> |
|
<li>When using session windows: the session merger is called whenever two sessions are being merged.</li> |
|
</ul> |
|
<p class="last">See the example at the bottom of this section for a visualization of the aggregation semantics.</p> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td><p class="first"><strong>Count</strong></p> |
|
<ul class="last simple"> |
|
<li>KGroupedStream → KTable</li> |
|
<li>KGroupedTable → KTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first"><strong>Rolling aggregation.</strong> Counts the number of records by the grouped key. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KGroupedStream.html">KGroupedStream details</a>, |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KGroupedTable.html">KGroupedTable details</a>)</p> |
|
<p>Several variants of <code class="docutils literal"><span class="pre">count</span></code> exist, see Javadocs for details.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">KGroupedStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">groupedStream</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KGroupedTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">groupedTable</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Counting a KGroupedStream</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">aggregatedStream</span> <span class="o">=</span> <span class="n">groupedStream</span><span class="o">.</span><span class="na">count</span><span class="o">();</span> |
|
|
|
<span class="c1">// Counting a KGroupedTable</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">aggregatedTable</span> <span class="o">=</span> <span class="n">groupedTable</span><span class="o">.</span><span class="na">count</span><span class="o">();</span> |
|
</pre></div> |
|
</div> |
|
<p>Detailed behavior for <code class="docutils literal"><span class="pre">KGroupedStream</span></code>:</p> |
|
<ul class="simple"> |
|
<li>Input records with <code class="docutils literal"><span class="pre">null</span></code> keys or values are ignored.</li> |
|
</ul> |
|
<p>Detailed behavior for <code class="docutils literal"><span class="pre">KGroupedTable</span></code>:</p> |
|
<ul class="last simple"> |
|
<li>Input records with <code class="docutils literal"><span class="pre">null</span></code> keys are ignored. Records with <code class="docutils literal"><span class="pre">null</span></code> values are not ignored but interpreted |
|
as “tombstones” for the corresponding key, which indicate the deletion of the key from the table.</li> |
|
</ul> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>Count (windowed)</strong></p> |
|
<ul class="last simple"> |
|
<li>KGroupedStream → KTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first"><strong>Windowed aggregation.</strong> |
|
Counts the number of records, <a class="reference internal" href="#streams-developer-guide-dsl-windowing"><span class="std std-ref">per window</span></a>, by the grouped key. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/TimeWindowedKStream.html">TimeWindowedKStream details</a>, |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/SessionWindowedKStream.html">SessionWindowedKStream details</a>)</p> |
|
<p>The windowed <code class="docutils literal"><span class="pre">count</span></code> turns a <code class="docutils literal"><span class="pre">TimeWindowedKStream<K,</span> <span class="pre">V></span></code> or <code class="docutils literal"><span class="pre">SessionWindowedKStream<K,</span> <span class="pre">V></span></code> |
|
into a windowed <code class="docutils literal"><span class="pre">KTable<Windowed<K>,</span> <span class="pre">V></span></code>.</p> |
|
<p>Several variants of <code class="docutils literal"><span class="pre">count</span></code> exist, see Javadocs for details.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">java.util.concurrent.TimeUnit</span><span class="o">;</span> |
|
<span class="n">KGroupedStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">groupedStream</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Counting a KGroupedStream with time-based windowing (here: with 5-minute tumbling windows)</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">Windowed</span><span class="o"><</span><span class="n">String</span><span class="o">>,</span> <span class="n">Long</span><span class="o">></span> <span class="n">aggregatedStream</span> <span class="o">=</span> <span class="n">groupedStream</span><span class="o">.</span><span class="na">windowedBy</span><span class="o">(</span> |
|
<span class="n">TimeWindows</span><span class="o">.</span><span class="na">of</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">)))</span> <span class="cm">/* time-based window */</span> |
|
<span class="o">.</span><span class="na">count</span><span class="o">();</span> |
|
|
|
<span class="c1">// Counting a KGroupedStream with session-based windowing (here: with 5-minute inactivity gaps)</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">Windowed</span><span class="o"><</span><span class="n">String</span><span class="o">>,</span> <span class="n">Long</span><span class="o">></span> <span class="n">aggregatedStream</span> <span class="o">=</span> <span class="n">groupedStream</span><span class="o">.</span><span class="na">windowedBy</span><span class="o">(</span> |
|
<span class="n">SessionWindows</span><span class="o">.</span><span class="na">with</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">)))</span> <span class="cm">/* session window */</span> |
|
<span class="o">.</span><span class="na">count</span><span class="o">();</span> |
|
</pre></div> |
|
</div> |
|
<p>Detailed behavior:</p> |
|
<ul class="last simple"> |
|
<li>Input records with <code class="docutils literal"><span class="pre">null</span></code> keys or values are ignored.</li> |
|
</ul> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td><p class="first"><strong>Reduce</strong></p> |
|
<ul class="last simple"> |
|
<li>KGroupedStream → KTable</li> |
|
<li>KGroupedTable → KTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first"><strong>Rolling aggregation.</strong> Combines the values of (non-windowed) records by the grouped key. |
|
The current record value is combined with the last reduced value, and a new reduced value is returned. |
|
The result value type cannot be changed, unlike <code class="docutils literal"><span class="pre">aggregate</span></code>. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KGroupedStream.html">KGroupedStream details</a>, |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KGroupedTable.html">KGroupedTable details</a>)</p> |
|
<p>When reducing a <em>grouped stream</em>, you must provide an “adder” reducer (e.g., <code class="docutils literal"><span class="pre">aggValue</span> <span class="pre">+</span> <span class="pre">curValue</span></code>). |
|
When reducing a <em>grouped table</em>, you must additionally provide a “subtractor” reducer (e.g., |
|
<code class="docutils literal"><span class="pre">aggValue</span> <span class="pre">-</span> <span class="pre">oldValue</span></code>).</p> |
|
<p>Several variants of <code class="docutils literal"><span class="pre">reduce</span></code> exist, see Javadocs for details.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">KGroupedStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">groupedStream</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KGroupedTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">groupedTable</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ examples, using lambda expressions</span> |
|
|
|
<span class="c1">// Reducing a KGroupedStream</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">aggregatedStream</span> <span class="o">=</span> <span class="n">groupedStream</span><span class="o">.</span><span class="na">reduce</span><span class="o">(</span> |
|
<span class="o">(</span><span class="n">aggValue</span><span class="o">,</span> <span class="n">newValue</span><span class="o">)</span> <span class="o">-></span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span> <span class="cm">/* adder */</span><span class="o">);</span> |
|
|
|
<span class="c1">// Reducing a KGroupedTable</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">aggregatedTable</span> <span class="o">=</span> <span class="n">groupedTable</span><span class="o">.</span><span class="na">reduce</span><span class="o">(</span> |
|
<span class="o">(</span><span class="n">aggValue</span><span class="o">,</span> <span class="n">newValue</span><span class="o">)</span> <span class="o">-></span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span><span class="o">,</span> <span class="cm">/* adder */</span> |
|
<span class="o">(</span><span class="n">aggValue</span><span class="o">,</span> <span class="n">oldValue</span><span class="o">)</span> <span class="o">-></span> <span class="n">aggValue</span> <span class="o">-</span> <span class="n">oldValue</span> <span class="cm">/* subtractor */</span><span class="o">);</span> |
|
|
|
|
|
<span class="c1">// Java 7 examples</span> |
|
|
|
<span class="c1">// Reducing a KGroupedStream</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">aggregatedStream</span> <span class="o">=</span> <span class="n">groupedStream</span><span class="o">.</span><span class="na">reduce</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">Reducer</span><span class="o"><</span><span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* adder */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Long</span> <span class="nf">apply</span><span class="o">(</span><span class="n">Long</span> <span class="n">aggValue</span><span class="o">,</span> <span class="n">Long</span> <span class="n">newValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
|
|
<span class="c1">// Reducing a KGroupedTable</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">aggregatedTable</span> <span class="o">=</span> <span class="n">groupedTable</span><span class="o">.</span><span class="na">reduce</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">Reducer</span><span class="o"><</span><span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* adder */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Long</span> <span class="nf">apply</span><span class="o">(</span><span class="n">Long</span> <span class="n">aggValue</span><span class="o">,</span> <span class="n">Long</span> <span class="n">newValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="k">new</span> <span class="n">Reducer</span><span class="o"><</span><span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* subtractor */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Long</span> <span class="nf">apply</span><span class="o">(</span><span class="n">Long</span> <span class="n">aggValue</span><span class="o">,</span> <span class="n">Long</span> <span class="n">oldValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">aggValue</span> <span class="o">-</span> <span class="n">oldValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
</pre></div> |
|
</div> |
|
<p>Detailed behavior for <code class="docutils literal"><span class="pre">KGroupedStream</span></code>:</p> |
|
<ul class="simple"> |
|
<li>Input records with <code class="docutils literal"><span class="pre">null</span></code> keys are ignored in general.</li> |
|
<li>When a record key is received for the first time, then the value of that record is used as the initial |
|
aggregate value.</li> |
|
<li>Whenever a record with a non-<code class="docutils literal"><span class="pre">null</span></code> value is received, the adder is called.</li> |
|
</ul> |
|
<p>Detailed behavior for <code class="docutils literal"><span class="pre">KGroupedTable</span></code>:</p> |
|
<ul class="simple"> |
|
<li>Input records with <code class="docutils literal"><span class="pre">null</span></code> keys are ignored in general.</li> |
|
<li>When a record key is received for the first time, then the value of that record is used as the initial |
|
aggregate value. |
|
Note that, in contrast to <code class="docutils literal"><span class="pre">KGroupedStream</span></code>, over time this initialization step may happen more than once |
|
for a key as a result of having received input tombstone records for that key (see below).</li> |
|
<li>When the first non-<code class="docutils literal"><span class="pre">null</span></code> value is received for a key (e.g., INSERT), then only the adder is called.</li> |
|
<li>When subsequent non-<code class="docutils literal"><span class="pre">null</span></code> values are received for a key (e.g., UPDATE), then (1) the subtractor is |
|
called with the old value as stored in the table and (2) the adder is called with the new value of the |
|
input record that was just received. The order of execution for the subtractor and adder is not defined.</li> |
|
<li>When a tombstone record – i.e. a record with a <code class="docutils literal"><span class="pre">null</span></code> value – is received for a key (e.g., DELETE), |
|
then only the subtractor is called. Note that, whenever the subtractor returns a <code class="docutils literal"><span class="pre">null</span></code> value itself, |
|
then the corresponding key is removed from the resulting <code class="docutils literal"><span class="pre">KTable</span></code>. If that happens, any next input |
|
record for that key will re-initialize its aggregate value.</li> |
|
</ul> |
|
<p class="last">See the example at the bottom of this section for a visualization of the aggregation semantics.</p> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>Reduce (windowed)</strong></p> |
|
<ul class="last simple"> |
|
<li>KGroupedStream → KTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first"><strong>Windowed aggregation.</strong> |
|
Combines the values of records, <a class="reference internal" href="#streams-developer-guide-dsl-windowing"><span class="std std-ref">per window</span></a>, by the grouped key. |
|
The current record value is combined with the last reduced value, and a new reduced value is returned. |
|
Records with <code class="docutils literal"><span class="pre">null</span></code> key or value are ignored. |
|
The result value type cannot be changed, unlike <code class="docutils literal"><span class="pre">aggregate</span></code>. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/TimeWindowedKStream.html">TimeWindowedKStream details</a>, |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/SessionWindowedKStream.html">SessionWindowedKStream details</a>)</p> |
|
<p>The windowed <code class="docutils literal"><span class="pre">reduce</span></code> turns a turns a <code class="docutils literal"><span class="pre">TimeWindowedKStream<K,</span> <span class="pre">V></span></code> or a <code class="docutils literal"><span class="pre">SessionWindowedKStream<K,</span> <span class="pre">V></span></code> |
|
into a windowed <code class="docutils literal"><span class="pre">KTable<Windowed<K>,</span> <span class="pre">V></span></code>.</p> |
|
<p>Several variants of <code class="docutils literal"><span class="pre">reduce</span></code> exist, see Javadocs for details.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">java.util.concurrent.TimeUnit</span><span class="o">;</span> |
|
<span class="n">KGroupedStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">groupedStream</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ examples, using lambda expressions</span> |
|
|
|
<span class="c1">// Aggregating with time-based windowing (here: with 5-minute tumbling windows)</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">Windowed</span><span class="o"><</span><span class="n">String</span><span class="o">>,</span> <span class="n">Long</span><span class="o">></span> <span class="n">timeWindowedAggregatedStream</span> <span class="o">=</span> <span class="n">groupedStream</span><span class="o">.</span><span class="na">windowedBy</span><span class="o">(</span> |
|
<span class="n">TimeWindows</span><span class="o">.</span><span class="na">of</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">))</span> <span class="cm">/* time-based window */</span><span class="o">)</span> |
|
<span class="o">.</span><span class="na">reduce</span><span class="o">(</span> |
|
<span class="o">(</span><span class="n">aggValue</span><span class="o">,</span> <span class="n">newValue</span><span class="o">)</span> <span class="o">-></span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span> <span class="cm">/* adder */</span> |
|
<span class="o">);</span> |
|
|
|
<span class="c1">// Aggregating with session-based windowing (here: with an inactivity gap of 5 minutes)</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">Windowed</span><span class="o"><</span><span class="n">String</span><span class="o">>,</span> <span class="n">Long</span><span class="o">></span> <span class="n">sessionzedAggregatedStream</span> <span class="o">=</span> <span class="n">groupedStream</span><span class="o">.</span><span class="na">windowedBy</span><span class="o">(</span> |
|
<span class="n">SessionWindows</span><span class="o">.</span><span class="na">with</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">)))</span> <span class="cm">/* session window */</span> |
|
<span class="o">.</span><span class="na">reduce</span><span class="o">(</span> |
|
<span class="o">(</span><span class="n">aggValue</span><span class="o">,</span> <span class="n">newValue</span><span class="o">)</span> <span class="o">-></span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span> <span class="cm">/* adder */</span> |
|
<span class="o">);</span> |
|
|
|
|
|
<span class="c1">// Java 7 examples</span> |
|
|
|
<span class="c1">// Aggregating with time-based windowing (here: with 5-minute tumbling windows)</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">Windowed</span><span class="o"><</span><span class="n">String</span><span class="o">>,</span> <span class="n">Long</span><span class="o">></span> <span class="n">timeWindowedAggregatedStream</span> <span class="o">=</span> <span class="n">groupedStream</span><span class="o">..</span><span class="na">windowedBy</span><span class="o">(</span> |
|
<span class="n">TimeWindows</span><span class="o">.</span><span class="na">of</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">))</span> <span class="cm">/* time-based window */</span><span class="o">)</span> |
|
<span class="o">.</span><span class="na">reduce</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">Reducer</span><span class="o"><</span><span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* adder */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Long</span> <span class="nf">apply</span><span class="o">(</span><span class="n">Long</span> <span class="n">aggValue</span><span class="o">,</span> <span class="n">Long</span> <span class="n">newValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
|
|
<span class="c1">// Aggregating with session-based windowing (here: with an inactivity gap of 5 minutes)</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">Windowed</span><span class="o"><</span><span class="n">String</span><span class="o">>,</span> <span class="n">Long</span><span class="o">></span> <span class="n">timeWindowedAggregatedStream</span> <span class="o">=</span> <span class="n">groupedStream</span><span class="o">.</span><span class="na">windowedBy</span><span class="o">(</span> |
|
<span class="n">SessionWindows</span><span class="o">.</span><span class="na">with</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">)))</span> <span class="cm">/* session window */</span> |
|
<span class="o">.</span><span class="na">reduce</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">Reducer</span><span class="o"><</span><span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* adder */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Long</span> <span class="nf">apply</span><span class="o">(</span><span class="n">Long</span> <span class="n">aggValue</span><span class="o">,</span> <span class="n">Long</span> <span class="n">newValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
</pre></div> |
|
</div> |
|
<p>Detailed behavior:</p> |
|
<ul class="simple"> |
|
<li>The windowed reduce behaves similar to the rolling reduce described above. The additional twist is that the |
|
behavior applies <em>per window</em>.</li> |
|
<li>Input records with <code class="docutils literal"><span class="pre">null</span></code> keys are ignored in general.</li> |
|
<li>When a record key is received for the first time for a given window, then the value of that record is used as |
|
the initial aggregate value.</li> |
|
<li>Whenever a record with a non-<code class="docutils literal"><span class="pre">null</span></code> value is received for a given window, the adder is called.</li> |
|
</ul> |
|
<p class="last">See the example at the bottom of this section for a visualization of the aggregation semantics.</p> |
|
</td> |
|
</tr> |
|
</tbody> |
|
</table> |
|
<p><strong>Example of semantics for stream aggregations:</strong> |
|
A <code class="docutils literal"><span class="pre">KGroupedStream</span></code> → <code class="docutils literal"><span class="pre">KTable</span></code> example is shown below. The streams and the table are initially empty. Bold |
|
font is used in the column for “KTable <code class="docutils literal"><span class="pre">aggregated</span></code>” to highlight changed state. An entry such as <code class="docutils literal"><span class="pre">(hello,</span> <span class="pre">1)</span></code> denotes a |
|
record with key <code class="docutils literal"><span class="pre">hello</span></code> and value <code class="docutils literal"><span class="pre">1</span></code>. To improve the readability of the semantics table you can assume that all records |
|
are processed in timestamp order.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="c1">// Key: word, value: count</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Integer</span><span class="o">></span> <span class="n">wordCounts</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="n">KGroupedStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Integer</span><span class="o">></span> <span class="n">groupedStream</span> <span class="o">=</span> <span class="n">wordCounts</span> |
|
<span class="o">.</span><span class="na">groupByKey</span><span class="o">(</span><span class="n">Serialized</span><span class="o">.</span><span class="na">with</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="n">Serdes</span><span class="o">.</span><span class="na">Integer</span><span class="o">()));</span> |
|
|
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Integer</span><span class="o">></span> <span class="n">aggregated</span> <span class="o">=</span> <span class="n">groupedStream</span><span class="o">.</span><span class="na">aggregate</span><span class="o">(</span> |
|
<span class="o">()</span> <span class="o">-></span> <span class="mi">0</span><span class="o">,</span> <span class="cm">/* initializer */</span> |
|
<span class="o">(</span><span class="n">aggKey</span><span class="o">,</span> <span class="n">newValue</span><span class="o">,</span> <span class="n">aggValue</span><span class="o">)</span> <span class="o">-></span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span><span class="o">,</span> <span class="cm">/* adder */</span> |
|
<span class="n">Materialized</span><span class="o">.<</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">,</span> <span class="n">KeyValueStore</span><span class="o"><</span><span class="n">Bytes</span><span class="o">,</span> <span class="kt">byte</span><span class="o">[]></span><span class="n">as</span><span class="o">(</span><span class="s">"aggregated-stream-store"</span> <span class="cm">/* state store name */</span><span class="o">)</span> |
|
<span class="o">.</span><span class="na">withKeySerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">())</span> <span class="cm">/* key serde */</span> |
|
<span class="o">.</span><span class="na">withValueSerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">Integer</span><span class="o">());</span> <span class="cm">/* serde for aggregate value */</span> |
|
</pre></div> |
|
</div> |
|
<div class="admonition note"> |
|
<p><b>Note</b></p> |
|
<p class="last"><strong>Impact of record caches</strong>: |
|
For illustration purposes, the column “KTable <code class="docutils literal"><span class="pre">aggregated</span></code>” below shows the table’s state changes over time in a |
|
very granular way. In practice, you would observe state changes in such a granular way only when |
|
<a class="reference internal" href="memory-mgmt.html#streams-developer-guide-memory-management-record-cache"><span class="std std-ref">record caches</span></a> are disabled (default: enabled). |
|
When record caches are enabled, what might happen for example is that the output results of the rows with timestamps |
|
4 and 5 would be <a class="reference internal" href="memory-mgmt.html#streams-developer-guide-memory-management-record-cache"><span class="std std-ref">compacted</span></a>, and there would only be |
|
a single state update for the key <code class="docutils literal"><span class="pre">kafka</span></code> in the KTable (here: from <code class="docutils literal"><span class="pre">(kafka</span> <span class="pre">1)</span></code> directly to <code class="docutils literal"><span class="pre">(kafka,</span> <span class="pre">3)</span></code>. |
|
Typically, you should only disable record caches for testing or debugging purposes – under normal circumstances it |
|
is better to leave record caches enabled.</p> |
|
</div> |
|
<table border="1" class="docutils"> |
|
<thead valign="bottom"> |
|
<tr class="row-odd"><th class="head"> </th> |
|
<th class="head" colspan="2">KStream <code class="docutils literal"><span class="pre">wordCounts</span></code></th> |
|
<th class="head" colspan="2">KGroupedStream <code class="docutils literal"><span class="pre">groupedStream</span></code></th> |
|
<th class="head">KTable <code class="docutils literal"><span class="pre">aggregated</span></code></th> |
|
</tr> |
|
<tr class="row-even"><th class="head">Timestamp</th> |
|
<th class="head">Input record</th> |
|
<th class="head">Grouping</th> |
|
<th class="head">Initializer</th> |
|
<th class="head">Adder</th> |
|
<th class="head">State</th> |
|
</tr> |
|
</thead> |
|
<tbody valign="top"> |
|
<tr class="row-odd"><td>1</td> |
|
<td>(hello, 1)</td> |
|
<td>(hello, 1)</td> |
|
<td>0 (for hello)</td> |
|
<td>(hello, 0 + 1)</td> |
|
<td><div class="first last line-block"> |
|
<div class="line"><strong>(hello, 1)</strong></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td>2</td> |
|
<td>(kafka, 1)</td> |
|
<td>(kafka, 1)</td> |
|
<td>0 (for kafka)</td> |
|
<td>(kafka, 0 + 1)</td> |
|
<td><div class="first last line-block"> |
|
<div class="line">(hello, 1)</div> |
|
<div class="line"><strong>(kafka, 1)</strong></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td>3</td> |
|
<td>(streams, 1)</td> |
|
<td>(streams, 1)</td> |
|
<td>0 (for streams)</td> |
|
<td>(streams, 0 + 1)</td> |
|
<td><div class="first last line-block"> |
|
<div class="line">(hello, 1)</div> |
|
<div class="line">(kafka, 1)</div> |
|
<div class="line"><strong>(streams, 1)</strong></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td>4</td> |
|
<td>(kafka, 1)</td> |
|
<td>(kafka, 1)</td> |
|
<td> </td> |
|
<td>(kafka, 1 + 1)</td> |
|
<td><div class="first last line-block"> |
|
<div class="line">(hello, 1)</div> |
|
<div class="line">(kafka, <strong>2</strong>)</div> |
|
<div class="line">(streams, 1)</div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td>5</td> |
|
<td>(kafka, 1)</td> |
|
<td>(kafka, 1)</td> |
|
<td> </td> |
|
<td>(kafka, 2 + 1)</td> |
|
<td><div class="first last line-block"> |
|
<div class="line">(hello, 1)</div> |
|
<div class="line">(kafka, <strong>3</strong>)</div> |
|
<div class="line">(streams, 1)</div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td>6</td> |
|
<td>(streams, 1)</td> |
|
<td>(streams, 1)</td> |
|
<td> </td> |
|
<td>(streams, 1 + 1)</td> |
|
<td><div class="first last line-block"> |
|
<div class="line">(hello, 1)</div> |
|
<div class="line">(kafka, 3)</div> |
|
<div class="line">(streams, <strong>2</strong>)</div> |
|
</div> |
|
</td> |
|
</tr> |
|
</tbody> |
|
</table> |
|
<p><strong>Example of semantics for table aggregations:</strong> |
|
A <code class="docutils literal"><span class="pre">KGroupedTable</span></code> → <code class="docutils literal"><span class="pre">KTable</span></code> example is shown below. The tables are initially empty. Bold font is used in the column |
|
for “KTable <code class="docutils literal"><span class="pre">aggregated</span></code>” to highlight changed state. An entry such as <code class="docutils literal"><span class="pre">(hello,</span> <span class="pre">1)</span></code> denotes a record with key |
|
<code class="docutils literal"><span class="pre">hello</span></code> and value <code class="docutils literal"><span class="pre">1</span></code>. To improve the readability of the semantics table you can assume that all records are processed |
|
in timestamp order.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="c1">// Key: username, value: user region (abbreviated to "E" for "Europe", "A" for "Asia")</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">userProfiles</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Re-group `userProfiles`. Don't read too much into what the grouping does:</span> |
|
<span class="c1">// its prime purpose in this example is to show the *effects* of the grouping</span> |
|
<span class="c1">// in the subsequent aggregation.</span> |
|
<span class="n">KGroupedTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Integer</span><span class="o">></span> <span class="n">groupedTable</span> <span class="o">=</span> <span class="n">userProfiles</span> |
|
<span class="o">.</span><span class="na">groupBy</span><span class="o">((</span><span class="n">user</span><span class="o">,</span> <span class="n">region</span><span class="o">)</span> <span class="o">-></span> <span class="n">KeyValue</span><span class="o">.</span><span class="na">pair</span><span class="o">(</span><span class="n">region</span><span class="o">,</span> <span class="n">user</span><span class="o">.</span><span class="na">length</span><span class="o">()),</span> <span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="n">Serdes</span><span class="o">.</span><span class="na">Integer</span><span class="o">());</span> |
|
|
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Integer</span><span class="o">></span> <span class="n">aggregated</span> <span class="o">=</span> <span class="n">groupedTable</span><span class="o">.</span><span class="na">aggregate</span><span class="o">(</span> |
|
<span class="o">()</span> <span class="o">-></span> <span class="mi">0</span><span class="o">,</span> <span class="cm">/* initializer */</span> |
|
<span class="o">(</span><span class="n">aggKey</span><span class="o">,</span> <span class="n">newValue</span><span class="o">,</span> <span class="n">aggValue</span><span class="o">)</span> <span class="o">-></span> <span class="n">aggValue</span> <span class="o">+</span> <span class="n">newValue</span><span class="o">,</span> <span class="cm">/* adder */</span> |
|
<span class="o">(</span><span class="n">aggKey</span><span class="o">,</span> <span class="n">oldValue</span><span class="o">,</span> <span class="n">aggValue</span><span class="o">)</span> <span class="o">-></span> <span class="n">aggValue</span> <span class="o">-</span> <span class="n">oldValue</span><span class="o">,</span> <span class="cm">/* subtractor */</span> |
|
<span class="n">Materialized</span><span class="o">.<</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">,</span> <span class="n">KeyValueStore</span><span class="o"><</span><span class="n">Bytes</span><span class="o">,</span> <span class="kt">byte</span><span class="o">[]></span><span class="n">as</span><span class="o">(</span><span class="s">"aggregated-table-store"</span> <span class="cm">/* state store name */</span><span class="o">)</span> |
|
<span class="o">.</span><span class="na">withKeySerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">())</span> <span class="cm">/* key serde */</span> |
|
<span class="o">.</span><span class="na">withValueSerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">Integer</span><span class="o">());</span> <span class="cm">/* serde for aggregate value */</span> |
|
</pre></div> |
|
</div> |
|
<div class="admonition note"> |
|
<p><b>Note</b></p> |
|
<p class="last"><strong>Impact of record caches</strong>: |
|
For illustration purposes, the column “KTable <code class="docutils literal"><span class="pre">aggregated</span></code>” below shows the table’s state changes over time in a |
|
very granular way. In practice, you would observe state changes in such a granular way only when |
|
<a class="reference internal" href="memory-mgmt.html#streams-developer-guide-memory-management-record-cache"><span class="std std-ref">record caches</span></a> are disabled (default: enabled). |
|
When record caches are enabled, what might happen for example is that the output results of the rows with timestamps |
|
4 and 5 would be <a class="reference internal" href="memory-mgmt.html#streams-developer-guide-memory-management-record-cache"><span class="std std-ref">compacted</span></a>, and there would only be |
|
a single state update for the key <code class="docutils literal"><span class="pre">kafka</span></code> in the KTable (here: from <code class="docutils literal"><span class="pre">(kafka</span> <span class="pre">1)</span></code> directly to <code class="docutils literal"><span class="pre">(kafka,</span> <span class="pre">3)</span></code>. |
|
Typically, you should only disable record caches for testing or debugging purposes – under normal circumstances it |
|
is better to leave record caches enabled.</p> |
|
</div> |
|
<table border="1" class="docutils"> |
|
<thead valign="bottom"> |
|
<tr class="row-odd"><th class="head"> </th> |
|
<th class="head" colspan="3">KTable <code class="docutils literal"><span class="pre">userProfiles</span></code></th> |
|
<th class="head" colspan="3">KGroupedTable <code class="docutils literal"><span class="pre">groupedTable</span></code></th> |
|
<th class="head">KTable <code class="docutils literal"><span class="pre">aggregated</span></code></th> |
|
</tr> |
|
<tr class="row-even"><th class="head">Timestamp</th> |
|
<th class="head">Input record</th> |
|
<th class="head">Interpreted as</th> |
|
<th class="head">Grouping</th> |
|
<th class="head">Initializer</th> |
|
<th class="head">Adder</th> |
|
<th class="head">Subtractor</th> |
|
<th class="head">State</th> |
|
</tr> |
|
</thead> |
|
<tbody valign="top"> |
|
<tr class="row-odd"><td>1</td> |
|
<td>(alice, E)</td> |
|
<td>INSERT alice</td> |
|
<td>(E, 5)</td> |
|
<td>0 (for E)</td> |
|
<td>(E, 0 + 5)</td> |
|
<td> </td> |
|
<td><div class="first last line-block"> |
|
<div class="line"><strong>(E, 5)</strong></div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td>2</td> |
|
<td>(bob, A)</td> |
|
<td>INSERT bob</td> |
|
<td>(A, 3)</td> |
|
<td>0 (for A)</td> |
|
<td>(A, 0 + 3)</td> |
|
<td> </td> |
|
<td><div class="first last line-block"> |
|
<div class="line"><strong>(A, 3)</strong></div> |
|
<div class="line">(E, 5)</div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td>3</td> |
|
<td>(charlie, A)</td> |
|
<td>INSERT charlie</td> |
|
<td>(A, 7)</td> |
|
<td> </td> |
|
<td>(A, 3 + 7)</td> |
|
<td> </td> |
|
<td><div class="first last line-block"> |
|
<div class="line">(A, <strong>10</strong>)</div> |
|
<div class="line">(E, 5)</div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td>4</td> |
|
<td>(alice, A)</td> |
|
<td>UPDATE alice</td> |
|
<td>(A, 5)</td> |
|
<td> </td> |
|
<td>(A, 10 + 5)</td> |
|
<td>(E, 5 - 5)</td> |
|
<td><div class="first last line-block"> |
|
<div class="line">(A, <strong>15</strong>)</div> |
|
<div class="line">(E, <strong>0</strong>)</div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td>5</td> |
|
<td>(charlie, null)</td> |
|
<td>DELETE charlie</td> |
|
<td>(null, 7)</td> |
|
<td> </td> |
|
<td> </td> |
|
<td>(A, 15 - 7)</td> |
|
<td><div class="first last line-block"> |
|
<div class="line">(A, <strong>8</strong>)</div> |
|
<div class="line">(E, 0)</div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td>6</td> |
|
<td>(null, E)</td> |
|
<td><em>ignored</em></td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
<td><div class="first last line-block"> |
|
<div class="line">(A, 8)</div> |
|
<div class="line">(E, 0)</div> |
|
</div> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td>7</td> |
|
<td>(bob, E)</td> |
|
<td>UPDATE bob</td> |
|
<td>(E, 3)</td> |
|
<td> </td> |
|
<td>(E, 0 + 3)</td> |
|
<td>(A, 8 - 3)</td> |
|
<td><div class="first last line-block"> |
|
<div class="line">(A, <strong>5</strong>)</div> |
|
<div class="line">(E, <strong>3</strong>)</div> |
|
</div> |
|
</td> |
|
</tr> |
|
</tbody> |
|
</table> |
|
</div> |
|
<div class="section" id="joining"> |
|
<span id="streams-developer-guide-dsl-joins"></span><h4><a class="toc-backref" href="#id13">Joining</a><a class="headerlink" href="#joining" title="Permalink to this headline"></a></h4> |
|
<p id="streams-developer-guide-dsl-joins-overview">Streams and tables can also be joined. Many stream processing applications in practice are coded as streaming joins. |
|
For example, applications backing an online shop might need to access multiple, updating database tables (e.g. sales |
|
prices, inventory, customer information) in order to enrich a new data record (e.g. customer transaction) with context |
|
information. That is, scenarios where you need to perform table lookups at very large scale and with a low processing |
|
latency. Here, a popular pattern is to make the information in the databases available in Kafka through so-called |
|
<em>change data capture</em> in combination with <a class="reference internal" href="../../#connect"><span class="std std-ref">Kafka’s Connect API</span></a>, and then implementing |
|
applications that leverage the Streams API to perform very fast and efficient local joins |
|
of such tables and streams, rather than requiring the application to make a query to a remote database over the network |
|
for each record. In this example, the KTable concept in Kafka Streams would enable you to track the latest state |
|
(e.g., snapshot) of each table in a local state store, thus greatly reducing the processing latency as well as |
|
reducing the load of the remote databases when doing such streaming joins.</p> |
|
<p>The following join operations are supported, see also the diagram in the |
|
<a class="reference internal" href="#streams-developer-guide-dsl-transformations-stateful-overview"><span class="std std-ref">overview section</span></a> of |
|
<a class="reference internal" href="#streams-developer-guide-dsl-transformations-stateful"><span class="std std-ref">Stateful Transformations</span></a>. |
|
Depending on the operands, joins are either <a class="reference internal" href="#streams-developer-guide-dsl-windowing"><span class="std std-ref">windowed</span></a> joins or |
|
non-windowed joins.</p> |
|
<table border="1" class="docutils"> |
|
<thead valign="bottom"> |
|
<tr class="row-odd"><th class="head">Join operands</th> |
|
<th class="head">Type</th> |
|
<th class="head">(INNER) JOIN</th> |
|
<th class="head">LEFT JOIN</th> |
|
<th class="head">OUTER JOIN</th> |
|
</tr> |
|
</thead> |
|
<tbody valign="top"> |
|
<tr class="row-even"><td>KStream-to-KStream</td> |
|
<td>Windowed</td> |
|
<td>Supported</td> |
|
<td>Supported</td> |
|
<td>Supported</td> |
|
</tr> |
|
<tr class="row-odd"><td>KTable-to-KTable</td> |
|
<td>Non-windowed</td> |
|
<td>Supported</td> |
|
<td>Supported</td> |
|
<td>Supported</td> |
|
</tr> |
|
<tr class="row-even"><td>KStream-to-KTable</td> |
|
<td>Non-windowed</td> |
|
<td>Supported</td> |
|
<td>Supported</td> |
|
<td>Not Supported</td> |
|
</tr> |
|
<tr class="row-odd"><td>KStream-to-GlobalKTable</td> |
|
<td>Non-windowed</td> |
|
<td>Supported</td> |
|
<td>Supported</td> |
|
<td>Not Supported</td> |
|
</tr> |
|
<tr class="row-even"><td>KTable-to-GlobalKTable</td> |
|
<td>N/A</td> |
|
<td>Not Supported</td> |
|
<td>Not Supported</td> |
|
<td>Not Supported</td> |
|
</tr> |
|
</tbody> |
|
</table> |
|
<p>Each case is explained in more detail in the subsequent sections.</p> |
|
<div class="section" id="join-co-partitioning-requirements"> |
|
<span id="streams-developer-guide-dsl-joins-co-partitioning"></span><h5><a class="toc-backref" href="#id14">Join co-partitioning requirements</a><a class="headerlink" href="#join-co-partitioning-requirements" title="Permalink to this headline"></a></h5> |
|
<p>Input data must be co-partitioned when joining. This ensures that input records with the same key, from both sides of the |
|
join, are delivered to the same stream task during processing. |
|
<strong>It is the responsibility of the user to ensure data co-partitioning when joining</strong>.</p> |
|
<div class="admonition tip"> |
|
<p><b>Tip</b></p> |
|
<p class="last">If possible, consider using <a class="reference internal" href="#streams_concepts_globalktable"><span class="std std-ref">global tables</span></a> (<code class="docutils literal"><span class="pre">GlobalKTable</span></code>) for joining because they do not require data co-partitioning.</p> |
|
</div> |
|
<p>The requirements for data co-partitioning are:</p> |
|
<ul class="simple"> |
|
<li>The input topics of the join (left side and right side) must have the <strong>same number of partitions</strong>.</li> |
|
<li>All applications that <em>write</em> to the input topics must have the <strong>same partitioning strategy</strong> so that records with |
|
the same key are delivered to same partition number. In other words, the keyspace of the input data must be |
|
distributed across partitions in the same manner. |
|
This means that, for example, applications that use Kafka’s <a class="reference internal" href="../../#producerapi"><span class="std std-ref">Java Producer API</span></a> must use the |
|
same partitioner (cf. the producer setting <code class="docutils literal"><span class="pre">"partitioner.class"</span></code> aka <code class="docutils literal"><span class="pre">ProducerConfig.PARTITIONER_CLASS_CONFIG</span></code>), |
|
and applications that use the Kafka’s Streams API must use the same <code class="docutils literal"><span class="pre">StreamPartitioner</span></code> for operations such as |
|
<code class="docutils literal"><span class="pre">KStream#to()</span></code>. The good news is that, if you happen to use the default partitioner-related settings across all |
|
applications, you do not need to worry about the partitioning strategy.</li> |
|
</ul> |
|
<p>Why is data co-partitioning required? Because |
|
<a class="reference internal" href="#streams-developer-guide-dsl-joins-kstream-kstream"><span class="std std-ref">KStream-KStream</span></a>, |
|
<a class="reference internal" href="#streams-developer-guide-dsl-joins-ktable-ktable"><span class="std std-ref">KTable-KTable</span></a>, and |
|
<a class="reference internal" href="#streams-developer-guide-dsl-joins-kstream-ktable"><span class="std std-ref">KStream-KTable</span></a> joins |
|
are performed based on the keys of records (e.g., <code class="docutils literal"><span class="pre">leftRecord.key</span> <span class="pre">==</span> <span class="pre">rightRecord.key</span></code>), it is required that the |
|
input streams/tables of a join are co-partitioned by key.</p> |
|
<p>The only exception are |
|
<a class="reference internal" href="#streams-developer-guide-dsl-joins-kstream-globalktable"><span class="std std-ref">KStream-GlobalKTable joins</span></a>. Here, co-partitioning is |
|
it not required because <em>all</em> partitions of the <code class="docutils literal"><span class="pre">GlobalKTable</span></code>‘s underlying changelog stream are made available to |
|
each <code class="docutils literal"><span class="pre">KafkaStreams</span></code> instance, i.e. each instance has a full copy of the changelog stream. Further, a |
|
<code class="docutils literal"><span class="pre">KeyValueMapper</span></code> allows for non-key based joins from the <code class="docutils literal"><span class="pre">KStream</span></code> to the <code class="docutils literal"><span class="pre">GlobalKTable</span></code>.</p> |
|
<div class="admonition note"> |
|
<p><b>Note</b></p> |
|
<p class="last"><strong>Kafka Streams partly verifies the co-partitioning requirement:</strong> |
|
During the partition assignment step, i.e. at runtime, Kafka Streams verifies whether the number of partitions for |
|
both sides of a join are the same. If they are not, a <code class="docutils literal"><span class="pre">TopologyBuilderException</span></code> (runtime exception) is being |
|
thrown. Note that Kafka Streams cannot verify whether the partitioning strategy matches between the input |
|
streams/tables of a join – it is up to the user to ensure that this is the case.</p> |
|
</div> |
|
<p><strong>Ensuring data co-partitioning:</strong> If the inputs of a join are not co-partitioned yet, you must ensure this manually. |
|
You may follow a procedure such as outlined below.</p> |
|
<ol class="arabic"> |
|
<li><p class="first">Identify the input KStream/KTable in the join whose underlying Kafka topic has the smaller number of partitions. |
|
Let’s call this stream/table “SMALLER”, and the other side of the join “LARGER”. To learn about the number of |
|
partitions of a Kafka topic you can use, for example, the CLI tool <code class="docutils literal"><span class="pre">bin/kafka-topics</span></code> with the <code class="docutils literal"><span class="pre">--describe</span></code> |
|
option.</p> |
|
</li> |
|
<li><p class="first">Pre-create a new Kafka topic for “SMALLER” that has the same number of partitions as “LARGER”. Let’s call this |
|
new topic “repartitioned-topic-for-smaller”. Typically, you’d use the CLI tool <code class="docutils literal"><span class="pre">bin/kafka-topics</span></code> with the |
|
<code class="docutils literal"><span class="pre">--create</span></code> option for this.</p> |
|
</li> |
|
<li><p class="first">Within your application, re-write the data of “SMALLER” into the new Kafka topic. You must ensure that, when writing |
|
the data with <code class="docutils literal"><span class="pre">to</span></code> or <code class="docutils literal"><span class="pre">through</span></code>, the same partitioner is used as for “LARGER”.</p> |
|
<blockquote> |
|
<div><ul class="simple"> |
|
<li>If “SMALLER” is a KStream: <code class="docutils literal"><span class="pre">KStream#to("repartitioned-topic-for-smaller")</span></code>.</li> |
|
<li>If “SMALLER” is a KTable: <code class="docutils literal"><span class="pre">KTable#to("repartitioned-topic-for-smaller")</span></code>.</li> |
|
</ul> |
|
</div></blockquote> |
|
</li> |
|
<li><p class="first">Within your application, re-read the data in “repartitioned-topic-for-smaller” into |
|
a new KStream/KTable.</p> |
|
<blockquote> |
|
<div><ul class="simple"> |
|
<li>If “SMALLER” is a KStream: <code class="docutils literal"><span class="pre">StreamsBuilder#stream("repartitioned-topic-for-smaller")</span></code>.</li> |
|
<li>If “SMALLER” is a KTable: <code class="docutils literal"><span class="pre">StreamsBuilder#table("repartitioned-topic-for-smaller")</span></code>.</li> |
|
</ul> |
|
</div></blockquote> |
|
</li> |
|
<li><p class="first">Within your application, perform the join between “LARGER” and the new stream/table.</p> |
|
</li> |
|
</ol> |
|
</div> |
|
<div class="section" id="kstream-kstream-join"> |
|
<span id="streams-developer-guide-dsl-joins-kstream-kstream"></span><h5><a class="toc-backref" href="#id15">KStream-KStream Join</a><a class="headerlink" href="#kstream-kstream-join" title="Permalink to this headline"></a></h5> |
|
<p>KStream-KStream joins are always <a class="reference internal" href="#windowing-sliding"><span class="std std-ref">windowed</span></a> joins, because otherwise the size of the |
|
internal state store used to perform the join – e.g., a <a class="reference internal" href="#windowing-sliding"><span class="std std-ref">sliding window</span></a> or “buffer” – would |
|
grow indefinitely. For stream-stream joins it’s important to highlight that a new input record on one side will |
|
produce a join output <em>for each</em> matching record on the other side, and there can be <em>multiple</em> such matching records |
|
in a given join window (cf. the row with timestamp 15 in the join semantics table below, for example).</p> |
|
<p>Join output records are effectively created as follows, leveraging the user-supplied <code class="docutils literal"><span class="pre">ValueJoiner</span></code>:</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">KeyValue</span><span class="o"><</span><span class="n">K</span><span class="o">,</span> <span class="n">LV</span><span class="o">></span> <span class="n">leftRecord</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KeyValue</span><span class="o"><</span><span class="n">K</span><span class="o">,</span> <span class="n">RV</span><span class="o">></span> <span class="n">rightRecord</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">ValueJoiner</span><span class="o"><</span><span class="n">LV</span><span class="o">,</span> <span class="n">RV</span><span class="o">,</span> <span class="n">JV</span><span class="o">></span> <span class="n">joiner</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="n">KeyValue</span><span class="o"><</span><span class="n">K</span><span class="o">,</span> <span class="n">JV</span><span class="o">></span> <span class="n">joinOutputRecord</span> <span class="o">=</span> <span class="n">KeyValue</span><span class="o">.</span><span class="na">pair</span><span class="o">(</span> |
|
<span class="n">leftRecord</span><span class="o">.</span><span class="na">key</span><span class="o">,</span> <span class="cm">/* by definition, leftRecord.key == rightRecord.key */</span> |
|
<span class="n">joiner</span><span class="o">.</span><span class="na">apply</span><span class="o">(</span><span class="n">leftRecord</span><span class="o">.</span><span class="na">value</span><span class="o">,</span> <span class="n">rightRecord</span><span class="o">.</span><span class="na">value</span><span class="o">)</span> |
|
<span class="o">);</span> |
|
</pre></div> |
|
</div> |
|
<table border="1" class="non-scrolling-table width-100-percent docutils"> |
|
<colgroup> |
|
<col width="15%" /> |
|
<col width="85%" /> |
|
</colgroup> |
|
<thead valign="bottom"> |
|
<tr class="row-odd"><th class="head">Transformation</th> |
|
<th class="head">Description</th> |
|
</tr> |
|
</thead> |
|
<tbody valign="top"> |
|
<tr class="row-even"><td><p class="first"><strong>Inner Join (windowed)</strong></p> |
|
<ul class="last simple"> |
|
<li>(KStream, KStream) |
|
→ KStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Performs an INNER JOIN of this stream with another stream. |
|
Even though this operation is windowed, the joined stream will be of type <code class="docutils literal"><span class="pre">KStream<K,</span> <span class="pre">...></span></code> rather than <code class="docutils literal"><span class="pre">KStream<Windowed<K>,</span> <span class="pre">...></span></code>. |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#join-org.apache.kafka.streams.kstream.KStream-org.apache.kafka.streams.kstream.ValueJoiner-org.apache.kafka.streams.kstream.JoinWindows-">(details)</a></p> |
|
<p><strong>Data must be co-partitioned</strong>: The input data for both sides must be <a class="reference internal" href="#streams-developer-guide-dsl-joins-co-partitioning"><span class="std std-ref">co-partitioned</span></a>.</p> |
|
<p><strong>Causes data re-partitioning of a stream if and only if the stream was marked for re-partitioning (if both are marked, both are re-partitioned).</strong></p> |
|
<p>Several variants of <code class="docutils literal"><span class="pre">join</span></code> exists, see the Javadocs for details.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">java.util.concurrent.TimeUnit</span><span class="o">;</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">left</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Double</span><span class="o">></span> <span class="n">right</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">join</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="o">(</span><span class="n">leftValue</span><span class="o">,</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">-></span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span><span class="o">,</span> <span class="cm">/* ValueJoiner */</span> |
|
<span class="n">JoinWindows</span><span class="o">.</span><span class="na">of</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">)),</span> |
|
<span class="n">Joined</span><span class="o">.</span><span class="na">with</span><span class="o">(</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="cm">/* key */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">(),</span> <span class="cm">/* left value */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">Double</span><span class="o">())</span> <span class="cm">/* right value */</span> |
|
<span class="o">);</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">join</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="k">new</span> <span class="n">ValueJoiner</span><span class="o"><</span><span class="n">Long</span><span class="o">,</span> <span class="n">Double</span><span class="o">,</span> <span class="n">String</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">String</span> <span class="nf">apply</span><span class="o">(</span><span class="n">Long</span> <span class="n">leftValue</span><span class="o">,</span> <span class="n">Double</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="n">JoinWindows</span><span class="o">.</span><span class="na">of</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">)),</span> |
|
<span class="n">Joined</span><span class="o">.</span><span class="na">with</span><span class="o">(</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="cm">/* key */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">(),</span> <span class="cm">/* left value */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">Double</span><span class="o">())</span> <span class="cm">/* right value */</span> |
|
<span class="o">);</span> |
|
</pre></div> |
|
</div> |
|
<p>Detailed behavior:</p> |
|
<ul> |
|
<li><p class="first">The join is <em>key-based</em>, i.e. with the join predicate <code class="docutils literal"><span class="pre">leftRecord.key</span> <span class="pre">==</span> <span class="pre">rightRecord.key</span></code>, and <em>window-based</em>, i.e. two input records are joined if and only if their |
|
timestamps are “close” to each other as defined by the user-supplied <code class="docutils literal"><span class="pre">JoinWindows</span></code>, i.e. the window defines an additional join predicate over the record timestamps.</p> |
|
</li> |
|
<li><p class="first">The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied <code class="docutils literal"><span class="pre">ValueJoiner</span></code> will be called to produce |
|
join output records.</p> |
|
<blockquote> |
|
<div><ul class="simple"> |
|
<li>Input records with a <code class="docutils literal"><span class="pre">null</span></code> key or a <code class="docutils literal"><span class="pre">null</span></code> value are ignored and do not trigger the join.</li> |
|
</ul> |
|
</div></blockquote> |
|
</li> |
|
</ul> |
|
<p class="last">See the semantics overview at the bottom of this section for a detailed description.</p> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>Left Join (windowed)</strong></p> |
|
<ul class="last simple"> |
|
<li>(KStream, KStream) |
|
→ KStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Performs a LEFT JOIN of this stream with another stream. |
|
Even though this operation is windowed, the joined stream will be of type <code class="docutils literal"><span class="pre">KStream<K,</span> <span class="pre">...></span></code> rather than <code class="docutils literal"><span class="pre">KStream<Windowed<K>,</span> <span class="pre">...></span></code>. |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#leftJoin-org.apache.kafka.streams.kstream.KStream-org.apache.kafka.streams.kstream.ValueJoiner-org.apache.kafka.streams.kstream.JoinWindows-">(details)</a></p> |
|
<p><strong>Data must be co-partitioned</strong>: The input data for both sides must be <a class="reference internal" href="#streams-developer-guide-dsl-joins-co-partitioning"><span class="std std-ref">co-partitioned</span></a>.</p> |
|
<p><strong>Causes data re-partitioning of a stream if and only if the stream was marked for re-partitioning (if both are marked, both are re-partitioned).</strong></p> |
|
<p>Several variants of <code class="docutils literal"><span class="pre">leftJoin</span></code> exists, see the Javadocs for details.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">java.util.concurrent.TimeUnit</span><span class="o">;</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">left</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Double</span><span class="o">></span> <span class="n">right</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">leftJoin</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="o">(</span><span class="n">leftValue</span><span class="o">,</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">-></span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span><span class="o">,</span> <span class="cm">/* ValueJoiner */</span> |
|
<span class="n">JoinWindows</span><span class="o">.</span><span class="na">of</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">)),</span> |
|
<span class="n">Joined</span><span class="o">.</span><span class="na">with</span><span class="o">(</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="cm">/* key */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">(),</span> <span class="cm">/* left value */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">Double</span><span class="o">())</span> <span class="cm">/* right value */</span> |
|
<span class="o">);</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">leftJoin</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="k">new</span> <span class="n">ValueJoiner</span><span class="o"><</span><span class="n">Long</span><span class="o">,</span> <span class="n">Double</span><span class="o">,</span> <span class="n">String</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">String</span> <span class="nf">apply</span><span class="o">(</span><span class="n">Long</span> <span class="n">leftValue</span><span class="o">,</span> <span class="n">Double</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="n">JoinWindows</span><span class="o">.</span><span class="na">of</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">)),</span> |
|
<span class="n">Joined</span><span class="o">.</span><span class="na">with</span><span class="o">(</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="cm">/* key */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">(),</span> <span class="cm">/* left value */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">Double</span><span class="o">())</span> <span class="cm">/* right value */</span> |
|
<span class="o">);</span> |
|
</pre></div> |
|
</div> |
|
<p>Detailed behavior:</p> |
|
<ul> |
|
<li><p class="first">The join is <em>key-based</em>, i.e. with the join predicate <code class="docutils literal"><span class="pre">leftRecord.key</span> <span class="pre">==</span> <span class="pre">rightRecord.key</span></code>, and <em>window-based</em>, i.e. two input records are joined if and only if their |
|
timestamps are “close” to each other as defined by the user-supplied <code class="docutils literal"><span class="pre">JoinWindows</span></code>, i.e. the window defines an additional join predicate over the record timestamps.</p> |
|
</li> |
|
<li><p class="first">The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied <code class="docutils literal"><span class="pre">ValueJoiner</span></code> will be called to produce |
|
join output records.</p> |
|
<blockquote> |
|
<div><ul class="simple"> |
|
<li>Input records with a <code class="docutils literal"><span class="pre">null</span></code> key or a <code class="docutils literal"><span class="pre">null</span></code> value are ignored and do not trigger the join.</li> |
|
</ul> |
|
</div></blockquote> |
|
</li> |
|
<li><p class="first">For each input record on the left side that does not have any match on the right side, the <code class="docutils literal"><span class="pre">ValueJoiner</span></code> will be called with <code class="docutils literal"><span class="pre">ValueJoiner#apply(leftRecord.value,</span> <span class="pre">null)</span></code>; |
|
this explains the row with timestamp=3 in the table below, which lists <code class="docutils literal"><span class="pre">[A,</span> <span class="pre">null]</span></code> in the LEFT JOIN column.</p> |
|
</li> |
|
</ul> |
|
<p class="last">See the semantics overview at the bottom of this section for a detailed description.</p> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td><p class="first"><strong>Outer Join (windowed)</strong></p> |
|
<ul class="last simple"> |
|
<li>(KStream, KStream) |
|
→ KStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Performs an OUTER JOIN of this stream with another stream. |
|
Even though this operation is windowed, the joined stream will be of type <code class="docutils literal"><span class="pre">KStream<K,</span> <span class="pre">...></span></code> rather than <code class="docutils literal"><span class="pre">KStream<Windowed<K>,</span> <span class="pre">...></span></code>. |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#outerJoin-org.apache.kafka.streams.kstream.KStream-org.apache.kafka.streams.kstream.ValueJoiner-org.apache.kafka.streams.kstream.JoinWindows-">(details)</a></p> |
|
<p><strong>Data must be co-partitioned</strong>: The input data for both sides must be <a class="reference internal" href="#streams-developer-guide-dsl-joins-co-partitioning"><span class="std std-ref">co-partitioned</span></a>.</p> |
|
<p><strong>Causes data re-partitioning of a stream if and only if the stream was marked for re-partitioning (if both are marked, both are re-partitioned).</strong></p> |
|
<p>Several variants of <code class="docutils literal"><span class="pre">outerJoin</span></code> exists, see the Javadocs for details.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">java.util.concurrent.TimeUnit</span><span class="o">;</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">left</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Double</span><span class="o">></span> <span class="n">right</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">outerJoin</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="o">(</span><span class="n">leftValue</span><span class="o">,</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">-></span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span><span class="o">,</span> <span class="cm">/* ValueJoiner */</span> |
|
<span class="n">JoinWindows</span><span class="o">.</span><span class="na">of</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">)),</span> |
|
<span class="n">Joined</span><span class="o">.</span><span class="na">with</span><span class="o">(</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="cm">/* key */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">(),</span> <span class="cm">/* left value */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">Double</span><span class="o">())</span> <span class="cm">/* right value */</span> |
|
<span class="o">);</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">outerJoin</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="k">new</span> <span class="n">ValueJoiner</span><span class="o"><</span><span class="n">Long</span><span class="o">,</span> <span class="n">Double</span><span class="o">,</span> <span class="n">String</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">String</span> <span class="nf">apply</span><span class="o">(</span><span class="n">Long</span> <span class="n">leftValue</span><span class="o">,</span> <span class="n">Double</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="n">JoinWindows</span><span class="o">.</span><span class="na">of</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">)),</span> |
|
<span class="n">Joined</span><span class="o">.</span><span class="na">with</span><span class="o">(</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="cm">/* key */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">(),</span> <span class="cm">/* left value */</span> |
|
<span class="n">Serdes</span><span class="o">.</span><span class="na">Double</span><span class="o">())</span> <span class="cm">/* right value */</span> |
|
<span class="o">);</span> |
|
</pre></div> |
|
</div> |
|
<p>Detailed behavior:</p> |
|
<ul> |
|
<li><p class="first">The join is <em>key-based</em>, i.e. with the join predicate <code class="docutils literal"><span class="pre">leftRecord.key</span> <span class="pre">==</span> <span class="pre">rightRecord.key</span></code>, and <em>window-based</em>, i.e. two input records are joined if and only if their |
|
timestamps are “close” to each other as defined by the user-supplied <code class="docutils literal"><span class="pre">JoinWindows</span></code>, i.e. the window defines an additional join predicate over the record timestamps.</p> |
|
</li> |
|
<li><p class="first">The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied <code class="docutils literal"><span class="pre">ValueJoiner</span></code> will be called to produce |
|
join output records.</p> |
|
<blockquote> |
|
<div><ul class="simple"> |
|
<li>Input records with a <code class="docutils literal"><span class="pre">null</span></code> key or a <code class="docutils literal"><span class="pre">null</span></code> value are ignored and do not trigger the join.</li> |
|
</ul> |
|
</div></blockquote> |
|
</li> |
|
<li><p class="first">For each input record on one side that does not have any match on the other side, the <code class="docutils literal"><span class="pre">ValueJoiner</span></code> will be called with <code class="docutils literal"><span class="pre">ValueJoiner#apply(leftRecord.value,</span> <span class="pre">null)</span></code> or |
|
<code class="docutils literal"><span class="pre">ValueJoiner#apply(null,</span> <span class="pre">rightRecord.value)</span></code>, respectively; this explains the row with timestamp=3 in the table below, which lists <code class="docutils literal"><span class="pre">[A,</span> <span class="pre">null]</span></code> in the OUTER JOIN column |
|
(unlike LEFT JOIN, <code class="docutils literal"><span class="pre">[null,</span> <span class="pre">x]</span></code> is possible, too, but no such example is shown in the table).</p> |
|
</li> |
|
</ul> |
|
<p class="last">See the semantics overview at the bottom of this section for a detailed description.</p> |
|
</td> |
|
</tr> |
|
</tbody> |
|
</table> |
|
<p><strong>Semantics of stream-stream joins:</strong> |
|
The semantics of the various stream-stream join variants are explained below. |
|
To improve the readability of the table, assume that (1) all records have the same key (and thus the key in the table is omitted), (2) all records belong to a single join window, and (3) all records are processed in timestamp order. |
|
The columns INNER JOIN, LEFT JOIN, and OUTER JOIN denote what is passed as arguments to the user-supplied |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/ValueJoiner.html">ValueJoiner</a> for the <code class="docutils literal"><span class="pre">join</span></code>, <code class="docutils literal"><span class="pre">leftJoin</span></code>, and |
|
<code class="docutils literal"><span class="pre">outerJoin</span></code> methods, respectively, whenever a new input record is received on either side of the join. An empty table |
|
cell denotes that the <code class="docutils literal"><span class="pre">ValueJoiner</span></code> is not called at all.</p> |
|
<table border="1" class="docutils"> |
|
<thead valign="bottom"> |
|
<tr class="row-odd"><th class="head">Timestamp</th> |
|
<th class="head">Left (KStream)</th> |
|
<th class="head">Right (KStream)</th> |
|
<th class="head">(INNER) JOIN</th> |
|
<th class="head">LEFT JOIN</th> |
|
<th class="head">OUTER JOIN</th> |
|
</tr> |
|
</thead> |
|
<tbody valign="top"> |
|
<tr class="row-even"><td>1</td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-odd"><td>2</td> |
|
<td> </td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-even"><td>3</td> |
|
<td>A</td> |
|
<td> </td> |
|
<td> </td> |
|
<td>[A, null]</td> |
|
<td>[A, null]</td> |
|
</tr> |
|
<tr class="row-odd"><td>4</td> |
|
<td> </td> |
|
<td>a</td> |
|
<td>[A, a]</td> |
|
<td>[A, a]</td> |
|
<td>[A, a]</td> |
|
</tr> |
|
<tr class="row-even"><td>5</td> |
|
<td>B</td> |
|
<td> </td> |
|
<td>[B, a]</td> |
|
<td>[B, a]</td> |
|
<td>[B, a]</td> |
|
</tr> |
|
<tr class="row-odd"><td>6</td> |
|
<td> </td> |
|
<td>b</td> |
|
<td>[A, b], [B, b]</td> |
|
<td>[A, b], [B, b]</td> |
|
<td>[A, b], [B, b]</td> |
|
</tr> |
|
<tr class="row-even"><td>7</td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-odd"><td>8</td> |
|
<td> </td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-even"><td>9</td> |
|
<td>C</td> |
|
<td> </td> |
|
<td>[C, a], [C, b]</td> |
|
<td>[C, a], [C, b]</td> |
|
<td>[C, a], [C, b]</td> |
|
</tr> |
|
<tr class="row-odd"><td>10</td> |
|
<td> </td> |
|
<td>c</td> |
|
<td>[A, c], [B, c], [C, c]</td> |
|
<td>[A, c], [B, c], [C, c]</td> |
|
<td>[A, c], [B, c], [C, c]</td> |
|
</tr> |
|
<tr class="row-even"><td>11</td> |
|
<td> </td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-odd"><td>12</td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-even"><td>13</td> |
|
<td> </td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-odd"><td>14</td> |
|
<td> </td> |
|
<td>d</td> |
|
<td>[A, d], [B, d], [C, d]</td> |
|
<td>[A, d], [B, d], [C, d]</td> |
|
<td>[A, d], [B, d], [C, d]</td> |
|
</tr> |
|
<tr class="row-even"><td>15</td> |
|
<td>D</td> |
|
<td> </td> |
|
<td>[D, a], [D, b], [D, c], [D, d]</td> |
|
<td>[D, a], [D, b], [D, c], [D, d]</td> |
|
<td>[D, a], [D, b], [D, c], [D, d]</td> |
|
</tr> |
|
</tbody> |
|
</table> |
|
</div> |
|
<div class="section" id="ktable-ktable-join"> |
|
<span id="streams-developer-guide-dsl-joins-ktable-ktable"></span><h5><a class="toc-backref" href="#id16">KTable-KTable Join</a><a class="headerlink" href="#ktable-ktable-join" title="Permalink to this headline"></a></h5> |
|
<p>KTable-KTable joins are always <em>non-windowed</em> joins. They are designed to be consistent with their counterparts in |
|
relational databases. The changelog streams of both KTables are materialized into local state stores to represent the |
|
latest snapshot of their <a class="reference internal" href="#streams_concepts_ktable"><span class="std std-ref">table duals</span></a>. |
|
The join result is a new KTable that represents the changelog stream of the join operation.</p> |
|
<p>Join output records are effectively created as follows, leveraging the user-supplied <code class="docutils literal"><span class="pre">ValueJoiner</span></code>:</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">KeyValue</span><span class="o"><</span><span class="n">K</span><span class="o">,</span> <span class="n">LV</span><span class="o">></span> <span class="n">leftRecord</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KeyValue</span><span class="o"><</span><span class="n">K</span><span class="o">,</span> <span class="n">RV</span><span class="o">></span> <span class="n">rightRecord</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">ValueJoiner</span><span class="o"><</span><span class="n">LV</span><span class="o">,</span> <span class="n">RV</span><span class="o">,</span> <span class="n">JV</span><span class="o">></span> <span class="n">joiner</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="n">KeyValue</span><span class="o"><</span><span class="n">K</span><span class="o">,</span> <span class="n">JV</span><span class="o">></span> <span class="n">joinOutputRecord</span> <span class="o">=</span> <span class="n">KeyValue</span><span class="o">.</span><span class="na">pair</span><span class="o">(</span> |
|
<span class="n">leftRecord</span><span class="o">.</span><span class="na">key</span><span class="o">,</span> <span class="cm">/* by definition, leftRecord.key == rightRecord.key */</span> |
|
<span class="n">joiner</span><span class="o">.</span><span class="na">apply</span><span class="o">(</span><span class="n">leftRecord</span><span class="o">.</span><span class="na">value</span><span class="o">,</span> <span class="n">rightRecord</span><span class="o">.</span><span class="na">value</span><span class="o">)</span> |
|
<span class="o">);</span> |
|
</pre></div> |
|
</div> |
|
<table border="1" class="non-scrolling-table width-100-percent docutils"> |
|
<colgroup> |
|
<col width="15%" /> |
|
<col width="85%" /> |
|
</colgroup> |
|
<thead valign="bottom"> |
|
<tr class="row-odd"><th class="head">Transformation</th> |
|
<th class="head">Description</th> |
|
</tr> |
|
</thead> |
|
<tbody valign="top"> |
|
<tr class="row-even"><td><p class="first"><strong>Inner Join</strong></p> |
|
<ul class="last simple"> |
|
<li>(KTable, KTable) |
|
→ KTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Performs an INNER JOIN of this table with another table. |
|
The result is an ever-updating KTable that represents the “current” result of the join. |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KTable.html#join-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-">(details)</a></p> |
|
<p><strong>Data must be co-partitioned</strong>: The input data for both sides must be <a class="reference internal" href="#streams-developer-guide-dsl-joins-co-partitioning"><span class="std std-ref">co-partitioned</span></a>.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">left</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Double</span><span class="o">></span> <span class="n">right</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">join</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="o">(</span><span class="n">leftValue</span><span class="o">,</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">-></span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span> <span class="cm">/* ValueJoiner */</span> |
|
<span class="o">);</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">join</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="k">new</span> <span class="n">ValueJoiner</span><span class="o"><</span><span class="n">Long</span><span class="o">,</span> <span class="n">Double</span><span class="o">,</span> <span class="n">String</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">String</span> <span class="nf">apply</span><span class="o">(</span><span class="n">Long</span> <span class="n">leftValue</span><span class="o">,</span> <span class="n">Double</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
</pre></div> |
|
</div> |
|
<p>Detailed behavior:</p> |
|
<ul> |
|
<li><p class="first">The join is <em>key-based</em>, i.e. with the join predicate <code class="docutils literal"><span class="pre">leftRecord.key</span> <span class="pre">==</span> <span class="pre">rightRecord.key</span></code>.</p> |
|
</li> |
|
<li><p class="first">The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied <code class="docutils literal"><span class="pre">ValueJoiner</span></code> will be called to produce |
|
join output records.</p> |
|
<blockquote> |
|
<div><ul class="simple"> |
|
<li>Input records with a <code class="docutils literal"><span class="pre">null</span></code> key are ignored and do not trigger the join.</li> |
|
<li>Input records with a <code class="docutils literal"><span class="pre">null</span></code> value are interpreted as <em>tombstones</em> for the corresponding key, which indicate the deletion of the key from the table. Tombstones do not |
|
trigger the join. When an input tombstone is received, then an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding |
|
key actually exists already in the join result KTable).</li> |
|
</ul> |
|
</div></blockquote> |
|
</li> |
|
</ul> |
|
<p class="last">See the semantics overview at the bottom of this section for a detailed description.</p> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>Left Join</strong></p> |
|
<ul class="last simple"> |
|
<li>(KTable, KTable) |
|
→ KTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Performs a LEFT JOIN of this table with another table. |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KTable.html#leftJoin-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-">(details)</a></p> |
|
<p><strong>Data must be co-partitioned</strong>: The input data for both sides must be <a class="reference internal" href="#streams-developer-guide-dsl-joins-co-partitioning"><span class="std std-ref">co-partitioned</span></a>.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">left</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Double</span><span class="o">></span> <span class="n">right</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">leftJoin</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="o">(</span><span class="n">leftValue</span><span class="o">,</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">-></span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span> <span class="cm">/* ValueJoiner */</span> |
|
<span class="o">);</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">leftJoin</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="k">new</span> <span class="n">ValueJoiner</span><span class="o"><</span><span class="n">Long</span><span class="o">,</span> <span class="n">Double</span><span class="o">,</span> <span class="n">String</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">String</span> <span class="nf">apply</span><span class="o">(</span><span class="n">Long</span> <span class="n">leftValue</span><span class="o">,</span> <span class="n">Double</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
</pre></div> |
|
</div> |
|
<p>Detailed behavior:</p> |
|
<ul> |
|
<li><p class="first">The join is <em>key-based</em>, i.e. with the join predicate <code class="docutils literal"><span class="pre">leftRecord.key</span> <span class="pre">==</span> <span class="pre">rightRecord.key</span></code>.</p> |
|
</li> |
|
<li><p class="first">The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied <code class="docutils literal"><span class="pre">ValueJoiner</span></code> will be called to produce |
|
join output records.</p> |
|
<blockquote> |
|
<div><ul class="simple"> |
|
<li>Input records with a <code class="docutils literal"><span class="pre">null</span></code> key are ignored and do not trigger the join.</li> |
|
<li>Input records with a <code class="docutils literal"><span class="pre">null</span></code> value are interpreted as <em>tombstones</em> for the corresponding key, which indicate the deletion of the key from the table. Tombstones do not |
|
trigger the join. When an input tombstone is received, then an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding |
|
key actually exists already in the join result KTable).</li> |
|
</ul> |
|
</div></blockquote> |
|
</li> |
|
<li><p class="first">For each input record on the left side that does not have any match on the right side, the <code class="docutils literal"><span class="pre">ValueJoiner</span></code> will be called with <code class="docutils literal"><span class="pre">ValueJoiner#apply(leftRecord.value,</span> <span class="pre">null)</span></code>; |
|
this explains the row with timestamp=3 in the table below, which lists <code class="docutils literal"><span class="pre">[A,</span> <span class="pre">null]</span></code> in the LEFT JOIN column.</p> |
|
</li> |
|
</ul> |
|
<p class="last">See the semantics overview at the bottom of this section for a detailed description.</p> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td><p class="first"><strong>Outer Join</strong></p> |
|
<ul class="last simple"> |
|
<li>(KTable, KTable) |
|
→ KTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Performs an OUTER JOIN of this table with another table. |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KTable.html#outerJoin-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-">(details)</a></p> |
|
<p><strong>Data must be co-partitioned</strong>: The input data for both sides must be <a class="reference internal" href="#streams-developer-guide-dsl-joins-co-partitioning"><span class="std std-ref">co-partitioned</span></a>.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">left</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Double</span><span class="o">></span> <span class="n">right</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">outerJoin</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="o">(</span><span class="n">leftValue</span><span class="o">,</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">-></span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span> <span class="cm">/* ValueJoiner */</span> |
|
<span class="o">);</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">outerJoin</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="k">new</span> <span class="n">ValueJoiner</span><span class="o"><</span><span class="n">Long</span><span class="o">,</span> <span class="n">Double</span><span class="o">,</span> <span class="n">String</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">String</span> <span class="nf">apply</span><span class="o">(</span><span class="n">Long</span> <span class="n">leftValue</span><span class="o">,</span> <span class="n">Double</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
</pre></div> |
|
</div> |
|
<p>Detailed behavior:</p> |
|
<ul> |
|
<li><p class="first">The join is <em>key-based</em>, i.e. with the join predicate <code class="docutils literal"><span class="pre">leftRecord.key</span> <span class="pre">==</span> <span class="pre">rightRecord.key</span></code>.</p> |
|
</li> |
|
<li><p class="first">The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied <code class="docutils literal"><span class="pre">ValueJoiner</span></code> will be called to produce |
|
join output records.</p> |
|
<blockquote> |
|
<div><ul class="simple"> |
|
<li>Input records with a <code class="docutils literal"><span class="pre">null</span></code> key are ignored and do not trigger the join.</li> |
|
<li>Input records with a <code class="docutils literal"><span class="pre">null</span></code> value are interpreted as <em>tombstones</em> for the corresponding key, which indicate the deletion of the key from the table. Tombstones do not |
|
trigger the join. When an input tombstone is received, then an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding |
|
key actually exists already in the join result KTable).</li> |
|
</ul> |
|
</div></blockquote> |
|
</li> |
|
<li><p class="first">For each input record on one side that does not have any match on the other side, the <code class="docutils literal"><span class="pre">ValueJoiner</span></code> will be called with <code class="docutils literal"><span class="pre">ValueJoiner#apply(leftRecord.value,</span> <span class="pre">null)</span></code> or |
|
<code class="docutils literal"><span class="pre">ValueJoiner#apply(null,</span> <span class="pre">rightRecord.value)</span></code>, respectively; this explains the rows with timestamp=3 and timestamp=7 in the table below, which list <code class="docutils literal"><span class="pre">[A,</span> <span class="pre">null]</span></code> and |
|
<code class="docutils literal"><span class="pre">[null,</span> <span class="pre">b]</span></code>, respectively, in the OUTER JOIN column.</p> |
|
</li> |
|
</ul> |
|
<p class="last">See the semantics overview at the bottom of this section for a detailed description.</p> |
|
</td> |
|
</tr> |
|
</tbody> |
|
</table> |
|
<p><strong>Semantics of table-table joins:</strong> |
|
The semantics of the various table-table join variants are explained below. |
|
To improve the readability of the table, you can assume that (1) all records have the same key (and thus the key in the table is omitted) and that (2) all records are processed in timestamp order. |
|
The columns INNER JOIN, LEFT JOIN, and OUTER JOIN denote what is passed as arguments to the user-supplied |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/ValueJoiner.html">ValueJoiner</a> for the <code class="docutils literal"><span class="pre">join</span></code>, <code class="docutils literal"><span class="pre">leftJoin</span></code>, and |
|
<code class="docutils literal"><span class="pre">outerJoin</span></code> methods, respectively, whenever a new input record is received on either side of the join. An empty table |
|
cell denotes that the <code class="docutils literal"><span class="pre">ValueJoiner</span></code> is not called at all.</p> |
|
<table border="1" class="docutils"> |
|
<colgroup> |
|
<col width="8%" /> |
|
<col width="13%" /> |
|
<col width="13%" /> |
|
<col width="22%" /> |
|
<col width="22%" /> |
|
<col width="22%" /> |
|
</colgroup> |
|
<thead valign="bottom"> |
|
<tr class="row-odd"><th class="head">Timestamp</th> |
|
<th class="head">Left (KTable)</th> |
|
<th class="head">Right (KTable)</th> |
|
<th class="head">(INNER) JOIN</th> |
|
<th class="head">LEFT JOIN</th> |
|
<th class="head">OUTER JOIN</th> |
|
</tr> |
|
</thead> |
|
<tbody valign="top"> |
|
<tr class="row-even"><td>1</td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-odd"><td>2</td> |
|
<td> </td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-even"><td>3</td> |
|
<td>A</td> |
|
<td> </td> |
|
<td> </td> |
|
<td>[A, null]</td> |
|
<td>[A, null]</td> |
|
</tr> |
|
<tr class="row-odd"><td>4</td> |
|
<td> </td> |
|
<td>a</td> |
|
<td>[A, a]</td> |
|
<td>[A, a]</td> |
|
<td>[A, a]</td> |
|
</tr> |
|
<tr class="row-even"><td>5</td> |
|
<td>B</td> |
|
<td> </td> |
|
<td>[B, a]</td> |
|
<td>[B, a]</td> |
|
<td>[B, a]</td> |
|
</tr> |
|
<tr class="row-odd"><td>6</td> |
|
<td> </td> |
|
<td>b</td> |
|
<td>[B, b]</td> |
|
<td>[B, b]</td> |
|
<td>[B, b]</td> |
|
</tr> |
|
<tr class="row-even"><td>7</td> |
|
<td>null</td> |
|
<td> </td> |
|
<td>null</td> |
|
<td>null</td> |
|
<td>[null, b]</td> |
|
</tr> |
|
<tr class="row-odd"><td>8</td> |
|
<td> </td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
<td>null</td> |
|
</tr> |
|
<tr class="row-even"><td>9</td> |
|
<td>C</td> |
|
<td> </td> |
|
<td> </td> |
|
<td>[C, null]</td> |
|
<td>[C, null]</td> |
|
</tr> |
|
<tr class="row-odd"><td>10</td> |
|
<td> </td> |
|
<td>c</td> |
|
<td>[C, c]</td> |
|
<td>[C, c]</td> |
|
<td>[C, c]</td> |
|
</tr> |
|
<tr class="row-even"><td>11</td> |
|
<td> </td> |
|
<td>null</td> |
|
<td>null</td> |
|
<td>[C, null]</td> |
|
<td>[C, null]</td> |
|
</tr> |
|
<tr class="row-odd"><td>12</td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
<td>null</td> |
|
<td>null</td> |
|
</tr> |
|
<tr class="row-even"><td>13</td> |
|
<td> </td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-odd"><td>14</td> |
|
<td> </td> |
|
<td>d</td> |
|
<td> </td> |
|
<td> </td> |
|
<td>[null, d]</td> |
|
</tr> |
|
<tr class="row-even"><td>15</td> |
|
<td>D</td> |
|
<td> </td> |
|
<td>[D, d]</td> |
|
<td>[D, d]</td> |
|
<td>[D, d]</td> |
|
</tr> |
|
</tbody> |
|
</table> |
|
</div> |
|
<div class="section" id="kstream-ktable-join"> |
|
<span id="streams-developer-guide-dsl-joins-kstream-ktable"></span><h5><a class="toc-backref" href="#id17">KStream-KTable Join</a><a class="headerlink" href="#kstream-ktable-join" title="Permalink to this headline"></a></h5> |
|
<p>KStream-KTable joins are always <em>non-windowed</em> joins. They allow you to perform <em>table lookups</em> against a KTable |
|
(changelog stream) upon receiving a new record from the KStream (record stream). An example use case would be to enrich |
|
a stream of user activities (KStream) with the latest user profile information (KTable).</p> |
|
<p>Join output records are effectively created as follows, leveraging the user-supplied <code class="docutils literal"><span class="pre">ValueJoiner</span></code>:</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">KeyValue</span><span class="o"><</span><span class="n">K</span><span class="o">,</span> <span class="n">LV</span><span class="o">></span> <span class="n">leftRecord</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KeyValue</span><span class="o"><</span><span class="n">K</span><span class="o">,</span> <span class="n">RV</span><span class="o">></span> <span class="n">rightRecord</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">ValueJoiner</span><span class="o"><</span><span class="n">LV</span><span class="o">,</span> <span class="n">RV</span><span class="o">,</span> <span class="n">JV</span><span class="o">></span> <span class="n">joiner</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="n">KeyValue</span><span class="o"><</span><span class="n">K</span><span class="o">,</span> <span class="n">JV</span><span class="o">></span> <span class="n">joinOutputRecord</span> <span class="o">=</span> <span class="n">KeyValue</span><span class="o">.</span><span class="na">pair</span><span class="o">(</span> |
|
<span class="n">leftRecord</span><span class="o">.</span><span class="na">key</span><span class="o">,</span> <span class="cm">/* by definition, leftRecord.key == rightRecord.key */</span> |
|
<span class="n">joiner</span><span class="o">.</span><span class="na">apply</span><span class="o">(</span><span class="n">leftRecord</span><span class="o">.</span><span class="na">value</span><span class="o">,</span> <span class="n">rightRecord</span><span class="o">.</span><span class="na">value</span><span class="o">)</span> |
|
<span class="o">);</span> |
|
</pre></div> |
|
</div> |
|
<table border="1" class="non-scrolling-table width-100-percent docutils"> |
|
<colgroup> |
|
<col width="15%" /> |
|
<col width="85%" /> |
|
</colgroup> |
|
<thead valign="bottom"> |
|
<tr class="row-odd"><th class="head">Transformation</th> |
|
<th class="head">Description</th> |
|
</tr> |
|
</thead> |
|
<tbody valign="top"> |
|
<tr class="row-even"><td><p class="first"><strong>Inner Join</strong></p> |
|
<ul class="last simple"> |
|
<li>(KStream, KTable) |
|
→ KStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Performs an INNER JOIN of this stream with the table, effectively doing a table lookup. |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#join-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-">(details)</a></p> |
|
<p><strong>Data must be co-partitioned</strong>: The input data for both sides must be <a class="reference internal" href="#streams-developer-guide-dsl-joins-co-partitioning"><span class="std std-ref">co-partitioned</span></a>.</p> |
|
<p><strong>Causes data re-partitioning of the stream if and only if the stream was marked for re-partitioning.</strong></p> |
|
<p>Several variants of <code class="docutils literal"><span class="pre">join</span></code> exists, see the Javadocs for details.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">left</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Double</span><span class="o">></span> <span class="n">right</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">join</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="o">(</span><span class="n">leftValue</span><span class="o">,</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">-></span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span><span class="o">,</span> <span class="cm">/* ValueJoiner */</span> |
|
<span class="n">Joined</span><span class="o">.</span><span class="na">keySerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">())</span> <span class="cm">/* key */</span> |
|
<span class="o">.</span><span class="na">withValueSerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">())</span> <span class="cm">/* left value */</span> |
|
<span class="o">);</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">join</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="k">new</span> <span class="n">ValueJoiner</span><span class="o"><</span><span class="n">Long</span><span class="o">,</span> <span class="n">Double</span><span class="o">,</span> <span class="n">String</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">String</span> <span class="nf">apply</span><span class="o">(</span><span class="n">Long</span> <span class="n">leftValue</span><span class="o">,</span> <span class="n">Double</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="n">Joined</span><span class="o">.</span><span class="na">keySerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">())</span> <span class="cm">/* key */</span> |
|
<span class="o">.</span><span class="na">withValueSerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">())</span> <span class="cm">/* left value */</span> |
|
<span class="o">);</span> |
|
</pre></div> |
|
</div> |
|
<p>Detailed behavior:</p> |
|
<ul> |
|
<li><p class="first">The join is <em>key-based</em>, i.e. with the join predicate <code class="docutils literal"><span class="pre">leftRecord.key</span> <span class="pre">==</span> <span class="pre">rightRecord.key</span></code>.</p> |
|
</li> |
|
<li><p class="first">The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied <code class="docutils literal"><span class="pre">ValueJoiner</span></code> will be called to produce |
|
join output records.</p> |
|
<blockquote> |
|
<div><ul class="simple"> |
|
<li>Only input records for the left side (stream) trigger the join. Input records for the right side (table) update only the internal right-side join state.</li> |
|
<li>Input records for the stream with a <code class="docutils literal"><span class="pre">null</span></code> key or a <code class="docutils literal"><span class="pre">null</span></code> value are ignored and do not trigger the join.</li> |
|
<li>Input records for the table with a <code class="docutils literal"><span class="pre">null</span></code> value are interpreted as <em>tombstones</em> for the corresponding key, which indicate the deletion of the key from the table. |
|
Tombstones do not trigger the join.</li> |
|
</ul> |
|
</div></blockquote> |
|
</li> |
|
</ul> |
|
<p class="last">See the semantics overview at the bottom of this section for a detailed description.</p> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>Left Join</strong></p> |
|
<ul class="last simple"> |
|
<li>(KStream, KTable) |
|
→ KStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Performs a LEFT JOIN of this stream with the table, effectively doing a table lookup. |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#leftJoin-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-">(details)</a></p> |
|
<p><strong>Data must be co-partitioned</strong>: The input data for both sides must be <a class="reference internal" href="#streams-developer-guide-dsl-joins-co-partitioning"><span class="std std-ref">co-partitioned</span></a>.</p> |
|
<p><strong>Causes data re-partitioning of the stream if and only if the stream was marked for re-partitioning.</strong></p> |
|
<p>Several variants of <code class="docutils literal"><span class="pre">leftJoin</span></code> exists, see the Javadocs for details.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">left</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Double</span><span class="o">></span> <span class="n">right</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">leftJoin</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="o">(</span><span class="n">leftValue</span><span class="o">,</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">-></span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span><span class="o">,</span> <span class="cm">/* ValueJoiner */</span> |
|
<span class="n">Joined</span><span class="o">.</span><span class="na">keySerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">())</span> <span class="cm">/* key */</span> |
|
<span class="o">.</span><span class="na">withValueSerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">())</span> <span class="cm">/* left value */</span> |
|
<span class="o">);</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">leftJoin</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="k">new</span> <span class="n">ValueJoiner</span><span class="o"><</span><span class="n">Long</span><span class="o">,</span> <span class="n">Double</span><span class="o">,</span> <span class="n">String</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">String</span> <span class="nf">apply</span><span class="o">(</span><span class="n">Long</span> <span class="n">leftValue</span><span class="o">,</span> <span class="n">Double</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="n">Joined</span><span class="o">.</span><span class="na">keySerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">())</span> <span class="cm">/* key */</span> |
|
<span class="o">.</span><span class="na">withValueSerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">())</span> <span class="cm">/* left value */</span> |
|
<span class="o">);</span> |
|
</pre></div> |
|
</div> |
|
<p>Detailed behavior:</p> |
|
<ul> |
|
<li><p class="first">The join is <em>key-based</em>, i.e. with the join predicate <code class="docutils literal"><span class="pre">leftRecord.key</span> <span class="pre">==</span> <span class="pre">rightRecord.key</span></code>.</p> |
|
</li> |
|
<li><p class="first">The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied <code class="docutils literal"><span class="pre">ValueJoiner</span></code> will be called to produce |
|
join output records.</p> |
|
<blockquote> |
|
<div><ul class="simple"> |
|
<li>Only input records for the left side (stream) trigger the join. Input records for the right side (table) update only the internal right-side join state.</li> |
|
<li>Input records for the stream with a <code class="docutils literal"><span class="pre">null</span></code> key or a <code class="docutils literal"><span class="pre">null</span></code> value are ignored and do not trigger the join.</li> |
|
<li>Input records for the table with a <code class="docutils literal"><span class="pre">null</span></code> value are interpreted as <em>tombstones</em> for the corresponding key, which indicate the deletion of the key from the table. |
|
Tombstones do not trigger the join.</li> |
|
</ul> |
|
</div></blockquote> |
|
</li> |
|
<li><p class="first">For each input record on the left side that does not have any match on the right side, the <code class="docutils literal"><span class="pre">ValueJoiner</span></code> will be called with <code class="docutils literal"><span class="pre">ValueJoiner#apply(leftRecord.value,</span> <span class="pre">null)</span></code>; |
|
this explains the row with timestamp=3 in the table below, which lists <code class="docutils literal"><span class="pre">[A,</span> <span class="pre">null]</span></code> in the LEFT JOIN column.</p> |
|
</li> |
|
</ul> |
|
<p class="last">See the semantics overview at the bottom of this section for a detailed description.</p> |
|
</td> |
|
</tr> |
|
</tbody> |
|
</table> |
|
<p><strong>Semantics of stream-table joins:</strong> |
|
The semantics of the various stream-table join variants are explained below. |
|
To improve the readability of the table we assume that (1) all records have the same key (and thus we omit the key in |
|
the table) and that (2) all records are processed in timestamp order. |
|
The columns INNER JOIN and LEFT JOIN denote what is passed as arguments to the user-supplied |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/ValueJoiner.html">ValueJoiner</a> for the <code class="docutils literal"><span class="pre">join</span></code> and <code class="docutils literal"><span class="pre">leftJoin</span></code> |
|
methods, respectively, whenever a new input record is received on either side of the join. An empty table |
|
cell denotes that the <code class="docutils literal"><span class="pre">ValueJoiner</span></code> is not called at all.</p> |
|
<table border="1" class="docutils"> |
|
<colgroup> |
|
<col> |
|
<col> |
|
<col> |
|
<col> |
|
<col> |
|
</colgroup> |
|
<thead valign="bottom"> |
|
<tr class="row-odd"><th class="head">Timestamp</th> |
|
<th class="head">Left (KStream)</th> |
|
<th class="head">Right (KTable)</th> |
|
<th class="head">(INNER) JOIN</th> |
|
<th class="head">LEFT JOIN</th> |
|
</tr> |
|
</thead> |
|
<tbody valign="top"> |
|
<tr class="row-even"><td>1</td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-odd"><td>2</td> |
|
<td> </td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-even"><td>3</td> |
|
<td>A</td> |
|
<td> </td> |
|
<td> </td> |
|
<td>[A, null]</td> |
|
</tr> |
|
<tr class="row-odd"><td>4</td> |
|
<td> </td> |
|
<td>a</td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-even"><td>5</td> |
|
<td>B</td> |
|
<td> </td> |
|
<td>[B, a]</td> |
|
<td>[B, a]</td> |
|
</tr> |
|
<tr class="row-odd"><td>6</td> |
|
<td> </td> |
|
<td>b</td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-even"><td>7</td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-odd"><td>8</td> |
|
<td> </td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-even"><td>9</td> |
|
<td>C</td> |
|
<td> </td> |
|
<td> </td> |
|
<td>[C, null]</td> |
|
</tr> |
|
<tr class="row-odd"><td>10</td> |
|
<td> </td> |
|
<td>c</td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-even"><td>11</td> |
|
<td> </td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-odd"><td>12</td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-even"><td>13</td> |
|
<td> </td> |
|
<td>null</td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-odd"><td>14</td> |
|
<td> </td> |
|
<td>d</td> |
|
<td> </td> |
|
<td> </td> |
|
</tr> |
|
<tr class="row-even"><td>15</td> |
|
<td>D</td> |
|
<td> </td> |
|
<td>[D, d]</td> |
|
<td>[D, d]</td> |
|
</tr> |
|
</tbody> |
|
</table> |
|
</div> |
|
<div class="section" id="kstream-globalktable-join"> |
|
<span id="streams-developer-guide-dsl-joins-kstream-globalktable"></span><h5><a class="toc-backref" href="#id18">KStream-GlobalKTable Join</a><a class="headerlink" href="#kstream-globalktable-join" title="Permalink to this headline"></a></h5> |
|
<p>KStream-GlobalKTable joins are always <em>non-windowed</em> joins. They allow you to perform <em>table lookups</em> against a |
|
<a class="reference internal" href="#streams_concepts_globalktable"><span class="std std-ref">GlobalKTable</span></a> (entire changelog stream) upon receiving a new record from the |
|
KStream (record stream). An example use case would be “star queries” or “star joins”, where you would enrich a stream |
|
of user activities (KStream) with the latest user profile information (GlobalKTable) and further context information |
|
(further GlobalKTables).</p> |
|
<p>At a high-level, KStream-GlobalKTable joins are very similar to |
|
<a class="reference internal" href="#streams-developer-guide-dsl-joins-kstream-ktable"><span class="std std-ref">KStream-KTable joins</span></a>. However, global tables provide you |
|
with much more flexibility at the <a class="reference internal" href="#streams_concepts_globalktable"><span class="std std-ref">some expense</span></a> when compared to partitioned |
|
tables:</p> |
|
<ul class="simple"> |
|
<li>They do not require <a class="reference internal" href="#streams-developer-guide-dsl-joins-co-partitioning"><span class="std std-ref">data co-partitioning</span></a>.</li> |
|
<li>They allow for efficient “star joins”; i.e., joining a large-scale “facts” stream against “dimension” tables</li> |
|
<li>They allow for joining against foreign keys; i.e., you can lookup data in the table not just by the keys of records in the |
|
stream, but also by data in the record values.</li> |
|
<li>They make many use cases feasible where you must work on heavily skewed data and thus suffer from hot partitions.</li> |
|
<li>They are often more efficient than their partitioned KTable counterpart when you need to perform multiple joins in |
|
succession.</li> |
|
</ul> |
|
<p>Join output records are effectively created as follows, leveraging the user-supplied <code class="docutils literal"><span class="pre">ValueJoiner</span></code>:</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">KeyValue</span><span class="o"><</span><span class="n">K</span><span class="o">,</span> <span class="n">LV</span><span class="o">></span> <span class="n">leftRecord</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KeyValue</span><span class="o"><</span><span class="n">K</span><span class="o">,</span> <span class="n">RV</span><span class="o">></span> <span class="n">rightRecord</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">ValueJoiner</span><span class="o"><</span><span class="n">LV</span><span class="o">,</span> <span class="n">RV</span><span class="o">,</span> <span class="n">JV</span><span class="o">></span> <span class="n">joiner</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="n">KeyValue</span><span class="o"><</span><span class="n">K</span><span class="o">,</span> <span class="n">JV</span><span class="o">></span> <span class="n">joinOutputRecord</span> <span class="o">=</span> <span class="n">KeyValue</span><span class="o">.</span><span class="na">pair</span><span class="o">(</span> |
|
<span class="n">leftRecord</span><span class="o">.</span><span class="na">key</span><span class="o">,</span> <span class="cm">/* by definition, leftRecord.key == rightRecord.key */</span> |
|
<span class="n">joiner</span><span class="o">.</span><span class="na">apply</span><span class="o">(</span><span class="n">leftRecord</span><span class="o">.</span><span class="na">value</span><span class="o">,</span> <span class="n">rightRecord</span><span class="o">.</span><span class="na">value</span><span class="o">)</span> |
|
<span class="o">);</span> |
|
</pre></div> |
|
</div> |
|
<table border="1" class="non-scrolling-table width-100-percent docutils"> |
|
<colgroup> |
|
<col width="15%" /> |
|
<col width="85%" /> |
|
</colgroup> |
|
<thead valign="bottom"> |
|
<tr class="row-odd"><th class="head">Transformation</th> |
|
<th class="head">Description</th> |
|
</tr> |
|
</thead> |
|
<tbody valign="top"> |
|
<tr class="row-even"><td><p class="first"><strong>Inner Join</strong></p> |
|
<ul class="last simple"> |
|
<li>(KStream, GlobalKTable) |
|
→ KStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Performs an INNER JOIN of this stream with the global table, effectively doing a table lookup. |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#join-org.apache.kafka.streams.kstream.GlobalKTable-org.apache.kafka.streams.kstream.KeyValueMapper-org.apache.kafka.streams.kstream.ValueJoiner-">(details)</a></p> |
|
<p>The <code class="docutils literal"><span class="pre">GlobalKTable</span></code> is fully bootstrapped upon (re)start of a <code class="docutils literal"><span class="pre">KafkaStreams</span></code> instance, which means the table is fully populated with all the data in the underlying topic that is |
|
available at the time of the startup. The actual data processing begins only once the bootstrapping has completed.</p> |
|
<p><strong>Causes data re-partitioning of the stream if and only if the stream was marked for re-partitioning.</strong></p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">left</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">GlobalKTable</span><span class="o"><</span><span class="n">Integer</span><span class="o">,</span> <span class="n">Double</span><span class="o">></span> <span class="n">right</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">join</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="o">(</span><span class="n">leftKey</span><span class="o">,</span> <span class="n">leftValue</span><span class="o">)</span> <span class="o">-></span> <span class="n">leftKey</span><span class="o">.</span><span class="na">length</span><span class="o">(),</span> <span class="cm">/* derive a (potentially) new key by which to lookup against the table */</span> |
|
<span class="o">(</span><span class="n">leftValue</span><span class="o">,</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">-></span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span> <span class="cm">/* ValueJoiner */</span> |
|
<span class="o">);</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">join</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="k">new</span> <span class="n">KeyValueMapper</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">,</span> <span class="n">Integer</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* derive a (potentially) new key by which to lookup against the table */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Integer</span> <span class="nf">apply</span><span class="o">(</span><span class="n">String</span> <span class="n">key</span><span class="o">,</span> <span class="n">Long</span> <span class="n">value</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">key</span><span class="o">.</span><span class="na">length</span><span class="o">();</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="k">new</span> <span class="n">ValueJoiner</span><span class="o"><</span><span class="n">Long</span><span class="o">,</span> <span class="n">Double</span><span class="o">,</span> <span class="n">String</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">String</span> <span class="nf">apply</span><span class="o">(</span><span class="n">Long</span> <span class="n">leftValue</span><span class="o">,</span> <span class="n">Double</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
</pre></div> |
|
</div> |
|
<p>Detailed behavior:</p> |
|
<ul class="last"> |
|
<li><p class="first">The join is indirectly <em>key-based</em>, i.e. with the join predicate <code class="docutils literal"><span class="pre">KeyValueMapper#apply(leftRecord.key,</span> <span class="pre">leftRecord.value)</span> <span class="pre">==</span> <span class="pre">rightRecord.key</span></code>.</p> |
|
</li> |
|
<li><p class="first">The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied <code class="docutils literal"><span class="pre">ValueJoiner</span></code> will be called to produce |
|
join output records.</p> |
|
<blockquote> |
|
<div><ul class="simple"> |
|
<li>Only input records for the left side (stream) trigger the join. Input records for the right side (table) update only the internal right-side join state.</li> |
|
<li>Input records for the stream with a <code class="docutils literal"><span class="pre">null</span></code> key or a <code class="docutils literal"><span class="pre">null</span></code> value are ignored and do not trigger the join.</li> |
|
<li>Input records for the table with a <code class="docutils literal"><span class="pre">null</span></code> value are interpreted as <em>tombstones</em>, which indicate the deletion of a record key from the table. Tombstones do not trigger the |
|
join.</li> |
|
</ul> |
|
</div></blockquote> |
|
</li> |
|
</ul> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>Left Join</strong></p> |
|
<ul class="last simple"> |
|
<li>(KStream, GlobalKTable) |
|
→ KStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Performs a LEFT JOIN of this stream with the global table, effectively doing a table lookup. |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#leftJoin-org.apache.kafka.streams.kstream.GlobalKTable-org.apache.kafka.streams.kstream.KeyValueMapper-org.apache.kafka.streams.kstream.ValueJoiner-">(details)</a></p> |
|
<p>The <code class="docutils literal"><span class="pre">GlobalKTable</span></code> is fully bootstrapped upon (re)start of a <code class="docutils literal"><span class="pre">KafkaStreams</span></code> instance, which means the table is fully populated with all the data in the underlying topic that is |
|
available at the time of the startup. The actual data processing begins only once the bootstrapping has completed.</p> |
|
<p><strong>Causes data re-partitioning of the stream if and only if the stream was marked for re-partitioning.</strong></p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">left</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">GlobalKTable</span><span class="o"><</span><span class="n">Integer</span><span class="o">,</span> <span class="n">Double</span><span class="o">></span> <span class="n">right</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Java 8+ example, using lambda expressions</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">leftJoin</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="o">(</span><span class="n">leftKey</span><span class="o">,</span> <span class="n">leftValue</span><span class="o">)</span> <span class="o">-></span> <span class="n">leftKey</span><span class="o">.</span><span class="na">length</span><span class="o">(),</span> <span class="cm">/* derive a (potentially) new key by which to lookup against the table */</span> |
|
<span class="o">(</span><span class="n">leftValue</span><span class="o">,</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">-></span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span> <span class="cm">/* ValueJoiner */</span> |
|
<span class="o">);</span> |
|
|
|
<span class="c1">// Java 7 example</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">></span> <span class="n">joined</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="na">leftJoin</span><span class="o">(</span><span class="n">right</span><span class="o">,</span> |
|
<span class="k">new</span> <span class="n">KeyValueMapper</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">,</span> <span class="n">Integer</span><span class="o">>()</span> <span class="o">{</span> <span class="cm">/* derive a (potentially) new key by which to lookup against the table */</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">Integer</span> <span class="nf">apply</span><span class="o">(</span><span class="n">String</span> <span class="n">key</span><span class="o">,</span> <span class="n">Long</span> <span class="n">value</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">key</span><span class="o">.</span><span class="na">length</span><span class="o">();</span> |
|
<span class="o">}</span> |
|
<span class="o">},</span> |
|
<span class="k">new</span> <span class="n">ValueJoiner</span><span class="o"><</span><span class="n">Long</span><span class="o">,</span> <span class="n">Double</span><span class="o">,</span> <span class="n">String</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="n">String</span> <span class="nf">apply</span><span class="o">(</span><span class="n">Long</span> <span class="n">leftValue</span><span class="o">,</span> <span class="n">Double</span> <span class="n">rightValue</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="s">"left="</span> <span class="o">+</span> <span class="n">leftValue</span> <span class="o">+</span> <span class="s">", right="</span> <span class="o">+</span> <span class="n">rightValue</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
</pre></div> |
|
</div> |
|
<p>Detailed behavior:</p> |
|
<ul class="last"> |
|
<li><p class="first">The join is indirectly <em>key-based</em>, i.e. with the join predicate <code class="docutils literal"><span class="pre">KeyValueMapper#apply(leftRecord.key,</span> <span class="pre">leftRecord.value)</span> <span class="pre">==</span> <span class="pre">rightRecord.key</span></code>.</p> |
|
</li> |
|
<li><p class="first">The join will be triggered under the conditions listed below whenever new input is received. When it is triggered, the user-supplied <code class="docutils literal"><span class="pre">ValueJoiner</span></code> will be called to produce |
|
join output records.</p> |
|
<blockquote> |
|
<div><ul class="simple"> |
|
<li>Only input records for the left side (stream) trigger the join. Input records for the right side (table) update only the internal right-side join state.</li> |
|
<li>Input records for the stream with a <code class="docutils literal"><span class="pre">null</span></code> key or a <code class="docutils literal"><span class="pre">null</span></code> value are ignored and do not trigger the join.</li> |
|
<li>Input records for the table with a <code class="docutils literal"><span class="pre">null</span></code> value are interpreted as <em>tombstones</em>, which indicate the deletion of a record key from the table. Tombstones do not trigger the |
|
join.</li> |
|
</ul> |
|
</div></blockquote> |
|
</li> |
|
<li><p class="first">For each input record on the left side that does not have any match on the right side, the <code class="docutils literal"><span class="pre">ValueJoiner</span></code> will be called with <code class="docutils literal"><span class="pre">ValueJoiner#apply(leftRecord.value,</span> <span class="pre">null)</span></code>.</p> |
|
</li> |
|
</ul> |
|
</td> |
|
</tr> |
|
</tbody> |
|
</table> |
|
<p><strong>Semantics of stream-table joins:</strong> |
|
The join semantics are identical to <a class="reference internal" href="#streams-developer-guide-dsl-joins-kstream-ktable"><span class="std std-ref">KStream-KTable joins</span></a>. |
|
The only difference is that, for KStream-GlobalKTable joins, the left input record is first “mapped” with |
|
a user-supplied <code class="docutils literal"><span class="pre">KeyValueMapper</span></code> into the table’s keyspace prior to the table lookup.</p> |
|
</div> |
|
</div> |
|
<div class="section" id="windowing"> |
|
<span id="streams-developer-guide-dsl-windowing"></span><h4><a class="toc-backref" href="#id19">Windowing</a><a class="headerlink" href="#windowing" title="Permalink to this headline"></a></h4> |
|
<p>Windowing lets you control how to group records that have the same key for stateful operations such as |
|
<a class="reference internal" href="#streams-developer-guide-dsl-aggregating"><span class="std std-ref">aggregations</span></a> or <a class="reference internal" href="#streams-developer-guide-dsl-joins"><span class="std std-ref">joins</span></a> into |
|
so-called windows. Windows are tracked per record key.</p> |
|
<div class="admonition note"> |
|
<p><b>Note</b></p> |
|
<p class="last">A related operation is <a class="reference internal" href="#streams-developer-guide-dsl-transformations-stateless"><span class="std std-ref">grouping</span></a>, which groups all |
|
records that have the same key to ensure that data is properly partitioned (“keyed”) for subsequent operations. |
|
Once grouped, windowing allows you to further sub-group the records of a key.</p> |
|
</div> |
|
<p>For example, in join operations, a windowing state store is used to store all the records received so far within the |
|
defined window boundary. In aggregating operations, a windowing state store is used to store the latest aggregation |
|
results per window. |
|
Old records in the state store are purged after the specified |
|
<a class="reference internal" href="../core-concepts.html#streams_concepts_windowing"><span class="std std-ref">window retention period</span></a>. |
|
Kafka Streams guarantees to keep a window for at least this specified time; the default value is one day and can be |
|
changed via <code class="docutils literal"><span class="pre">Windows#until()</span></code> and <code class="docutils literal"><span class="pre">SessionWindows#until()</span></code>.</p> |
|
<p>The DSL supports the following types of windows:</p> |
|
<table border="1" class="docutils"> |
|
<colgroup> |
|
<col width="34%" /> |
|
<col width="10%" /> |
|
<col width="56%" /> |
|
</colgroup> |
|
<thead valign="bottom"> |
|
<tr class="row-odd"><th class="head">Window name</th> |
|
<th class="head">Behavior</th> |
|
<th class="head">Short description</th> |
|
</tr> |
|
</thead> |
|
<tbody valign="top"> |
|
<tr class="row-even"><td><a class="reference internal" href="#windowing-tumbling"><span class="std std-ref">Tumbling time window</span></a></td> |
|
<td>Time-based</td> |
|
<td>Fixed-size, non-overlapping, gap-less windows</td> |
|
</tr> |
|
<tr class="row-odd"><td><a class="reference internal" href="#windowing-hopping"><span class="std std-ref">Hopping time window</span></a></td> |
|
<td>Time-based</td> |
|
<td>Fixed-size, overlapping windows</td> |
|
</tr> |
|
<tr class="row-even"><td><a class="reference internal" href="#windowing-sliding"><span class="std std-ref">Sliding time window</span></a></td> |
|
<td>Time-based</td> |
|
<td>Fixed-size, overlapping windows that work on differences between record timestamps</td> |
|
</tr> |
|
<tr class="row-odd"><td><a class="reference internal" href="#windowing-session"><span class="std std-ref">Session window</span></a></td> |
|
<td>Session-based</td> |
|
<td>Dynamically-sized, non-overlapping, data-driven windows</td> |
|
</tr> |
|
</tbody> |
|
</table> |
|
<div class="section" id="tumbling-time-windows"> |
|
<span id="windowing-tumbling"></span><h5><a class="toc-backref" href="#id20">Tumbling time windows</a><a class="headerlink" href="#tumbling-time-windows" title="Permalink to this headline"></a></h5> |
|
<p>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 <em>size</em>. |
|
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.</p> |
|
<div class="figure align-center" id="id3"> |
|
<img class="centered" src="/{{version}}/images/streams-time-windows-tumbling.png"> |
|
<p class="caption"><span class="caption-text">This diagram shows windowing a stream of data records with tumbling windows. Windows do not overlap because, by |
|
definition, the advance interval is identical to the window size. In this diagram the time numbers represent minutes; |
|
e.g. t=5 means “at the five-minute mark”. In reality, the unit of time in Kafka Streams is milliseconds, which means |
|
the time numbers would need to be multiplied with 60 * 1,000 to convert from minutes to milliseconds (e.g. t=5 would |
|
become t=300,000).</span></p> |
|
</div> |
|
<p>Tumbling time windows are <em>aligned to the epoch</em>, with the lower interval bound being inclusive and the upper bound |
|
being exclusive. “Aligned to the epoch” means that the first window starts at timestamp zero. For example, tumbling |
|
windows with a size of 5000ms have predictable window boundaries <code class="docutils literal"><span class="pre">[0;5000),[5000;10000),...</span></code> — and <strong>not</strong> |
|
<code class="docutils literal"><span class="pre">[1000;6000),[6000;11000),...</span></code> or even something “random” like <code class="docutils literal"><span class="pre">[1452;6452),[6452;11452),...</span></code>.</p> |
|
<p>The following code defines a tumbling window with a size of 5 minutes:</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">java.util.concurrent.TimeUnit</span><span class="o">;</span> |
|
<span class="kn">import</span> <span class="nn">org.apache.kafka.streams.kstream.TimeWindows</span><span class="o">;</span> |
|
|
|
<span class="c1">// A tumbling time window with a size of 5 minutes (and, by definition, an implicit</span> |
|
<span class="c1">// advance interval of 5 minutes).</span> |
|
<span class="kt">long</span> <span class="n">windowSizeMs</span> <span class="o">=</span> <span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">);</span> <span class="c1">// 5 * 60 * 1000L</span> |
|
<span class="n">TimeWindows</span><span class="o">.</span><span class="na">of</span><span class="o">(</span><span class="n">windowSizeMs</span><span class="o">);</span> |
|
|
|
<span class="c1">// The above is equivalent to the following code:</span> |
|
<span class="n">TimeWindows</span><span class="o">.</span><span class="na">of</span><span class="o">(</span><span class="n">windowSizeMs</span><span class="o">).</span><span class="na">advanceBy</span><span class="o">(</span><span class="n">windowSizeMs</span><span class="o">);</span> |
|
</pre></div> |
|
</div> |
|
</div> |
|
<div class="section" id="hopping-time-windows"> |
|
<span id="windowing-hopping"></span><h5><a class="toc-backref" href="#id21">Hopping time windows</a><a class="headerlink" href="#hopping-time-windows" title="Permalink to this headline"></a></h5> |
|
<p>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 <em>size</em> and its <em>advance interval</em> (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.</p> |
|
<div class="admonition note"> |
|
<p><b>Note</b></p> |
|
<p class="last"><strong>Hopping windows vs. sliding windows:</strong> |
|
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.</p> |
|
</div> |
|
<p>The following code defines a hopping window with a size of 5 minutes and an advance interval of 1 minute:</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">java.util.concurrent.TimeUnit</span><span class="o">;</span> |
|
<span class="kn">import</span> <span class="nn">org.apache.kafka.streams.kstream.TimeWindows</span><span class="o">;</span> |
|
|
|
<span class="c1">// A hopping time window with a size of 5 minutes and an advance interval of 1 minute.</span> |
|
<span class="c1">// The window's name -- the string parameter -- is used to e.g. name the backing state store.</span> |
|
<span class="kt">long</span> <span class="n">windowSizeMs</span> <span class="o">=</span> <span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">);</span> <span class="c1">// 5 * 60 * 1000L</span> |
|
<span class="kt">long</span> <span class="n">advanceMs</span> <span class="o">=</span> <span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">1</span><span class="o">);</span> <span class="c1">// 1 * 60 * 1000L</span> |
|
<span class="n">TimeWindows</span><span class="o">.</span><span class="na">of</span><span class="o">(</span><span class="n">windowSizeMs</span><span class="o">).</span><span class="na">advanceBy</span><span class="o">(</span><span class="n">advanceMs</span><span class="o">);</span> |
|
</pre></div> |
|
</div> |
|
<div class="figure align-center" id="id4"> |
|
<img class="centered" src="/{{version}}/images/streams-time-windows-hopping.png"> |
|
<p class="caption"><span class="caption-text">This diagram shows windowing a stream of data records with hopping windows. In this diagram the time numbers |
|
represent minutes; e.g. t=5 means “at the five-minute mark”. In reality, the unit of time in Kafka Streams is |
|
milliseconds, which means the time numbers would need to be multiplied with 60 * 1,000 to convert from minutes to |
|
milliseconds (e.g. t=5 would become t=300,000).</span></p> |
|
</div> |
|
<p>Hopping time windows are <em>aligned to the epoch</em>, with the lower interval bound being inclusive and the upper bound |
|
being exclusive. “Aligned to the epoch” means that the first window starts at timestamp zero. For example, hopping |
|
windows with a size of 5000ms and an advance interval (“hop”) of 3000ms have predictable window boundaries |
|
<code class="docutils literal"><span class="pre">[0;5000),[3000;8000),...</span></code> — and <strong>not</strong> <code class="docutils literal"><span class="pre">[1000;6000),[4000;9000),...</span></code> or even something “random” like |
|
<code class="docutils literal"><span class="pre">[1452;6452),[4452;9452),...</span></code>.</p> |
|
<p>Unlike non-windowed aggregates that we have seen previously, windowed aggregates return a <em>windowed KTable</em> whose keys |
|
type is <code class="docutils literal"><span class="pre">Windowed<K></span></code>. 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 <code class="docutils literal"><span class="pre">Windowed#window()</span></code> and <code class="docutils literal"><span class="pre">Windowed#key()</span></code>, |
|
respectively.</p> |
|
</div> |
|
<div class="section" id="sliding-time-windows"> |
|
<span id="windowing-sliding"></span><h5><a class="toc-backref" href="#id22">Sliding time windows</a><a class="headerlink" href="#sliding-time-windows" title="Permalink to this headline"></a></h5> |
|
<p>Sliding windows are actually quite different from hopping and tumbling windows. In Kafka Streams, sliding windows |
|
are used only for <a class="reference internal" href="#streams-developer-guide-dsl-joins"><span class="std std-ref">join operations</span></a>, and can be specified through the |
|
<code class="docutils literal"><span class="pre">JoinWindows</span></code> class.</p> |
|
<p>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 (in the case of symmetric windows) the difference of their timestamps is |
|
within the window size. Thus, sliding windows are not aligned to the epoch, but to the data record timestamps. In |
|
contrast to hopping and tumbling windows, the lower and upper window time interval bounds of sliding windows are |
|
<em>both inclusive</em>.</p> |
|
</div> |
|
<div class="section" id="session-windows"> |
|
<span id="windowing-session"></span><h5><a class="toc-backref" href="#id23">Session Windows</a><a class="headerlink" href="#session-windows" title="Permalink to this headline"></a></h5> |
|
<p>Session windows are used to aggregate key-based events into so-called <em>sessions</em>, the process of which is referred to |
|
as <em>sessionization</em>. Sessions represent a <strong>period of activity</strong> separated by a defined <strong>gap of inactivity</strong> (or |
|
“idleness”). Any events processed that fall within the inactivity gap of any existing sessions are merged into the |
|
existing sessions. If an event falls outside of the session gap, then a new session will be created.</p> |
|
<p>Session windows are different from the other window types in that:</p> |
|
<ul class="simple"> |
|
<li>all windows are tracked independently across keys – e.g. windows of different keys typically have different start |
|
and end times</li> |
|
<li>their window sizes sizes vary – even windows for the same key typically have different sizes</li> |
|
</ul> |
|
<p>The prime area of application for session windows is <strong>user behavior analysis</strong>. Session-based analyses can range from |
|
simple metrics (e.g. count of user visits on a news website or social platform) to more complex metrics (e.g. customer |
|
conversion funnel and event flows).</p> |
|
<p>The following code defines a session window with an inactivity gap of 5 minutes:</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">java.util.concurrent.TimeUnit</span><span class="o">;</span> |
|
<span class="kn">import</span> <span class="nn">org.apache.kafka.streams.kstream.SessionWindows</span><span class="o">;</span> |
|
|
|
<span class="c1">// A session window with an inactivity gap of 5 minutes.</span> |
|
<span class="n">SessionWindows</span><span class="o">.</span><span class="na">with</span><span class="o">(</span><span class="n">TimeUnit</span><span class="o">.</span><span class="na">MINUTES</span><span class="o">.</span><span class="na">toMillis</span><span class="o">(</span><span class="mi">5</span><span class="o">));</span> |
|
</pre></div> |
|
</div> |
|
<p>Given the previous session window example, here’s what would happen on an input stream of six records. |
|
When the first three records arrive (upper part of in the diagram below), we’d have three sessions (see lower part) |
|
after having processed those records: two for the green record key, with one session starting and ending at the |
|
0-minute mark (only due to the illustration it looks as if the session goes from 0 to 1), and another starting and |
|
ending at the 6-minute mark; and one session for the blue record key, starting and ending at the 2-minute mark.</p> |
|
<div class="figure align-center" id="id5"> |
|
<img class="centered" src="/{{version}}/images/streams-session-windows-01.png"> |
|
<p class="caption"><span class="caption-text">Detected sessions after having received three input records: two records for the green record key at t=0 and t=6, and |
|
one record for the blue record key at t=2. |
|
In this diagram the time numbers represent minutes; e.g. t=5 means “at the five-minute mark”. In reality, the unit |
|
of time in Kafka Streams is milliseconds, which means the time numbers would need to be multiplied with 60 * 1,000 to |
|
convert from minutes to milliseconds (e.g. t=5 would become t=300,000).</span></p> |
|
</div> |
|
<p>If we then receive three additional records (including two late-arriving records), what would happen is that the two |
|
existing sessions for the green record key will be merged into a single session starting at time 0 and ending at time 6, |
|
consisting of a total of three records. The existing session for the blue record key will be extended to end at time 5, |
|
consisting of a total of two records. And, finally, there will be a new session for the blue key starting and ending at |
|
time 11.</p> |
|
<div class="figure align-center" id="id6"> |
|
<img class="centered" src="/{{version}}/images/streams-session-windows-02.png"> |
|
<p class="caption"><span class="caption-text">Detected sessions after having received six input records. Note the two late-arriving data records at t=4 (green) and |
|
t=5 (blue), which lead to a merge of sessions and an extension of a session, respectively.</span></p> |
|
</div> |
|
</div> |
|
</div> |
|
</div> |
|
<div class="section" id="applying-processors-and-transformers-processor-api-integration"> |
|
<span id="streams-developer-guide-dsl-process"></span><h3><a class="toc-backref" href="#id24">Applying processors and transformers (Processor API integration)</a><a class="headerlink" href="#applying-processors-and-transformers-processor-api-integration" title="Permalink to this headline"></a></h3> |
|
<p>Beyond the aforementioned <a class="reference internal" href="#streams-developer-guide-dsl-transformations-stateless"><span class="std std-ref">stateless</span></a> and |
|
<a class="reference internal" href="#streams-developer-guide-dsl-transformations-stateless"><span class="std std-ref">stateful</span></a> transformations, you may also |
|
leverage the <a class="reference internal" href="processor-api.html#streams-developer-guide-processor-api"><span class="std std-ref">Processor API</span></a> from the DSL. |
|
There are a number of scenarios where this may be helpful:</p> |
|
<ul class="simple"> |
|
<li><strong>Customization:</strong> You need to implement special, customized logic that is not or not yet available in the DSL.</li> |
|
<li><strong>Combining ease-of-use with full flexibility where it’s needed:</strong> Even though you generally prefer to use |
|
the expressiveness of the DSL, there are certain steps in your processing that require more flexibility and |
|
tinkering than the DSL provides. For example, only the Processor API provides access to a |
|
record’s metadata such as its topic, partition, and offset information. |
|
However, you don’t want to switch completely to the Processor API just because of that.</li> |
|
<li><strong>Migrating from other tools:</strong> You are migrating from other stream processing technologies that provide an |
|
imperative API, and migrating some of your legacy code to the Processor API was faster and/or easier than to |
|
migrate completely to the DSL right away.</li> |
|
</ul> |
|
<table border="1" class="non-scrolling-table width-100-percent docutils"> |
|
<colgroup> |
|
<col width="19%" /> |
|
<col width="81%" /> |
|
</colgroup> |
|
<thead valign="bottom"> |
|
<tr class="row-odd"><th class="head">Transformation</th> |
|
<th class="head">Description</th> |
|
</tr> |
|
</thead> |
|
<tbody valign="top"> |
|
<tr class="row-even"><td><p class="first"><strong>Process</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream -> void</li> |
|
</ul> |
|
</td> |
|
<td><p class="first"><strong>Terminal operation.</strong> Applies a <code class="docutils literal"><span class="pre">Processor</span></code> to each record. |
|
<code class="docutils literal"><span class="pre">process()</span></code> allows you to leverage the <a class="reference internal" href="processor-api.html#streams-developer-guide-processor-api"><span class="std std-ref">Processor API</span></a> from the DSL. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#process-org.apache.kafka.streams.processor.ProcessorSupplier-java.lang.String...-">details</a>)</p> |
|
<p>This is essentially equivalent to adding the <code class="docutils literal"><span class="pre">Processor</span></code> via <code class="docutils literal"><span class="pre">Topology#addProcessor()</span></code> to your |
|
<a class="reference internal" href="../core-concepts.html#streams_topology"><span class="std std-ref">processor topology</span></a>.</p> |
|
<p class="last">An example is available in the |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#process-org.apache.kafka.streams.processor.ProcessorSupplier-java.lang.String...-">javadocs</a>.</p> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>Transform</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream -> KStream</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Applies a <code class="docutils literal"><span class="pre">Transformer</span></code> to each record. |
|
<code class="docutils literal"><span class="pre">transform()</span></code> allows you to leverage the <a class="reference internal" href="processor-api.html#streams-developer-guide-processor-api"><span class="std std-ref">Processor API</span></a> from the DSL. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#transform-org.apache.kafka.streams.kstream.TransformerSupplier-java.lang.String...-">details</a>)</p> |
|
<p>Each input record is transformed into zero, one, or more output records (similar to the stateless <code class="docutils literal"><span class="pre">flatMap</span></code>). |
|
The <code class="docutils literal"><span class="pre">Transformer</span></code> must return <code class="docutils literal"><span class="pre">null</span></code> for zero output. |
|
You can modify the record’s key and value, including their types.</p> |
|
<p><strong>Marks the stream for data re-partitioning:</strong> |
|
Applying a grouping or a join after <code class="docutils literal"><span class="pre">transform</span></code> will result in re-partitioning of the records. |
|
If possible use <code class="docutils literal"><span class="pre">transformValues</span></code> instead, which will not cause data re-partitioning.</p> |
|
<p><code class="docutils literal"><span class="pre">transform</span></code> is essentially equivalent to adding the <code class="docutils literal"><span class="pre">Transformer</span></code> via <code class="docutils literal"><span class="pre">Topology#addProcessor()</span></code> to your |
|
<a class="reference internal" href="../core-concepts.html#streams_topology"><span class="std std-ref">processor topology</span></a>.</p> |
|
<p class="last">An example is available in the |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#transform-org.apache.kafka.streams.kstream.TransformerSupplier-java.lang.String...-">javadocs</a>. |
|
</p> |
|
</td> |
|
</tr> |
|
<tr class="row-even"><td><p class="first"><strong>Transform (values only)</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream -> KStream</li> |
|
<li>KTable -> KTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Applies a <code class="docutils literal"><span class="pre">ValueTransformer</span></code> to each record, while retaining the key of the original record. |
|
<code class="docutils literal"><span class="pre">transformValues()</span></code> allows you to leverage the <a class="reference internal" href="processor-api.html#streams-developer-guide-processor-api"><span class="std std-ref">Processor API</span></a> from the DSL. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#transformValues-org.apache.kafka.streams.kstream.ValueTransformerSupplier-java.lang.String...-">details</a>)</p> |
|
<p>Each input record is transformed into exactly one output record (zero output records or multiple output records are not possible). |
|
The <code class="docutils literal"><span class="pre">ValueTransformer</span></code> may return <code class="docutils literal"><span class="pre">null</span></code> as the new value for a record.</p> |
|
<p><code class="docutils literal"><span class="pre">transformValues</span></code> is preferable to <code class="docutils literal"><span class="pre">transform</span></code> because it will not cause data re-partitioning.</p> |
|
<p><code class="docutils literal"><span class="pre">transformValues</span></code> is essentially equivalent to adding the <code class="docutils literal"><span class="pre">ValueTransformer</span></code> via <code class="docutils literal"><span class="pre">Topology#addProcessor()</span></code> to your |
|
<a class="reference internal" href="../core-concepts.html#streams_topology"><span class="std std-ref">processor topology</span></a>.</p> |
|
<p class="last">An example is available in the |
|
<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#transformValues-org.apache.kafka.streams.kstream.ValueTransformerSupplier-java.lang.String...-">javadocs</a>.</p> |
|
</td> |
|
</tr> |
|
</tbody> |
|
</table> |
|
<p>The following example shows how to leverage, via the <code class="docutils literal"><span class="pre">KStream#process()</span></code> method, a custom <code class="docutils literal"><span class="pre">Processor</span></code> that sends an |
|
email notification whenever a page view count reaches a predefined threshold.</p> |
|
<p>First, we need to implement a custom stream processor, <code class="docutils literal"><span class="pre">PopularPageEmailAlert</span></code>, that implements the <code class="docutils literal"><span class="pre">Processor</span></code> |
|
interface:</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="c1">// A processor that sends an alert message about a popular page to a configurable email address</span> |
|
<span class="kd">public</span> <span class="kd">class</span> <span class="nc">PopularPageEmailAlert</span> <span class="kd">implements</span> <span class="n">Processor</span><span class="o"><</span><span class="n">PageId</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="o">{</span> |
|
|
|
<span class="kd">private</span> <span class="kd">final</span> <span class="n">String</span> <span class="n">emailAddress</span><span class="o">;</span> |
|
<span class="kd">private</span> <span class="n">ProcessorContext</span> <span class="n">context</span><span class="o">;</span> |
|
|
|
<span class="kd">public</span> <span class="nf">PopularPageEmailAlert</span><span class="o">(</span><span class="n">String</span> <span class="n">emailAddress</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">this</span><span class="o">.</span><span class="na">emailAddress</span> <span class="o">=</span> <span class="n">emailAddress</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
|
|
<span class="nd">@Override</span> |
|
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">init</span><span class="o">(</span><span class="n">ProcessorContext</span> <span class="n">context</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">this</span><span class="o">.</span><span class="na">context</span> <span class="o">=</span> <span class="n">context</span><span class="o">;</span> |
|
|
|
<span class="c1">// Here you would perform any additional initializations such as setting up an email client.</span> |
|
<span class="o">}</span> |
|
|
|
<span class="nd">@Override</span> |
|
<span class="kt">void</span> <span class="nf">process</span><span class="o">(</span><span class="n">PageId</span> <span class="n">pageId</span><span class="o">,</span> <span class="n">Long</span> <span class="n">count</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="c1">// Here you would format and send the alert email.</span> |
|
<span class="c1">//</span> |
|
<span class="c1">// In this specific example, you would be able to include information about the page's ID and its view count</span> |
|
<span class="c1">// (because the class implements `Processor<PageId, Long>`).</span> |
|
<span class="o">}</span> |
|
|
|
<span class="nd">@Override</span> |
|
<span class="kt">void</span> <span class="nf">close</span><span class="o">()</span> <span class="o">{</span> |
|
<span class="c1">// Any code for clean up would go here. This processor instance will not be used again after this call.</span> |
|
<span class="o">}</span> |
|
|
|
<span class="o">}</span> |
|
</pre></div> |
|
</div> |
|
<div class="admonition tip"> |
|
<p><b>Tip</b></p> |
|
<p class="last">Even though we do not demonstrate it in this example, a stream processor can access any available state stores by |
|
calling <code class="docutils literal"><span class="pre">ProcessorContext#getStateStore()</span></code>. Only such state stores are available that (1) have been named in the |
|
corresponding <code class="docutils literal"><span class="pre">KStream#process()</span></code> method call (note that this is a different method than <code class="docutils literal"><span class="pre">Processor#process()</span></code>), |
|
plus (2) all global stores. Note that global stores do not need to be attached explicitly; however, they only |
|
allow for read-only access.</p> |
|
</div> |
|
<p>Then we can leverage the <code class="docutils literal"><span class="pre">PopularPageEmailAlert</span></code> processor in the DSL via <code class="docutils literal"><span class="pre">KStream#process</span></code>.</p> |
|
<p>In Java 8+, using lambda expressions:</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">GenericRecord</span><span class="o">></span> <span class="n">pageViews</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Send an email notification when the view count of a page reaches one thousand.</span> |
|
<span class="n">pageViews</span><span class="o">.</span><span class="na">groupByKey</span><span class="o">()</span> |
|
<span class="o">.</span><span class="na">count</span><span class="o">()</span> |
|
<span class="o">.</span><span class="na">filter</span><span class="o">((</span><span class="n">PageId</span> <span class="n">pageId</span><span class="o">,</span> <span class="n">Long</span> <span class="n">viewCount</span><span class="o">)</span> <span class="o">-></span> <span class="n">viewCount</span> <span class="o">==</span> <span class="mi">1000</span><span class="o">)</span> |
|
<span class="c1">// PopularPageEmailAlert is your custom processor that implements the</span> |
|
<span class="c1">// `Processor` interface, see further down below.</span> |
|
<span class="o">.</span><span class="na">process</span><span class="o">(()</span> <span class="o">-></span> <span class="k">new</span> <span class="n">PopularPageEmailAlert</span><span class="o">(</span><span class="s">"alerts@yourcompany.com"</span><span class="o">));</span> |
|
</pre></div> |
|
</div> |
|
<p>In Java 7:</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="c1">// Send an email notification when the view count of a page reaches one thousand.</span> |
|
<span class="n">pageViews</span><span class="o">.</span><span class="na">groupByKey</span><span class="o">().</span> |
|
<span class="o">.</span><span class="na">count</span><span class="o">()</span> |
|
<span class="o">.</span><span class="na">filter</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">Predicate</span><span class="o"><</span><span class="n">PageId</span><span class="o">,</span> <span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="kd">public</span> <span class="kt">boolean</span> <span class="nf">test</span><span class="o">(</span><span class="n">PageId</span> <span class="n">pageId</span><span class="o">,</span> <span class="n">Long</span> <span class="n">viewCount</span><span class="o">)</span> <span class="o">{</span> |
|
<span class="k">return</span> <span class="n">viewCount</span> <span class="o">==</span> <span class="mi">1000</span><span class="o">;</span> |
|
<span class="o">}</span> |
|
<span class="o">})</span> |
|
<span class="o">.</span><span class="na">process</span><span class="o">(</span> |
|
<span class="k">new</span> <span class="n">ProcessorSupplier</span><span class="o"><</span><span class="n">PageId</span><span class="o">,</span> <span class="n">Long</span><span class="o">>()</span> <span class="o">{</span> |
|
<span class="kd">public</span> <span class="n">Processor</span><span class="o"><</span><span class="n">PageId</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="nf">get</span><span class="o">()</span> <span class="o">{</span> |
|
<span class="c1">// PopularPageEmailAlert is your custom processor that implements</span> |
|
<span class="c1">// the `Processor` interface, see further down below.</span> |
|
<span class="k">return</span> <span class="k">new</span> <span class="n">PopularPageEmailAlert</span><span class="o">(</span><span class="s">"alerts@yourcompany.com"</span><span class="o">);</span> |
|
<span class="o">}</span> |
|
<span class="o">});</span> |
|
</pre></div> |
|
</div> |
|
</div> |
|
</div> |
|
<div class="section" id="writing-streams-back-to-kafka"> |
|
<span id="streams-developer-guide-dsl-destinations"></span><h2><a class="toc-backref" href="#id25">Writing streams back to Kafka</a><a class="headerlink" href="#writing-streams-back-to-kafka" title="Permalink to this headline"></a></h2> |
|
<p>Any streams and tables may be (continuously) written back to a Kafka topic. As we will describe in more detail below, the output data might be |
|
re-partitioned on its way to Kafka, depending on the situation.</p> |
|
<table border="1" class="non-scrolling-table width-100-percent docutils"> |
|
<colgroup> |
|
<col width="22%" /> |
|
<col width="78%" /> |
|
</colgroup> |
|
<thead valign="bottom"> |
|
<tr class="row-odd"><th class="head">Writing to Kafka</th> |
|
<th class="head">Description</th> |
|
</tr> |
|
</thead> |
|
<tbody valign="top"> |
|
<tr class="row-even"><td><p class="first"><strong>To</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream -> void</li> |
|
</ul> |
|
</td> |
|
<td><p class="first"><strong>Terminal operation.</strong> Write the records to Kafka topic(s). |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#to(java.lang.String)">KStream details</a>)</p> |
|
<p>When to provide serdes explicitly:</p> |
|
<ul class="simple"> |
|
<li>If you do not specify SerDes explicitly, the default SerDes from the |
|
<a class="reference internal" href="config-streams.html#streams-developer-guide-configuration"><span class="std std-ref">configuration</span></a> are used.</li> |
|
<li>You <strong>must specify SerDes explicitly</strong> via the <code class="docutils literal"><span class="pre">Produced</span></code> class if the key and/or value types of the |
|
<code class="docutils literal"><span class="pre">KStream</span></code> do not match the configured default SerDes.</li> |
|
<li>See <a class="reference internal" href="datatypes.html#streams-developer-guide-serdes"><span class="std std-ref">Data Types and Serialization</span></a> for information about configuring default SerDes, available SerDes, |
|
and implementing your own custom SerDes.</li> |
|
</ul> |
|
<p>A variant of <code class="docutils literal"><span class="pre">to</span></code> exists that enables you to specify how the data is produced by using a <code class="docutils literal"><span class="pre">Produced</span></code> |
|
instance to specify, for example, a <code class="docutils literal"><span class="pre">StreamPartitioner</span></code> that gives you control over |
|
how output records are distributed across the partitions of the output topic.</p> |
|
<p>Another variant of <code class="docutils literal"><span class="pre">to</span></code> exists that enables you to dynamically choose which topic to send to for each record via a <code class="docutils literal"><span class="pre">TopicNameExtractor</span></code> |
|
instance.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">table</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
|
|
<span class="c1">// Write the stream to the output topic, using the configured default key</span> |
|
<span class="c1">// and value serdes.</span> |
|
<span class="n">stream</span><span class="o">.</span><span class="na">to</span><span class="o">(</span><span class="s">"my-stream-output-topic"</span><span class="o">);</span> |
|
|
|
<span class="c1">// Same for table</span> |
|
<span class="n">table</span><span class="o">.</span><span class="na">to</span><span class="o">(</span><span class="s">"my-table-output-topic"</span><span class="o">);</span> |
|
|
|
<span class="c1">// Write the stream to the output topic, using explicit key and value serdes,</span> |
|
<span class="c1">// (thus overriding the defaults in the config properties).</span> |
|
<span class="n">stream</span><span class="o">.</span><span class="na">to</span><span class="o">(</span><span class="s">"my-stream-output-topic"</span><span class="o">,</span> <span class="n">Produced</span><span class="o">.</span><span class="na">with</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">());</span> |
|
</pre></div> |
|
</div> |
|
<p><strong>Causes data re-partitioning if any of the following conditions is true:</strong></p> |
|
<ol class="last arabic simple"> |
|
<li>If the output topic has a different number of partitions than the stream/table.</li> |
|
<li>If the <code class="docutils literal"><span class="pre">KStream</span></code> was marked for re-partitioning.</li> |
|
<li>If you provide a custom <code class="docutils literal"><span class="pre">StreamPartitioner</span></code> to explicitly control how to distribute the output records |
|
across the partitions of the output topic.</li> |
|
<li>If the key of an output record is <code class="docutils literal"><span class="pre">null</span></code>.</li> |
|
</ol> |
|
</td> |
|
</tr> |
|
<tr class="row-odd"><td><p class="first"><strong>Through</strong></p> |
|
<ul class="last simple"> |
|
<li>KStream -> KStream</li> |
|
<li>KTable -> KTable</li> |
|
</ul> |
|
</td> |
|
<td><p class="first">Write the records to a Kafka topic and create a new stream/table from that topic. |
|
Essentially a shorthand for <code class="docutils literal"><span class="pre">KStream#to()</span></code> followed by <code class="docutils literal"><span class="pre">StreamsBuilder#stream()</span></code>, same for tables. |
|
(<a class="reference external" href="../../../javadoc/org/apache/kafka/streams/kstream/KStream.html#through(java.lang.String)">KStream details</a>)</p> |
|
<p>When to provide SerDes explicitly:</p> |
|
<ul class="simple"> |
|
<li>If you do not specify SerDes explicitly, the default SerDes from the |
|
<a class="reference internal" href="config-streams.html#streams-developer-guide-configuration"><span class="std std-ref">configuration</span></a> are used.</li> |
|
<li>You <strong>must specify SerDes explicitly</strong> if the key and/or value types of the <code class="docutils literal"><span class="pre">KStream</span></code> or <code class="docutils literal"><span class="pre">KTable</span></code> do not |
|
match the configured default SerDes.</li> |
|
<li>See <a class="reference internal" href="datatypes.html#streams-developer-guide-serdes"><span class="std std-ref">Data Types and Serialization</span></a> for information about configuring default SerDes, available SerDes, |
|
and implementing your own custom SerDes.</li> |
|
</ul> |
|
<p>A variant of <code class="docutils literal"><span class="pre">through</span></code> exists that enables you to specify how the data is produced by using a <code class="docutils literal"><span class="pre">Produced</span></code> |
|
instance to specify, for example, a <code class="docutils literal"><span class="pre">StreamPartitioner</span></code> that gives you control over |
|
how output records are distributed across the partitions of the output topic.</p> |
|
<div class="highlight-java"><div class="highlight"><pre><span></span><span class="n">StreamsBuilder</span> <span class="n">builder</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span> <span class="o">...;</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">table</span> <span class="o">=</span> <span class="o">...;</span> |
|
|
|
<span class="c1">// Variant 1: Imagine that your application needs to continue reading and processing</span> |
|
<span class="c1">// the records after they have been written to a topic via ``to()``. Here, one option</span> |
|
<span class="c1">// is to write to an output topic, then read from the same topic by constructing a</span> |
|
<span class="c1">// new stream from it, and then begin processing it (here: via `map`, for example).</span> |
|
<span class="n">stream</span><span class="o">.</span><span class="na">to</span><span class="o">(</span><span class="s">"my-stream-output-topic"</span><span class="o">);</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">newStream</span> <span class="o">=</span> <span class="n">builder</span><span class="o">.</span><span class="na">stream</span><span class="o">(</span><span class="s">"my-stream-output-topic"</span><span class="o">).</span><span class="na">map</span><span class="o">(...);</span> |
|
|
|
<span class="c1">// Variant 2 (better): Since the above is a common pattern, the DSL provides the</span> |
|
<span class="c1">// convenience method ``through`` that is equivalent to the code above.</span> |
|
<span class="c1">// Note that you may need to specify key and value serdes explicitly, which is</span> |
|
<span class="c1">// not shown in this simple example.</span> |
|
<span class="n">KStream</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">newStream</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">through</span><span class="o">(</span><span class="s">"user-clicks-topic"</span><span class="o">).</span><span class="na">map</span><span class="o">(...);</span> |
|
|
|
<span class="c1">// ``through`` is also available for tables</span> |
|
<span class="n">KTable</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">></span> <span class="n">newTable</span> <span class="o">=</span> <span class="n">table</span><span class="o">.</span><span class="na">through</span><span class="o">(</span><span class="s">"my-table-output-topic"</span><span class="o">).</span><span class="na">map</span><span class="o">(...);</span> |
|
</pre></div> |
|
</div> |
|
<p><strong>Causes data re-partitioning if any of the following conditions is true:</strong></p> |
|
<ol class="last arabic simple"> |
|
<li>If the output topic has a different number of partitions than the stream/table.</li> |
|
<li>If the <code class="docutils literal"><span class="pre">KStream</span></code> was marked for re-partitioning.</li> |
|
<li>If you provide a custom <code class="docutils literal"><span class="pre">StreamPartitioner</span></code> to explicitly control how to distribute the output records |
|
across the partitions of the output topic.</li> |
|
<li>If the key of an output record is <code class="docutils literal"><span class="pre">null</span></code>.</li> |
|
</ol> |
|
</td> |
|
</tr> |
|
</tbody> |
|
</table> |
|
<div class="admonition note"> |
|
<p><b>Note</b></p> |
|
<p class="last"><strong>When you want to write to systems other than Kafka:</strong> |
|
Besides writing the data back to Kafka, you can also apply a |
|
<a class="reference internal" href="#streams-developer-guide-dsl-process"><span class="std std-ref">custom processor</span></a> as a stream sink at the end of the processing to, for |
|
example, write to external databases. First, doing so is not a recommended pattern – we strongly suggest to use the |
|
<a class="reference internal" href="../../connect/index.html#kafka-connect"><span class="std std-ref">Kafka Connect API</span></a> instead. However, if you do use such a sink processor, please be aware that |
|
it is now your responsibility to guarantee message delivery semantics when talking to such external systems (e.g., to |
|
retry on delivery failure or to prevent message duplication).</p> |
|
</div> |
|
</div> |
|
<div class="section" id="testing-a-streams-app"> |
|
<a class="headerlink" href="#testing-a-streams-app" title="Permalink to this headline"><h2>Testing a Streams application</h2></a> |
|
Kafka Streams comes with a <code>test-utils</code> module to help you test your application <a href="testing.html">here</a>. |
|
</div> |
|
</div> |
|
<div class="section" id="scala-dsl"> |
|
<span id="streams-developer-guide-dsl-scala-dsl"></span><h2><a class="toc-backref" href="#id27">Kafka Streams DSL for Scala</a><a class="headerlink" href="#scala-dsl" title="Permalink to this headline"></a></h2> |
|
<p id="scala-dsl-motivation">The Kafka Streams DSL Java APIs are based on the Builder design pattern, which allows users to incrementally build the target functionality using lower level compositional fluent APIs. These APIs can be called from Scala, but there are several issues:</p> |
|
<ol class="last arabic simple"> |
|
<li><strong>Additional type annotations</strong> - The Java APIs use Java generics in a way that are not fully compatible with the type inferencer of the Scala compiler. Hence the user has to add type annotations to the Scala code, which seems rather non-idiomatic in Scala.</li> |
|
<li><strong>Verbosity</strong> - In some cases the Java APIs appear too verbose compared to idiomatic Scala.</li> |
|
<li><strong>Type Unsafety</strong> - The Java APIs offer some options where the compile time type safety is sometimes subverted and can result in runtime errors. This stems from the fact that the SerDes defined as part of config are not type checked during compile time. Hence any missing SerDes can result in runtime errors.</li> |
|
</ol> |
|
<p id="scala-dsl-overview">The Kafka Streams DSL for Scala library is a wrapper over the existing Java APIs for Kafka Streams DSL that addresses the concerns raised above. |
|
It does not attempt to provide idiomatic Scala APIs that one would implement in a Scala library developed from scratch. The intention is to make the Java APIs more usable in Scala through better type inferencing, enhanced expressiveness, and lesser boilerplates. |
|
</p> |
|
<p>The library wraps Java Stream DSL APIs in Scala thereby providing:</p> |
|
<ol class="last arabic simple"> |
|
<li>Better type inference in Scala.</li> |
|
<li>Less boilerplate in application code.</li> |
|
<li>The usual builder-style composition that developers get with the original Java API.</li> |
|
<li>Implicit serializers and de-serializers leading to better abstraction and less verbosity.</li> |
|
<li>Better type safety during compile time.</li> |
|
</ol> |
|
<p>All functionality provided by Kafka Streams DSL for Scala are under the root package name of <code class="docutils literal"><span class="pre">org.apache.kafka.streams.scala</span></code>.</p> |
|
<p>Many of the public facing types from the Java API are wrapped. The following Scala abstractions are available to the user:</p> |
|
<ul> |
|
<li><code class="docutils literal"><span class="pre">org.apache.kafka.streams.scala.StreamsBuilder</span></code></li> |
|
<li><code class="docutils literal"><span class="pre">org.apache.kafka.streams.scala.kstream.KStream</span></code></li> |
|
<li><code class="docutils literal"><span class="pre">org.apache.kafka.streams.scala.kstream.KTable</span></code></li> |
|
<li><code class="docutils literal"><span class="pre">org.apache.kafka.streams.scala.kstream.KGroupedStream</span></code></li> |
|
<li><code class="docutils literal"><span class="pre">org.apache.kafka.streams.scala.kstream.KGroupedTable</span></code></li> |
|
<li><code class="docutils literal"><span class="pre">org.apache.kafka.streams.scala.kstream.SessionWindowedKStream</span></code></li> |
|
<li><code class="docutils literal"><span class="pre">org.apache.kafka.streams.scala.kstream.TimeWindowedKStream</span></code></li> |
|
</ul> |
|
<p>The library also has several utility abstractions and modules that the user needs to use for proper semantics.</p> |
|
<ul> |
|
<li><code class="docutils literal"><span class="pre">org.apache.kafka.streams.scala.ImplicitConversions</span></code>: Module that brings into scope the implicit conversions between the Scala and Java classes.</li> |
|
<li><code class="docutils literal"><span class="pre">org.apache.kafka.streams.scala.Serdes</span></code>: Module that contains all primitive SerDes that can be imported as implicits and a helper to create custom SerDes.</li> |
|
</ul> |
|
<p>The library is cross-built with Scala 2.11 and 2.12. To reference the library compiled against Scala 2.11 include the following in your maven <code>pom.xml</code> add the following:</p> |
|
<pre class="brush: xml;"> |
|
<dependency> |
|
<groupId>org.apache.kafka</groupId> |
|
<artifactId>kafka-streams-scala_2.11</artifactId> |
|
<version>{{fullDotVersion}}</version> |
|
</dependency> |
|
</pre> |
|
<p>To use the library compiled against Scala 2.12 replace the <code class="docutils literal"><span class="pre">artifactId</span></code> with <code class="docutils literal"><span class="pre">kafka-streams-scala_2.12</span></code>.</p> |
|
<p>When using SBT then you can reference the correct library using the following:</p> |
|
<pre class="brush: scala;"> |
|
libraryDependencies += "org.apache.kafka" %% "kafka-streams-scala" % "{{fullDotVersion}}" |
|
</pre> |
|
<div class="section" id="scala-dsl-sample-usage"> |
|
<span id="streams-developer-guide-dsl-sample-usage"></span><h3><a class="toc-backref" href="#id28">Sample Usage</a><a class="headerlink" href="#scala-dsl-sample-usage" title="Permalink to this headline"></a></h3> |
|
<p>The library works by wrapping the original Java abstractions of Kafka Streams within a Scala wrapper object and then using implicit conversions between them. All the Scala abstractions are named identically as the corresponding Java abstraction, but they reside in a different package of the library e.g. the Scala class <code class="docutils literal"><span class="pre">org.apache.kafka.streams.scala.StreamsBuilder</span></code> is a wrapper around <code class="docutils literal"><span class="pre">org.apache.kafka.streams.StreamsBuilder</span></code>, <code class="docutils literal"><span class="pre">org.apache.kafka.streams.scala.kstream.KStream</span></code> is a wrapper around <code class="docutils literal"><span class="pre">org.apache.kafka.streams.kstream.KStream</span></code>, and so on.</p> |
|
<p>Here's an example of the classic WordCount program that uses the Scala <code class="docutils literal"><span class="pre">StreamsBuilder</span></code> that builds an instance of <code class="docutils literal"><span class="pre">KStream</span></code> which is a wrapper around Java <code class="docutils literal"><span class="pre">KStream</span></code>. Then we reify to a table and get a <code class="docutils literal"><span class="pre">KTable</span></code>, which, again is a wrapper around Java <code class="docutils literal"><span class="pre">KTable</span></code>.</p> |
|
<p>The net result is that the following code is structured just like using the Java API, but with Scala and with far fewer type annotations compared to using the Java API directly from Scala. The difference in type annotation usage is more obvious when given an example. Below is an example WordCount implementation that will be used to demonstrate the differences between the Scala and Java API.</p> |
|
<pre class="brush: scala;"> |
|
import java.util.Properties |
|
import java.util.concurrent.TimeUnit |
|
|
|
import org.apache.kafka.streams.kstream.Materialized |
|
import org.apache.kafka.streams.scala.ImplicitConversions._ |
|
import org.apache.kafka.streams.scala._ |
|
import org.apache.kafka.streams.scala.kstream._ |
|
import org.apache.kafka.streams.{KafkaStreams, StreamsConfig} |
|
|
|
object WordCountApplication extends App { |
|
import Serdes._ |
|
|
|
val props: Properties = { |
|
val p = new Properties() |
|
p.put(StreamsConfig.APPLICATION_ID_CONFIG, "wordcount-application") |
|
p.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092") |
|
p |
|
} |
|
|
|
val builder: StreamsBuilder = new StreamsBuilder |
|
val textLines: KStream[String, String] = builder.stream[String, String]("TextLinesTopic") |
|
val wordCounts: KTable[String, Long] = textLines |
|
.flatMapValues(textLine => textLine.toLowerCase.split("\\W+")) |
|
.groupBy((_, word) => word) |
|
.count(Materialized.as("counts-store")) |
|
wordCounts.toStream.to("WordsWithCountsTopic") |
|
|
|
val streams: KafkaStreams = new KafkaStreams(builder.build(), props) |
|
streams.start() |
|
|
|
sys.ShutdownHookThread { |
|
streams.close(10, TimeUnit.SECONDS) |
|
} |
|
} |
|
</pre> |
|
<p>In the above code snippet, we don't have to provide any SerDes, <code class="docutils literal"><span class="pre">Serialized</span></code>, <code class="docutils literal"><span class="pre">Produced</span></code>, <code class="docutils literal"><span class="pre">Consumed</span></code> or <code class="docutils literal"><span class="pre">Joined</span></code> explicitly. They will also not be dependent on any SerDes specified in the config. <strong>In fact all SerDes specified in the config will be ignored by the Scala APIs</strong>. All SerDes and <code class="docutils literal"><span class="pre">Serialized</span></code>, <code class="docutils literal"><span class="pre">Produced</span></code>, <code class="docutils literal"><span class="pre">Consumed</span></code> or <code class="docutils literal"><span class="pre">Joined</span></code> will be handled through implicit SerDes as discussed later in the <a href="#scala-dsl-implicit-serdes">Implicit SerDes</a> section. The complete independence from configuration based SerDes is what makes this library completely typesafe. Any missing instances of SerDes, <code class="docutils literal"><span class="pre">Serialized</span></code>, <code class="docutils literal"><span class="pre">Produced</span></code>, <code class="docutils literal"><span class="pre">Consumed</span></code> or <code class="docutils literal"><span class="pre">Joined</span></code> will be flagged as a compile time error.</p> |
|
</div> |
|
<div class="section" id="scala-dsl-implicit-serdes"> |
|
<span id="streams-developer-guide-dsl-scala-dsl-implicit-serdes"></span><h3><a class="toc-backref" href="#id29">Implicit SerDes</a><a class="headerlink" href="#scala-dsl-implicit-serdes" title="Permalink to this headline"></a></h3> |
|
<p>One of the common complaints of Scala users with the Java API has been the repetitive usage of the SerDes in API invocations. Many of the APIs need to take the SerDes through abstractions like <code class="docutils literal"><span class="pre">Serialized</span></code>, <code class="docutils literal"><span class="pre">Produced</span></code>, <code class="docutils literal"><span class="pre">Consumed</span></code> or <code class="docutils literal"><span class="pre">Joined</span></code>. And the user has to supply them every time through the with function of these classes.</p> |
|
<p>The library uses the power of <a href="https://docs.scala-lang.org/tour/implicit-parameters.html">Scala implicit parameters</a> to alleviate this concern. As a user you can provide implicit SerDes or implicit values of <code class="docutils literal"><span class="pre">Serialized</span></code>, <code class="docutils literal"><span class="pre">Produced</span></code>, <code class="docutils literal"><span class="pre">Consumed</span></code> or <code class="docutils literal"><span class="pre">Joined</span></code> once and make your code less verbose. In fact you can just have the implicit SerDes in scope and the library will make the instances of <code class="docutils literal"><span class="pre">Serialized</span></code>, <code class="docutils literal"><span class="pre">Produced</span></code>, <code class="docutils literal"><span class="pre">Consumed</span></code> or <code class="docutils literal"><span class="pre">Joined</span></code> available in scope.</p> |
|
<p>The library also bundles all implicit SerDes of the commonly used primitive types in a Scala module - so just import the module vals and have all SerDes in scope. A similar strategy of modular implicits can be adopted for any user-defined SerDes as well (User-defined SerDes are discussed in the next section).</p> |
|
<p>Here's an example:</p> |
|
<pre class="brush: scala;"> |
|
// DefaultSerdes brings into scope implicit SerDes (mostly for primitives) |
|
// that will set up all Serialized, Produced, Consumed and Joined instances. |
|
// So all APIs below that accept Serialized, Produced, Consumed or Joined will |
|
// get these instances automatically |
|
import Serdes._ |
|
|
|
val builder = new StreamsBuilder() |
|
|
|
val userClicksStream: KStream[String, Long] = builder.stream(userClicksTopic) |
|
|
|
val userRegionsTable: KTable[String, String] = builder.table(userRegionsTopic) |
|
|
|
// The following code fragment does not have a single instance of Serialized, |
|
// Produced, Consumed or Joined supplied explicitly. |
|
// All of them are taken care of by the implicit SerDes imported by DefaultSerdes |
|
val clicksPerRegion: KTable[String, Long] = |
|
userClicksStream |
|
.leftJoin(userRegionsTable)((clicks, region) => (if (region == null) "UNKNOWN" else region, clicks)) |
|
.map((_, regionWithClicks) => regionWithClicks) |
|
.groupByKey |
|
.reduce(_ + _) |
|
|
|
clicksPerRegion.toStream.to(outputTopic) |
|
</pre> |
|
<p>Quite a few things are going on in the above code snippet that may warrant a few lines of elaboration:</p> |
|
<ol> |
|
<li>The code snippet does not depend on any config defined SerDes. In fact any SerDes defined as part of the config will be ignored.</li> |
|
<li>All SerDes are picked up from the implicits in scope. And <code class="docutils literal"><span class="pre">import Serdes._</span></code> brings all necessary SerDes in scope.</li> |
|
<li>This is an example of compile time type safety that we don't have in the Java APIs.</li> |
|
<li>The code looks less verbose and more focused towards the actual transformation that it does on the data stream.</li> |
|
</ol> |
|
</div> |
|
<div class="section" id="scala-dsl-user-defined-serdes"> |
|
<span id="streams-developer-guide-dsl-scala-dsl-user-defined-serdes"></span><h3><a class="toc-backref" href="#id30">User-Defined SerDes</a><a class="headerlink" href="#scala-dsl-user-defined-serdes" title="Permalink to this headline"></a></h3> |
|
<p>When the default primitive SerDes are not enough and we need to define custom SerDes, the usage is exactly the same as above. Just define the implicit SerDes and start building the stream transformation. Here's an example with <code class="docutils literal"><span class="pre">AvroSerde</span></code>:</p> |
|
<pre class="brush: scala;"> |
|
// domain object as a case class |
|
case class UserClicks(clicks: Long) |
|
|
|
// An implicit Serde implementation for the values we want to |
|
// serialize as avro |
|
implicit val userClicksSerde: Serde[UserClicks] = new AvroSerde |
|
|
|
// Primitive SerDes |
|
import Serdes._ |
|
|
|
// And then business as usual .. |
|
|
|
val userClicksStream: KStream[String, UserClicks] = builder.stream(userClicksTopic) |
|
|
|
val userRegionsTable: KTable[String, String] = builder.table(userRegionsTopic) |
|
|
|
// Compute the total per region by summing the individual click counts per region. |
|
val clicksPerRegion: KTable[String, Long] = |
|
userClicksStream |
|
|
|
// Join the stream against the table. |
|
.leftJoin(userRegionsTable)((clicks, region) => (if (region == null) "UNKNOWN" else region, clicks.clicks)) |
|
|
|
// Change the stream from <user> -> <region, clicks> to <region> -> <clicks> |
|
.map((_, regionWithClicks) => regionWithClicks) |
|
|
|
// Compute the total per region by summing the individual click counts per region. |
|
.groupByKey |
|
.reduce(_ + _) |
|
|
|
// Write the (continuously updating) results to the output topic. |
|
clicksPerRegion.toStream.to(outputTopic) |
|
</pre> |
|
<p>A complete example of user-defined SerDes can be found in a test class within the library.</p> |
|
</div> |
|
</div> |
|
</div> |
|
|
|
|
|
</div> |
|
</div> |
|
<div class="pagination"> |
|
<a href="/{{version}}/documentation/streams/developer-guide/config-streams" class="pagination__btn pagination__btn__prev">Previous</a> |
|
<a href="/{{version}}/documentation/streams/developer-guide/processor-api" class="pagination__btn pagination__btn__next">Next</a> |
|
</div> |
|
</script> |
|
|
|
<!--#include virtual="../../../includes/_header.htm" --> |
|
<!--#include virtual="../../../includes/_top.htm" --> |
|
<div class="content documentation documentation--current"> |
|
<!--#include virtual="../../../includes/_nav.htm" --> |
|
<div class="right"> |
|
<!--#include virtual="../../../includes/_docs_banner.htm" --> |
|
<ul class="breadcrumbs"> |
|
<li><a href="/documentation">Documentation</a></li> |
|
<li><a href="/documentation/streams">Kafka Streams</a></li> |
|
<li><a href="/documentation/streams/developer-guide/">Developer Guide</a></li> |
|
</ul> |
|
<div class="p-content"></div> |
|
</div> |
|
</div> |
|
<!--#include virtual="../../../includes/_footer.htm" --> |
|
<script> |
|
$(function() { |
|
// Show selected style on nav item |
|
$('.b-nav__streams').addClass('selected'); |
|
|
|
//sticky secondary nav |
|
var $navbar = $(".sub-nav-sticky"), |
|
y_pos = $navbar.offset().top, |
|
height = $navbar.height(); |
|
|
|
$(window).scroll(function() { |
|
var scrollTop = $(window).scrollTop(); |
|
|
|
if (scrollTop > y_pos - height) { |
|
$navbar.addClass("navbar-fixed") |
|
} else if (scrollTop <= y_pos) { |
|
$navbar.removeClass("navbar-fixed") |
|
} |
|
}); |
|
|
|
// Display docs subnav items |
|
$('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded'); |
|
}); |
|
</script>
|
|
|