kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mj...@apache.org
Subject [kafka] branch 3.0 updated: MINOR: update stream-stream join docs (#11172)
Date Thu, 05 Aug 2021 18:36:42 GMT
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/3.0 by this push:
     new 98f1ab9  MINOR: update stream-stream join docs (#11172)
98f1ab9 is described below

commit 98f1ab94fba4c2ebb2847b41573a0a40a85b781c
Author: Matthias J. Sax <matthias@confluent.io>
AuthorDate: Thu Aug 5 11:33:20 2021 -0700

    MINOR: update stream-stream join docs (#11172)
    
    Reviewers: Guozhang Wang <guozhang@confluent.io>
---
 docs/streams/developer-guide/dsl-api.html | 75 ++++++++++++++++++++++++++++---
 1 file changed, 68 insertions(+), 7 deletions(-)

diff --git a/docs/streams/developer-guide/dsl-api.html b/docs/streams/developer-guide/dsl-api.html
index 0800536..ba3b3c8 100644
--- a/docs/streams/developer-guide/dsl-api.html
+++ b/docs/streams/developer-guide/dsl-api.html
@@ -1891,7 +1891,8 @@ KStream&lt;String, String&gt; joined = left.leftJoin(right,
                                                 </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>
+                                            this explains the row with timestamp=60 and timestampe=80
in the table below, which lists <code class="docutils literal"><span class="pre">[E,</span>
<span class="pre">null]</span></code> and <code class="docutils literal"><span
class="pre">[F,</span> <span class="pre">null]</span></code>in
the LEFT JOIN column.
+                                            Note that these left results are emitted after
the specified grace period passed. <strong>Caution:</strong> using the deprecated
<code class="docutils literal"><span class="pre">JoinWindows.of(...).grace(...)</span></code>
API might result in eagerly emitted spurious left results.</p>
                                         </li>
                                     </ul>
                                     <p class="last">See the semantics overview at the
bottom of this section for a detailed description.</p>
@@ -1951,8 +1952,9 @@ KStream&lt;String, String&gt; joined = left.outerJoin(right,
                                                 </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>
+                                            <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=60, timestamp=80, and timestamp=100 in
the table below, which lists <code class="docutils literal"><span class="pre">[E,</span>
<span class="pre">null]</span></code>,
+                                            <code class="docutils literal"><span
class="pre">[F,</span> <span class="pre">null]</span></code>, and
<code class="docutils literal"><span class="pre">[null,</span> <span
class="pre">f]</span></code> in the OUTER JOIN column.
+                                            Note that these left and right results are emitted
after the specified grace period passed. <strong>Caution:</strong> using the deprecated
<code class="docutils literal"><span class="pre">JoinWindows.of(...).grace(...)</span></code>
API might result in eagerly emitted spurious left/right results.</p>
                                         </li>
                                     </ul>
                                     <p class="last">See the semantics overview at the
bottom of this section for a detailed description.</p>
@@ -1962,8 +1964,11 @@ KStream&lt;String, String&gt; joined = left.outerJoin(right,
                         </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
+                            To improve the readability of the table, assume that (1) all
records have the same key (and thus the key in the table is omitted), and (2) all records
are processed in timestamp order.
+                            We assume a join window size of 10 seconds with a grace period
of 5 seconds.</p>
+                        <p><b>Note:</b> If you use the old and now deprecated
API to specify the grace period, i.e., <code class="docutils literal"><span class="pre">JoinWindows.of(...).grace(...)</span></code>,
+                            left/outer join results are emitted eagerly, and the observed
result might differ from the result shown below.</p>
+                        <p>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
                             <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>
@@ -1996,8 +2001,8 @@ KStream&lt;String, String&gt; joined = left.outerJoin(right,
                                 <td>A</td>
                                 <td>&nbsp;</td>
                                 <td>&nbsp;</td>
-                                <td>[A, null]</td>
-                                <td>[A, null]</td>
+                                <td> </td>
+                                <td> </td>
                             </tr>
                             <tr class="row-odd"><td>4</td>
                                 <td>&nbsp;</td>
@@ -2083,6 +2088,62 @@ KStream&lt;String, String&gt; joined = left.outerJoin(right,
                                 <td>[D, a], [D, b], [D, c], [D, d]</td>
                                 <td>[D, a], [D, b], [D, c], [D, d]</td>
                             </tr>
+                            <tr class="row-odd"><td> ... </td>
+                                <td> </td>
+                                <td> </td>
+                                <td> </td>
+                                <td> </td>
+                                <td> </td>
+                            </tr>
+                            <tr class="even-row"><td>40</td>
+                                <td>E</td>
+                                <td> </td>
+                                <td> </td>
+                                <td> </td>
+                                <td> </td>
+                            </tr>
+                            <tr class="row-odd"><td> ... </td>
+                                <td> </td>
+                                <td> </td>
+                                <td> </td>
+                                <td> </td>
+                                <td> </td>
+                            </tr>
+                            <tr class="even-row"><td>60</td>
+                                <td>F</td>
+                                <td> </td>
+                                <td> </td>
+                                <td>[E, null]</td>
+                                <td>[E, null]</td>
+                            </tr>
+                            <tr class="row-odd"><td> ... </td>
+                                <td> </td>
+                                <td> </td>
+                                <td> </td>
+                                <td> </td>
+                                <td> </td>
+                            </tr>
+                            <tr class="even-row"><td>80</td>
+                                <td> </td>
+                                <td>f</td>
+                                <td> </td>
+                                <td>[F, null]</td>
+                                <td>[F, null]</td>
+                            </tr>
+                            <tr class="row-odd"><td> ... </td>
+                                <td> </td>
+                                <td> </td>
+                                <td> </td>
+                                <td> </td>
+                                <td> </td>
+                            </tr>
+                            <tr class="even-row"><td>100</td>
+                                <td>G</td>
+                                <td> </td>
+                                <td> </td>
+                                <td> </td>
+                                <td>[null, f]</td>
+                            </tr>
                             </tbody>
                         </table>
                     </div>

Mime
View raw message