This is an automated email from the ASF dual-hosted git repository.

mjsax pushed a commit to branch 2.4
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.4 by this push:
     new f69eb75  MINOR: Add documentation for foreign-key joins (KIP-213) 
(#7535)
f69eb75 is described below

commit f69eb750d8e99128f7c41fb1868e6e42b1db7554
Author: Adam Bellemare <[email protected]>
AuthorDate: Thu Oct 24 14:35:55 2019 -0400

    MINOR: Add documentation for foreign-key joins (KIP-213) (#7535)
    
    Reviewers: John Roesler <[email protected]>, Matthias J. Sax 
<[email protected]>
---
 docs/streams/developer-guide/dsl-api.html | 351 ++++++++++++++++++++++++++++--
 docs/streams/upgrade-guide.html           |  13 +-
 2 files changed, 339 insertions(+), 25 deletions(-)

diff --git a/docs/streams/developer-guide/dsl-api.html 
b/docs/streams/developer-guide/dsl-api.html
index c99b03e..5be029e 100644
--- a/docs/streams/developer-guide/dsl-api.html
+++ b/docs/streams/developer-guide/dsl-api.html
@@ -48,7 +48,8 @@
                         <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="#ktable-ktable-equi-join" id="id16">KTable-KTable Equi-Join</a></li>
+                            <li><a class="reference internal" 
href="#ktable-ktable-fk-join" id="id16">KTable-KTable Foreign-Key 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>
@@ -1628,19 +1629,25 @@
                             <td>Supported</td>
                             <td>Supported</td>
                         </tr>
-                        <tr class="row-even"><td>KStream-to-KTable</td>
+                        <tr class="row-even"><td>KTable-to-KTable Foreign-Key 
Join</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>
+                        <tr class="row-odd"><td>KStream-to-KTable</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>
+                        <tr class="row-even"><td>KStream-to-GlobalKTable</td>
+                            <td>Non-windowed</td>
+                            <td>Supported</td>
+                            <td>Supported</td>
+                            <td>Not Supported</td>
+                        </tr>
+                        <tr class="row-odd"><td>KTable-to-GlobalKTable</td>
                             <td>N/A</td>
                             <td>Not Supported</td>
                             <td>Not Supported</td>
@@ -1651,13 +1658,11 @@
                     <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
+                        <p>For equi-joins, input data must be co-partitioned 
when joining. This ensures that input records with the same key from both sides 
of the
                             join, are delivered to the same stream task during 
processing.
-                            <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>
+                            <strong>It is your responsibility to ensure data 
co-partitioning when joining</strong>.
+                            Co-partitioning is not required when performing <a 
class="reference internal" 
href="#streams-developer-guide-dsl-joins-ktable-ktable-fk-join"><span 
class="std std-ref">KTable-KTable Foreign-Key joins</span></a> and <a 
class="reference internal" href="#streams_concepts_globalktable"><span 
class="std std-ref">Global KTable joins</span></a>.
+                            </p>
                         <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>
@@ -1676,11 +1681,14 @@
                             <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>&#8216;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>
+                        <p>There are two exceptions where co-partitioning is 
not required. For
+                            <a class="reference internal" 
href="#streams-developer-guide-dsl-joins-kstream-globalktable"><span class="std 
std-ref">KStream-GlobalKTable joins</span></a> joins, co-partitioning is
+                            not required because <em>all</em> partitions of 
the <code class="docutils literal"><span 
class="pre">GlobalKTable</span></code>&#8216;s underlying changelog stream are 
made available to
+                             each <code class="docutils literal"><span 
class="pre">KafkaStreams</span></code> instance. That is, 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>.
+                            <a class="reference internal" 
href="#streams-developer-guide-dsl-joins-ktable-ktable-fk-join"><span 
class="std std-ref">KTable-KTable Foreign-Key joins</span></a> also do not 
require co-partitioning. Kafka Streams internally ensures co-partitioning for 
Foreign-Key joins.
+                            </p>
+
                         <div class="admonition note">
                             <p><b>Note</b></p>
                             <p class="last"><strong>Kafka Streams partly 
verifies the co-partitioning requirement:</strong>
@@ -2064,9 +2072,9 @@
                             </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
+                    <div class="section" id="ktable-ktable-equi-join">
+                        <span 
id="streams-developer-guide-dsl-joins-ktable-ktable"></span><h5><a 
class="toc-backref" href="#id16">KTable-KTable Equi-Join</a><a 
class="headerlink" href="#ktable-ktable-equi-join" title="Permalink to this 
headline"></a></h5>
+                        <p>KTable-KTable equi-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>
@@ -2240,8 +2248,8 @@
                             </tr>
                             </tbody>
                         </table>
-                        <p><strong>Semantics of table-table joins:</strong>
-                            The semantics of the various table-table join 
variants are explained below.
+                        <p><strong>Semantics of table-table 
equi-joins:</strong>
+                            The semantics of the various table-table equi-join 
variants are explained below.
                             To improve the readability of the table, you can 
assume that (1) all records have the same key (and thus the key in the table is 
omitted) and that (2) all records are processed in timestamp order.
                             The columns INNER JOIN, LEFT JOIN, and OUTER JOIN 
denote what is passed as arguments to the user-supplied
                             <a class="reference external" 
href="/{{version}}/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
@@ -2374,6 +2382,309 @@
                             </tbody>
                         </table>
                     </div>
+                    <div class="section" id="ktable-ktable-fk-join"> <span
+                        
id="streams-developer-guide-dsl-joins-ktable-ktable-fk"></span>
+                      <h5><a class="toc-backref" href="#id16">KTable-KTable 
Foreign-Key
+                          Join</a></h5>
+                      <p>KTable-KTable foreign-key joins are always 
<em>non-windowed</em>
+                        joins. Foreign-key joins are analogous to joins in 
SQL. As a rough example: 
+                        <div><code class="docutils literal"><span class="pre">
+                        SELECT ... FROM {this KTable} 
+                        JOIN {other KTable} 
+                        ON {other.key} = {result of 
foreignKeyExtractor(this.value)} ... 
+                        </span></code></div>
+                        The output of the operation is a new KTable containing 
the join result.
+                        </p>
+                        <p> 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>. A foreign-key 
extractor
+                        function is applied to the left record, with a new 
intermediate
+                        record created and is used to lookup and join with the 
corresponding 
+                        primary key on the right hand side table.
+                        The result is a new KTable that represents the 
changelog stream
+                        of the join operation.</p>
+                      The left KTable can have multiple records which map to 
the same
+                      key on the right KTable. An update to a single left 
KTable entry
+                      may result in a single output event, provided the 
corresponding
+                      key exists in the right KTable. Consequently, a single 
update to a
+                      right KTable entry will result in an update for each 
record in the
+                      left KTable that has the same foreign key.<br>
+                      <br>
+                      <div class="highlight-java"> </div>
+                      <table class="non-scrolling-table width-100-percent 
docutils"
+                        border="1">
+                        <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 a foreign-key INNER 
JOIN of this
+                                table with another table. The result is an 
ever-updating
+                                KTable that represents the “current” result of 
the join.
+                                <a class="reference external"
+                
href="/%7B%7Bversion%7D%7D/javadoc/org/apache/kafka/streams/kstream/KTable.html#join-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-">(details)</a></p>
+                              <div class="highlight-java">
+                                <div class="highlight">
+                                  <pre><span></span><span 
class="n">KTable</span><span class="o">&lt;</span><span 
class="n">String</span><span class="o">,</span> Long<span class="o">&gt;</span> 
<span class="n">left</span> <span class="o">=</span> <span class="o">...;</span>
+                <span class="n">KTable</span><span 
class="o">&lt;Long</span><span class="o">,</span> <span 
class="n">Double</span><span class="o">&gt;</span> <span class="n">right</span> 
<span class="o">=</span> <span class="o">...;<br>//This </span><span 
class="o"><span class="o"><span class="n">foreignKeyExtractor</span></span> 
simply uses the left-value to map to the right-key.<br></span><span 
class="o"><span class="n">Function</span><span class="o">&lt;Long</span><span 
class="o">,</s [...]
+                <span class="n">KTable</span><span class="o">&lt;</span><span 
class="n">String</span><span class="o">,</span> <span 
class="n">String</span><span class="o">&gt;</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><br>    
<span class="o"><span class="n">foreignKeyExtractor,</span></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">-&gt;</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>
+                </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: </p>
+                                  <pre><code class="docutils literal"><span 
class="pre">foreignKeyExtractor.apply(leftRecord.value)</span> <span 
class="pre">==</span> <span class="pre">rightRecord.key</span></code></pre>
+                                </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>
+                                            Records for which the <code 
class="docutils literal"><span class="pre">foreignKeyExtractor</span></code> 
produces <code class="docutils literal"><span class="pre">null</span></code> 
are ignored and do not trigger a join.
+                                            If you want to join with <code 
class="docutils literal"><span class="pre">null</span></code> foreign keys, use 
a suitable sentinel value to do so (i.e. <code class="docutils literal"><span 
class="pre">"NULL"</span></code> for a String field, or <code class="docutils 
literal"><span class="pre">-1</span></code> for an auto-incrementing integer 
field).
+                                        </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. 
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 foreign-key LEFT 
JOIN of this
+                                table with another table. <a class="reference 
external"
+                
href="/%7B%7Bversion%7D%7D/javadoc/org/apache/kafka/streams/kstream/KTable.html#leftJoin-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-">(details)</a></p>
+                              <div class="highlight-java">
+                                <div class="highlight">
+                                  <pre><span></span><span 
class="n">KTable</span><span class="o">&lt;</span><span 
class="n">String</span><span class="o">,</span> Long<span class="o">&gt;</span> 
<span class="n">left</span> <span class="o">=</span> <span class="o">...;</span>
+                <span class="n">KTable</span><span 
class="o">&lt;Long</span><span class="o">,</span> <span 
class="n">Double</span><span class="o">&gt;</span> <span class="n">right</span> 
<span class="o">=</span> <span class="o">...;<br>//This </span><span 
class="o"><span class="o"><span class="n">foreignKeyExtractor</span></span> 
simply uses the left-value to map to the right-key.<br></span><span 
class="o"><span class="n">Function</span><span class="o">&lt;Long</span><span 
class="o">,</s [...]
+                <span class="n">KTable</span><span class="o">&lt;</span><span 
class="n">String</span><span class="o">,</span> <span 
class="n">String</span><span class="o">&gt;</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><br>    
<span class="o"><span class="n">foreignKeyExtractor,</span></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">-&gt;</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>
+                </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: </p>
+                                  <pre><code class="docutils literal"><span 
class="pre">foreignKeyExtractor.apply(leftRecord.value)</span> <span 
class="pre">==</span> <span class="pre">rightRecord.key</span></code></pre>
+                                </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>
+                                            Records for which the <code 
class="docutils literal"><span class="pre">foreignKeyExtractor</span></code> 
produces <code class="docutils literal"><span class="pre">null</span></code> 
are ignored and do not trigger a join.
+                                            If you want to join with <code 
class="docutils literal"><span class="pre">null</span></code> foreign keys, use 
a suitable sentinel value to do so (i.e. <code class="docutils literal"><span 
class="pre">"NULL"</span></code> for a String field, or <code class="docutils 
literal"><span class="pre">-1</span></code> for an auto-incrementing integer 
field).
+                                        </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. 
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=7 &amp; 8 
in the
+                                    table below, which lists <code 
class="docutils
+                                      literal"><span class="pre">(q,10,null) 
and (r,10,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 table-table foreign-key 
joins:</strong>
+                        The semantics of the table-table foreign-key INNER and 
LEFT JOIN
+                        variants are demonstrated below.
+                        The key is shown alongside the value for each record.
+                        Records are processed in incrementing offset order.
+                        The columns INNER JOIN and LEFT JOIN denote what is
+                        passed as arguments to the user-supplied <a 
class="reference
+                          external"
+                
href="/%7B%7Bversion%7D%7D/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. For the purpose of this example, 
<code
+                          class="docutils literal"><span class="pre">Function
+                            foreignKeyExtractor</span></code> simply uses the 
left-value
+                        as the output. </p>
+                      <table class="docutils" border="1">
+                        <colgroup> <col width="12%"> <col width="13%"> <col
+                            width="13%"> <col width="31%"> <col width="31%"> 
</colgroup>
+                        <thead valign="bottom"> <tr class="row-odd">
+                            <th class="head">Record Offset</th>
+                            <th class="head">Left KTable (K, extracted-FK)</th>
+                            <th class="head">Right KTable (FK, VR)</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>(k,1)</td>
+                            <td>&nbsp;(1,foo)</td>
+                            <td>
+                              <meta http-equiv="content-type" 
content="text/html;
+                                charset=UTF-8">
+                              (k,1,foo) <br>
+                            </td>
+                            <td>
+                              <meta http-equiv="content-type" 
content="text/html;
+                                charset=UTF-8">
+                              (k,1,foo)</td>
+                          </tr>
+                          <tr class="row-odd">
+                            <td>2</td>
+                            <td>(k,2)
+                              <meta http-equiv="content-type" 
content="text/html;
+                                charset=UTF-8">
+                            </td>
+                            <td>
+                              <meta http-equiv="content-type" 
content="text/html;
+                                charset=UTF-8">
+                              <br>
+                            </td>
+                            <td>(k,null)</td>
+                            <td>(k,2,null) <br>
+                            </td>
+                          </tr>
+                          <tr class="row-even">
+                            <td>3</td>
+                            <td>(k,3)<br>
+                            </td>
+                            <td>&nbsp;</td>
+                            <td>(k,null)</td>
+                            <td>(k,3,null)<br>
+                            </td>
+                          </tr>
+                          <tr class="row-odd">
+                            <td>4</td>
+                            <td>&nbsp;</td>
+                            <td>(3,bar)<br>
+                            </td>
+                            <td>(k,3,bar)<br>
+                            </td>
+                            <td>(k,3,bar)<br>
+                            </td>
+                          </tr>
+                          <tr class="row-even">
+                            <td>5</td>
+                            <td>(k,null)<br>
+                            </td>
+                            <td>&nbsp;</td>
+                            <td>(k,null)<br>
+                            </td>
+                            <td>(k,null,null)
+                              <meta http-equiv="content-type" 
content="text/html;
+                                charset=UTF-8">
+                            </td>
+                          </tr>
+                          <tr class="row-odd">
+                            <td>6</td>
+                            <td>(k,1)</td>
+                            <td><br>
+                            </td>
+                            <td>(k,1,foo)<br>
+                            </td>
+                            <td>(k,1,foo)<br>
+                            </td>
+                          </tr>
+                          <tr class="row-even">
+                            <td>7</td>
+                            <td>(q,10)<br>
+                            </td>
+                            <td>&nbsp;</td>
+                            <td><br>
+                            </td>
+                            <td>(q,10,null) </td>
+                          </tr>
+                          <tr class="row-odd">
+                            <td>8</td>
+                            <td>(r,10)</td>
+                            <td><br>
+                            </td>
+                            <td>&nbsp;</td>
+                            <td>(r,10,null)</td>
+                          </tr>
+                          <tr class="row-even">
+                            <td>9</td>
+                            <td><br>
+                            </td>
+                            <td>(10,baz)</td>
+                            <td>&nbsp;</td>
+                            <td>(q,10,baz), (r,10,baz) </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
diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html
index 04ec734..4d47f70 100644
--- a/docs/streams/upgrade-guide.html
+++ b/docs/streams/upgrade-guide.html
@@ -73,8 +73,12 @@
     </p>
 
     <h3><a id="streams_api_changes_240" 
href="#streams_api_changes_240">Streams API changes in 2.4.0</a></h3>
-
-    <!-- Placeholder KIP-213 -->
+    <p>     
+         As of 2.4.0 Kafka Streams offers a KTable-KTable foreign-key join (as 
per <a 
href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable";>KIP-213</a>).
 
+         This joiner allows for records to be joined between two KTables with 
different keys. 
+         Both <a 
href="/{{version}}/documentation/streams/developer-guide/dsl-api.html#ktable-ktable-fk-join">INNER
 and LEFT foreign-key joins</a> 
+         are supported.
+    </p>
     <p>
         In the 2.4 release, you now can name all operators in a Kafka Streams 
DSL topology via
         <a 
href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL";>KIP-307</a>.
@@ -90,7 +94,7 @@
 
     </p>
     <p>
-        Before the 2.4.0 version of Kafa Streams, users of the DSL could not 
name the state stores involved in a stream-stream join.
+        Before the 2.4.0 version of Kafka Streams, users of the DSL could not 
name the state stores involved in a stream-stream join.
         If users changed their topology and added a operator before the
         join, the internal names of the state stores would shift, requiring an 
application reset when redeploying.
         In the 2.4.0 release, Kafka Streams adds the <code>StreamJoined</code>
@@ -107,7 +111,6 @@
         new overloaded methods.  You can get more details from
         <a 
href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-479%3A+Add+StreamJoined+config+object+to+Join";>KIP-479</a>.
     </p>
-
     <p>
         With the introduction of incremental cooperative rebalancing, Streams 
no longer requires all tasks be revoked at the beginning of a rebalance. 
Instead, at the completion of the rebalance only those tasks which are to be 
migrated to another consumer
         for overall load balance will need to be closed and revoked. This 
changes the semantics of the <code>StateListener</code> a bit, as it will not 
necessarily transition to <code>REBALANCING</code> at the beginning of a 
rebalance anymore. Note that
@@ -158,7 +161,7 @@
         Hence, this feature won't be supported in the future any longer and 
you need to updated your code accordingly.
         If you use a custom <code>PartitionGrouper</code> and stop to use it, 
the created tasks might change.
         Hence, you will need to reset your application to upgrade it.
-    </p>
+    
 
     <h3><a id="streams_api_changes_230" 
href="#streams_api_changes_230">Streams API changes in 2.3.0</a></h3>
 

Reply via email to