27/upgrade.html [73:910]:
- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
KIP-651 adds support
for using PEM files for key and trust stores.
KIP-612 adds support
for enforcing broker-wide and per-listener connection create rates. The 2.7.0 release contains
the first part of KIP-612 with dynamic configuration coming in the 2.8.0 release.
The ability to throttle topic and partition creations or
topics deletions to prevent a cluster from being harmed via
KIP-599
When new features become available in Kafka there are two main issues:
- How do Kafka clients become aware of broker capabilities?
- How does the broker decide which features to enable?
KIP-584
provides a flexible and operationally easy solution for client discovery, feature gating and rolling upgrades using a single restart.
The ability to print record offsets and headers with the ConsoleConsumer
is now possible
via KIP-431
The addition of KIP-554
continues progress towards the goal of Zookeeper removal from Kafka. The addition of KIP-554
means you don't have to connect directly to ZooKeeper anymore for managing SCRAM credentials.
Altering non-reconfigurable configs of existent listeners causes InvalidRequestException
.
By contrast, the previous (unintended) behavior would have caused the updated configuration to be persisted,
but it wouldn't
take effect until the broker was restarted. See KAFKA-10479 for more discussion.
See DynamicBrokerConfig.DynamicSecurityConfigs
and SocketServer.ListenerReconfigurableConfigs
for the supported reconfigurable configs of existent listeners.
Kafka Streams adds support for
Sliding Windows Aggregations
in the KStreams DSL.
Reverse iteration over state stores enabling more efficient most recent update searches with
KIP-617
End-to-End latency metrics in Kafka Steams see
KIP-613
for more details
Kafka Streams added metrics reporting default RocksDB properties with
KIP-607
Better Scala implicit Serdes support from
KIP-616
If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets.
Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override
the inter-broker protocol version.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g.,
2.5
, 2.4
, etc.)
- 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.
It is still possible to downgrade at this point if there are any problems.
- Once the cluster's behavior and performance has been verified, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 2.6
.
- Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest
protocol version, it will no longer be possible to downgrade the cluster to an older version.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 2.6 on each broker and restart them one by one. Note that the older Scala clients,
which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs
(or to take advantage of exactly once semantics),
the newer Java clients must be used.
- Kafka Streams adds a new processing mode (requires broker 2.5 or newer) that improves application
scalability using exactly-once guarantees
(cf. KIP-447)
- TLSv1.3 has been enabled by default for Java 11 or newer. The client and server will negotiate TLSv1.3 if
both support it and fallback to TLSv1.2 otherwise. See
KIP-573 for more details.
- The default value for the
client.dns.lookup
configuration has been changed from default
to use_all_dns_ips
. If a hostname resolves to multiple IP addresses, clients and brokers will now
attempt to connect to each IP in sequence until the connection is successfully established. See
KIP-602
for more details.
NotLeaderForPartitionException
has been deprecated and replaced with NotLeaderOrFollowerException
.
Fetch requests and other requests intended only for the leader or follower return NOT_LEADER_OR_FOLLOWER(6) instead of REPLICA_NOT_AVAILABLE(9)
if the broker is not a replica, ensuring that this transient error during reassignments is handled by all clients as a retriable exception.
If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets.
Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override
the inter-broker protocol version.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g.,
2.4
, 2.3
, etc.)
- 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.
It is still possible to downgrade at this point if there are any problems.
- Once the cluster's behavior and performance has been verified, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 2.5
.
- Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest
protocol version, it will no longer be possible to downgrade the cluster to an older version.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 2.5 on each broker and restart them one by one. Note that the older Scala clients,
which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs
(or to take advantage of exactly once semantics),
the newer Java clients must be used.
- There are several notable changes to the reassignment tool
kafka-reassign-partitions.sh
following the completion of
KIP-455.
This tool now requires the --additional
flag to be provided when changing the throttle of an
active reassignment. Reassignment cancellation is now possible using the
--cancel
command. Finally, reassignment with --zookeeper
has been deprecated in favor of --bootstrap-server
. See the KIP for more detail.
- When
RebalanceProtocol#COOPERATIVE
is used, Consumer#poll
can still return data
while it is in the middle of a rebalance for those partitions still owned by the consumer; in addition
Consumer#commitSync
now may throw a non-fatal RebalanceInProgressException
to notify
users of such an event, in order to distinguish from the fatal CommitFailedException
and allow
users to complete the ongoing rebalance and then reattempt committing offsets for those still-owned partitions.
- For improved resiliency in typical network environments, the default value of
zookeeper.session.timeout.ms
has been increased from 6s to 18s and
replica.lag.time.max.ms
from 10s to 30s.
- New DSL operator
cogroup()
has been added for aggregating multiple streams together at once.
- Added a new
KStream.toTable()
API to translate an input event stream into a KTable.
- Added a new Serde type
Void
to represent null keys or null values from input topic.
- Deprecated
UsePreviousTimeOnInvalidTimestamp
and replaced it with UsePartitionTimeOnInvalidTimeStamp
.
- Improved exactly-once semantics by adding a pending offset fencing mechanism and stronger transactional commit
consistency check, which greatly simplifies the implementation of a scalable exactly-once application.
We also added a new exactly-once semantics code example under
examples folder. Check out
KIP-447
for the full details.
- Added a new public api
KafkaStreams.queryMetadataForKey(String, K, Serializer) to get detailed information on the key being queried.
It provides information about the partition number where the key resides in addition to hosts containing the active and standby partitions for the key.
- Provided support to query stale stores (for high availability) and the stores belonging to a specific partition by deprecating
KafkaStreams.store(String, QueryableStoreType)
and replacing it with KafkaStreams.store(StoreQueryParameters)
.
- Added a new public api to access lag information for stores local to an instance with
KafkaStreams.allLocalStorePartitionLags()
.
- Scala 2.11 is no longer supported. See
KIP-531
for details.
- All Scala classes from the package
kafka.security.auth
have been deprecated. See
KIP-504
for details of the new Java authorizer API added in 2.4.0. Note that kafka.security.auth.Authorizer
and kafka.security.auth.SimpleAclAuthorizer
were deprecated in 2.4.0.
- TLSv1 and TLSv1.1 have been disabled by default since these have known security vulnerabilities. Only TLSv1.2 is now
enabled by default. You can continue to use TLSv1 and TLSv1.1 by explicitly enabling these in the configuration options
ssl.protocol
and ssl.enabled.protocols
.
- ZooKeeper has been upgraded to 3.5.7, and a ZooKeeper upgrade from 3.4.X to 3.5.7 can fail if there are no snapshot files in the 3.4 data directory.
This usually happens in test upgrades where ZooKeeper 3.5.7 is trying to load an existing 3.4 data dir in which no snapshot file has been created.
For more details about the issue please refer to ZOOKEEPER-3056.
A fix is given in ZOOKEEPER-3056, which is to set
snapshot.trust.empty=true
config in zookeeper.properties
before the upgrade.
- ZooKeeper version 3.5.7 supports TLS-encrypted connectivity to ZooKeeper both with or without client certificates,
and additional Kafka configurations are available to take advantage of this.
See KIP-515 for details.
If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets.
Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.10.0, 0.11.0, 1.0, 2.0, 2.2).
- log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact
following the upgrade for the details on what this configuration does.)
If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override
the inter-broker protocol version.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1, 2.0, 2.1, 2.2, 2.3).
- 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.
It is still possible to downgrade at this point if there are any problems.
- Once the cluster's behavior and performance has been verified, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 2.4.
- Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest
protocol version, it will no longer be possible to downgrade the cluster to an older version.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 2.4 on each broker and restart them one by one. Note that the older Scala clients,
which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs
(or to take advantage of exactly once semantics),
the newer Java clients must be used.
Additional Upgrade Notes:
- ZooKeeper has been upgraded to 3.5.6. ZooKeeper upgrade from 3.4.X to 3.5.6 can fail if there are no snapshot files in 3.4 data directory.
This usually happens in test upgrades where ZooKeeper 3.5.6 is trying to load an existing 3.4 data dir in which no snapshot file has been created.
For more details about the issue please refer to ZOOKEEPER-3056.
A fix is given in ZOOKEEPER-3056, which is to set
snapshot.trust.empty=true
config in zookeeper.properties
before the upgrade. But we have observed data loss in standalone cluster upgrades when using
snapshot.trust.empty=true
config. For more details about the issue please refer to ZOOKEEPER-3644.
So we recommend the safe workaround of copying empty snapshot file to the 3.4 data directory,
if there are no snapshot files in 3.4 data directory. For more details about the workaround please refer to ZooKeeper Upgrade FAQ.
-
An embedded Jetty based AdminServer added in ZooKeeper 3.5.
AdminServer is enabled by default in ZooKeeper and is started on port 8080.
AdminServer is disabled by default in the ZooKeeper config (
zookeeper.properties
) provided by the Apache Kafka distribution.
Make sure to update your local zookeeper.properties
file with admin.enableServer=false
if you wish to disable the AdminServer.
Please refer AdminServer config to configure the AdminServer.
- A new Admin API has been added for partition reassignments. Due to changing the way Kafka propagates reassignment information,
it is possible to lose reassignment state in failure edge cases while upgrading to the new version. It is not recommended to start reassignments while upgrading.
- ZooKeeper has been upgraded from 3.4.14 to 3.5.6. TLS and dynamic reconfiguration are supported by the new version.
- The
bin/kafka-preferred-replica-election.sh
command line tool has been deprecated. It has been replaced by bin/kafka-leader-election.sh
.
- The methods
electPreferredLeaders
in the Java AdminClient
class have been deprecated in favor of the methods electLeaders
.
- Scala code leveraging the
NewTopic(String, int, short)
constructor with literal values will need to explicitly call toShort
on the second literal.
- The argument in the constructor
GroupAuthorizationException(String)
is now used to specify an exception message.
Previously it referred to the group that failed authorization. This was done for consistency with other exception types and to
avoid potential misuse. The constructor TopicAuthorizationException(String)
which was previously used for a single
unauthorized topic was changed similarly.
- The internal
PartitionAssignor
interface has been deprecated and replaced with a new ConsumerPartitionAssignor
in the public API. Some
methods/signatures are slightly different between the two interfaces. Users implementing a custom PartitionAssignor should migrate to the new interface as soon as possible.
- The
DefaultPartitioner
now uses a sticky partitioning strategy. This means that records for specific topic with null keys and no assigned partition
will be sent to the same partition until the batch is ready to be sent. When a new batch is created, a new partition is chosen. This decreases latency to produce, but
it may result in uneven distribution of records across partitions in edge cases. Generally users will not be impacted, but this difference may be noticeable in tests and
other situations producing records for a very short amount of time.
- The blocking
KafkaConsumer#committed
methods have been extended to allow a list of partitions as input parameters rather than a single partition.
It enables fewer request/response iterations between clients and brokers fetching for the committed offsets for the consumer group.
The old overloaded functions are deprecated and we would recommend users to make their code changes to leverage the new methods (details
can be found in KIP-520).
- We've introduced a new
INVALID_RECORD
error in the produce response to distinguish from the CORRUPT_MESSAGE
error.
To be more concrete, previously when a batch of records were sent as part of a single request to the broker and one or more of the records failed
the validation due to various causes (mismatch magic bytes, crc checksum errors, null key for log compacted topics, etc), the whole batch would be rejected
with the same and misleading CORRUPT_MESSAGE
, and the caller of the producer client would see the corresponding exception from either
the future object of RecordMetadata
returned from the send
call as well as in the Callback#onCompletion(RecordMetadata metadata, Exception exception)
Now with the new error code and improved error messages of the exception, producer callers would be better informed about the root cause why their sent records were failed.
- We are introducing incremental cooperative rebalancing to the clients' group protocol, which allows consumers to keep all of their assigned partitions during a rebalance
and at the end revoke only those which must be migrated to another consumer for overall cluster balance. The
ConsumerCoordinator
will choose the latest RebalanceProtocol
that is commonly supported by all of the consumer's supported assignors. You can use the new built-in CooperativeStickyAssignor
or plug in your own custom cooperative assignor. To do
so you must implement the ConsumerPartitionAssignor
interface and include RebalanceProtocol.COOPERATIVE
in the list returned by ConsumerPartitionAssignor#supportedProtocols
.
Your custom assignor can then leverage the ownedPartitions
field in each consumer's Subscription
to give partitions back to their previous owners whenever possible. Note that when
a partition is to be reassigned to another consumer, it must be removed from the new assignment until it has been revoked from its original owner. Any consumer that has to revoke a partition will trigger
a followup rebalance to allow the revoked partition to safely be assigned to its new owner. See the
ConsumerPartitionAssignor RebalanceProtocol javadocs for more information.
To upgrade from the old (eager) protocol, which always revokes all partitions before rebalancing, to cooperative rebalancing, you must follow a specific upgrade path to get all clients on the same ConsumerPartitionAssignor
that supports the cooperative protocol. This can be done with two rolling bounces, using the CooperativeStickyAssignor
for the example: during the first one, add "cooperative-sticky" to the list of supported assignors
for each member (without removing the previous assignor -- note that if previously using the default, you must include that explicitly as well). You then bounce and/or upgrade it.
Once the entire group is on 2.4+ and all members have the "cooperative-sticky" among their supported assignors, remove the other assignor(s) and perform a second rolling bounce so that by the end all members support only the
cooperative protocol. For further details on the cooperative rebalancing protocol and upgrade path, see KIP-429.
- There are some behavioral changes to the
ConsumerRebalanceListener
, as well as a new API. Exceptions thrown during any of the listener's three callbacks will no longer be swallowed, and will instead be re-thrown
all the way up to the Consumer.poll()
call. The onPartitionsLost
method has been added to allow users to react to abnormal circumstances where a consumer may have lost ownership of its partitions
(such as a missed rebalance) and cannot commit offsets. By default, this will simply call the existing onPartitionsRevoked
API to align with previous behavior. Note however that onPartitionsLost
will not
be called when the set of lost partitions is empty. This means that no callback will be invoked at the beginning of the first rebalance of a new consumer joining the group.
The semantics of the ConsumerRebalanceListener's
callbacks are further changed when following the cooperative rebalancing protocol described above. In addition to onPartitionsLost
, onPartitionsRevoked
will also never be called when the set of revoked partitions is empty. The callback will generally be invoked only at the end of a rebalance, and only on the set of partitions that are being moved to another consumer. The
onPartitionsAssigned
callback will however always be called, even with an empty set of partitions, as a way to notify users of a rebalance event (this is true for both cooperative and eager). For details on
the new callback semantics, see the ConsumerRebalanceListener javadocs.
- The Scala trait
kafka.security.auth.Authorizer
has been deprecated and replaced with a new Java API
org.apache.kafka.server.authorizer.Authorizer
. The authorizer implementation class
kafka.security.auth.SimpleAclAuthorizer
has also been deprecated and replaced with a new
implementation kafka.security.authorizer.AclAuthorizer
. AclAuthorizer
uses features
supported by the new API to improve authorization logging and is compatible with SimpleAclAuthorizer
.
For more details, see KIP-504.
If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets.
Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0, 1.0, 1.1).
- log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact
following the upgrade for the details on what this configuration does.)
If you are upgrading from 0.11.0.x, 1.0.x, 1.1.x, 2.0.x, or 2.1.x, and you have not overridden the message format, then you only need to override
the inter-broker protocol version.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1, 2.0, 2.1, 2.2).
- 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.
It is still possible to downgrade at this point if there are any problems.
- Once the cluster's behavior and performance has been verified, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 2.3.
- Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest
protocol version, it will no longer be possible to downgrade the cluster to an older version.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 2.3 on each broker and restart them one by one. Note that the older Scala clients,
which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs
(or to take advantage of exactly once semantics),
the newer Java clients must be used.
- We are introducing a new rebalancing protocol for Kafka Connect based on
incremental cooperative rebalancing.
The new protocol does not require stopping all the tasks during a rebalancing phase between Connect workers. Instead, only the tasks that need to be exchanged
between workers are stopped and they are started in a follow up rebalance. The new Connect protocol is enabled by default beginning with 2.3.0.
For more details on how it works and how to enable the old behavior of eager rebalancing, checkout
incremental cooperative rebalancing design.
- We are introducing static membership towards consumer user. This feature reduces unnecessary rebalances during normal application upgrades or rolling bounces.
For more details on how to use it, checkout static membership design.
- Kafka Streams DSL switches its used store types. While this change is mainly transparent to users, there are some corner cases that may require code changes.
See the Kafka Streams upgrade section for more details.
- Kafka Streams 2.3.0 requires 0.11 message format or higher and does not work with older message format.
If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets.
Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0, 1.0, 1.1).
- log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact
following the upgrade for the details on what this configuration does.)
If you are upgrading from 0.11.0.x, 1.0.x, 1.1.x, or 2.0.x and you have not overridden the message format, then you only need to override
the inter-broker protocol version.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1, 2.0).
- 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.
It is still possible to downgrade at this point if there are any problems.
- Once the cluster's behavior and performance has been verified, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 2.2.
- Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest
protocol version, it will no longer be possible to downgrade the cluster to an older version.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 2.2 on each broker and restart them one by one. Note that the older Scala clients,
which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs
(or to take advantage of exactly once semantics),
the newer Java clients must be used.
- Kafka Streams 2.2.1 requires 0.11 message format or higher and does not work with older message format.
- The default consumer group id has been changed from the empty string (
""
) to null
. Consumers who use the new default group id will not be able to subscribe to topics,
and fetch or commit offsets. The empty string as consumer group id is deprecated but will be supported until a future major release. Old clients that rely on the empty string group id will now
have to explicitly provide it as part of their consumer config. For more information see
KIP-289.
- The
bin/kafka-topics.sh
command line tool is now able to connect directly to brokers with --bootstrap-server
instead of zookeeper. The old --zookeeper
option is still available for now. Please read KIP-377 for more information.
- Kafka Streams depends on a newer version of RocksDBs that requires MacOS 10.13 or higher.
Note that 2.1.x contains a change to the internal schema used to store consumer offsets. Once the upgrade is
complete, it will not be possible to downgrade to previous versions. See the rolling upgrade notes below for more detail.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0, 1.0, 1.1).
- log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact
following the upgrade for the details on what this configuration does.)
If you are upgrading from 0.11.0.x, 1.0.x, 1.1.x, or 2.0.x and you have not overridden the message format, then you only need to override
the inter-broker protocol version.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1, 2.0).
- 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.
It is still possible to downgrade at this point if there are any problems.
- Once the cluster's behavior and performance has been verified, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 2.1.
- Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest
protocol version, it will no longer be possible to downgrade the cluster to an older version.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 2.1 on each broker and restart them one by one. Note that the older Scala clients,
which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs
(or to take advantage of exactly once semantics),
the newer Java clients must be used.
Additional Upgrade Notes:
- Offset expiration semantics has slightly changed in this version. According to the new semantics, offsets of partitions in a group will
not be removed while the group is subscribed to the corresponding topic and is still active (has active consumers). If group becomes
empty all its offsets will be removed after default offset retention period (or the one set by broker) has passed (unless the group becomes
active again). Offsets associated with standalone (simple) consumers, that do not use Kafka group management, will be removed after default
offset retention period (or the one set by broker) has passed since their last commit.
- The default for console consumer's
enable.auto.commit
property when no group.id
is provided is now set to false
.
This is to avoid polluting the consumer coordinator cache as the auto-generated group is not likely to be used by other consumers.
- The default value for the producer's
retries
config was changed to Integer.MAX_VALUE
, as we introduced delivery.timeout.ms
in KIP-91,
which sets an upper bound on the total time between sending a record and receiving acknowledgement from the broker. By default,
the delivery timeout is set to 2 minutes.
- By default, MirrorMaker now overrides
delivery.timeout.ms
to Integer.MAX_VALUE
when
configuring the producer. If you have overridden the value of retries
in order to fail faster,
you will instead need to override delivery.timeout.ms
.
- The
ListGroup
API now expects, as a recommended alternative, Describe Group
access to the groups a user should be able to list.
Even though the old Describe Cluster
access is still supported for backward compatibility, using it for this API is not advised.
- KIP-336 deprecates the ExtendedSerializer and ExtendedDeserializer interfaces and
propagates the usage of Serializer and Deserializer. ExtendedSerializer and ExtendedDeserializer were introduced with
KIP-82 to provide record headers for serializers and deserializers
in a Java 7 compatible fashion. Now we consolidated these interfaces as Java 7 support has been dropped since.
- Jetty has been upgraded to 9.4.12, which excludes TLS_RSA_* ciphers by default because they do not support forward
secrecy, see https://github.com/eclipse/jetty.project/issues/2807 for more information.
- Unclean leader election is automatically enabled by the controller when
unclean.leader.election.enable
config is dynamically updated by using per-topic config override.
- The
AdminClient
has added a method AdminClient#metrics()
. Now any application using the AdminClient
can gain more information
and insight by viewing the metrics captured from the AdminClient
. For more information
see KIP-324
- Kafka now supports Zstandard compression from KIP-110.
You must upgrade the broker as well as clients to make use of it. Consumers prior to 2.1.0 will not be able to read from topics which use
Zstandard compression, so you should not enable it for a topic until all downstream consumers are upgraded. See the KIP for more detail.
Kafka 2.0.0 introduces wire protocol changes. By following the recommended rolling upgrade plan below,
you guarantee no downtime during the upgrade. However, please review the notable changes in 2.0.0 before upgrading.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0, 1.0, 1.1).
- log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact
following the upgrade for the details on what this configuration does.)
If you are upgrading from 0.11.0.x, 1.0.x, or 1.1.x and you have not overridden the message format, then you only need to override
the inter-broker protocol format.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1).
- Upgrade the brokers one at a time: shut down the broker, update the code, and restart it.
- Once the entire cluster is upgraded, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 2.0.
- Restart the brokers one by one for the new protocol version to take effect.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 2.0 on each broker and restart them one by one. Note that the older Scala consumer
does not support the new message format introduced in 0.11, so to avoid the performance cost of down-conversion (or to
take advantage of exactly once semantics), the newer Java consumer must be used.
Additional Upgrade Notes:
- If you are willing to accept downtime, you can simply take all the brokers down, update the code and start them back up. They will start
with the new protocol by default.
- Bumping the protocol version and restarting can be done any time after the brokers are upgraded. It does not have to be immediately after.
Similarly for the message format version.
- If you are using Java8 method references in your Kafka Streams code you might need to update your code to resolve method ambiguities.
Hot-swapping the jar-file only might not work.
- ACLs should not be added to prefixed resources,
(added in KIP-290),
until all brokers in the cluster have been updated.
NOTE: any prefixed ACLs added to a cluster, even after the cluster is fully upgraded, will be ignored should the cluster be downgraded again.
- KIP-186 increases the default offset retention time from 1 day to 7 days. This makes it less likely to "lose" offsets in an application that commits infrequently. It also increases the active set of offsets and therefore can increase memory usage on the broker. Note that the console consumer currently enables offset commit by default and can be the source of a large number of offsets which this change will now preserve for 7 days instead of 1. You can preserve the existing behavior by setting the broker config
offsets.retention.minutes
to 1440.
- Support for Java 7 has been dropped, Java 8 is now the minimum version required.
- The default value for
ssl.endpoint.identification.algorithm
was changed to https
, which performs hostname verification (man-in-the-middle attacks are possible otherwise). Set ssl.endpoint.identification.algorithm
to an empty string to restore the previous behaviour.
- KAFKA-5674 extends the lower interval of
max.connections.per.ip
minimum to zero and therefore allows IP-based filtering of inbound connections.
- KIP-272
added API version tag to the metric
kafka.network:type=RequestMetrics,name=RequestsPerSec,request={Produce|FetchConsumer|FetchFollower|...}
.
This metric now becomes kafka.network:type=RequestMetrics,name=RequestsPerSec,request={Produce|FetchConsumer|FetchFollower|...},version={0|1|2|3|...}
. This will impact
JMX monitoring tools that do not automatically aggregate. To get the total count for a specific request type, the tool needs to be
updated to aggregate across different versions.
- KIP-225 changed the metric "records.lag" to use tags for topic and partition. The original version with the name format "{topic}-{partition}.records-lag" has been removed.
- The Scala consumers, which have been deprecated since 0.11.0.0, have been removed. The Java consumer has been the recommended option
since 0.10.0.0. Note that the Scala consumers in 1.1.0 (and older) will continue to work even if the brokers are upgraded to 2.0.0.
- The Scala producers, which have been deprecated since 0.10.0.0, have been removed. The Java producer has been the recommended option
since 0.9.0.0. Note that the behaviour of the default partitioner in the Java producer differs from the default partitioner
in the Scala producers. Users migrating should consider configuring a custom partitioner that retains the previous behaviour.
Note that the Scala producers in 1.1.0 (and older) will continue to work even if the brokers are upgraded to 2.0.0.
- MirrorMaker and ConsoleConsumer no longer support the Scala consumer, they always use the Java consumer.
- The ConsoleProducer no longer supports the Scala producer, it always uses the Java producer.
- A number of deprecated tools that rely on the Scala clients have been removed: ReplayLogProducer, SimpleConsumerPerformance, SimpleConsumerShell, ExportZkOffsets, ImportZkOffsets, UpdateOffsetsInZK, VerifyConsumerRebalance.
- The deprecated kafka.tools.ProducerPerformance has been removed, please use org.apache.kafka.tools.ProducerPerformance.
- New Kafka Streams configuration parameter
upgrade.from
added that allows rolling bounce upgrade from older version.
- KIP-284 changed the retention time for Kafka Streams repartition topics by setting its default value to
Long.MAX_VALUE
.
- Updated
ProcessorStateManager
APIs in Kafka Streams for registering state stores to the processor topology. For more details please read the Streams Upgrade Guide.
-
In earlier releases, Connect's worker configuration required the
internal.key.converter
and internal.value.converter
properties.
In 2.0, these are no longer required and default to the JSON converter.
You may safely remove these properties from your Connect standalone and distributed worker configurations:
internal.key.converter=org.apache.kafka.connect.json.JsonConverter
internal.key.converter.schemas.enable=false
internal.value.converter=org.apache.kafka.connect.json.JsonConverter
internal.value.converter.schemas.enable=false
- KIP-266 adds a new consumer configuration
default.api.timeout.ms
to specify the default timeout to use for KafkaConsumer
APIs that could block. The KIP also adds overloads for such blocking
APIs to support specifying a specific timeout to use for each of them instead of using the default timeout set by default.api.timeout.ms
.
In particular, a new poll(Duration)
API has been added which does not block for dynamic partition assignment.
The old poll(long)
API has been deprecated and will be removed in a future version. Overloads have also been added
for other KafkaConsumer
methods like partitionsFor
, listTopics
, offsetsForTimes
,
beginningOffsets
, endOffsets
and close
that take in a Duration
.
- Also as part of KIP-266, the default value of
request.timeout.ms
has been changed to 30 seconds.
The previous value was a little higher than 5 minutes to account for maximum time that a rebalance would take.
Now we treat the JoinGroup request in the rebalance as a special case and use a value derived from
max.poll.interval.ms
for the request timeout. All other request types use the timeout defined
by request.timeout.ms
- The internal method
kafka.admin.AdminClient.deleteRecordsBefore
has been removed. Users are encouraged to migrate to org.apache.kafka.clients.admin.AdminClient.deleteRecords
.
- The AclCommand tool
--producer
convenience option uses the KIP-277 finer grained ACL on the given topic.
- KIP-176 removes
the
--new-consumer
option for all consumer based tools. This option is redundant since the new consumer is automatically
used if --bootstrap-server is defined.
- KIP-290 adds the ability
to define ACLs on prefixed resources, e.g. any topic starting with 'foo'.
- KIP-283 improves message down-conversion
handling on Kafka broker, which has typically been a memory-intensive operation. The KIP adds a mechanism by which the operation becomes less memory intensive
by down-converting chunks of partition data at a time which helps put an upper bound on memory consumption. With this improvement, there is a change in
FetchResponse
protocol behavior where the broker could send an oversized message batch towards the end of the response with an invalid offset.
Such oversized messages must be ignored by consumer clients, as is done by KafkaConsumer
.
KIP-283 also adds new topic and broker configurations message.downconversion.enable
and log.message.downconversion.enable
respectively
to control whether down-conversion is enabled. When disabled, broker does not perform any down-conversion and instead sends an UNSUPPORTED_VERSION
error to the client.
- Dynamic broker configuration options can be stored in ZooKeeper using kafka-configs.sh before brokers are started.
This option can be used to avoid storing clear passwords in server.properties as all password configs may be stored encrypted in ZooKeeper.
- ZooKeeper hosts are now re-resolved if connection attempt fails. But if your ZooKeeper host names resolve
to multiple addresses and some of them are not reachable, then you may need to increase the connection timeout
zookeeper.connection.timeout.ms
.
- KIP-279: OffsetsForLeaderEpochResponse v1 introduces a partition-level
leader_epoch
field.
- KIP-219: Bump up the protocol versions of non-cluster action requests and responses that are throttled on quota violation.
- KIP-290: Bump up the protocol versions ACL create, describe and delete requests and responses.
- Upgrading your Streams application from 1.1 to 2.0 does not require a broker upgrade.
A Kafka Streams 2.0 application can connect to 2.0, 1.1, 1.0, 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
- Note that in 2.0 we have removed the public APIs that are deprecated prior to 1.0; users leveraging on those deprecated APIs need to make code changes accordingly.
See Streams API changes in 2.0.0 for more details.
Kafka 1.1.0 introduces wire protocol changes. By following the recommended rolling upgrade plan below,
you guarantee no downtime during the upgrade. However, please review the notable changes in 1.1.0 before upgrading.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0, 1.0).
- log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact
following the upgrade for the details on what this configuration does.)
If you are upgrading from 0.11.0.x or 1.0.x and you have not overridden the message format, then you only need to override
the inter-broker protocol format.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0 or 1.0).
- Upgrade the brokers one at a time: shut down the broker, update the code, and restart it.
- Once the entire cluster is upgraded, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 1.1.
- Restart the brokers one by one for the new protocol version to take effect.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 1.1 on each broker and restart them one by one. Note that the older Scala consumer
does not support the new message format introduced in 0.11, so to avoid the performance cost of down-conversion (or to
take advantage of exactly once semantics), the newer Java consumer must be used.
Additional Upgrade Notes:
- If you are willing to accept downtime, you can simply take all the brokers down, update the code and start them back up. They will start
with the new protocol by default.
- Bumping the protocol version and restarting can be done any time after the brokers are upgraded. It does not have to be immediately after.
Similarly for the message format version.
- If you are using Java8 method references in your Kafka Streams code you might need to update your code to resolve method ambiguties.
Hot-swapping the jar-file only might not work.
- New Kafka Streams configuration parameter
upgrade.from
added that allows rolling bounce upgrade from version 0.10.0.x
- See the Kafka Streams upgrade guide for details about this new config.
- The kafka artifact in Maven no longer depends on log4j or slf4j-log4j12. Similarly to the kafka-clients artifact, users
can now choose the logging back-end by including the appropriate slf4j module (slf4j-log4j12, logback, etc.). The release
tarball still includes log4j and slf4j-log4j12.
- KIP-225 changed the metric "records.lag" to use tags for topic and partition. The original version with the name format "{topic}-{partition}.records-lag" is deprecated and will be removed in 2.0.0.
- Kafka Streams is more robust against broker communication errors. Instead of stopping the Kafka Streams client with a fatal exception,
Kafka Streams tries to self-heal and reconnect to the cluster. Using the new
AdminClient
you have better control of how often
Kafka Streams retries and can configure
fine-grained timeouts (instead of hard coded retries as in older version).
- Kafka Streams rebalance time was reduced further making Kafka Streams more responsive.
- Kafka Connect now supports message headers in both sink and source connectors, and to manipulate them via simple message transforms. Connectors must be changed to explicitly use them. A new
HeaderConverter
is introduced to control how headers are (de)serialized, and the new "SimpleHeaderConverter" is used by default to use string representations of values.
- kafka.tools.DumpLogSegments now automatically sets deep-iteration option if print-data-log is enabled
explicitly or implicitly due to any of the other options like decoder.
- KIP-226 introduced DescribeConfigs Request/Response v1.
- KIP-227 introduced Fetch Request/Response v7.
- Upgrading your Streams application from 1.0 to 1.1 does not require a broker upgrade.
A Kafka Streams 1.1 application can connect to 1.0, 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
- See Streams API changes in 1.1.0 for more details.
Kafka 1.0.0 introduces wire protocol changes. By following the recommended rolling upgrade plan below,
you guarantee no downtime during the upgrade. However, please review the notable changes in 1.0.0 before upgrading.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0).
- log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact
following the upgrade for the details on what this configuration does.)
If you are upgrading from 0.11.0.x and you have not overridden the message format, you must set
both the message format version and the inter-broker protocol version to 0.11.0.
- inter.broker.protocol.version=0.11.0
- log.message.format.version=0.11.0
- Upgrade the brokers one at a time: shut down the broker, update the code, and restart it.
- Once the entire cluster is upgraded, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 1.0.
- Restart the brokers one by one for the new protocol version to take effect.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 1.0 on each broker and restart them one by one. If you are upgrading from
0.11.0 and log.message.format.version is set to 0.11.0, you can update the config and skip the rolling restart.
Note that the older Scala consumer does not support the new message format introduced in 0.11, so to avoid the
performance cost of down-conversion (or to take advantage of exactly once semantics),
the newer Java consumer must be used.
Additional Upgrade Notes:
- If you are willing to accept downtime, you can simply take all the brokers down, update the code and start them back up. They will start
with the new protocol by default.
- Bumping the protocol version and restarting can be done any time after the brokers are upgraded. It does not have to be immediately after.
Similarly for the message format version.
- New Kafka Streams configuration parameter
upgrade.from
added that allows rolling bounce upgrade from version 0.10.0.x
- See the Kafka Streams upgrade guide for details about this new config.
- Restored binary compatibility of AdminClient's Options classes (e.g. CreateTopicsOptions, DeleteTopicsOptions, etc.) with
0.11.0.x. Binary (but not source) compatibility had been broken inadvertently in 1.0.0.
- Topic deletion is now enabled by default, since the functionality is now stable. Users who wish to
to retain the previous behavior should set the broker config
delete.topic.enable
to false
. Keep in mind that topic deletion removes data and the operation is not reversible (i.e. there is no "undelete" operation)
- For topics that support timestamp search if no offset can be found for a partition, that partition is now included in the search result with a null offset value. Previously, the partition was not included in the map.
This change was made to make the search behavior consistent with the case of topics not supporting timestamp search.
- If the
inter.broker.protocol.version
is 1.0 or later, a broker will now stay online to serve replicas
on live log directories even if there are offline log directories. A log directory may become offline due to IOException
caused by hardware failure. Users need to monitor the per-broker metric offlineLogDirectoryCount
to check
whether there is offline log directory.
- Added KafkaStorageException which is a retriable exception. KafkaStorageException will be converted to NotLeaderForPartitionException in the response
- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
32/upgrade.html [450:1287]:
- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
-
KIP-651 adds support
for using PEM files for key and trust stores.
-
KIP-612 adds support
for enforcing broker-wide and per-listener connection create rates. The 2.7.0 release contains
the first part of KIP-612 with dynamic configuration coming in the 2.8.0 release.
-
The ability to throttle topic and partition creations or
topics deletions to prevent a cluster from being harmed via
KIP-599
-
When new features become available in Kafka there are two main issues:
- How do Kafka clients become aware of broker capabilities?
- How does the broker decide which features to enable?
KIP-584
provides a flexible and operationally easy solution for client discovery, feature gating and rolling upgrades using a single restart.
-
The ability to print record offsets and headers with the
ConsoleConsumer
is now possible
via KIP-431
-
The addition of KIP-554
continues progress towards the goal of Zookeeper removal from Kafka. The addition of KIP-554
means you don't have to connect directly to ZooKeeper anymore for managing SCRAM credentials.
- Altering non-reconfigurable configs of existent listeners causes
InvalidRequestException
.
By contrast, the previous (unintended) behavior would have caused the updated configuration to be persisted,
but it wouldn't
take effect until the broker was restarted. See KAFKA-10479 for more discussion.
See DynamicBrokerConfig.DynamicSecurityConfigs
and SocketServer.ListenerReconfigurableConfigs
for the supported reconfigurable configs of existent listeners.
-
Kafka Streams adds support for
Sliding Windows Aggregations
in the KStreams DSL.
-
Reverse iteration over state stores enabling more efficient most recent update searches with
KIP-617
-
End-to-End latency metrics in Kafka Steams see
KIP-613
for more details
-
Kafka Streams added metrics reporting default RocksDB properties with
KIP-607
-
Better Scala implicit Serdes support from
KIP-616
If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets.
Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override
the inter-broker protocol version.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g.,
2.5
, 2.4
, etc.)
- 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.
It is still possible to downgrade at this point if there are any problems.
- Once the cluster's behavior and performance has been verified, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 2.6
.
- Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest
protocol version, it will no longer be possible to downgrade the cluster to an older version.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 2.6 on each broker and restart them one by one. Note that the older Scala clients,
which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs
(or to take advantage of exactly once semantics),
the newer Java clients must be used.
- Kafka Streams adds a new processing mode (requires broker 2.5 or newer) that improves application
scalability using exactly-once guarantees
(cf. KIP-447)
- TLSv1.3 has been enabled by default for Java 11 or newer. The client and server will negotiate TLSv1.3 if
both support it and fallback to TLSv1.2 otherwise. See
KIP-573 for more details.
- The default value for the
client.dns.lookup
configuration has been changed from default
to use_all_dns_ips
. If a hostname resolves to multiple IP addresses, clients and brokers will now
attempt to connect to each IP in sequence until the connection is successfully established. See
KIP-602
for more details.
NotLeaderForPartitionException
has been deprecated and replaced with NotLeaderOrFollowerException
.
Fetch requests and other requests intended only for the leader or follower return NOT_LEADER_OR_FOLLOWER(6) instead of REPLICA_NOT_AVAILABLE(9)
if the broker is not a replica, ensuring that this transient error during reassignments is handled by all clients as a retriable exception.
If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets.
Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override
the inter-broker protocol version.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g.,
2.4
, 2.3
, etc.)
- 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.
It is still possible to downgrade at this point if there are any problems.
- Once the cluster's behavior and performance has been verified, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 2.5
.
- Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest
protocol version, it will no longer be possible to downgrade the cluster to an older version.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 2.5 on each broker and restart them one by one. Note that the older Scala clients,
which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs
(or to take advantage of exactly once semantics),
the newer Java clients must be used.
- There are several notable changes to the reassignment tool
kafka-reassign-partitions.sh
following the completion of
KIP-455.
This tool now requires the --additional
flag to be provided when changing the throttle of an
active reassignment. Reassignment cancellation is now possible using the
--cancel
command. Finally, reassignment with --zookeeper
has been deprecated in favor of --bootstrap-server
. See the KIP for more detail.
- When
RebalanceProtocol#COOPERATIVE
is used, Consumer#poll
can still return data
while it is in the middle of a rebalance for those partitions still owned by the consumer; in addition
Consumer#commitSync
now may throw a non-fatal RebalanceInProgressException
to notify
users of such an event, in order to distinguish from the fatal CommitFailedException
and allow
users to complete the ongoing rebalance and then reattempt committing offsets for those still-owned partitions.
- For improved resiliency in typical network environments, the default value of
zookeeper.session.timeout.ms
has been increased from 6s to 18s and
replica.lag.time.max.ms
from 10s to 30s.
- New DSL operator
cogroup()
has been added for aggregating multiple streams together at once.
- Added a new
KStream.toTable()
API to translate an input event stream into a KTable.
- Added a new Serde type
Void
to represent null keys or null values from input topic.
- Deprecated
UsePreviousTimeOnInvalidTimestamp
and replaced it with UsePartitionTimeOnInvalidTimeStamp
.
- Improved exactly-once semantics by adding a pending offset fencing mechanism and stronger transactional commit
consistency check, which greatly simplifies the implementation of a scalable exactly-once application.
We also added a new exactly-once semantics code example under
examples folder. Check out
KIP-447
for the full details.
- Added a new public api
KafkaStreams.queryMetadataForKey(String, K, Serializer) to get detailed information on the key being queried.
It provides information about the partition number where the key resides in addition to hosts containing the active and standby partitions for the key.
- Provided support to query stale stores (for high availability) and the stores belonging to a specific partition by deprecating
KafkaStreams.store(String, QueryableStoreType)
and replacing it with KafkaStreams.store(StoreQueryParameters)
.
- Added a new public api to access lag information for stores local to an instance with
KafkaStreams.allLocalStorePartitionLags()
.
- Scala 2.11 is no longer supported. See
KIP-531
for details.
- All Scala classes from the package
kafka.security.auth
have been deprecated. See
KIP-504
for details of the new Java authorizer API added in 2.4.0. Note that kafka.security.auth.Authorizer
and kafka.security.auth.SimpleAclAuthorizer
were deprecated in 2.4.0.
- TLSv1 and TLSv1.1 have been disabled by default since these have known security vulnerabilities. Only TLSv1.2 is now
enabled by default. You can continue to use TLSv1 and TLSv1.1 by explicitly enabling these in the configuration options
ssl.protocol
and ssl.enabled.protocols
.
- ZooKeeper has been upgraded to 3.5.7, and a ZooKeeper upgrade from 3.4.X to 3.5.7 can fail if there are no snapshot files in the 3.4 data directory.
This usually happens in test upgrades where ZooKeeper 3.5.7 is trying to load an existing 3.4 data dir in which no snapshot file has been created.
For more details about the issue please refer to ZOOKEEPER-3056.
A fix is given in ZOOKEEPER-3056, which is to set
snapshot.trust.empty=true
config in zookeeper.properties
before the upgrade.
- ZooKeeper version 3.5.7 supports TLS-encrypted connectivity to ZooKeeper both with or without client certificates,
and additional Kafka configurations are available to take advantage of this.
See KIP-515 for details.
If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets.
Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.10.0, 0.11.0, 1.0, 2.0, 2.2).
- log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact
following the upgrade for the details on what this configuration does.)
If you are upgrading from version 0.11.0.x or above, and you have not overridden the message format, then you only need to override
the inter-broker protocol version.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1, 2.0, 2.1, 2.2, 2.3).
- 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.
It is still possible to downgrade at this point if there are any problems.
- Once the cluster's behavior and performance has been verified, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 2.4.
- Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest
protocol version, it will no longer be possible to downgrade the cluster to an older version.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 2.4 on each broker and restart them one by one. Note that the older Scala clients,
which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs
(or to take advantage of exactly once semantics),
the newer Java clients must be used.
Additional Upgrade Notes:
- ZooKeeper has been upgraded to 3.5.6. ZooKeeper upgrade from 3.4.X to 3.5.6 can fail if there are no snapshot files in 3.4 data directory.
This usually happens in test upgrades where ZooKeeper 3.5.6 is trying to load an existing 3.4 data dir in which no snapshot file has been created.
For more details about the issue please refer to ZOOKEEPER-3056.
A fix is given in ZOOKEEPER-3056, which is to set
snapshot.trust.empty=true
config in zookeeper.properties
before the upgrade. But we have observed data loss in standalone cluster upgrades when using
snapshot.trust.empty=true
config. For more details about the issue please refer to ZOOKEEPER-3644.
So we recommend the safe workaround of copying empty snapshot file to the 3.4 data directory,
if there are no snapshot files in 3.4 data directory. For more details about the workaround please refer to ZooKeeper Upgrade FAQ.
-
An embedded Jetty based AdminServer added in ZooKeeper 3.5.
AdminServer is enabled by default in ZooKeeper and is started on port 8080.
AdminServer is disabled by default in the ZooKeeper config (
zookeeper.properties
) provided by the Apache Kafka distribution.
Make sure to update your local zookeeper.properties
file with admin.enableServer=false
if you wish to disable the AdminServer.
Please refer AdminServer config to configure the AdminServer.
- A new Admin API has been added for partition reassignments. Due to changing the way Kafka propagates reassignment information,
it is possible to lose reassignment state in failure edge cases while upgrading to the new version. It is not recommended to start reassignments while upgrading.
- ZooKeeper has been upgraded from 3.4.14 to 3.5.6. TLS and dynamic reconfiguration are supported by the new version.
- The
bin/kafka-preferred-replica-election.sh
command line tool has been deprecated. It has been replaced by bin/kafka-leader-election.sh
.
- The methods
electPreferredLeaders
in the Java AdminClient
class have been deprecated in favor of the methods electLeaders
.
- Scala code leveraging the
NewTopic(String, int, short)
constructor with literal values will need to explicitly call toShort
on the second literal.
- The argument in the constructor
GroupAuthorizationException(String)
is now used to specify an exception message.
Previously it referred to the group that failed authorization. This was done for consistency with other exception types and to
avoid potential misuse. The constructor TopicAuthorizationException(String)
which was previously used for a single
unauthorized topic was changed similarly.
- The internal
PartitionAssignor
interface has been deprecated and replaced with a new ConsumerPartitionAssignor
in the public API. Some
methods/signatures are slightly different between the two interfaces. Users implementing a custom PartitionAssignor should migrate to the new interface as soon as possible.
- The
DefaultPartitioner
now uses a sticky partitioning strategy. This means that records for specific topic with null keys and no assigned partition
will be sent to the same partition until the batch is ready to be sent. When a new batch is created, a new partition is chosen. This decreases latency to produce, but
it may result in uneven distribution of records across partitions in edge cases. Generally users will not be impacted, but this difference may be noticeable in tests and
other situations producing records for a very short amount of time.
- The blocking
KafkaConsumer#committed
methods have been extended to allow a list of partitions as input parameters rather than a single partition.
It enables fewer request/response iterations between clients and brokers fetching for the committed offsets for the consumer group.
The old overloaded functions are deprecated and we would recommend users to make their code changes to leverage the new methods (details
can be found in KIP-520).
- We've introduced a new
INVALID_RECORD
error in the produce response to distinguish from the CORRUPT_MESSAGE
error.
To be more concrete, previously when a batch of records were sent as part of a single request to the broker and one or more of the records failed
the validation due to various causes (mismatch magic bytes, crc checksum errors, null key for log compacted topics, etc), the whole batch would be rejected
with the same and misleading CORRUPT_MESSAGE
, and the caller of the producer client would see the corresponding exception from either
the future object of RecordMetadata
returned from the send
call as well as in the Callback#onCompletion(RecordMetadata metadata, Exception exception)
Now with the new error code and improved error messages of the exception, producer callers would be better informed about the root cause why their sent records were failed.
- We are introducing incremental cooperative rebalancing to the clients' group protocol, which allows consumers to keep all of their assigned partitions during a rebalance
and at the end revoke only those which must be migrated to another consumer for overall cluster balance. The
ConsumerCoordinator
will choose the latest RebalanceProtocol
that is commonly supported by all of the consumer's supported assignors. You can use the new built-in CooperativeStickyAssignor
or plug in your own custom cooperative assignor. To do
so you must implement the ConsumerPartitionAssignor
interface and include RebalanceProtocol.COOPERATIVE
in the list returned by ConsumerPartitionAssignor#supportedProtocols
.
Your custom assignor can then leverage the ownedPartitions
field in each consumer's Subscription
to give partitions back to their previous owners whenever possible. Note that when
a partition is to be reassigned to another consumer, it must be removed from the new assignment until it has been revoked from its original owner. Any consumer that has to revoke a partition will trigger
a followup rebalance to allow the revoked partition to safely be assigned to its new owner. See the
ConsumerPartitionAssignor RebalanceProtocol javadocs for more information.
To upgrade from the old (eager) protocol, which always revokes all partitions before rebalancing, to cooperative rebalancing, you must follow a specific upgrade path to get all clients on the same ConsumerPartitionAssignor
that supports the cooperative protocol. This can be done with two rolling bounces, using the CooperativeStickyAssignor
for the example: during the first one, add "cooperative-sticky" to the list of supported assignors
for each member (without removing the previous assignor -- note that if previously using the default, you must include that explicitly as well). You then bounce and/or upgrade it.
Once the entire group is on 2.4+ and all members have the "cooperative-sticky" among their supported assignors, remove the other assignor(s) and perform a second rolling bounce so that by the end all members support only the
cooperative protocol. For further details on the cooperative rebalancing protocol and upgrade path, see KIP-429.
- There are some behavioral changes to the
ConsumerRebalanceListener
, as well as a new API. Exceptions thrown during any of the listener's three callbacks will no longer be swallowed, and will instead be re-thrown
all the way up to the Consumer.poll()
call. The onPartitionsLost
method has been added to allow users to react to abnormal circumstances where a consumer may have lost ownership of its partitions
(such as a missed rebalance) and cannot commit offsets. By default, this will simply call the existing onPartitionsRevoked
API to align with previous behavior. Note however that onPartitionsLost
will not
be called when the set of lost partitions is empty. This means that no callback will be invoked at the beginning of the first rebalance of a new consumer joining the group.
The semantics of the ConsumerRebalanceListener's
callbacks are further changed when following the cooperative rebalancing protocol described above. In addition to onPartitionsLost
, onPartitionsRevoked
will also never be called when the set of revoked partitions is empty. The callback will generally be invoked only at the end of a rebalance, and only on the set of partitions that are being moved to another consumer. The
onPartitionsAssigned
callback will however always be called, even with an empty set of partitions, as a way to notify users of a rebalance event (this is true for both cooperative and eager). For details on
the new callback semantics, see the ConsumerRebalanceListener javadocs.
- The Scala trait
kafka.security.auth.Authorizer
has been deprecated and replaced with a new Java API
org.apache.kafka.server.authorizer.Authorizer
. The authorizer implementation class
kafka.security.auth.SimpleAclAuthorizer
has also been deprecated and replaced with a new
implementation kafka.security.authorizer.AclAuthorizer
. AclAuthorizer
uses features
supported by the new API to improve authorization logging and is compatible with SimpleAclAuthorizer
.
For more details, see KIP-504.
If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets.
Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0, 1.0, 1.1).
- log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact
following the upgrade for the details on what this configuration does.)
If you are upgrading from 0.11.0.x, 1.0.x, 1.1.x, 2.0.x, or 2.1.x, and you have not overridden the message format, then you only need to override
the inter-broker protocol version.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1, 2.0, 2.1, 2.2).
- 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.
It is still possible to downgrade at this point if there are any problems.
- Once the cluster's behavior and performance has been verified, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 2.3.
- Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest
protocol version, it will no longer be possible to downgrade the cluster to an older version.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 2.3 on each broker and restart them one by one. Note that the older Scala clients,
which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs
(or to take advantage of exactly once semantics),
the newer Java clients must be used.
- We are introducing a new rebalancing protocol for Kafka Connect based on
incremental cooperative rebalancing.
The new protocol does not require stopping all the tasks during a rebalancing phase between Connect workers. Instead, only the tasks that need to be exchanged
between workers are stopped and they are started in a follow up rebalance. The new Connect protocol is enabled by default beginning with 2.3.0.
For more details on how it works and how to enable the old behavior of eager rebalancing, checkout
incremental cooperative rebalancing design.
- We are introducing static membership towards consumer user. This feature reduces unnecessary rebalances during normal application upgrades or rolling bounces.
For more details on how to use it, checkout static membership design.
- Kafka Streams DSL switches its used store types. While this change is mainly transparent to users, there are some corner cases that may require code changes.
See the Kafka Streams upgrade section for more details.
- Kafka Streams 2.3.0 requires 0.11 message format or higher and does not work with older message format.
If you are upgrading from a version prior to 2.1.x, please see the note below about the change to the schema used to store consumer offsets.
Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0, 1.0, 1.1).
- log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact
following the upgrade for the details on what this configuration does.)
If you are upgrading from 0.11.0.x, 1.0.x, 1.1.x, or 2.0.x and you have not overridden the message format, then you only need to override
the inter-broker protocol version.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1, 2.0).
- 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.
It is still possible to downgrade at this point if there are any problems.
- Once the cluster's behavior and performance has been verified, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 2.2.
- Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest
protocol version, it will no longer be possible to downgrade the cluster to an older version.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 2.2 on each broker and restart them one by one. Note that the older Scala clients,
which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs
(or to take advantage of exactly once semantics),
the newer Java clients must be used.
- Kafka Streams 2.2.1 requires 0.11 message format or higher and does not work with older message format.
- The default consumer group id has been changed from the empty string (
""
) to null
. Consumers who use the new default group id will not be able to subscribe to topics,
and fetch or commit offsets. The empty string as consumer group id is deprecated but will be supported until a future major release. Old clients that rely on the empty string group id will now
have to explicitly provide it as part of their consumer config. For more information see
KIP-289.
- The
bin/kafka-topics.sh
command line tool is now able to connect directly to brokers with --bootstrap-server
instead of zookeeper. The old --zookeeper
option is still available for now. Please read KIP-377 for more information.
- Kafka Streams depends on a newer version of RocksDBs that requires MacOS 10.13 or higher.
Note that 2.1.x contains a change to the internal schema used to store consumer offsets. Once the upgrade is
complete, it will not be possible to downgrade to previous versions. See the rolling upgrade notes below for more detail.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0, 1.0, 1.1).
- log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact
following the upgrade for the details on what this configuration does.)
If you are upgrading from 0.11.0.x, 1.0.x, 1.1.x, or 2.0.x and you have not overridden the message format, then you only need to override
the inter-broker protocol version.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1, 2.0).
- 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.
It is still possible to downgrade at this point if there are any problems.
- Once the cluster's behavior and performance has been verified, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 2.1.
- Restart the brokers one by one for the new protocol version to take effect. Once the brokers begin using the latest
protocol version, it will no longer be possible to downgrade the cluster to an older version.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 2.1 on each broker and restart them one by one. Note that the older Scala clients,
which are no longer maintained, do not support the message format introduced in 0.11, so to avoid conversion costs
(or to take advantage of exactly once semantics),
the newer Java clients must be used.
Additional Upgrade Notes:
- Offset expiration semantics has slightly changed in this version. According to the new semantics, offsets of partitions in a group will
not be removed while the group is subscribed to the corresponding topic and is still active (has active consumers). If group becomes
empty all its offsets will be removed after default offset retention period (or the one set by broker) has passed (unless the group becomes
active again). Offsets associated with standalone (simple) consumers, that do not use Kafka group management, will be removed after default
offset retention period (or the one set by broker) has passed since their last commit.
- The default for console consumer's
enable.auto.commit
property when no group.id
is provided is now set to false
.
This is to avoid polluting the consumer coordinator cache as the auto-generated group is not likely to be used by other consumers.
- The default value for the producer's
retries
config was changed to Integer.MAX_VALUE
, as we introduced delivery.timeout.ms
in KIP-91,
which sets an upper bound on the total time between sending a record and receiving acknowledgement from the broker. By default,
the delivery timeout is set to 2 minutes.
- By default, MirrorMaker now overrides
delivery.timeout.ms
to Integer.MAX_VALUE
when
configuring the producer. If you have overridden the value of retries
in order to fail faster,
you will instead need to override delivery.timeout.ms
.
- The
ListGroup
API now expects, as a recommended alternative, Describe Group
access to the groups a user should be able to list.
Even though the old Describe Cluster
access is still supported for backward compatibility, using it for this API is not advised.
- KIP-336 deprecates the ExtendedSerializer and ExtendedDeserializer interfaces and
propagates the usage of Serializer and Deserializer. ExtendedSerializer and ExtendedDeserializer were introduced with
KIP-82 to provide record headers for serializers and deserializers
in a Java 7 compatible fashion. Now we consolidated these interfaces as Java 7 support has been dropped since.
- Jetty has been upgraded to 9.4.12, which excludes TLS_RSA_* ciphers by default because they do not support forward
secrecy, see https://github.com/eclipse/jetty.project/issues/2807 for more information.
- Unclean leader election is automatically enabled by the controller when
unclean.leader.election.enable
config is dynamically updated by using per-topic config override.
- The
AdminClient
has added a method AdminClient#metrics()
. Now any application using the AdminClient
can gain more information
and insight by viewing the metrics captured from the AdminClient
. For more information
see KIP-324
- Kafka now supports Zstandard compression from KIP-110.
You must upgrade the broker as well as clients to make use of it. Consumers prior to 2.1.0 will not be able to read from topics which use
Zstandard compression, so you should not enable it for a topic until all downstream consumers are upgraded. See the KIP for more detail.
Kafka 2.0.0 introduces wire protocol changes. By following the recommended rolling upgrade plan below,
you guarantee no downtime during the upgrade. However, please review the notable changes in 2.0.0 before upgrading.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0, 1.0, 1.1).
- log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact
following the upgrade for the details on what this configuration does.)
If you are upgrading from 0.11.0.x, 1.0.x, or 1.1.x and you have not overridden the message format, then you only need to override
the inter-broker protocol format.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0, 1.0, 1.1).
- Upgrade the brokers one at a time: shut down the broker, update the code, and restart it.
- Once the entire cluster is upgraded, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 2.0.
- Restart the brokers one by one for the new protocol version to take effect.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 2.0 on each broker and restart them one by one. Note that the older Scala consumer
does not support the new message format introduced in 0.11, so to avoid the performance cost of down-conversion (or to
take advantage of exactly once semantics), the newer Java consumer must be used.
Additional Upgrade Notes:
- If you are willing to accept downtime, you can simply take all the brokers down, update the code and start them back up. They will start
with the new protocol by default.
- Bumping the protocol version and restarting can be done any time after the brokers are upgraded. It does not have to be immediately after.
Similarly for the message format version.
- If you are using Java8 method references in your Kafka Streams code you might need to update your code to resolve method ambiguities.
Hot-swapping the jar-file only might not work.
- ACLs should not be added to prefixed resources,
(added in KIP-290),
until all brokers in the cluster have been updated.
NOTE: any prefixed ACLs added to a cluster, even after the cluster is fully upgraded, will be ignored should the cluster be downgraded again.
- KIP-186 increases the default offset retention time from 1 day to 7 days. This makes it less likely to "lose" offsets in an application that commits infrequently. It also increases the active set of offsets and therefore can increase memory usage on the broker. Note that the console consumer currently enables offset commit by default and can be the source of a large number of offsets which this change will now preserve for 7 days instead of 1. You can preserve the existing behavior by setting the broker config
offsets.retention.minutes
to 1440.
- Support for Java 7 has been dropped, Java 8 is now the minimum version required.
- The default value for
ssl.endpoint.identification.algorithm
was changed to https
, which performs hostname verification (man-in-the-middle attacks are possible otherwise). Set ssl.endpoint.identification.algorithm
to an empty string to restore the previous behaviour.
- KAFKA-5674 extends the lower interval of
max.connections.per.ip
minimum to zero and therefore allows IP-based filtering of inbound connections.
- KIP-272
added API version tag to the metric
kafka.network:type=RequestMetrics,name=RequestsPerSec,request={Produce|FetchConsumer|FetchFollower|...}
.
This metric now becomes kafka.network:type=RequestMetrics,name=RequestsPerSec,request={Produce|FetchConsumer|FetchFollower|...},version={0|1|2|3|...}
. This will impact
JMX monitoring tools that do not automatically aggregate. To get the total count for a specific request type, the tool needs to be
updated to aggregate across different versions.
- KIP-225 changed the metric "records.lag" to use tags for topic and partition. The original version with the name format "{topic}-{partition}.records-lag" has been removed.
- The Scala consumers, which have been deprecated since 0.11.0.0, have been removed. The Java consumer has been the recommended option
since 0.10.0.0. Note that the Scala consumers in 1.1.0 (and older) will continue to work even if the brokers are upgraded to 2.0.0.
- The Scala producers, which have been deprecated since 0.10.0.0, have been removed. The Java producer has been the recommended option
since 0.9.0.0. Note that the behaviour of the default partitioner in the Java producer differs from the default partitioner
in the Scala producers. Users migrating should consider configuring a custom partitioner that retains the previous behaviour.
Note that the Scala producers in 1.1.0 (and older) will continue to work even if the brokers are upgraded to 2.0.0.
- MirrorMaker and ConsoleConsumer no longer support the Scala consumer, they always use the Java consumer.
- The ConsoleProducer no longer supports the Scala producer, it always uses the Java producer.
- A number of deprecated tools that rely on the Scala clients have been removed: ReplayLogProducer, SimpleConsumerPerformance, SimpleConsumerShell, ExportZkOffsets, ImportZkOffsets, UpdateOffsetsInZK, VerifyConsumerRebalance.
- The deprecated kafka.tools.ProducerPerformance has been removed, please use org.apache.kafka.tools.ProducerPerformance.
- New Kafka Streams configuration parameter
upgrade.from
added that allows rolling bounce upgrade from older version.
- KIP-284 changed the retention time for Kafka Streams repartition topics by setting its default value to
Long.MAX_VALUE
.
- Updated
ProcessorStateManager
APIs in Kafka Streams for registering state stores to the processor topology. For more details please read the Streams Upgrade Guide.
-
In earlier releases, Connect's worker configuration required the
internal.key.converter
and internal.value.converter
properties.
In 2.0, these are no longer required and default to the JSON converter.
You may safely remove these properties from your Connect standalone and distributed worker configurations:
internal.key.converter=org.apache.kafka.connect.json.JsonConverter
internal.key.converter.schemas.enable=false
internal.value.converter=org.apache.kafka.connect.json.JsonConverter
internal.value.converter.schemas.enable=false
- KIP-266 adds a new consumer configuration
default.api.timeout.ms
to specify the default timeout to use for KafkaConsumer
APIs that could block. The KIP also adds overloads for such blocking
APIs to support specifying a specific timeout to use for each of them instead of using the default timeout set by default.api.timeout.ms
.
In particular, a new poll(Duration)
API has been added which does not block for dynamic partition assignment.
The old poll(long)
API has been deprecated and will be removed in a future version. Overloads have also been added
for other KafkaConsumer
methods like partitionsFor
, listTopics
, offsetsForTimes
,
beginningOffsets
, endOffsets
and close
that take in a Duration
.
- Also as part of KIP-266, the default value of
request.timeout.ms
has been changed to 30 seconds.
The previous value was a little higher than 5 minutes to account for maximum time that a rebalance would take.
Now we treat the JoinGroup request in the rebalance as a special case and use a value derived from
max.poll.interval.ms
for the request timeout. All other request types use the timeout defined
by request.timeout.ms
- The internal method
kafka.admin.AdminClient.deleteRecordsBefore
has been removed. Users are encouraged to migrate to org.apache.kafka.clients.admin.AdminClient.deleteRecords
.
- The AclCommand tool
--producer
convenience option uses the KIP-277 finer grained ACL on the given topic.
- KIP-176 removes
the
--new-consumer
option for all consumer based tools. This option is redundant since the new consumer is automatically
used if --bootstrap-server is defined.
- KIP-290 adds the ability
to define ACLs on prefixed resources, e.g. any topic starting with 'foo'.
- KIP-283 improves message down-conversion
handling on Kafka broker, which has typically been a memory-intensive operation. The KIP adds a mechanism by which the operation becomes less memory intensive
by down-converting chunks of partition data at a time which helps put an upper bound on memory consumption. With this improvement, there is a change in
FetchResponse
protocol behavior where the broker could send an oversized message batch towards the end of the response with an invalid offset.
Such oversized messages must be ignored by consumer clients, as is done by KafkaConsumer
.
KIP-283 also adds new topic and broker configurations message.downconversion.enable
and log.message.downconversion.enable
respectively
to control whether down-conversion is enabled. When disabled, broker does not perform any down-conversion and instead sends an UNSUPPORTED_VERSION
error to the client.
- Dynamic broker configuration options can be stored in ZooKeeper using kafka-configs.sh before brokers are started.
This option can be used to avoid storing clear passwords in server.properties as all password configs may be stored encrypted in ZooKeeper.
- ZooKeeper hosts are now re-resolved if connection attempt fails. But if your ZooKeeper host names resolve
to multiple addresses and some of them are not reachable, then you may need to increase the connection timeout
zookeeper.connection.timeout.ms
.
- KIP-279: OffsetsForLeaderEpochResponse v1 introduces a partition-level
leader_epoch
field.
- KIP-219: Bump up the protocol versions of non-cluster action requests and responses that are throttled on quota violation.
- KIP-290: Bump up the protocol versions ACL create, describe and delete requests and responses.
- Upgrading your Streams application from 1.1 to 2.0 does not require a broker upgrade.
A Kafka Streams 2.0 application can connect to 2.0, 1.1, 1.0, 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
- Note that in 2.0 we have removed the public APIs that are deprecated prior to 1.0; users leveraging on those deprecated APIs need to make code changes accordingly.
See Streams API changes in 2.0.0 for more details.
Kafka 1.1.0 introduces wire protocol changes. By following the recommended rolling upgrade plan below,
you guarantee no downtime during the upgrade. However, please review the notable changes in 1.1.0 before upgrading.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0, 1.0).
- log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact
following the upgrade for the details on what this configuration does.)
If you are upgrading from 0.11.0.x or 1.0.x and you have not overridden the message format, then you only need to override
the inter-broker protocol format.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (0.11.0 or 1.0).
- Upgrade the brokers one at a time: shut down the broker, update the code, and restart it.
- Once the entire cluster is upgraded, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 1.1.
- Restart the brokers one by one for the new protocol version to take effect.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 1.1 on each broker and restart them one by one. Note that the older Scala consumer
does not support the new message format introduced in 0.11, so to avoid the performance cost of down-conversion (or to
take advantage of exactly once semantics), the newer Java consumer must be used.
Additional Upgrade Notes:
- If you are willing to accept downtime, you can simply take all the brokers down, update the code and start them back up. They will start
with the new protocol by default.
- Bumping the protocol version and restarting can be done any time after the brokers are upgraded. It does not have to be immediately after.
Similarly for the message format version.
- If you are using Java8 method references in your Kafka Streams code you might need to update your code to resolve method ambiguties.
Hot-swapping the jar-file only might not work.
- New Kafka Streams configuration parameter
upgrade.from
added that allows rolling bounce upgrade from version 0.10.0.x
- See the Kafka Streams upgrade guide for details about this new config.
- The kafka artifact in Maven no longer depends on log4j or slf4j-log4j12. Similarly to the kafka-clients artifact, users
can now choose the logging back-end by including the appropriate slf4j module (slf4j-log4j12, logback, etc.). The release
tarball still includes log4j and slf4j-log4j12.
- KIP-225 changed the metric "records.lag" to use tags for topic and partition. The original version with the name format "{topic}-{partition}.records-lag" is deprecated and will be removed in 2.0.0.
- Kafka Streams is more robust against broker communication errors. Instead of stopping the Kafka Streams client with a fatal exception,
Kafka Streams tries to self-heal and reconnect to the cluster. Using the new
AdminClient
you have better control of how often
Kafka Streams retries and can configure
fine-grained timeouts (instead of hard coded retries as in older version).
- Kafka Streams rebalance time was reduced further making Kafka Streams more responsive.
- Kafka Connect now supports message headers in both sink and source connectors, and to manipulate them via simple message transforms. Connectors must be changed to explicitly use them. A new
HeaderConverter
is introduced to control how headers are (de)serialized, and the new "SimpleHeaderConverter" is used by default to use string representations of values.
- kafka.tools.DumpLogSegments now automatically sets deep-iteration option if print-data-log is enabled
explicitly or implicitly due to any of the other options like decoder.
- KIP-226 introduced DescribeConfigs Request/Response v1.
- KIP-227 introduced Fetch Request/Response v7.
- Upgrading your Streams application from 1.0 to 1.1 does not require a broker upgrade.
A Kafka Streams 1.1 application can connect to 1.0, 0.11.0, 0.10.2 and 0.10.1 brokers (it is not possible to connect to 0.10.0 brokers though).
- See Streams API changes in 1.1.0 for more details.
Kafka 1.0.0 introduces wire protocol changes. By following the recommended rolling upgrade plan below,
you guarantee no downtime during the upgrade. However, please review the notable changes in 1.0.0 before upgrading.
For a rolling upgrade:
- Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the message format version currently in use. If you have previously
overridden the message format version, you should keep its current value. Alternatively, if you are upgrading from a version prior
to 0.11.0.x, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
- inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0).
- log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See potential performance impact
following the upgrade for the details on what this configuration does.)
If you are upgrading from 0.11.0.x and you have not overridden the message format, you must set
both the message format version and the inter-broker protocol version to 0.11.0.
- inter.broker.protocol.version=0.11.0
- log.message.format.version=0.11.0
- Upgrade the brokers one at a time: shut down the broker, update the code, and restart it.
- Once the entire cluster is upgraded, bump the protocol version by editing
inter.broker.protocol.version
and setting it to 1.0.
- Restart the brokers one by one for the new protocol version to take effect.
- If you have overridden the message format version as instructed above, then you need to do one more rolling restart to
upgrade it to its latest version. Once all (or most) consumers have been upgraded to 0.11.0 or later,
change log.message.format.version to 1.0 on each broker and restart them one by one. If you are upgrading from
0.11.0 and log.message.format.version is set to 0.11.0, you can update the config and skip the rolling restart.
Note that the older Scala consumer does not support the new message format introduced in 0.11, so to avoid the
performance cost of down-conversion (or to take advantage of exactly once semantics),
the newer Java consumer must be used.
Additional Upgrade Notes:
- If you are willing to accept downtime, you can simply take all the brokers down, update the code and start them back up. They will start
with the new protocol by default.
- Bumping the protocol version and restarting can be done any time after the brokers are upgraded. It does not have to be immediately after.
Similarly for the message format version.
- New Kafka Streams configuration parameter
upgrade.from
added that allows rolling bounce upgrade from version 0.10.0.x
- See the Kafka Streams upgrade guide for details about this new config.
- Restored binary compatibility of AdminClient's Options classes (e.g. CreateTopicsOptions, DeleteTopicsOptions, etc.) with
0.11.0.x. Binary (but not source) compatibility had been broken inadvertently in 1.0.0.
- Topic deletion is now enabled by default, since the functionality is now stable. Users who wish to
to retain the previous behavior should set the broker config
delete.topic.enable
to false
. Keep in mind that topic deletion removes data and the operation is not reversible (i.e. there is no "undelete" operation)
- For topics that support timestamp search if no offset can be found for a partition, that partition is now included in the search result with a null offset value. Previously, the partition was not included in the map.
This change was made to make the search behavior consistent with the case of topics not supporting timestamp search.
- If the
inter.broker.protocol.version
is 1.0 or later, a broker will now stay online to serve replicas
on live log directories even if there are offline log directories. A log directory may become offline due to IOException
caused by hardware failure. Users need to monitor the per-broker metric offlineLogDirectoryCount
to check
whether there is offline log directory.
- Added KafkaStorageException which is a retriable exception. KafkaStorageException will be converted to NotLeaderForPartitionException in the response
- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -