<h5><aid="upgrade_410_notable"href="#upgrade_410_notable">Notable changes in 4.1.0</a></h5>
<ul>
<li><b>Producer</b>
<ul>
<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.
<h4><aid="upgrade_4_0_0"href="#upgrade_4_0_0">Upgrading to 4.0.0 from any version 3.3.x through 3.9.x</a></h4>
<p><b>For a rolling upgrade:</b></p>
<ol>
<li>Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the
brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations.
</li>
<li>Once the cluster's behavior and performance has been verified, bump the metadata.version by running
<code>
bin/kafka-features.sh upgrade --metadata 4.0
</code>
</li>
<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>
<ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-896%3A+Remove+old+client+protocol+API+versions+in+Kafka+4.0">KIP-896</a> for the details.
Apache Kafka 4.0 ships with a brand-new group coordinator implementation (See <ahref="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=217387038#KIP848:TheNextGenerationoftheConsumerRebalanceProtocol-GroupCoordinator">here</a>.
The Next Generation of the Consumer Rebalance Protocol (<ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol">KIP-848</a>)
is now Generally Available (GA) in Apache Kafka 4.0. The protocol is automatically enabled on the server when the upgrade to 4.0 is finalized.
Transactions Server Side Defense (<ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense">KIP-890</a>)
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>A number of deprecated classes, methods, configurations and tools have been removed.
<ul>
<li><b>Common</b>
<ul>
<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
and <code>kafka.clients.admin.DescribeLogDirsResult.allDescriptions()</code>instead.
</li>
<li>The <code>kafka.common.requests.DescribeLogDirsResponse.ReplicaInfo</code> class was removed. Please use the <code>kafka.clients.admin.DescribeLogDirsResult.descriptions()</code> class
and <code>kafka.clients.admin.DescribeLogDirsResult.allDescriptions()</code>instead.
<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> class was 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 <code>remote.log.manager.copier.thread.pool.size</code> configuration default value was changed to 10 from -1.
Values of -1 are no longer valid. A minimum of 1 or higher is valid.
See <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1030%3A+Change+constraints+and+default+values+for+various+configurations">KIP-1030</a>
</li>
<li>
The <code>remote.log.manager.expiration.thread.pool.size</code> configuration default value was changed to 10 from -1.
Values of -1 are no longer valid. A minimum of 1 or higher is valid.
See <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1030%3A+Change+constraints+and+default+values+for+various+configurations">KIP-1030</a>
</li>
<li>
The <code>remote.log.manager.thread.pool.size</code> configuration default value was changed to 2 from 10.
See <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1030%3A+Change+constraints+and+default+values+for+various+configurations">KIP-1030</a>
The minimum <code>segment.bytes/log.segment.bytes</code> has changed from 14 bytes to 1MB.
See <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1030%3A+Change+constraints+and+default+values+for+various+configurations">KIP-1030</a>
The modified behavior is 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.
</li>
<li>The <code>add.source.alias.to.metrics</code> configuration was removed from <code>MirrorSourceConnector</code>.
The source cluster alias is now always added to the metrics.
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>kafka-configs.sh now uses incrementalAlterConfigs API to alter broker configurations instead of the deprecated alterConfigs API,
and it will fall directly if the broker doesn't support incrementalAlterConfigs API, which means the broker version is prior to 2.3.x.
See <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1011%3A+Use+incrementalAlterConfigs+when+updating+broker+configs+by+kafka-configs.sh">KIP-1011</a> for more details.
<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 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 <ahref="https://issues.apache.org/jira/browse/KAFKA-17531">KAFKA-17531</a> for details.
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,
For implementors of RemoteLogMetadataManager (RLMM), a new API `nextSegmentWithTxnIndex` is
introduced in RLMM to allow the implementation to return the next segment metadata with a transaction
index. This API is used when the consumers are enabled with isolation level as READ_COMMITTED.
See <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1058:+Txn+consumer+exerts+pressure+on+remote+storage+when+collecting+aborted+transactions">KIP-1058</a> for more details.
The criteria for identifying internal topics in ReplicationPolicy and DefaultReplicationPolicy have
been updated to enable the replication of topics that appear to be internal but aren't truly internal to Kafka and Mirror Maker 2.
See <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1074%3A+Allow+the+replication+of+user+internal+topics">KIP-1074</a> for more details.
KIP-714 is now enabled for Kafka Streams via <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1076%3A++Metrics+for+client+applications+KIP-714+extension">KIP-1076</a>.
This allows to not only collect the metric of the internally used clients of a Kafka Streams appliction via a broker-side plugin,
but also to collect the <ahref="/{{version}}/documentation/#kafka_streams_monitoring">metrics</a> of the Kafka Streams runtime itself.
The default value of 'num.recovery.threads.per.data.dir' has been changed from 1 to 2. The impact of this is faster
recovery post unclean shutdown at the expense of extra IO cycles.
See <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1030%3A+Change+constraints+and+default+values+for+various+configurations">KIP-1030</a>
</li>
<li>
The default value of 'message.timestamp.after.max.ms' has been changed from Long.Max to 1 hour. The impact of this messages with a
timestamp of more than 1 hour in the future will be rejected when message.timestamp.type=CreateTime is set.
See <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1030%3A+Change+constraints+and+default+values+for+various+configurations">KIP-1030</a>
<li>Introduced in KIP-890, 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>
<ul>
<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>
<li> See <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense">KIP-890</a> and
<ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1050%3A+Consistent+error+handling+for+Transactions">KIP-1050</a> for more details </li>