mirror of https://github.com/apache/kafka.git
KAFKA-5019; Upgrades notes for idempotent/transactional features and new message format
Author: Jason Gustafson <jason@confluent.io> Reviewers: Ismael Juma <ismael@juma.me.uk> Closes #3212 from hachikuji/KAFKA-5019
This commit is contained in:
parent
f389b71570
commit
8711e5937f
|
@ -20,8 +20,8 @@
|
||||||
<script id="upgrade-template" type="text/x-handlebars-template">
|
<script id="upgrade-template" type="text/x-handlebars-template">
|
||||||
|
|
||||||
<h4><a id="upgrade_11_0_0" href="#upgrade_11_0_0">Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x or 0.10.2.x to 0.11.0.0</a></h4>
|
<h4><a id="upgrade_11_0_0" href="#upgrade_11_0_0">Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x or 0.10.2.x to 0.11.0.0</a></h4>
|
||||||
<p>0.11.0.0 has wire protocol changes. By following the recommended rolling upgrade plan below, you guarantee no downtime during the upgrade.
|
<p>Kafka 0.11.0.0 introduces a new message format version as well as wire protocol changes. By following the recommended rolling upgrade plan below,
|
||||||
However, please review the <a href="#upgrade_1100_notable">notable changes in 0.11.0.0</a> before upgrading.
|
you guarantee no downtime during the upgrade. However, please review the <a href="#upgrade_1100_notable">notable changes in 0.11.0.0</a> before upgrading.
|
||||||
</p>
|
</p>
|
||||||
|
|
||||||
<p>Starting with version 0.10.2, Java clients (producer and consumer) have acquired the ability to communicate with older brokers. Version 0.11.0
|
<p>Starting with version 0.10.2, Java clients (producer and consumer) have acquired the ability to communicate with older brokers. Version 0.11.0
|
||||||
|
@ -32,24 +32,37 @@
|
||||||
<p><b>For a rolling upgrade:</b></p>
|
<p><b>For a rolling upgrade:</b></p>
|
||||||
|
|
||||||
<ol>
|
<ol>
|
||||||
<li> Update server.properties file on all brokers and add the following properties:
|
<li> 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 current message format version currently in use. If you have
|
||||||
|
not overridden the message format previously, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
|
||||||
<ul>
|
<ul>
|
||||||
<li>inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1 or 0.10.2).</li>
|
<li>inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1 or 0.10.2).</li>
|
||||||
<li>log.message.format.version=CURRENT_KAFKA_VERSION (See <a href="#upgrade_10_performance_impact">potential performance impact following the upgrade</a> for the details on what this configuration does.)
|
<li>log.message.format.version=CURRENT_MESSAGE_FORMAT_VERSION (See <a href="#upgrade_10_performance_impact">potential performance impact
|
||||||
|
following the upgrade</a> for the details on what this configuration does.)</li>
|
||||||
</ul>
|
</ul>
|
||||||
</li>
|
</li>
|
||||||
<li> Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. </li>
|
<li> Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. </li>
|
||||||
<li> Once the entire cluster is upgraded, bump the protocol version by editing inter.broker.protocol.version and setting it to 0.11.0. </li>
|
<li> Once the entire cluster is upgraded, bump the protocol version by editing <code>inter.broker.protocol.version</code> and setting it to 0.11.0, but
|
||||||
<li> If your previous message format is 0.10.0, change log.message.format.version to 0.11.0 (this is a no-op as the message format is the same for 0.10.0, 0.10.1, 0.10.2 and 0.11.0).
|
do not change <code>log.message.format.version</code> yet. </li>
|
||||||
If your previous message format version is lower than 0.10.0, do not change log.message.format.version yet - this parameter should only change once all consumers have been upgraded to 0.10.0.0 or later.</li>
|
|
||||||
<li> Restart the brokers one by one for the new protocol version to take effect. </li>
|
<li> Restart the brokers one by one for the new protocol version to take effect. </li>
|
||||||
<li> If log.message.format.version is still lower than 0.10.0 at this point, wait until all consumers have been upgraded to 0.10.0 or later,
|
<li> Once all (or most) consumers have been upgraded to 0.11.0 or later, then change log.message.format.version to 0.11.0 on each
|
||||||
then change log.message.format.version to 0.11.0 on each broker and restart them one by one. </li>
|
broker and restart them one by one. Note that the older Scala consumer does not support the new message format, so to avoid
|
||||||
|
the performance cost of down-conversion (or to take advantage of <a href="#upgrade_11_exactly_once_semantics">exactly once semantics</a>),
|
||||||
|
the new Java consumer must be used.</li>
|
||||||
</ol>
|
</ol>
|
||||||
|
|
||||||
<p><b>Note:</b> If you are willing to accept downtime, you can simply take all the brokers down, update the code and start all of them. They will start with the new protocol by default.
|
<p><b>Additional Upgrade Notes:</b></p>
|
||||||
|
|
||||||
<p><b>Note:</b> Bumping the protocol version and restarting can be done any time after the brokers were upgraded. It does not have to be immediately after.
|
<ol>
|
||||||
|
<li>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.</li>
|
||||||
|
<li>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.</li>
|
||||||
|
<li>It is also possible to enable the 0.11.0 message format on individual topics using the topic admin tool (<code>bin/kafka-topics.sh</code>)
|
||||||
|
prior to updating the global setting <code>log.message.format.version</code>.</li>
|
||||||
|
<li>If you are upgrading from a version prior to 0.10.0, it is NOT necessary to first update the message format to 0.10.0
|
||||||
|
before you switch to 0.11.0.</li>
|
||||||
|
</ol>
|
||||||
|
|
||||||
<h5><a id="upgrade_1100_notable" href="#upgrade_1100_notable">Notable changes in 0.11.0.0</a></h5>
|
<h5><a id="upgrade_1100_notable" href="#upgrade_1100_notable">Notable changes in 0.11.0.0</a></h5>
|
||||||
<ul>
|
<ul>
|
||||||
|
@ -88,6 +101,10 @@
|
||||||
<li>Streams API configuration parameters <code>timestamp.extractor</code>, <code>key.serde</code>, and <code>value.serde</code> were deprecated and
|
<li>Streams API configuration parameters <code>timestamp.extractor</code>, <code>key.serde</code>, and <code>value.serde</code> were deprecated and
|
||||||
replaced by <code>default.timestamp.extractor</code>, <code>default.key.serde</code>, and <code>default.value.serde</code>, respectively.
|
replaced by <code>default.timestamp.extractor</code>, <code>default.key.serde</code>, and <code>default.value.serde</code>, respectively.
|
||||||
</li>
|
</li>
|
||||||
|
<li>For offset commit failures in the Java consumer's <code>commitAsync</code> APIs, we no longer expose the underlying
|
||||||
|
cause when instances of <code>RetriableCommitFailedException</code> are passed to the commit callback. See
|
||||||
|
<a href="https://issues.apache.org/jira/browse/KAFKA-5052">KAFKA-5052</a> for more detail.
|
||||||
|
</li>
|
||||||
</ul>
|
</ul>
|
||||||
|
|
||||||
<h5><a id="upgrade_1100_new_protocols" href="#upgrade_1100_new_protocols">New Protocol Versions</a></h5>
|
<h5><a id="upgrade_1100_new_protocols" href="#upgrade_1100_new_protocols">New Protocol Versions</a></h5>
|
||||||
|
@ -98,6 +115,49 @@
|
||||||
<li> <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-82+-+Add+Record+Headers">KIP-82</a>: FetchResponse v5 introduces an array of <code>header</code> in the message protocol, containing <code>key</code> field and <code>value</code> field.</li>
|
<li> <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-82+-+Add+Record+Headers">KIP-82</a>: FetchResponse v5 introduces an array of <code>header</code> in the message protocol, containing <code>key</code> field and <code>value</code> field.</li>
|
||||||
</ul>
|
</ul>
|
||||||
|
|
||||||
|
<h5><a id="upgrade_11_exactly_once_semantics" href="#upgrade_11_exactly_once_semantics">Notes on Exactly Once Semantics</a></h5>
|
||||||
|
<p>Kafka 0.11.0 includes support for idempotent and transactional capabilities in the producer. Idempotent delivery
|
||||||
|
ensures that messages are delivered exactly once to a particular topic partition during the lifetime of a single producer.
|
||||||
|
Transactional delivery allows producers to send data to multiple partitions such that either all messages are successfully
|
||||||
|
delivered, or none of them are. Together, these capabilities enable "exactly once semantics" in Kafka. More details on these
|
||||||
|
features are available in the user guide, but below we add a few specific notes on enabling them in an upgraded cluster.
|
||||||
|
Note that enabling EoS is not required and there is no impact on the broker's behavior if unused.</p>
|
||||||
|
|
||||||
|
<ol>
|
||||||
|
<li>Only the new Java producer and consumer support exactly once semantics.</li>
|
||||||
|
<li>These features depend crucially on the <a href="#upgrade_11_message_format">0.11.0 message format</a>. Attempting to use them
|
||||||
|
on an older format will result in unsupported version errors.</li>
|
||||||
|
<li>Transaction state is stored in a new internal topic <code>__transaction_state</code>. This topic is not created until the
|
||||||
|
the first attempt to use a transactional request API. Similar to the consumer offsets topic, there are several settings
|
||||||
|
to control the topic's configuration. For example, <code>transaction.state.log.min.isr</code> controls the minimum ISR for
|
||||||
|
this topic. See the configuration section in the user guide for a full list of options.</li>
|
||||||
|
<li>For secure clusters, the transactional APIs require new ACLs which can be turned on with the <code>bin/kafka-acls.sh</code>.
|
||||||
|
tool.</li>
|
||||||
|
<li>EoS in Kafka introduces new request APIs and modifies several existing ones. See
|
||||||
|
<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-ExactlyOnceDeliveryandTransactionalMessaging-RPCProtocolSummary">KIP-98</a>
|
||||||
|
for the full details</code></li>
|
||||||
|
</ol>
|
||||||
|
|
||||||
|
<h5><a id="upgrade_11_message_format" href="#upgrade_11_message_format">Notes on the new message format in 0.11.0</a></h5>
|
||||||
|
<p>The 0.11.0 message format includes several major enhancements in order to support better delivery semantics for the producer
|
||||||
|
(see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging">KIP-98</a>)
|
||||||
|
and improved replication fault tolerance
|
||||||
|
(see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Epoch+rather+than+High+Watermark+for+Truncation">KIP-101</a>).
|
||||||
|
Although the new format contains more information to make these improvements possible, we have made the batch format much
|
||||||
|
more efficient. As long as the number of messages per batch is more than 2, you can expect lower overall overhead. For smaller
|
||||||
|
batches, however, there may be a small performance impact. See <a href="bit.ly/kafka-eos-perf">here</a> for the results of our
|
||||||
|
initial performance analysis of the new message format. You can also find more detail on the message format in the
|
||||||
|
<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-ExactlyOnceDeliveryandTransactionalMessaging-MessageFormat">KIP-98</a> proposal.
|
||||||
|
</p>
|
||||||
|
<p>Most of the discussion on the performance impact of <a href="#upgrade_10_performance_impact">upgrading to the 0.10.0 message format</a>
|
||||||
|
remains pertinent to the 0.11.0 upgrade. This mainly affects clusters that are not secured with TLS since "zero-copy" transfer
|
||||||
|
is already not possible in that case. In order to avoid the cost of down-conversion, you should ensure that consumer applications
|
||||||
|
are upgraded to the latest 0.11.0 client. Significantly, since the old consumer has been deprecated in 0.11.0.0, it does not support
|
||||||
|
the new message format. You must upgrade to use the new consumer to use the new message format without the cost of down-conversion.
|
||||||
|
Note that 0.11.0 consumers support backwards compability with brokers 0.10.0 brokers and upward, so it is possible to upgrade the
|
||||||
|
clients first before the brokers.
|
||||||
|
</p>
|
||||||
|
|
||||||
<h4><a id="upgrade_10_2_0" href="#upgrade_10_2_0">Upgrading from 0.8.x, 0.9.x, 0.10.0.x or 0.10.1.x to 0.10.2.0</a></h4>
|
<h4><a id="upgrade_10_2_0" href="#upgrade_10_2_0">Upgrading from 0.8.x, 0.9.x, 0.10.0.x or 0.10.1.x to 0.10.2.0</a></h4>
|
||||||
<p>0.10.2.0 has wire protocol changes. By following the recommended rolling upgrade plan below, you guarantee no downtime during the upgrade.
|
<p>0.10.2.0 has wire protocol changes. By following the recommended rolling upgrade plan below, you guarantee no downtime during the upgrade.
|
||||||
However, please review the <a href="#upgrade_1020_notable">notable changes in 0.10.2.0</a> before upgrading.
|
However, please review the <a href="#upgrade_1020_notable">notable changes in 0.10.2.0</a> before upgrading.
|
||||||
|
|
Loading…
Reference in New Issue