Skip to content
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
30 changes: 27 additions & 3 deletions docs/streams/developer-guide/dsl-api.html
Original file line number Diff line number Diff line change
Expand Up @@ -3130,15 +3130,20 @@ <h4>Operations and concepts</h4>
<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>
Expand Down Expand Up @@ -3456,6 +3461,25 @@ <h3>
</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&#39;s resume the
Expand Down
35 changes: 24 additions & 11 deletions docs/streams/upgrade-guide.html
Original file line number Diff line number Diff line change
Expand Up @@ -148,7 +148,15 @@ <h3><a id="streams_api_changes_400" href="#streams_api_changes_400">Streams API
<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>
Expand Down Expand Up @@ -231,22 +239,22 @@ <h3><a id="streams_api_changes_400" href="#streams_api_changes_400">Streams API
</p>

<p>
You can now configure your topology with a <code>ProcessorWrapper</code>, which allows you to access and optionally wrap/replace
any processor in the topology by injecting an alternative <code>ProcessorSupplier</code> in its place. This can be used to peek
records and access the processor context even for DSL operators, for example to implement a logging or tracing framework, or to
aid in testing or debugging scenarios. You must implement the <code>ProcessorWrapper</code> interface and then pass the class
or class name into the configs via the new <code>StreamsConfig#PROCESSOR_WRAPPER_CLASS_CONFIG</code> config. NOTE: this config is
applied during the topology building phase, and therefore will not take effect unless the config is passed in when creating
the StreamsBuilder (DSL) or Topology(PAPI) objects. You MUST use the StreamsBuilder/Topology constructor overload that
accepts a TopologyConfig parameter for the <code>StreamsConfig#PROCESSOR_WRAPPER_CLASS_CONFIG</code> to be picked up.
See <a href="https://cwiki.apache.org/confluence/x/TZCMEw">KIP-1112</a> for more details.
You can now configure your topology with a <code>ProcessorWrapper</code>, which allows you to access and optionally wrap/replace
any processor in the topology by injecting an alternative <code>ProcessorSupplier</code> in its place. This can be used to peek
records and access the processor context even for DSL operators, for example to implement a logging or tracing framework, or to
aid in testing or debugging scenarios. You must implement the <code>ProcessorWrapper</code> interface and then pass the class
or class name into the configs via the new <code>StreamsConfig#PROCESSOR_WRAPPER_CLASS_CONFIG</code> config. NOTE: this config is
applied during the topology building phase, and therefore will not take effect unless the config is passed in when creating
the StreamsBuilder (DSL) or Topology(PAPI) objects. You MUST use the StreamsBuilder/Topology constructor overload that
accepts a TopologyConfig parameter for the <code>StreamsConfig#PROCESSOR_WRAPPER_CLASS_CONFIG</code> to be picked up.
See <a href="https://cwiki.apache.org/confluence/x/TZCMEw">KIP-1112</a> for more details.
</p>

<p>
Upgraded RocksDB dependency to version 9.7.3 (from 7.9.2). This upgrade incorporates various improvements and optimizations within RocksDB. However, it also introduces some API changes.
The <code>org.rocksdb.AccessHint</code> class, along with its associated methods, has been removed.
Several methods related to compressed block cache configuration in the <code>BlockBasedTableConfig</code> class have been removed, including <code>blockCacheCompressedNumShardBits</code>, <code>blockCacheCompressedSize</code>, and their corresponding setters. These functionalities are now consolidated under the <code>cache</code> option, and developers should configure their compressed block cache using the <code>setCache</code> method instead.
The <code>NO_FILE_CLOSES</code> field has been removed from the <code>org.rocksdb.TickerTypeenum</code> as a result the <code>number-open-files</code> metrics does not work as expected. Metric <code>number-open-files</code> returns constant -1 from now on until it will officially be removed.
The <code>NO_FILE_CLOSES</code> field has been removed from the <code>org.rocksdb.TickerTypeenum</code> as a result the <code>number-open-files</code> metrics does not work as expected. Metric <code>number-open-files</code> returns constant -1 from now on until it will officially be removed.
The <code>org.rocksdb.Options.setLogger()</code> method now accepts a <code>LoggerInterface</code> as a parameter instead of the previous <code>Logger</code>.
Some data types used in RocksDB's Java API have been modified. These changes, along with the removed class, field, and new methods, are primarily relevant to users implementing custom RocksDB configurations.
These changes are expected to be largely transparent to most Kafka Streams users. However, those employing advanced RocksDB customizations within their Streams applications, particularly through the <code>rocksdb.config.setter</code>, are advised to consult the detailed RocksDB 9.7.3 changelog to ensure a smooth transition and adapt their configurations as needed. Specifically, users leveraging the removed <code>AccessHint</code> class, the removed methods from the <code>BlockBasedTableConfig</code> class, the <code>NO_FILE_CLOSES</code> field from <code>TickerType</code>, or relying on the previous signature of <code>setLogger()</code> will need to update their implementations.
Expand Down Expand Up @@ -525,6 +533,11 @@ <h3><a id="streams_api_changes_330" href="#streams_api_changes_330">Streams API
<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>
Expand Down
18 changes: 14 additions & 4 deletions docs/upgrade.html
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,11 @@ <h5><a id="upgrade_servers_401_notable" href="#upgrade_servers_401_notable">Nota
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>
Expand Down Expand Up @@ -341,14 +346,19 @@ <h5><a id="upgrade_servers_400_notable" href="#upgrade_servers_400_notable">Nota
<li><a id="upgrade_400_notable_kafka_streams" href="#upgrade_400_notable_kafka_streams"><b>Kafka Streams</b></a>
<ul>
<li>
All public API, deprecated in Apache Kafka 3.6 or an earlier release, have been removed, with the exception of <code>JoinWindows.of()</code> and <code>JoinWindows#grace()</code>.
See <a href="https://issues.apache.org/jira/browse/KAFKA-17531">KAFKA-17531</a> for details.
All public API, deprecated in Apache Kafka 3.6 or an earlier release, have been removed, with the exception of <code>JoinWindows.of()</code> and <code>JoinWindows#grace()</code>.
See <a href="https://issues.apache.org/jira/browse/KAFKA-17531">KAFKA-17531</a> for details.
</li>
<li>
The most important changes are highlighted in the <a href="/{{version}}/documentation/streams/upgrade-guide.html#streams_api_changes_400">Kafka Streams upgrade guide</a>.
</li>
<li>
The most important changes are highlighted in the <a href="/{{version}}/documentation/streams/upgrade-guide.html#streams_api_changes_400">Kafka Streams upgrade guide</a>.
For a full list of changes, see <a href="https://issues.apache.org/jira/browse/KAFKA-12822">KAFKA-12822</a>.
</li>
<li>
For a full list of changes, see <a href="https://issues.apache.org/jira/browse/KAFKA-12822">KAFKA-12822</a>.
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>
Expand Down
Loading