KAFKA-17298: Update upgrade notes for 4.0 KIP-848 (#18756)

Reviewers: David Jacot <djacot@confluent.io>
This commit is contained in:
Lianet Magrans 2025-02-13 11:51:56 -05:00 committed by GitHub
parent d08e6a8e58
commit c465cf6b4b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
3 changed files with 94 additions and 4 deletions

View File

@ -121,9 +121,9 @@ public class ConsumerConfig extends AbstractConfig {
*/ */
public static final String GROUP_REMOTE_ASSIGNOR_CONFIG = "group.remote.assignor"; public static final String GROUP_REMOTE_ASSIGNOR_CONFIG = "group.remote.assignor";
public static final String DEFAULT_GROUP_REMOTE_ASSIGNOR = null; public static final String DEFAULT_GROUP_REMOTE_ASSIGNOR = null;
public static final String GROUP_REMOTE_ASSIGNOR_DOC = "The server-side assignor to use. If no assignor is specified, " + public static final String GROUP_REMOTE_ASSIGNOR_DOC = "The name of the server-side assignor to use. " +
"the group coordinator will pick one. This configuration is applied only if <code>group.protocol</code> is " + "If not specified, the group coordinator will pick the first assignor defined in the broker config group.consumer.assignors." +
"set to \"consumer\"."; "This configuration is applied only if <code>group.protocol</code> is set to \"consumer\".";
/** /**
* <code>bootstrap.servers</code> * <code>bootstrap.servers</code>

View File

@ -4056,6 +4056,81 @@ $ bin/kafka-topics.sh --create --topic tieredTopic --bootstrap-server localhost:
<p>For more information, please check <a href="https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Tiered+Storage+GA+Release+Notes">Kafka Tiered Storage GA Release Notes</a>. <p>For more information, please check <a href="https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Tiered+Storage+GA+Release+Notes">Kafka Tiered Storage GA Release Notes</a>.
</p> </p>
<h3 class="anchor-heading"><a id="consumer_rebalance_protocol" class="anchor-link"></a><a href="#consumer_rebalance_protocol">6.10 Consumer Rebalance Protocol</a></h3>
<h4 class="anchor-heading"><a id="consumer_rebalance_protocol_overview" class="anchor-link"></a><a href="#consumer_rebalance_protocol_overview">Overview</a></h4>
<p>Starting from Apache Kafka 4.0, the Next Generation of the Consumer Rebalance Protocol (<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol">KIP-848</a>)
is Generally Available (GA). It improves the scalability of consumer groups while simplifying consumers. It also decreases rebalance times, thanks to
its fully incremental design, which no longer relies on a global synchronization barrier.</p>
<p>Consumer Groups using the new protocol are now referred to as <code>Consumer</code> groups, while groups using the old protocol are referred to as <code>Classic</code> groups. Note
that Classic groups can still be used to form consumer groups using the old protocol.</p>
<h4 class="anchor-heading"><a id="consumer_rebalance_protocol_server" class="anchor-link"></a><a href="#consumer_rebalance_protocol_server">Server</a></h4>
<p>The new consumer protocol is automatically enabled on the server since Apache Kafka 4.0. Enabling and disabling the protocol is controlled by the
<code>group.version</code> feature flag.</p>
<p>The consumer heartbeat interval and the session timeout are controlled by the server now with the following configs:</p>
<ul>
<li><code>group.consumer.heartbeat.interval.ms</code></li>
<li><code>group.consumer.session.timeout.ms</code></li>
</ul>
<p>The assignment strategy is also controlled by the server. The <code>group.consumer.assignors</code> configuration can be used to specify the list of available
assignors for <code>Consumer</code> groups. By default, the <code>uniform</code> assignor and the <code>range</code> assignor are configured. The first assignor
in the list is used by default unless the Consumer selects a different one. It is also possible to implement custom assignment strategies on the server side
by implementing the <code>org.apache.kafka.coordinator.group.api.assignor.ConsumerGroupPartitionAssignor</code> interface and specifying the full class name in the configuration.</p>
<h4 class="anchor-heading"><a id="consumer_rebalance_protocol_consumer" class="anchor-link"></a><a href="#consumer_rebalance_protocol_consumer">Consumer</a></h4>
<p>Since Apache Kafka 4.0, the Consumer supports the new consumer rebalance protocol. However, the protocol is not enabled by default. The <code>group.protocol</code>
configuration must be set to <code>consumer</code> to enable it. When enabled, the new consumer protocol is used alongside an improved threading model.</p>
<p>The <code>group.remote.assignor</code> configuration is introduced as an optional configuration to overwrite the default assignment strategy configured
on the server side.</p>
<p>The <code>subscribe(SubscriptionPattern)</code> and <code>subscribe(SubscriptionPattern, ConsumerRebalanceListener)</code> methods have been added to
subscribe to a regular expression with the new consumer rebalance protocol. With these methods, the regular expression uses the RE2J format and is now evaluated
on the server side.</p>
<p>New metrics have been added to the Consumer when using the new rebalance protocol, mainly providing visibility over the improved threading model.
See <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-1068%3A+New+metrics+for+the+new+KafkaConsumer">New Consumer Metrics</a>.</p>
<p>When the new rebalance protocol is enabled, the following configurations and APIs are no longer usable:</p>
<ul>
<li><code>heartbeat.interval.ms</code></li>
<li><code>session.timeout.ms</code></li>
<li><code>partition.assignment.strategy</code></li>
<li><code>enforceRebalance(String)</code> and <code>enforceRebalance()</code></li>
</ul>
<h4 class="anchor-heading"><a id="consumer_rebalance_protocol_upgrade" class="anchor-link"></a><a href="#consumer_rebalance_protocol_upgrade">Upgrade & Downgrade</a></h4>
<h5 class="anchor-heading"><a id="consumer_rebalance_protocol_upgrade_offline" class="anchor-link"></a><a href="#consumer_rebalance_protocol_upgrade_offline">Offline</a></h5>
<p>Consumer groups are automatically converted from <code>Classic</code> to <code>Consumer</code> and vice versa when they are empty. Hence, it is possible to change the protocol
used by the group by shutting down all the consumers and bringing them back up with the <code>group.protocol=consumer</code> configuration. The downside is that it requires taking
the consumer group down.</p>
<h5 class="anchor-heading"><a id="consumer_rebalance_protocol_upgrade_online" class="anchor-link"></a><a href="#consumer_rebalance_protocol_upgrade_online">Online</a></h5>
<p>Consumer groups can be upgraded without downtime by rolling out the consumer with the <code>group.protocol=consumer</code> configuration. When the first consumer using
the new consumer rebalance protocol joins the group, the group is converted from <code>Classic</code> to <code>Consumer</code>, and the classic rebalance protocol is
interoperated to work with the new consumer rebalance protocol. This is only possible when the classic group uses an assignor that does not embed custom metadata.</p>
<p>Consumer groups can be downgraded using the opposite process. In this case, the group is converted from <code>Consumer</code> to <code>Classic</code> when the last
consumer using the new consumer rebalance protocol leaves the group.</p>
<h4 class="anchor-heading"><a id="consumer_rebalance_protocol_limitations" class="anchor-link"></a><a href="#consumer_rebalance_protocol_limitations">Limitations</a></h4>
<p>While the new consumer rebalance protocol works for most use cases, it is still important to be aware of the following limitations:</p>
<ul>
<li>Client-side assignors are not supported. (see <a href="https://issues.apache.org/jira/browse/KAFKA-18327">KAFKA-18327</a>)</li>
<li>Rack-aware assignment strategies are not fully supported. (see <a href="https://issues.apache.org/jira/browse/KAFKA-17747">KAFKA-17747</a>)</li>
</ul>
</script> </script>
<div class="p-ops"></div> <div class="p-ops"></div>

View File

@ -45,6 +45,21 @@
latest Kafka releases. latest Kafka releases.
<!--#include virtual="zk2kraft.html" --> <!--#include virtual="zk2kraft.html" -->
</li> </li>
<li>
Apache Kafka 4.0 ships with a brand-new group coordinator implementation (See <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=217387038#KIP848:TheNextGenerationoftheConsumerRebalanceProtocol-GroupCoordinator">here</a>.
Functionally speaking, it implements all the same APIs. However, the group coordinator runs in its own dedicated threads.
The number of threads can be defined with the <code>group.coordinator.threads</code> configuration. It defaults to 1 and
must certainly be adjusted for production deployments.
</li>
<li>
The Next Generation of the Consumer Rebalance Protocol (<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol">KIP-848</a>)
is now Generally Available (GA) in Apache Kafka 4.0. The protocol is automatically enabled on the server when the upgrade to 4.0 is finalized.
Note that once the new protocol is used by consumer groups, care must be taken to downgrade to a previous version. Downgrading
to version 3.4.1 or above is safe. Downgrading to an earlier version requires to ensure that all the records in the
<code>__consumer_offsets</code> are removed because those versions cannot handle them. This can be achieved by deleting all the
consumer groups using the new protocol, disabling the new protocol and letting compaction remove the tombstones.
Check <a href="/{{version}}/documentation.html#consumer_rebalance_protocol">here</a> for details.
</li>
<li>A number of deprecated classes, methods, configurations and tools have been removed. <li>A number of deprecated classes, methods, configurations and tools have been removed.
<ul> <ul>
<li><b>Common</b> <li><b>Common</b>
@ -285,7 +300,7 @@
See <a href="https://issues.apache.org/jira/browse/KAFKA-17531">KAFKA-17531</a> for details. See <a href="https://issues.apache.org/jira/browse/KAFKA-17531">KAFKA-17531</a> for details.
</li> </li>
<li> <li>
The most important changes are highlighted in the <a href=""/{{version}}/documentation/streams/upgrade-guide.html#streams_api_changes_400">Kafka Streams upgrade guide</a>. The most important changes are highlighted in the <a href="/{{version}}/documentation/streams/upgrade-guide.html#streams_api_changes_400">Kafka Streams upgrade guide</a>.
</li> </li>
<li> <li>
For a full list of changes, see <a href="https://issues.apache.org/jira/browse/KAFKA-12822">KAFKA-12822</a>. For a full list of changes, see <a href="https://issues.apache.org/jira/browse/KAFKA-12822">KAFKA-12822</a>.