Skip to content

Commit c47c280

Browse files
authored
MINOR: update docs for KAFKA-19668 (#721)
1 parent 04502e9 commit c47c280

File tree

13 files changed

+139
-34
lines changed

13 files changed

+139
-34
lines changed

33/streams/developer-guide/dsl-api.html

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3496,6 +3496,9 @@ <h5><a class="toc-backref" href="#id34">KTable-KTable Foreign-Key
34963496
</tr>
34973497
</tbody>
34983498
</table>
3499+
3500+
<p><b>CAUTION:</b> If you are using "merge repartition topics" optimization, it is not recommended to use <code>KStream#processValues</code> to avoid compatibility issues for future upgrades to newer versions of Kafka Streams.
3501+
For more details, see the <a href="/40/documentation/streams/developer-guide/dsl-api.htm#transformers-removal-and-migration-to-processors">migration guide</a> in the Kafka Streams 4.0 docs.</p>
34993502
<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
35003503
email notification whenever a page view count reaches a predefined threshold.</p>
35013504
<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>

34/streams/developer-guide/dsl-api.html

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3496,6 +3496,9 @@ <h5><a class="toc-backref" href="#id34">KTable-KTable Foreign-Key
34963496
</tr>
34973497
</tbody>
34983498
</table>
3499+
3500+
<p><b>CAUTION:</b> If you are using "merge repartition topics" optimization, it is not recommended to use <code>KStream#processValues</code> to avoid compatibility issues for future upgrades to newer versions of Kafka Streams.
3501+
For more details, see the <a href="/40/documentation/streams/developer-guide/dsl-api.htm#transformers-removal-and-migration-to-processors">migration guide</a> in the Kafka Streams 4.0 docs.</p>
34993502
<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
35003503
email notification whenever a page view count reaches a predefined threshold.</p>
35013504
<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>

35/streams/developer-guide/dsl-api.html

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3524,6 +3524,9 @@ <h5><a class="toc-backref" href="#id34">KTable-KTable Foreign-Key
35243524
</tr>
35253525
</tbody>
35263526
</table>
3527+
3528+
<p><b>CAUTION:</b> If you are using "merge repartition topics" optimization, it is not recommended to use <code>KStream#processValues</code> to avoid compatibility issues for future upgrades to newer versions of Kafka Streams.
3529+
For more details, see the <a href="/40/documentation/streams/developer-guide/dsl-api.htm#transformers-removal-and-migration-to-processors">migration guide</a> in the Kafka Streams 4.0 docs.</p>
35273530
<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
35283531
email notification whenever a page view count reaches a predefined threshold.</p>
35293532
<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>

36/streams/developer-guide/dsl-api.html

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3540,6 +3540,9 @@ <h5><a class="toc-backref" href="#id34">KTable-KTable Foreign-Key
35403540
</tr>
35413541
</tbody>
35423542
</table>
3543+
3544+
<p><b>CAUTION:</b> If you are using "merge repartition topics" optimization, it is not recommended to use <code>KStream#processValues</code> to avoid compatibility issues for future upgrades to newer versions of Kafka Streams.
3545+
For more details, see the <a href="/40/documentation/streams/developer-guide/dsl-api.htm#transformers-removal-and-migration-to-processors">migration guide</a> in the Kafka Streams 4.0 docs.</p>
35433546
<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
35443547
email notification whenever a page view count reaches a predefined threshold.</p>
35453548
<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>

37/streams/developer-guide/dsl-api.html

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3536,6 +3536,9 @@ <h5><a class="toc-backref" href="#id34">KTable-KTable Foreign-Key
35363536
</tr>
35373537
</tbody>
35383538
</table>
3539+
3540+
<p><b>CAUTION:</b> If you are using "merge repartition topics" optimization, it is not recommended to use <code>KStream#processValues</code> to avoid compatibility issues for future upgrades to newer versions of Kafka Streams.
3541+
For more details, see the <a href="/40/documentation/streams/developer-guide/dsl-api.htm#transformers-removal-and-migration-to-processors">migration guide</a> in the Kafka Streams 4.0 docs.</p>
35393542
<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
35403543
email notification whenever a page view count reaches a predefined threshold.</p>
35413544
<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>

38/streams/developer-guide/dsl-api.html

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3180,6 +3180,9 @@ <h5><a class="toc-backref" href="#id34">KTable-KTable Foreign-Key
31803180
</tr>
31813181
</tbody>
31823182
</table>
3183+
3184+
<p><b>CAUTION:</b> If you are using "merge repartition topics" optimization, it is not recommended to use <code>KStream#processValues</code> to avoid compatibility issues for future upgrades to newer versions of Kafka Streams.
3185+
For more details, see the <a href="/40/documentation/streams/developer-guide/dsl-api.htm#transformers-removal-and-migration-to-processors">migration guide</a> in the Kafka Streams 4.0 docs.</p>
31833186
<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
31843187
email notification whenever a page view count reaches a predefined threshold.</p>
31853188
<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>

39/streams/developer-guide/dsl-api.html

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3180,6 +3180,9 @@ <h5><a class="toc-backref" href="#id34">KTable-KTable Foreign-Key
31803180
</tr>
31813181
</tbody>
31823182
</table>
3183+
3184+
<p><b>CAUTION:</b> If you are using "merge repartition topics" optimization, it is not recommended to use <code>KStream#processValues</code> to avoid compatibility issues for future upgrades to newer versions of Kafka Streams.
3185+
For more details, see the <a href="/40/documentation/streams/developer-guide/dsl-api.htm#transformers-removal-and-migration-to-processors">migration guide</a> in the Kafka Streams 4.0 docs.</p>
31833186
<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
31843187
email notification whenever a page view count reaches a predefined threshold.</p>
31853188
<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>

40/streams/developer-guide/dsl-api.html

Lines changed: 27 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -3130,15 +3130,20 @@ <h4>Operations and concepts</h4>
31303130
<code>Processor</code> (provided by a given <code>ProcessorSupplier</code>);
31313131
</li>
31323132
<li><code>KStream#processValues</code>: Process all records in a stream, one record at a time, by applying a
3133-
<code>FixedKeyProcessor</code> (provided by a given <code>FixedKeyProcessorSupplier</code>);
3133+
<code>FixedKeyProcessor</code> (provided by a given <code>FixedKeyProcessorSupplier</code>)
3134+
[<b>CAUTION:</b> If you are deploying a new Kafka Streams application, and you are using the
3135+
"merge repartition topics" optimization, you should enable the fix for
3136+
<a href="https://issues.apache.org/jira/browse/KAFKA-19668">KAFKA-19668</a> to avoid compatibility
3137+
issues for future upgrades to newer versions of Kafka Streams;
3138+
For more details, see the <a href="#transformers-removal-and-migration-to-processors">migration guide</a> below];
31343139
</li>
31353140
<li><code>Processor</code>: A processor of key-value pair records;</li>
31363141
<li><code>ContextualProcessor</code>: An abstract implementation of <code>Processor</code> that manages the
3137-
<code>ProcessorContext</code> instance.
3142+
<code>ProcessorContext</code> instance;
31383143
</li>
31393144
<li><code>FixedKeyProcessor</code>: A processor of key-value pair records where keys are immutable;</li>
31403145
<li><code>ContextualFixedKeyProcessor</code>: An abstract implementation of <code>FixedKeyProcessor</code> that
3141-
manages the <code>FixedKeyProcessorContext</code> instance.
3146+
manages the <code>FixedKeyProcessorContext</code> instance;
31423147
</li>
31433148
<li><code>ProcessorSupplier</code>: A processor supplier that can create one or more <code>Processor</code>
31443149
instances; and</li>
@@ -3456,6 +3461,25 @@ <h3>
34563461
</ul>
34573462
<p>The Processor API now serves as a unified replacement for all these methods. It simplifies the API surface
34583463
while maintaining support for both stateless and stateful operations.</p>
3464+
3465+
<p><b>CAUTION:</b> If you are using <code>KStream.transformValues()</code> and you have the "merge repartition topics"
3466+
optimization enabled, rewriting your program to <code>KStream.processValues()</code> might not be safe due to
3467+
<a href="https://issues.apache.org/jira/browse/KAFKA-19668">KAFKA-19668</a>. For this case, you should not upgrade
3468+
to Kafka Streams 4.0.0 or 4.1.0, but use Kafka Streams 4.0.1 instead, which contains a fix.
3469+
Note, that the fix is not enabled by default for backward compatibility reasons, and you would need to
3470+
enable the fix by setting config <code>__enable.process.processValue.fix__ = true</code> and pass it
3471+
into <code>StreamsBuilder()</code> constructor.</p>
3472+
<pre class="line-numbers"><code class="language-java">final Properties properties = new Properties();
3473+
properties.put(StreamsConfig.APPLICATION_ID_CONFIG, ...);
3474+
properties.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ...);
3475+
properties.put(TopologyConfig.InternalConfig.ENABLE_PROCESS_PROCESSVALUE_FIX, true);
3476+
3477+
final StreamsBuilder builder = new StreamsBuilder(new TopologyConfig(new StreamsConfig(properties)));</code></pre>
3478+
3479+
<p>It is recommended, that you compare the output of <code>Topology.describe()</code> for the old and new topology,
3480+
to verify if the rewrite to <code>processValues()</code> is correct, and that it does not introduce any incompatibilities.
3481+
You should also test the upgrade in a non-production environment.</p>
3482+
34593483
<h4>Migration Examples</h4>
34603484
<p>To migrate from the deprecated <code>transform</code>, <code>transformValues</code>, <code>flatTransform</code>, and
34613485
<code>flatTransformValues</code> methods to the Processor API (PAPI) in Kafka Streams, let&#39;s resume the

0 commit comments

Comments
 (0)