Upgrading from any older version to {{fullDotVersion}} is possible: if upgrading from 3.4 or below, you will need to do two rolling bounces, where during the first rolling bounce phase you set the config <code>upgrade.from="older version"</code>
(possible values are <code>"2.4" - "3.4"</code>) and during the second you remove it. This is required to safely handle 2 changes. The first is a change in foreign-key join serialization format.
The second is a change in the serialization format for an internal repartition topic. For more details, please refer to <ahref="https://cwiki.apache.org/confluence/x/P5VbDg">KIP-904</a>:
<li> prepare your application instances for a rolling bounce and make sure that config <code>upgrade.from</code> is set to the version from which it is being upgrade.</li>
<li> prepare your newly deployed {{fullDotVersion}} application instances for a second round of rolling bounces; make sure to remove the value for config <code>upgrade.from</code></li>
<p> As an alternative, an offline upgrade is also possible. Upgrading from any versions as old as 0.11.0.x to {{fullDotVersion}} in offline mode require the following steps: </p>
<h3class="anchor-heading"><aid="streams_notable_changes"class="anchor-link"></a><ahref="#streams_notable_changes">Notable compatibility changes in past releases</a></h3>
Kafka Streams does not support running multiple instances of the same application as different processes on the same physical state directory. Starting in 2.8.0 (as well as 2.7.1 and 2.6.2),
this restriction will be enforced. If you wish to run more than one instance of Kafka Streams, you must configure them with different values for <code>state.dir</code>.
<p><b>Note:</b> Kafka Streams 4.1.0 contains a critical memory leak bug (<ahref="https://issues.apache.org/jira/browse/KAFKA-19748">KAFKA-19748</a>) that affects users of range scans and certain DSL operators (session windows, sliding windows, stream-stream joins, foreign-key joins). Users running Kafka Streams should consider upgrading directly to 4.1.1 when available.</p>
enables you to enforce explicit naming for all internal resources of the topology, including internal topics (e.g., changelog and repartition topics) and their associated state stores.
This ensures that every internal resource is named before the Kafka Streams application is deployed, which is essential for upgrading your topology.
You can enable this feature via <code>StreamsConfig</code> using the <code>StreamsConfig#ENSURE_EXPLICIT_INTERNAL_RESOURCE_NAMING_CONFIG</code> parameter.
When set to <code>true</code>, the application will refuse to start if any internal resource has an auto-generated name.
<ahref="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 <ahref="https://issues.apache.org/jira/browse/KAFKA-19668">KAFKA-19668</a>.
Please refer to the <ahref="/{{version}}/documentation/streams/developer-guide/dsl-api.html#transformers-removal-and-migration-to-processors">migration guide</a> for more details.
In this release two configs <code>default.deserialization.exception.handler</code> and <code>default.production.exception.handler</code> are deprecated, as they don't have any overwrites, which is described in
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> (<ahref="https://cwiki.apache.org/confluence/x/LQ6TEg">KIP-1065</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.
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
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
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.
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.
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>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.
enables you to provide a processing exception handler to manage exceptions during the processing of a record rather than throwing the exception all the way out of your streams application.
You can provide the configs via the <code>StreamsConfig</code> as <code>StreamsConfig#PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG</code>.
The specified handler must implement the <code>org.apache.kafka.streams.errors.ProcessingExceptionHandler</code> interface.
Kafka Streams now allows to customize the logging interval of stream-thread runtime summary, via the newly added config <code>log.summary.interval.ms</code>.
By default, the summary is logged every 2 minutes. More details can be found in
in which users can provide their customized implementation of the newly added <code>StandbyUpdateListener</code> interface to continuously monitor changes to standby tasks.
IQv2 supports <code>RangeQuery</code> that allows to specify unbounded, bounded, or half-open key-ranges, which return data in unordered (byte[]-lexicographical) order (per partition).
<ahref="https://cwiki.apache.org/confluence/x/eKCzDw">KIP-985</a> extends this functionality by adding <code>.withDescendingKeys()</code> and <code>.withAscendingKeys()</code>to allow user to receive data in descending or ascending order.
namely <code>TimestampedKeyQuery</code> and <code>TimestampedRangeQuery</code>. Both should be used to query a timestamped key-value store, to retrieve a <code>ValueAndTimestamp</code> result.
The existing <code>KeyQuery</code> and <code>RangeQuery</code> are changed to always return the value only for timestamped key-value stores.
The non-null key requirements for Kafka Streams join operators were relaxed as part of <ahref="https://cwiki.apache.org/confluence/x/f5CzDw">KIP-962</a>.
<li>left join KStream-KStream: no longer drop left records with null-key and call ValueJoiner with 'null' for right value.</li>
<li>outer join KStream-KStream: no longer drop left/right records with null-key and call ValueJoiner with 'null' for right/left value.</li>
<li>left-foreign-key join KTable-KTable: no longer drop left records with null-foreign-key returned by the ForeignKeyExtractor and call ValueJoiner with 'null' for right value.</li>
<li>left join KStream-KTable: no longer drop left records with null-key and call ValueJoiner with 'null' for right value.</li>
<li>left join KStream-GlobalTable: no longer drop records when KeyValueMapper returns 'null' and call ValueJoiner with 'null' for right value.</li>
</ul>
Stream-DSL users who want to keep the current behavior can prepend a .filter() operator to the aforementioned operators and filter accordingly.
The following snippets illustrate how to keep the old behavior.
A new configuration option <code>balance_subtopology</code> for <code>rack.aware.assignment.strategy</code> was introduced in 3.7 release.
For more information, including how it can be enabled and further configured, see the <ahref="/{{version}}/documentation/streams/developer-guide/config-streams.html#rack-aware-assignment-strategy"><b>Kafka Streams Developer Guide</b></a>.
Rack aware task assignment can be enabled for <code>StickyTaskAssignor</code> or <code>HighAvailabilityTaskAssignor</code> to compute task assignments which can minimize cross rack traffic under certain conditions.
For more information, including how it can be enabled and further configured, see the <ahref="/{{version}}/documentation/streams/developer-guide/config-streams.html#rack-aware-assignment-strategy"><b>Kafka Streams Developer Guide</b></a>.
</p>
<p>
IQv2 supports a <code>RangeQuery</code> that allows to specify unbounded, bounded, or half-open key-ranges. Users have to use <code>withUpperBound(K)</code>, <code>withLowerBound(K)</code>,
or <code>withNoBounds()</code> to specify half-open or unbounded ranges, but cannot use <code>withRange(K lower, K upper)</code> for the same.
For more information, including how it can be enabled and further configured, see the <ahref="/{{version}}/documentation/streams/developer-guide/config-streams.html#rack-aware-assignment-strategy"><b>Kafka Streams Developer Guide</b></a>.
improves the implementation of KTable aggregations. In general, an input KTable update triggers a result refinent for two rows;
however, prior to KIP-904, if both refinements happen to the same result row, two independent updates to the same row are applied, resulting in spurious itermediate results.
config <code>cache.max.bytes.buffering</code> in favor of the newly introduced config <code>statestore.cache.max.bytes</code>.
To improve monitoring, two new metrics <code>input-buffer-bytes-total</code> and <code>cache-size-bytes-total</code>
were added at the DEBUG level. Note, that the KIP is only partially implemented in the 3.4.0 release, and config
<code>input.buffer.max.bytes</code> is not available yet.
</p>
<p>
<ahref="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211883356">KIP-873</a> enables you to multicast
result records to multiple partition of downstream sink topics and adds functionality for choosing to drop result records without sending.
The <code>Integer StreamPartitioner.partition()</code> method is deprecated and replaced by the newly added
<code>Optiona≶Set<Integer>>StreamPartitioner.partitions()</code> method, which enables returning a set of partitions to send the record to.
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 <ahref="/{{version}}/documentation/streams/developer-guide/dsl-api.html#transformers-removal-and-migration-to-processors">migration guide</a>.
However, the KIP was only partially implemented, and is now completed with the 3.2.0 release.
For a full list of available RocksDB metrics, please consult the <ahref="/{{version}}/documentation/#kafka_streams_client_monitoring">monitoring documentation</a>.
</p>
<p>
Kafka Streams ships with RocksDB and in-memory store implementations and users can pick which one to use.
However, for the DSL, the choice is a per-operator one, making it cumbersome to switch from the default RocksDB
store to in-memory store for all operators, especially for larger topologies.
Interactive Queries may throw new exceptions for different errors:
</p>
<ul>
<li><code>UnknownStateStoreException</code>: If the specified store name does not exist in the topology, an <code>UnknownStateStoreException</code> will be thrown instead of the former <code>InvalidStateStoreException</code>.</li>
<li><code>StreamsNotStartedException</code>: If Streams state is <code>CREATED</code>, a <code>StreamsNotStartedException</code> will be thrown.</li>
<li><code>InvalidStateStorePartitionException</code>: If the specified partition does not exist, a <code>InvalidStateStorePartitionException</code> will be thrown.</li>
We deprecated the StreamsConfig <code>processing.guarantee</code> configuration value <code>"exactly_once"</code> (for EOS version 1) in favor of the improved EOS version 2, formerly configured via
<code>"exactly_once_beta</code>. To avoid confusion about the term "beta" in the config name and highlight the production-readiness of EOS version 2, we have also renamed "eos-beta" to "eos-v2"
and deprecated the configuration value <code>"exactly_once_beta"</code>, replacing it with a new configuration value <code>"exactly_once_v2"</code>
Users of exactly-once semantics should plan to migrate to the eos-v2 config and prepare for the removal of the deprecated configs in 4.0 or after at least a year
from the release of 3.0, whichever comes last. Note that eos-v2 requires broker version 2.5 or higher, like eos-beta, so users should begin to upgrade their kafka cluster if necessary. See
<ahref="https://cwiki.apache.org/confluence/x/zJONCg">KIP-732</a> for more details.
Additionally, in older versions Kafka Streams emitted stream-stream left/outer join results eagerly. This behavior may lead to spurious left/outer join result records.
In this release, we changed the behavior to avoid spurious results and left/outer join result are only emitted after the join window is closed, i.e., after the grace period elapsed.
To maintain backward compatibility, the old API <code>JoinWindows#of(timeDifference)</code> preserves the old eager-emit behavior and only the new
The public <code>topicGroupId</code> and <code>partition</code> fields on TaskId have been deprecated and replaced with getters. Please migrate to using the new <code>TaskId.subtopology()</code>
(which replaces <code>topicGroupId</code>) and <code>TaskId.partition()</code> APIs instead. Also, the <code>TaskId#readFrom</code> and <code>TaskId#writeTo</code> methods have been deprecated
and will be removed, as they were never intended for public use. We have also deprecated the <code>org.apache.kafka.streams.processor.TaskMetadata</code> class and introduced a new interface
<code>org.apache.kafka.streams.TaskMetadata</code> to be used instead. This change was introduced to better reflect the fact that <code>TaskMetadata</code> was not meant to be instantiated outside
of Kafka codebase.
Please note that the new <code>TaskMetadata</code> offers APIs that better represent the task id as an actual <code>TaskId</code> object instead of a String. Please migrate to the new
<code>org.apache.kafka.streams.TaskMetadata</code> which offers these better methods, for example, by using the new <code>ThreadMetadata#activeTasks</code> and <code>ThreadMetadata#standbyTasks</code>.
<code>org.apache.kafka.streams.processor.ThreadMetadata</code> class is also now deprecated and the newly introduced interface <code>org.apache.kafka.streams.ThreadMetadata</code> is to be used instead. In this new <code>ThreadMetadata</code>
interface, any reference to the deprecated <code>TaskMetadata</code> is replaced by the new interface.
Finally, also <code>org.apache.kafka.streams.state.StreamsMetadata</code> has been deprecated. Please migrate to the new <code>org.apache.kafka.streams.StreamsMetadata</code>.
We have deprecated several methods under <code>org.apache.kafka.streams.KafkaStreams</code> that returned the aforementioned deprecated classes:
<li>Users of <code>KafkaStreams#allMetadata</code> are meant to migrate to the new <code>KafkaStreams#metadataForAllStreamsClients</code>.</li>
<li>Users of <code>KafkaStreams#allMetadataForStore(String)</code> are meant to migrate to the new <code>KafkaStreams#streamsMetadataForStore(String)</code>.</li>
<li>Users of <code>KafkaStreams#localThreadsMetadata</code> are meant to migrate to the new <code>KafkaStreams#metadataForLocalThreads</code>.</li>
</ul>
<p>See <ahref="https://cwiki.apache.org/confluence/x/vYTOCg">KIP-740</a> and <ahref="https://cwiki.apache.org/confluence/x/XIrOCg">KIP-744</a> for more details.</p>
<li><code>--zookeeper</code> flag of the application reset tool: deprecated in Kafka 1.0.0 (<ahref="https://cwiki.apache.org/confluence/x/6J1jB">KIP-198</a>).</li>
<li><code>--execute</code> flag of the application reset tool: deprecated in Kafka 1.1.0 (<ahref="https://cwiki.apache.org/confluence/x/ApI7B">KIP-171</a>).</li>
<li><code>StreamsBuilder#addGlobalStore</code> (one overload): deprecated in Kafka 1.1.0 (<ahref="https://cwiki.apache.org/confluence/x/vKpzB">KIP-233</a>).</li>
<li><code>ProcessorContext#forward</code> (some overloads): deprecated in Kafka 2.0.0 (<ahref="https://cwiki.apache.org/confluence/x/Ih6HB">KIP-251</a>).</li>
<li><code>WindowBytesStoreSupplier#segments</code>: deprecated in Kafka 2.1.0 (<ahref="https://cwiki.apache.org/confluence/x/mQU0BQ">KIP-319</a>).</li>
<li><code>segments, until, maintainMs</code> on <code>TimeWindows</code>, <code>JoinWindows</code>, and <code>SessionWindows</code>: deprecated in Kafka 2.1.0 (<ahref="https://cwiki.apache.org/confluence/x/sQU0BQ">KIP-328</a>).</li>
<li> Overloaded <code>JoinWindows#of, before, after</code>, <code>SessionWindows#with</code>, <code>TimeWindows#of, advanceBy</code>, <code>UnlimitedWindows#startOn</code> and <code>KafkaStreams#close</code> with <code>long</code> typed parameters: deprecated in Kafka 2.1.0 (<ahref="https://cwiki.apache.org/confluence/x/IBNPBQ">KIP-358</a>).</li>
<li> Overloaded <code>KStream#groupBy, groupByKey</code> and <code>KTable#groupBy</code> with <code>Serialized</code> parameter: deprecated in Kafka 2.1.0 (<ahref="https://cwiki.apache.org/confluence/x/mgJ1BQ">KIP-372</a>).</li>
<li><code>Joined#named, name</code>: deprecated in Kafka 2.3.0 (<ahref="https://cwiki.apache.org/confluence/x/xikYBQ">KIP-307</a>).</li>
<li><code>TopologyTestDriver#pipeInput, readOutput</code>, <code>OutputVerifier</code> and <code>ConsumerRecordFactory</code> classes (<ahref="https://cwiki.apache.org/confluence/x/tI-iBg">KIP-470</a>).</li>
<li><code>KafkaClientSupplier#getAdminClient</code>: deprecated in Kafka 2.4.0 (<ahref="https://cwiki.apache.org/confluence/x/V9XiBg">KIP-476</a>).</li>
<li> Overloaded <code>KStream#join, leftJoin, outerJoin</code> with <code>KStream</code> and <code>Joined</code> parameters: deprecated in Kafka 2.4.0 (<ahref="https://cwiki.apache.org/confluence/x/EBEgBw">KIP-479</a>).</li>
<li><code>WindowStore#put(K key, V value)</code>: deprecated in Kafka 2.4.0 (<ahref="https://cwiki.apache.org/confluence/x/kcviBg">KIP-474</a>).</li>
<li><code>UsePreviousTimeOnInvalidTimestamp</code>: deprecated in Kafka 2.5.0 as renamed to <code>UsePartitionTimeOnInvalidTimestamp</code> (<ahref="https://cwiki.apache.org/confluence/x/BxXABw">KIP-530</a>).</li>
<li> Overloaded <code>KafkaStreams#metadataForKey</code>: deprecated in Kafka 2.5.0 (<ahref="https://cwiki.apache.org/confluence/x/Xg-jBw">KIP-535</a>).</li>
<li> Overloaded <code>KafkaStreams#store</code>: deprecated in Kafka 2.5.0 (<ahref="https://cwiki.apache.org/confluence/x/QYyvC">KIP-562</a>).</li>
The following dependencies were removed from Kafka Streams:
</p>
<ul>
<li>Connect-json: As of Kafka Streams no longer has a compile time dependency on "connect:json" module (<ahref="https://issues.apache.org/jira/browse/KAFKA-5146">KAFKA-5146</a>).
Projects that were relying on this transitive dependency will have to explicitly declare it.</li>
We added two new methods to <code>KafkaStreams</code>, namely <code>KafkaStreams#addStreamThread()</code> and <code>KafkaStreams#removeStreamThread()</code> in
The <code>TimeWindowedDeserializer</code> constructor <code>TimeWindowedDeserializer(final Deserializer inner)</code>
was deprecated to encourage users to properly set their window size through <code>TimeWindowedDeserializer(final Deserializer inner, Long windowSize)</code>.
An additional streams config, <code>window.size.ms</code>, was added for users that cannot set the window size through
We changed the default value of <code>default.key.serde</code> and <code>default.value.serde</code> to be <code>null</code> instead of <code>ByteArraySerde</code>.
Users will now see a <code>ConfigException</code> if their serdes are not correctly configured through those configs or passed in explicitly.
For more highly available stateful applications, we've modified the task assignment algorithm to delay the movement of stateful active tasks to instances
that aren't yet caught up with that task's state. Instead, to migrate a task from one instance to another (eg when scaling out),
Streams will assign a warmup replica to the target instance so it can begin restoring the state while the active task stays available on an instance
that already had the task. The instances warming up tasks will communicate their progress to the group so that, once ready, Streams can move active
tasks to their new owners in the background. Check out <ahref="https://cwiki.apache.org/confluence/x/0i4lBg">KIP-441</a>
for full details, including several new configs for control over this new feature.
</p>
<p>
New end-to-end latency metrics have been added. These task-level metrics will be logged at the INFO level and report the min and max end-to-end latency of a record at the beginning/source node(s)
and end/terminal node(s) of a task. See <ahref="https://cwiki.apache.org/confluence/x/gBkRCQ">KIP-613</a> for more information.
If you need to write into and read back from a topic that you manage, you can fall back to use <code>KStream.to()</code> in combination with <code>StreamsBuilder#stream()</code>.
Please refer to the <ahref="/{{version}}/documentation/streams/developer-guide/dsl-api.html">developer guide</a> for more details about <code>KStream.repartition()</code>.
<h3class="anchor-heading"><aid="streams_api_changes_250"class="anchor-link"></a><ahref="#streams_api_changes_250">Streams API changes in 2.5.0</a></h3>
Deprecated <code>KafkaStreams.store(String, QueryableStoreType)</code> and replaced it with <code>KafkaStreams.store(StoreQueryParameters)</code> to allow querying
<h3class="anchor-heading"><aid="streams_api_changes_240"class="anchor-link"></a><ahref="#streams_api_changes_240">Streams API changes in 2.4.0</a></h3>
With the introduction of incremental cooperative rebalancing, Streams no longer requires all tasks be revoked at the beginning of a rebalance. Instead, at the completion of the rebalance only those tasks which are to be migrated to another consumer
for overall load balance will need to be closed and revoked. This changes the semantics of the <code>StateListener</code> a bit, as it will not necessarily transition to <code>REBALANCING</code> at the beginning of a rebalance anymore. Note that
this means IQ will now be available at all times except during state restoration, including while a rebalance is in progress. If restoration is occurring when a rebalance begins, we will continue to actively restore the state stores and/or process
standby tasks during a cooperative rebalance. Note that with this new rebalancing protocol, you may sometimes see a rebalance be followed by a second short rebalance that ensures all tasks are safely distributed. For details on please see
<h3class="anchor-heading"><aid="streams_api_broker_compat"class="anchor-link"></a><ahref="#streams_api_broker_compat">Streams API broker compatibility</a></h3>
<p>The following table shows which versions of the Kafka Streams API are compatible with various Kafka broker versions. For Kafka Stream version older than 2.4.x, please check <ahref="/39/documentation/streams/upgrade-guide">3.9 upgrade document</a>.</p>