Skip to content

Commit

Permalink
MINOR: Update 4.0.0 documentation for Kafka Streams (#18124)
Browse files Browse the repository at this point in the history
Reviewers: Matthias J. Sax <[email protected]>
  • Loading branch information
aliehsaeedii authored Jan 14, 2025
1 parent 9f95597 commit 2d22f52
Showing 1 changed file with 69 additions and 0 deletions.
69 changes: 69 additions & 0 deletions docs/streams/upgrade-guide.html
Original file line number Diff line number Diff line change
Expand Up @@ -141,6 +141,20 @@ <h3 class="anchor-heading"><a id="streams_notable_changes" class="anchor-link"><

<h3><a id="streams_api_changes_400" href="#streams_api_changes_400">Streams API changes in 4.0.0</a></h3>

<p>
In this release, eos-v1 (Exactly Once Semantics version 1) is no longer supported. To use eos-v2, brokers must be running version 2.5 or later.
Additionally, all deprecated methods, classes, APIs, and config parameters up to and including AK 3.5 release have been removed.
A few important ones are listed below. The full list can be found <a href="https://issues.apache.org/jira/browse/KAFKA-12822">here</a>.
<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-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>
</ul>
</p>

<p>
In this release the <code>ClientInstanceIds</code> instance stores the global consumer<code>Uuid</code> for the
<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability#KIP714:Clientmetricsandobservability-Clientidentificationandtheclientinstanceid">KIP-714</a>
Expand All @@ -161,6 +175,61 @@ <h3><a id="streams_api_changes_400" href="#streams_api_changes_400">Streams API
<code>TransformerSupplier</code>, <code>ValueTransformer</code>, and <code>ValueTransformerSupplier</code>.
</p>

<p>
Previously, the <code>ProductionExceptionHandler</code> was not invoked on a (retriable) <code>TimeoutException</code>. With Kafka Streams 4.0, the handler is called, and the default handler would return <code>RETRY</code> to not change existing behavior.
However, a custom handler can now decide to break the infinite retry loop by returning either <code>CONTINUE</code> or <code>FAIL</code> (<a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=311627309">KIP-1065</a>).
</p>

<p>
In this release, Kafka Streams metrics can be collected broker side via the KIP-714 broker-plugin.
For more detailed information, refer to <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1076%3A++Metrics+for+client+applications+KIP-714+extension">KIP-1076</a> document please.
</p>

<p>
<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1077%3A+Deprecate+%60ForeachProcessor%60+and+move+to+internal+package">KIP-1077</a>
deprecates the <code>ForeachProcessor</code> class.
This change is aimed at improving the organization and clarity of the Kafka Streams API by ensuring that internal classes are not exposed in public packages.
</p>

<p>
<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1078%3A+Remove+Leaking+Getter+Methods+in+Joined+Helper+Class">KIP-1078</a> deprecates the leaking getter methods in the <code>Joined</code> helper class.
These methods are deprecated without a replacement for future removal, as they don't add any value to Kafka Streams users.
</p>

<p>
To ensures better encapsulation and organization of configuration documentation within Kafka Streams,
<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1085%3A+Fix+leaking+*_DOC+variables+in+StreamsConfig">KIP-1085</a>
deprecate certain public doc description variables that are only used within the <code>StreamsConfig</code> or <code>TopologyConfig</code> classes.
Additionally, the unused variable <code>DUMMY_THREAD_INDEX</code> will also be deprecated.
</p>

<p>
Due to the removal of the already deprecated <code>#through</code> method in Kafka Streams, the <code>intermediateTopicsOption</code> of <code>StreamsResetter</code> tool in Apache Kafka is
not needed any more and therefore is deprecated (<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1087%3A+Removing+intermediateTopicsOption+from+StreamsResetter">KIP-1087</a>).
</p>

<p>
Since string metrics cannot be collected on the broker side (KIP-714), <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1091%3A+Improved+Kafka+Streams+operator+metrics">KIP-1091</a>
introduces numeric counterparts to allow proper broker-side metric collection for Kafka Streams applications.
These metrics will be available at the <code>INFO</code> recording level, and a thread-level metric with a String value will be available for users leveraging Java Management Extensions (<code>JMX</code>).
</p>

<p>
In order to reduce storage overhead and improve API usability, a new method in the Java and Scala APIs that accepts a BiFunction for foreign key extraction is introduced by
<a href="">KIP-1104</a>.
KIP-1104 allows foreign key extraction from both the key and value in KTable joins in Apache Kafka.
Previously, foreign key joins in KTables only allowed extraction from the value, which led to data duplication and potential inconsistencies.
This enhancement introduces a new method in the Java and Scala APIs that accepts a BiFunction for foreign key extraction, enabling more intuitive and efficient joins.
The existing methods will be deprecated but not removed, ensuring backward compatibility. This change aims to reduce storage overhead and improve API usability.
</p>

<p>
With introduction of <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1106%3A+Add+duration+based+offset+reset+option+for+consumer+clients">KIP-1106</a>,
the existing <code>Topology.AutoOffsetReset</code> is deprecated and replaced with a new class <code>org.apache.kafka.streams.AutoOffsetReset</code> to capture the reset strategies.
New methods will be added to the <code>org.apache.kafka.streams.Topology</code> and <code>org.apache.kafka.streams.kstream.Consumed</code> classes to support the new reset strategy.
These changes aim to provide more flexibility and efficiency in managing offsets, especially in scenarios involving long-term storage and infinite retention.
</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
Expand Down

0 comments on commit 2d22f52

Please sign in to comment.