<h4><aid="upgrade_4_2_0"href="#upgrade_4_2_0">Upgrading to 4.2.0</a></h4>
<h5><aid="upgrade_4_2_0_from"href="#upgrade_4_2_0_from">Upgrading Servers to 4.2.0 from any version 3.3.x through 4.1.x</a></h5>
<h5><aid="upgrade_420_notable"href="#upgrade_420_notable">Notable changes in 4.2.0</a></h5>
<ul>
<li>The <code>org.apache.kafka.disallowed.login.modules</code> config was deprecated. Please use the <code>org.apache.kafka.allowed.login.modules</code>
The <code>remote.log.manager.thread.pool.size</code> config was deprecated. Please use the <code>remote.log.manager.follower.thread.pool.size</code> instead.
<li>The <code>KafkaPrincipalBuilder</code> now extends <code>KafkaPrincipalSerde</code>. Force developer to implement <code>KafkaPrincipalSerde</code> interface for custom <code>KafkaPrincipalBuilder</code>.
For further details, please refer to <ahref="https://cwiki.apache.org/confluence/x/1gq9F">KIP-1157</a>.
Apache Kafka 4.1 ships with a preview of Queues for Kafka (<ahref="https://cwiki.apache.org/confluence/x/4hA0Dw">KIP-932</a>). This feature introduces a new kind of group called
share groups, as an alternative to consumer groups. Consumers in a share group cooperatively consume records from topics, without assigning each partition to just one consumer.
Share groups also introduce per-record acknowledgement and counting of delivery attempts. Use share groups in cases where records are processed one at a time, rather than as part
of an ordered stream. To enable share groups, use the <code>kafka-features.sh</code> tool to upgrade to <code>share.version=1</code>.
For more information, please read the <ahref="https://cwiki.apache.org/confluence/x/CIq3FQ"> release notes</a>.
The logger class name for LogCleaner has been updated from <code>kafka.log.LogCleaner</code> to <code>org.apache.kafka.storage.internals.log.LogCleaner</code> in the log4j2.yaml configuration file.
Added loggers for <code>org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread</code> and <code>org.apache.kafka.storage.internals.log.Cleaner</code> classes to CleanerAppender.
<li>The <code>flush</code> method now detects potential deadlocks and prohibits its use inside a callback. This change prevents unintended blocking behavior, which was a known risk in earlier versions.
The <code>window.size.ms</code> and <code>window.inner.serde.class</code> in <code>StreamsConfig</code> are deprecated.
Use the corresponding string constants defined in <code>TimeWindowedSerializer</code>, <code>TimeWindowedDeserializer</code>, <code>SessionWindowedSerializer</code> and <code>SessionWindowedDeserializer</code> instead.
<p>Note: Apache Kafka 4.0 only supports KRaft mode - ZooKeeper mode has been removed. As such, <b>broker upgrades to 4.0.0 (and higher) require KRaft mode and
the software and metadata versions must be at least 3.3.x</b> (the first version when KRaft mode was deemed production ready). For clusters in KRaft mode
with versions older than 3.3.x, we recommend upgrading to 3.9.x before upgrading to 4.0.x. Clusters in ZooKeeper mode
have to be <ahref="/{{version}}/documentation.html#kraft_zk_migration">migrated to KRaft mode</a> before they can be upgraded to 4.0.x.
<li>Note that cluster metadata downgrade is not supported in this version since it has metadata changes.
Every <ahref="https://github.com/apache/kafka/blob/trunk/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java">MetadataVersion</a>
has a boolean parameter that indicates if there are metadata changes (i.e. <code>IBP_4_0_IV1(23, "4.0", "IV1", true)</code> means this version has metadata changes).
Given your current and target versions, a downgrade is only possible if there are no metadata changes in the versions between.</li>
brings a strengthened transactional protocol to Apache Kafka 4.0. The new and improved transactional protocol is enabled when the upgrade to 4.0 is finalized.
When using 4.0 producer clients, the producer epoch is bumped on every transaction to ensure every transaction includes the intended messages and duplicates are not
written as part of the next transaction. Downgrading the protocol is safe. For more information check <ahref="/{{version}}/documentation.html#transaction_protocol">here</a>.
<li>The <code>metrics.jmx.blacklist</code> and <code>metrics.jmx.whitelist</code> configurations were removed from the <code>org.apache.kafka.common.metrics.JmxReporter</code>
Please use <code>metrics.jmx.exclude</code> and <code>metrics.jmx.include</code> respectively instead.
<li>The <code>bufferpool-wait-time-total</code>, <code>io-waittime-total</code>, and <code>iotime-total</code> metrics were removed.
Please use <code>bufferpool-wait-time-ns-total</code>, <code>io-wait-time-ns-total</code>, and <code>io-time-ns-total</code> metrics as replacements, respectively.
<li>The <code>kafka.common.requests.DescribeLogDirsResponse.LogDirInfo</code> class was removed. Please use the <code>kafka.clients.admin.DescribeLogDirsResult.descriptions()</code> class
<li>The <code>kafka.common.requests.DescribeLogDirsResponse.ReplicaInfo</code> class was removed. Please use the <code>kafka.clients.admin.DescribeLogDirsResult.descriptions()</code> class
<li>The <code>org.apache.kafka.common.errors.NotLeaderForPartitionException</code> class was removed. The <code>org.apache.kafka.common.errors.NotLeaderOrFollowerException</code> is returned
if a request could not be processed because the broker is not the leader or follower for a topic partition.
<li>The <code>org.apache.kafka.clients.producer.internals.DefaultPartitioner</code> and <code>org.apache.kafka.clients.producer.UniformStickyPartitioner</code> classes were removed.
<li>The default properties files for KRaft mode are no longer stored in the separate <code>config/kraft</code> directory since Zookeeper has been removed. These files have been consolidated with other configuration files.
Now all configuration files are in <code>config</code> directory.
<li>The valid format for <code>--bootstrap-server</code> only supports comma-separated value, such as <code>host1:port1,host2:port2,...</code>.
Providing other formats, like space-separated bootstrap servers (e.g., <code>host1:port1 host2:port2 host3:port3</code>), will result in an exception, even though this was allowed in Apache Kafka versions prior to 4.0.
The modified behavior is now identical to the previous <code>required</code> configuration, therefore users should ensure that brokers in the target cluster are at least running 2.3.0.
interface to build custom readers for the <code>kafka-console-producer</code> tool.
</li>
<li>The <code>kafka.tools.DefaultMessageFormatter</code> class was removed. Please use the <code>org.apache.kafka.tools.consumer.DefaultMessageFormatter</code> class instead.
</li>
<li>The <code>kafka.tools.LoggingMessageFormatter</code> class was removed. Please use the <code>org.apache.kafka.tools.consumer.LoggingMessageFormatter</code> class instead.
<li>The <code>kafka.tools.NoOpMessageFormatter</code> class was removed. Please use the <code>org.apache.kafka.tools.consumer.NoOpMessageFormatter</code> class instead.
<li>The <code>--authorizer</code>, <code>--authorizer-properties</code>, and <code>--zk-tls-config-file</code> options were removed from the <code>kafka-acls</code> command line tool.
Please use <code>--bootstrap-server</code> or <code>--bootstrap-controller</code> instead.
interface to build custom decoders for the <code>kafka-dump-log</code> tool.
</li>
<li>The <code>kafka.coordinator.group.OffsetsMessageFormatter</code> class was removed. Please use the <code>org.apache.kafka.tools.consumer.OffsetsMessageFormatter</code> class instead.
</li>
<li>The <code>kafka.coordinator.group.GroupMetadataMessageFormatter</code> class was removed. Please use the <code>org.apache.kafka.tools.consumer.GroupMetadataMessageFormatter</code> class instead.
</li>
<li>The <code>kafka.coordinator.transaction.TransactionLogMessageFormatter</code> class was removed. Please use the <code>org.apache.kafka.tools.consumer.TransactionLogMessageFormatter</code> class instead.
</li>
<li>The <code>--topic-white-list</code> option was removed from the <code>kafka-replica-verification</code> command line tool.
<li>The <code>whitelist</code> and <code>blacklist</code> configurations were removed from the <code>org.apache.kafka.connect.transforms.ReplaceField</code> transformation.
Please use <code>include</code> and <code>exclude</code> respectively instead.
<li>The <code>onPartitionsRevoked(Collection<TopicPartition>)</code> and <code>onPartitionsAssigned(Collection<TopicPartition>)</code> methods
<li>The <code>enable.idempotence</code> configuration will no longer automatically fall back when the <code>max.in.flight.requests.per.connection</code> value exceeds 5.
<li>The deprecated <code>sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata>, String)</code> method has been removed from the Producer API.
<li>The <code>org.apache.kafka.common.ConsumerGroupState</code> enumeration and related methods have been deprecated. Please use <code>GroupState</code> instead
which applies to all types of group.
</li>
<li>The <code>Admin.describeConsumerGroups</code> method used to return a <code>ConsumerGroupDescription</code> in state
<code>DEAD</code> if the group ID was not found. In Apache Kafka 4.0, the <code>GroupIdNotFoundException</code>
is thrown instead as part of the support for new types of group.
All public APIs, 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>.
The most important changes are highlighted in the <ahref="/{{version}}/documentation/streams/upgrade-guide.html#streams_api_changes_400">Kafka Streams upgrade guide</a>.
Logging framework has been migrated from Log4j to Log4j2.
Users can use the log4j-transform-cli tool to automatically convert their existing Log4j configuration files to Log4j2 format.
See <ahref="https://logging.staged.apache.org/log4j/transform/cli.html#log4j-transform-cli">log4j-transform-cli</a> for more details.
Log4j2 provides limited compatibility for Log4j configurations.
See <ahref="https://logging.apache.org/log4j/2.x/migrate-from-log4j1.html#ConfigurationCompatibility">Use Log4j 1 to Log4j 2 bridge</a> for more information,
<ahref="https://cwiki.apache.org/confluence/x/2xRRCg">KIP-714</a> is now enabled for Kafka Streams via <ahref="https://cwiki.apache.org/confluence/x/XA-OEg">KIP-1076</a>.
<li>Introduced in <ahref="https://cwiki.apache.org/confluence/x/B40ODg">KIP-890</a>, the <code>TransactionAbortableException</code> enhances error handling within transactional operations by clearly indicating scenarios where transactions should be aborted due to errors. It is important for applications to properly manage both <code>TimeoutException</code> and <code>TransactionAbortableException</code> when working with transaction producers.</li>
<li><b>TimeoutException:</b> This exception indicates that a transactional operation has timed out. Given the risk of message duplication that can arise from retrying operations after a timeout (potentially violating exactly-once semantics), applications should treat timeouts as reasons to abort the ongoing transaction.</li>
<li><b>TransactionAbortableException:</b> Specifically introduced to signal errors that should lead to transaction abortion, ensuring this exception is properly handled is critical for maintaining the integrity of transactional processing.</li>
<li>To ensure seamless operation and compatibility with future Kafka versions, developers are encouraged to update their error-handling logic to treat both exceptions as triggers for aborting transactions. This approach is pivotal for preserving exactly-once semantics.</li>