mirror of https://github.com/apache/kafka.git
KAFKA-19668: update upgrade docs (#20484)
Docs change for KAFKA-19668 bug fix. Reviewers: Bill Bejeck <bill@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
This commit is contained in:
parent
ed6472bcf3
commit
a1ffe5b6d2
|
@ -3130,15 +3130,20 @@ grouped
|
|||
<code>Processor</code> (provided by a given <code>ProcessorSupplier</code>);
|
||||
</li>
|
||||
<li><code>KStream#processValues</code>: Process all records in a stream, one record at a time, by applying a
|
||||
<code>FixedKeyProcessor</code> (provided by a given <code>FixedKeyProcessorSupplier</code>);
|
||||
<code>FixedKeyProcessor</code> (provided by a given <code>FixedKeyProcessorSupplier</code>)
|
||||
[<b>CAUTION:</b> If you are deploying a new Kafka Streams application, and you are using the
|
||||
"merge repartition topics" optimization, you should enable the fix for
|
||||
<a href="https://issues.apache.org/jira/browse/KAFKA-19668">KAFKA-19668</a> to avoid compatibility
|
||||
issues for future upgrades to newer versions of Kafka Streams;
|
||||
For more details, see the <a href="#transformers-removal-and-migration-to-processors">migration guide</a> below];
|
||||
</li>
|
||||
<li><code>Processor</code>: A processor of key-value pair records;</li>
|
||||
<li><code>ContextualProcessor</code>: An abstract implementation of <code>Processor</code> that manages the
|
||||
<code>ProcessorContext</code> instance.
|
||||
<code>ProcessorContext</code> instance;
|
||||
</li>
|
||||
<li><code>FixedKeyProcessor</code>: A processor of key-value pair records where keys are immutable;</li>
|
||||
<li><code>ContextualFixedKeyProcessor</code>: An abstract implementation of <code>FixedKeyProcessor</code> that
|
||||
manages the <code>FixedKeyProcessorContext</code> instance.
|
||||
manages the <code>FixedKeyProcessorContext</code> instance;
|
||||
</li>
|
||||
<li><code>ProcessorSupplier</code>: A processor supplier that can create one or more <code>Processor</code>
|
||||
instances; and</li>
|
||||
|
@ -3456,6 +3461,25 @@ grouped
|
|||
</ul>
|
||||
<p>The Processor API now serves as a unified replacement for all these methods. It simplifies the API surface
|
||||
while maintaining support for both stateless and stateful operations.</p>
|
||||
|
||||
<p><b>CAUTION:</b> If you are using <code>KStream.transformValues()</code> and you have the "merge repartition topics"
|
||||
optimization enabled, rewriting your program to <code>KStream.processValues()</code> might not be safe due to
|
||||
<a href="https://issues.apache.org/jira/browse/KAFKA-19668">KAFKA-19668</a>. For this case, you should not upgrade
|
||||
to Kafka Streams 4.0.0 or 4.1.0, but use Kafka Streams 4.0.1 instead, which contains a fix.
|
||||
Note, that the fix is not enabled by default for backward compatibility reasons, and you would need to
|
||||
enable the fix by setting config <code>__enable.process.processValue.fix__ = true</code> and pass it
|
||||
into <code>StreamsBuilder()</code> constructor.</p>
|
||||
<pre class="line-numbers"><code class="language-java">final Properties properties = new Properties();
|
||||
properties.put(StreamsConfig.APPLICATION_ID_CONFIG, ...);
|
||||
properties.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ...);
|
||||
properties.put(TopologyConfig.InternalConfig.ENABLE_PROCESS_PROCESSVALUE_FIX, true);
|
||||
|
||||
final StreamsBuilder builder = new StreamsBuilder(new TopologyConfig(new StreamsConfig(properties)));</code></pre>
|
||||
|
||||
<p>It is recommended, that you compare the output of <code>Topology.describe()</code> for the old and new topology,
|
||||
to verify if the rewrite to <code>processValues()</code> is correct, and that it does not introduce any incompatibilities.
|
||||
You should also test the upgrade in a non-production environment.</p>
|
||||
|
||||
<h4>Migration Examples</h4>
|
||||
<p>To migrate from the deprecated <code>transform</code>, <code>transformValues</code>, <code>flatTransform</code>, and
|
||||
<code>flatTransformValues</code> methods to the Processor API (PAPI) in Kafka Streams, let's resume the
|
||||
|
|
|
@ -262,7 +262,15 @@
|
|||
<ul>
|
||||
<li><a href="https://issues.apache.org/jira/browse/KAFKA-12829">Old processor APIs</a></li>
|
||||
<li><a href="https://issues.apache.org/jira/browse/KAFKA-12823">KStream#through() in both Java and Scala</a></li>
|
||||
<li><a href="https://issues.apache.org/jira/browse/KAFKA-16339">"transformer" methods and classes in both Java and Scala</a></li>
|
||||
<li>
|
||||
<a href="https://issues.apache.org/jira/browse/KAFKA-16339">"transformer" methods and classes in both Java and Scala</a>
|
||||
<ul>
|
||||
<li>migrating from <code>KStreams#transformValues()</code> to <code>KStreams.processValues()</code> might not be safe
|
||||
due to <a href="https://issues.apache.org/jira/browse/KAFKA-19668">KAFKA-19668</a>.
|
||||
Please refer to the <a href="/{{version}}/documentation/streams/developer-guide/dsl-api.html#transformers-removal-and-migration-to-processors">migration guide</a> for more details.
|
||||
</li>
|
||||
</ul>
|
||||
</li>
|
||||
<li><a href="https://issues.apache.org/jira/browse/KAFKA-12824">kstream.KStream#branch in both Java and Scala</a></li>
|
||||
<li><a href="https://issues.apache.org/jira/browse/KAFKA-16332">builder methods for Time/Session/Join/SlidingWindows</a></li>
|
||||
<li><a href="https://issues.apache.org/jira/browse/KAFKA-12827">KafkaStreams#setUncaughtExceptionHandler()</a></li>
|
||||
|
@ -639,6 +647,11 @@
|
|||
<code>FixedKeyProcessorContext</code>, and <code>ContextualFixedKeyProcessor</code> are introduced to guard against
|
||||
disallowed key modification inside <code>processValues()</code>. Furthermore, <code>ProcessingContext</code> is
|
||||
added for a better interface hierarchy.
|
||||
<b>CAUTION:</b> The newly added <code>KStream.processValues()</code> method introduced a regression bug
|
||||
(<a href="https://issues.apache.org/jira/browse/KAFKA-19668">KAFKA-19668</a>).
|
||||
If you have "merge repartition topics" optimization enabled, it is not safe to migrate from <code>transformValues()</code>
|
||||
to <code>processValues()</code> in 3.3.0 release. The bug is only fixed with Kafka Streams 4.0.1, 4.1.1, and 4.2.0.
|
||||
For more details, please refer to the <a href="/{{version}}/documentation/streams/developer-guide/dsl-api.html#transformers-removal-and-migration-to-processors">migration guide</a>.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
|
|
|
@ -130,6 +130,11 @@
|
|||
The filename for rotated <code>state-change.log</code> files has been updated from <code>stage-change.log.[date]</code> to <code>state-change.log.[date]</code> in the log4j2.yaml configuration file.
|
||||
See <a href="https://issues.apache.org/jira/browse/KAFKA-19576">KAFKA-19576</a> for details.
|
||||
</li>
|
||||
<li>
|
||||
Kafka Streams include a critical fix to upgrade from <code>KStreams#transformValues()</code> (remove with 4.0.0 release)
|
||||
to <code>KStreams#processValues()</code>.
|
||||
For more details, see the <a href="/{{version}}/documentation/streams/developer-guide/dsl-api.html#transformers-removal-and-migration-to-processors">migration guide</a>.
|
||||
</li>
|
||||
</ul>
|
||||
<h5><a id="upgrade_servers_400_notable" href="#upgrade_servers_400_notable">Notable changes in 4.0.0</a></h5>
|
||||
<ul>
|
||||
|
@ -418,6 +423,11 @@
|
|||
<li>
|
||||
For a full list of changes, see <a href="https://issues.apache.org/jira/browse/KAFKA-12822">KAFKA-12822</a>.
|
||||
</li>
|
||||
<li>
|
||||
If you are using <code>KStream#transformValues()</code> which was removed with Apache Kafka 4.0.0 release,
|
||||
and you need to rewrite your program to use <code>KStreams#processValues()</code> instead,
|
||||
pay close attention to the <a href="/{{version}}/documentation/streams/developer-guide/dsl-api.html#transformers-removal-and-migration-to-processors">migration guide</a>.
|
||||
</li>
|
||||
</ul>
|
||||
</li>
|
||||
</ul>
|
||||
|
|
Loading…
Reference in New Issue