mirror of https://github.com/apache/kafka.git
KAFKA-14834: [13/N] Docs updates for versioned store semantics (#13622)
Reviewers: Matthias J. Sax <matthias@confluent.io>
This commit is contained in:
parent
a7d0b3f753
commit
ced1f62c1b
|
@ -309,7 +309,7 @@
|
|||
<h3 class="anchor-heading"><a id="streams_out_of_ordering" class="anchor-link"></a><a href="#streams_out_of_ordering">Out-of-Order Handling</a></h3>
|
||||
|
||||
<p>
|
||||
Besides the guarantee that each record will be processed exactly-once, another issue that many stream processing application will face is how to
|
||||
Besides the guarantee that each record will be processed exactly-once, another issue that many stream processing applications will face is how to
|
||||
handle <a href="https://dl.acm.org/citation.cfm?id=3242155">out-of-order data</a> that may impact their business logic. In Kafka Streams, there are two causes that could potentially
|
||||
result in out-of-order data arrivals with respect to their timestamps:
|
||||
</p>
|
||||
|
@ -328,13 +328,16 @@
|
|||
for stateful operations such as aggregations and joins, however, out-of-order data could cause the processing logic to be incorrect. If users want to handle such out-of-order data, generally they need to allow their applications
|
||||
to wait for longer time while bookkeeping their states during the wait time, i.e. making trade-off decisions between latency, cost, and correctness.
|
||||
In Kafka Streams specifically, users can configure their window operators for windowed aggregations to achieve such trade-offs (details can be found in <a href="/{{version}}/documentation/streams/developer-guide"><b>Developer Guide</b></a>).
|
||||
As for Joins, users have to be aware that some of the out-of-order data cannot be handled by increasing on latency and cost in Streams yet:
|
||||
As for Joins, users may use <a href="/{{version}}/documentation/streams/developer-guide/dsl-api.html#versioned-state-stores">versioned state stores</a> to address concerns with out-of-order data, but out-of-order data will not be handled by default:
|
||||
</p>
|
||||
|
||||
<ul>
|
||||
<li> For Stream-Stream joins, all three types (inner, outer, left) handle out-of-order records correctly, but the resulted stream may contain unnecessary leftRecord-null for left joins, and leftRecord-null or null-rightRecord for outer joins. </li>
|
||||
<li> For Stream-Table joins, out-of-order records are not handled (i.e., Streams applications don't check for out-of-order records and just process all records in offset order), and hence it may produce unpredictable results. </li>
|
||||
<li> For Table-Table joins, out-of-order records are not handled (i.e., Streams applications don't check for out-of-order records and just process all records in offset order). However, the join result is a changelog stream and hence will be eventually consistent. </li>
|
||||
<li> For Stream-Stream joins, all three types (inner, outer, left) handle out-of-order records correctly.</li>
|
||||
<li> For Stream-Table joins, if not using versioned stores, then out-of-order records are not handled (i.e., Streams applications don't check for out-of-order records and just process all records in offset order),
|
||||
and hence it may produce unpredictable results. With versioned stores, stream-side out-of-order data will be properly handled by performing a timestamp-based lookup in the table. Table-side out-of-order data is still not handled.</li>
|
||||
<li> For Table-Table joins, if not using versioned stores, then out-of-order records are not handled (i.e., Streams applications don't check for out-of-order records and just process all records in offset order).
|
||||
However, the join result is a changelog stream and hence will be eventually consistent. With versioned stores, table-table join semantics change from offset-based semantics to
|
||||
<a href="/{{version}}/documentation/streams/developer-guide/dsl-api.html#versioned-state-stores">timestamp-based semantics</a> and out-of-order records are handled accordingly.</li>
|
||||
</ul>
|
||||
|
||||
<div class="pagination">
|
||||
|
|
|
@ -69,6 +69,7 @@
|
|||
</li>
|
||||
<li><a class="reference internal" href="#naming-a-streams-app" id="id33">Naming Operators in a Streams DSL application</a></li>
|
||||
<li><a class="reference internal" href="#controlling-emit-rate" id="id32">Controlling KTable update rate</a></li>
|
||||
<li><a class="reference internal" href="#versioned-state-stores" id="id36">Using timestamp-based semantics for table processors</a></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>
|
||||
|
@ -2217,6 +2218,9 @@ KTable<String, String> joined = left.join(right,
|
|||
<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>
|
||||
<li>When joining <a class="reference internal" href="#versioned-state-stores"><span class="std std-ref">versioned tables</span></a>,
|
||||
out-of-order input records, i.e., those for which another record from the same table, with the same key and a larger timestamp,
|
||||
has already been processed, are ignored and do not trigger the join.</li>
|
||||
</ul>
|
||||
</div></blockquote>
|
||||
</li>
|
||||
|
@ -2261,6 +2265,9 @@ KTable<String, String> joined = left.leftJoin(right,
|
|||
<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. Right-tombstones trigger the join,
|
||||
but left-tombstones don't: when an input tombstone is received, 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>
|
||||
<li>When joining <a class="reference internal" href="#versioned-state-stores"><span class="std std-ref">versioned tables</span></a>,
|
||||
out-of-order input records, i.e., those for which another record from the same table, with the same key and a larger timestamp,
|
||||
has already been processed, are ignored and do not trigger the join.</li>
|
||||
</ul>
|
||||
</div></blockquote>
|
||||
</li>
|
||||
|
@ -2308,6 +2315,9 @@ KTable<String, String> joined = left.outerJoin(right,
|
|||
<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 may trigger joins,
|
||||
depending on the content in the left and right tables. When an input tombstone is received, 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>
|
||||
<li>When joining <a class="reference internal" href="#versioned-state-stores"><span class="std std-ref">versioned tables</span></a>,
|
||||
out-of-order input records, i.e., those for which another record from the same table, with the same key and a larger timestamp,
|
||||
has already been processed, are ignored and do not trigger the join.</li>
|
||||
</ul>
|
||||
</div></blockquote>
|
||||
</li>
|
||||
|
@ -2547,6 +2557,9 @@ Function<Long, Long> foreignKeyExtractor = (x) -> x;
|
|||
result KTable if required (i.e. only if the
|
||||
corresponding key actually exists already in
|
||||
the join result KTable).</li>
|
||||
<li>When joining <a class="reference internal" href="#versioned-state-stores"><span class="std std-ref">versioned tables</span></a>,
|
||||
out-of-order input records, i.e., those for which another record from the same table, with the same key and a larger timestamp,
|
||||
has already been processed, are ignored and do not trigger the join.</li>
|
||||
</ul>
|
||||
</div>
|
||||
</blockquote>
|
||||
|
@ -2607,6 +2620,9 @@ Function<Long, Long> foreignKeyExtractor = (x) -> x;
|
|||
result KTable if required (i.e. only if the
|
||||
corresponding key actually exists already in
|
||||
the join result KTable).</li>
|
||||
<li>When joining <a class="reference internal" href="#versioned-state-stores"><span class="std std-ref">versioned tables</span></a>,
|
||||
out-of-order input records, i.e., those for which another record from the same table, with the same key and a larger timestamp,
|
||||
has already been processed, are ignored and do not trigger the join.</li>
|
||||
</ul>
|
||||
</div>
|
||||
</blockquote>
|
||||
|
@ -2830,6 +2846,9 @@ KStream<String, String> joined = left.join(right,
|
|||
</ul>
|
||||
</div></blockquote>
|
||||
</li>
|
||||
<li>When the table is <a class="reference internal" href="#versioned-state-stores"><span class="std std-ref">versioned</span></a>,
|
||||
the table record to join with is determined by performing a timestamped lookup, i.e., the table record which is joined will be the latest-by-timestamp record with timestamp
|
||||
less than or equal to the stream record timestamp. If the stream record timestamp is older than the table's history retention, then the record is dropped.</li>
|
||||
</ul>
|
||||
<p class="last">See the semantics overview at the bottom of this section for a detailed description.</p>
|
||||
</td>
|
||||
|
@ -2884,6 +2903,9 @@ KStream<String, String> joined = left.leftJoin(right,
|
|||
<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>
|
||||
<li>When the table is <a class="reference internal" href="#versioned-state-stores"><span class="std std-ref">versioned</span></a>,
|
||||
the table record to join with is determined by performing a timestamped lookup, i.e., the table record which is joined will be the latest-by-timestamp record with timestamp
|
||||
less than or equal to the stream record timestamp. If the stream record timestamp is older than the table's history retention, then the record that is joined will be <code class="docutils literal"><span class="pre">null</span></code>.</li>
|
||||
</ul>
|
||||
<p class="last">See the semantics overview at the bottom of this section for a detailed description.</p>
|
||||
</td>
|
||||
|
@ -3609,6 +3631,52 @@ groupedTable
|
|||
and <a href="https://cwiki.apache.org/confluence/x/sQU0BQ" title="KIP-328">KIP-328</a>.
|
||||
</p>
|
||||
</div>
|
||||
<div class="section" id="versioned-state-stores">
|
||||
<h2><a class="headerlink" href="#versioned-state-stores" title="Permalink to this headline">Using timestamp-based semantics for table processors</a></h2>
|
||||
<p>By default, tables in Kafka Streams use offset-based semantics. When multiple records arrive for the same key, the one with the largest record offset
|
||||
is considered the latest record for the key, and is the record that appears in aggregation and join results computed on the table. This is true even
|
||||
in the event of <a href="/{{version}}/documentation/streams/core-concepts.html#streams_out_of_ordering">out-of-order data</a>. The record with the
|
||||
largest offset is considered to be the latest record for the key, even if this record does not have the largest timestamp.</p>
|
||||
<p>An alternative to offset-based semantics is timestamp-based semantics. With timestamp-based semantics, the record with the largest timestamp is
|
||||
considered the latest record, even if there is another record with a larger offset (and smaller timestamp). If there is no out-of-order data (per key),
|
||||
then offset-based semantics and timestamp-based semantics are equivalent; the difference only appears when there is out-of-order data.</p>
|
||||
<p>Starting with Kafka Streams 3.5, Kafka Streams supports timestamp-based semantics through the use of
|
||||
<a href="/{{version}}/documentation/streams/developer-guide/processor-api.html#versioned-state-stores">versioned state stores</a>.
|
||||
When a table is materialized with a versioned state store, it is a versioned table and will result in different processor semantics in the presence of
|
||||
out-of-order data.</p>
|
||||
<ul class="simple">
|
||||
<li>When performing a stream-table join, stream-side records will join with the latest-by-timestamp table record which has a timestamp less than or equal to
|
||||
the stream record's timestamp. This is in contrast to joining a stream to an unversioned table, in which case the latest-by-offset table record will
|
||||
be joined, even if the stream-side record is out-of-order and has a lower timestamp.</li>
|
||||
<li>Aggregations computed on the table will include the latest-by-timestamp record for each key, instead of the latest-by-offset record. Out-of-order
|
||||
updates (per key) will not trigger a new aggregation result. This is true for <code class="docutils literal"><span class="pre">count</span></code>
|
||||
and <code class="docutils literal"><span class="pre">reduce</span></code> operations as well, in addition to
|
||||
<code class="docutils literal"><span class="pre">aggregate</span></code> operations.</li>
|
||||
<li>Table joins will use the latest-by-timestamp record for each key, instead of the latest-by-offset record. Out-of-order updates (per key) will not
|
||||
trigger a new join result. This is true for both primary-key table-table joins and also foreign-key table-table joins. If a
|
||||
versioned table is joined with an unversioned table, the result will be the join of the latest-by-timestamp record from the versioned table with
|
||||
the latest-by-offset record from the unversioned table.</li>
|
||||
<li>Table filter operations will no longer suppress consecutive tombstones, so users may observe more <code class="docutils literal"><span class="pre">null</span></code>
|
||||
records downstream of the filter than compared to when filtering an unversioned table. This is done in order to preserve a complete version history downstream,
|
||||
in the event of out-of-order data.</li>
|
||||
<li><code class="docutils literal"><span class="pre">suppress</span></code> operations are not allowed on versioned tables, as this would collapse the version history
|
||||
and lead to undefined behavior.</li>
|
||||
</ul>
|
||||
<p>Once a table is materialized with a versioned store, downstream tables are also considered versioned until any of the following occurs:</p>
|
||||
<ul class="simple">
|
||||
<li>A downstream table is explicitly materialized, either with an unversioned store supplier or with no store supplier (all stores are unversioned by default, including the default store supplier)</li>
|
||||
<li>Any stateful transformation occurs, including aggregations and joins</li>
|
||||
<li>A table is converted to a stream and back.</li>
|
||||
</ul>
|
||||
<p>The results of certain processors should not be materialized with versioned stores, as these processors do not produce a complete older version history,
|
||||
and therefore materialization as a versioned table would lead to unpredictable results:</p>
|
||||
<ul class="simple">
|
||||
<li>Aggregate processors, for both table and stream aggregations. This includes <code class="docutils literal"><span class="pre">aggregate</span></code>,
|
||||
<code class="docutils literal"><span class="pre">count</span></code> and <code class="docutils literal"><span class="pre">reduce</span></code> operations.</li>
|
||||
<li>Table-table join processors, including both primary-key and foreign-key joins.</li>
|
||||
</ul>
|
||||
<p>For more on versioned stores and how to start using them in your application, see <a href="/{{version}}/documentation/streams/developer-guide/processor-api.html#versioned-state-stores">here</a>.</p>
|
||||
</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
|
||||
|
|
|
@ -437,7 +437,7 @@ StoreBuilder<KeyValueStore<String, Long>> countStoreSupplier = Store
|
|||
advised as performance is expected to be worse than when using non-versioned
|
||||
stores.</p>
|
||||
<p>Versioned stores do not support caching or interactive queries at this time.
|
||||
Also, window stores may not be versioned.</p>
|
||||
Also, window stores and global tables may not be versioned.</p>
|
||||
<b>Upgrade note:</b> Versioned state stores are opt-in only; no automatic
|
||||
upgrades from non-versioned to versioned stores will take place.
|
||||
<p>Upgrades are supported from persistent, non-versioned key-value stores
|
||||
|
|
|
@ -134,14 +134,15 @@
|
|||
<h3><a id="streams_api_changes_350" href="#streams_api_changes_350">Streams API changes in 3.5.0</a></h3>
|
||||
<p>
|
||||
A new state store type, versioned key-value stores, was introduced in
|
||||
<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-889%3A+Versioned+State+Stores">KIP-889</a>.
|
||||
<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-889%3A+Versioned+State+Stores">KIP-889</a> and
|
||||
<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-914%3A+DSL+Processor+Semantics+for+Versioned+Stores">KIP-914</a>.
|
||||
Rather than storing a single record version (value and timestamp) per key,
|
||||
versioned state stores may store multiple record versions per key. This
|
||||
allows versioned state stores to support timestamped retrieval operations
|
||||
to return the latest record (per key) as of a specified timestamp.
|
||||
For more information, including how to upgrade from a non-versioned key-value
|
||||
store to a versioned store in an existing application, see the
|
||||
<a href="/{{version}}/documentation/streams/developer-guide/processor-api.html#versioned-state-stores">Developer Guide</a> section.
|
||||
<a href="/{{version}}/documentation/streams/developer-guide/dsl-api.html#versioned-state-stores">Developer Guide</a>.
|
||||
Versioned key-value stores are opt-in only; existing applications will not be
|
||||
affected upon upgrading to 3.5 without explicit code changes.
|
||||
</p>
|
||||
|
|
|
@ -35,6 +35,7 @@
|
|||
for storing multiple record versions per key, thereby enabling timestamped retrieval
|
||||
operations to return the latest record (per key) as of a specified timestamp.
|
||||
See <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-889%3A+Versioned+State+Stores">KIP-889</a>
|
||||
and <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-914%3A+DSL+Processor+Semantics+for+Versioned+Stores">KIP-914</a>
|
||||
for more details.
|
||||
If the new store typed is used in the DSL, improved processing semantics are applied as described in
|
||||
<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-914%3A+DSL+Processor+Semantics+for+Versioned+Stores">KIP-914</a>.
|
||||
|
|
Loading…
Reference in New Issue