You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by js...@apache.org on 2022/09/26 18:27:58 UTC

[kafka] branch 3.3 updated: KAFKA-14207; KRaft Operations documentation (#12642)

This is an automated email from the ASF dual-hosted git repository.

jsancio pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
     new 9b8a48ca2a5 KAFKA-14207; KRaft Operations documentation (#12642)
9b8a48ca2a5 is described below

commit 9b8a48ca2a5207204c4a61ae34e2bf4be6f1e61b
Author: José Armando García Sancio <js...@users.noreply.github.com>
AuthorDate: Mon Sep 26 11:19:48 2022 -0700

    KAFKA-14207; KRaft Operations documentation (#12642)
    
    Reviewers: Colin Patrick McCabe <cm...@apache.org>, Chase Thomas <fo...@users.noreply.github.com>
---
 config/kraft/README.md |  15 ++-----
 docs/ops.html          | 119 ++++++++++++++++++++++++++++++++++++++++++++++++-
 2 files changed, 120 insertions(+), 14 deletions(-)

diff --git a/config/kraft/README.md b/config/kraft/README.md
index d280f87958b..c8506551333 100644
--- a/config/kraft/README.md
+++ b/config/kraft/README.md
@@ -1,10 +1,9 @@
-KRaft (aka KIP-500) mode Preview Release
+KRaft (aka KIP-500) mode
 =========================================================
 
 # Introduction
 It is now possible to run Apache Kafka without Apache ZooKeeper!  We call this the [Kafka Raft metadata mode](https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum), typically shortened to `KRaft mode`.
-`KRaft` is intended to be pronounced like `craft` (as in `craftsmanship`). It is currently *PREVIEW AND SHOULD NOT BE USED IN PRODUCTION*, but it
-is available for testing in the Kafka 3.1 release.
+`KRaft` is intended to be pronounced like `craft` (as in `craftsmanship`).
 
 When the Kafka cluster is in KRaft mode, it does not store its metadata in ZooKeeper.  In fact, you do not have to run ZooKeeper at all, because it stores its metadata in a KRaft quorum of controller nodes.
 
@@ -13,10 +12,6 @@ Most important of all, KRaft mode is more scalable.  We expect to be able to [su
 
 # Quickstart
 
-## Warning
-KRaft mode in Kafka 3.1 is provided for testing only, *NOT* for production.  We do not yet support upgrading existing ZooKeeper-based Kafka clusters into this mode.  
-There may be bugs, including serious ones.  You should *assume that your data could be lost at any time* if you try the preview release of KRaft mode.
-
 ## Generate a cluster ID
 The first step is to generate an ID for your new cluster, using the kafka-storage tool:
 
@@ -110,19 +105,15 @@ This is particularly important for the metadata log maintained by the controller
 nothing in the log, which would cause all metadata to be lost.
 
 # Missing Features
-We don't support any kind of upgrade right now, either to or from KRaft mode.  This is an important gap that we are working on.
 
-Finally, the following Kafka features have not yet been fully implemented:
+The following features have not yet been fully implemented:
 
 * Configuring SCRAM users via the administrative API
 * Supporting JBOD configurations with multiple storage directories
 * Modifying certain dynamic configurations on the standalone KRaft controller
-* Support for some configurations, like enabling unclean leader election by default or dynamically changing broker endpoints
 * Delegation tokens
 * Upgrade from ZooKeeper mode
 
-We've tried to make it clear when a feature is not supported in the preview release, but you may encounter some rough edges. We will cover these feature gaps incrementally in the `trunk` branch.
-
 # Debugging
 If you encounter an issue, you might want to take a look at the metadata log.
 
diff --git a/docs/ops.html b/docs/ops.html
index 0b25384e763..9ce051316f4 100644
--- a/docs/ops.html
+++ b/docs/ops.html
@@ -1269,11 +1269,11 @@ $ bin/kafka-acls.sh \
   Java 8, Java 11, and Java 17 are supported. Note that Java 8 support has been deprecated since Apache Kafka 3.0 and will be removed in Apache Kafka 4.0.
   Java 11 and later versions perform significantly better if TLS is enabled, so they are highly recommended (they also include a number of other
   performance improvements: G1GC, CRC32C, Compact Strings, Thread-Local Handshakes and more).
-  
+
   From a security perspective, we recommend the latest released patch version as older freely available versions have disclosed security vulnerabilities.
 
   Typical arguments for running Kafka with OpenJDK-based Java implementations (including Oracle JDK) are:
-  
+
   <pre class="line-numbers"><code class="language-text">  -Xmx6g -Xms6g -XX:MetaspaceSize=96m -XX:+UseG1GC
   -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:G1HeapRegionSize=16M
   -XX:MinMetaspaceFreeRatio=50 -XX:MaxMetaspaceFreeRatio=80 -XX:+ExplicitGCInvokesConcurrent</code></pre>
@@ -3401,6 +3401,121 @@ for built-in state stores, currently we have:
     <li>Don't overbuild the cluster: large clusters, especially in a write heavy usage pattern, means a lot of intracluster communication (quorums on the writes and subsequent cluster member updates), but don't underbuild it (and risk swamping the cluster). Having more servers adds to your read capacity.</li>
   </ul>
   Overall, we try to keep the ZooKeeper system as small as will handle the load (plus standard growth capacity planning) and as simple as possible. We try not to do anything fancy with the configuration or application layout as compared to the official release as well as keep it as self contained as possible. For these reasons, we tend to skip the OS packaged versions, since it has a tendency to try to put things in the OS standard hierarchy, which can be 'messy', for want of a better wa [...]
+
+  <h3 class="anchor-heading"><a id="kraft" class="anchor-link"></a><a href="#kraft">6.10 KRaft</a></h3>
+
+  <h4 class="anchor-heading"><a id="kraft_config" class="anchor-link"></a><a href="#kraft_config">Configuration</a></h4>
+
+  <h5 class="anchor-heading"><a id="kraft_role" class="anchor-link"></a><a href="#kraft_role">Process Roles</a></h5>
+
+  <p>In KRaft mode each Kafka server can be configured as a controller, a broker, or both using the <code>process.roles<code> property. This property can have the following values:</p>
+
+  <ul>
+    <li>If <code>process.roles</code> is set to <code>broker</code>, the server acts as a broker.</li>
+    <li>If <code>process.roles</code> is set to <code>controller</code>, the server acts as a controller.</li>
+    <li>If <code>process.roles</code> is set to <code>broker,controller</code>, the server acts as both a broker and a controller.</li>
+    <li>If <code>process.roles</code> is not set at all, it is assumed to be in ZooKeeper mode.</li>
+  </ul>
+
+  <p>Kafka servers that act as both brokers and controllers are referred to as "combined" servers. Combined servers are simpler to operate for small use cases like a development environment. The key disadvantage is that the controller will be less isolated from the rest of the system. For example, it is not possible to roll or scale the controllers separately from the brokers in combined mode. Combined mode is not recommended in critical deployment environments.</p>
+
+
+  <h5 class="anchor-heading"><a id="kraft_voter" class="anchor-link"></a><a href="#kraft_voter">Controllers</a></h5>
+
+  <p>In KRaft mode, specific Kafka servers are selected to be controllers (unlike the ZooKeeper-based mode, where any server can become the Controller). The servers selected to be controllers will participate in the metadata quorum. Each controller is either an active or a hot standby for the current active controller.</p>
+
+  <p>A Kafka admin will typically select 3 or 5 servers for this role, depending on factors like cost and the number of concurrent failures your system should withstand without availability impact. A majority of the controllers must be alive in order to maintain availability. With 3 controllers, the cluster can tolerate 1 controller failure; with 5 controllers, the cluster can tolerate 2 controller failures.</p>
+
+  <p>All of the servers in a Kafka cluster discover the quorum voters using the <code>controller.quorum.voters</code> property. This identifies the quorum controller servers that should be used. All the controllers must be enumerated. Each controller is identified with their <code>id</code>, <code>host</code> and <code>port</code> information. For example:</p>
+
+  <pre class="line-numbers"><code class="language-bash">controller.quorum.voters=id1@host1:port1,id2@host2:port2,id3@host3:port3</code></pre>
+
+  <p>If a Kafka cluster has 3 controllers named controller1, controller2 and controller3, then controller1 may have the following configuration:</p>
+
+  <pre class="line-numbers"><code class="language-bash">
+process.roles=controller
+node.id=1
+listeners=CONTROLLER://controller1.example.com:9093
+controller.quorum.voters=1@controller1.example.com:9093,2@controller2.example.com:9093,3@controller3.example.com:9093</code></pre>
+
+  <p>Every broker and controller must set the <code>controller.quorum.voters</code> property. The node ID supplied in the <code>controller.quorum.voters</code> property must match the corresponding id on the controller servers. For example, on controller1, node.id must be set to 1, and so forth. Each node ID must be unique across all the servers in a particular cluster. No two servers can have the same node ID regardless of their <code>process.roles<code> values.
+
+  <h4 class="anchor-heading"><a id="kraft_storage" class="anchor-link"></a><a href="#kraft_storage">Storage Tool</a></h4>
+  <p></p>
+  The <code>kafka-storage.sh random-uuid</code> command can be used to generate a cluster ID for your new cluster. This cluster ID must be used when formatting each server in the cluster with the <code>kafka-storage.sh format</code> command.
+
+  <p>This is different from how Kafka has operated in the past. Previously, Kafka would format blank storage directories automatically, and also generate a new cluster ID automatically. One reason for the change is that auto-formatting can sometimes obscure an error condition. This is particularly important for the metadata log maintained by the controller and broker servers. If a majority of the controllers were able to start with an empty log directory, a leader might be able to be ele [...]
+
+  <h4 class="anchor-heading"><a id="kraft_debug" class="anchor-link"></a><a href="#kraft_debug">Debugging</a></h4>
+
+  <h5 class="anchor-heading"><a id="kraft_metadata_tool" class="anchor-link"></a><a href="#kraft_metadata_tool">Metadata Quorum Tool</a></h5>
+
+  <p>The <code>kafka-metadata-quorum</code> tool can be used to describe the runtime state of the cluster metadata partition. For example, the following command displays a summary of the metadata quorum:</p>
+
+  <pre class="line-numbers"><code class="language-bash">  &gt; bin/kafka-metadata-quorum.sh --bootstrap-server  broker_host:port describe --status
+ClusterId:              fMCL8kv1SWm87L_Md-I2hg
+LeaderId:               3002
+LeaderEpoch:            2
+HighWatermark:          10
+MaxFollowerLag:         0
+MaxFollowerLagTimeMs:   -1
+CurrentVoters:          [3000,3001,3002]
+CurrentObservers:       [0,1,2]</code></pre>
+
+  <h5 class="anchor-heading"><a id="kraft_dump_log" class="anchor-link"></a><a href="#kraft_dump_log">Dump Log Tool</a></h5>
+
+  <p>The <code>kafka-dump-log</code> tool can be used to debug the log segments and snapshots for the cluster metadata directory. The tool will scan the provided files and decode the metadata records. For example, this command decodes and prints the records in the first log segment:</p>
+
+  <pre class="line-numbers"><code class="language-bash">  &gt; bin/kafka-dump-log.sh --cluster-metadata-decoder --files metadata_log_dir/__cluster_metadata-0/00000000000000000000.log</code></pre>
+
+  <p>This command decodes and prints the recrods in the a cluster metadata snapshot:</p>
+
+  <pre class="line-numbers"><code class="language-bash">  &gt; bin/kafka-dump-log.sh --cluster-metadata-decoder --files metadata_log_dir/__cluster_metadata-0/00000000000000000100-0000000001.checkpoint</code></pre>
+
+  <h5 class="anchor-heading"><a id="kraft_shell_tool" class="anchor-link"></a><a href="#kraft_shell_tool">Metadata Shell</a></h5>
+
+  <p>The <code>kafka-metadata-shell<code> tool can be used to interactively inspect the state of the cluster metadata partition:</p>
+
+  <pre class="line-numbers"><code class="language-bash">
+  &gt; bin/kafka-metadata-shell.sh  --snapshot metadata_log_dir/__cluster_metadata-0/00000000000000000000.log
+&gt;&gt; ls /
+brokers  local  metadataQuorum  topicIds  topics
+&gt;&gt; ls /topics
+foo
+&gt;&gt; cat /topics/foo/0/data
+{
+  "partitionId" : 0,
+  "topicId" : "5zoAlv-xEh9xRANKXt1Lbg",
+  "replicas" : [ 1 ],
+  "isr" : [ 1 ],
+  "removingReplicas" : null,
+  "addingReplicas" : null,
+  "leader" : 1,
+  "leaderEpoch" : 0,
+  "partitionEpoch" : 0
+}
+&gt;&gt; exit
+  </code></pre>
+
+  <h4 class="anchor-heading"><a id="kraft_deployment" class="anchor-link"></a><a href="#kraft_deployment">Deploying Considerations</a></h4>
+
+  <ul>
+    <li>Kafka server's <code>process.role</code> should be set to either <code>broker</code> or <code>controller</code> but not both. Combined mode can be used in development enviroment but it should be avoided in critical deployment evironments.</li>
+    <li>For redundancy, a Kafka cluster should use 3 controllers. More than 3 servers is not recommended in critical environments. In the rare case of a partial network failure it is possible for the cluster metadata quorum to become unavailable. This limitation will be addresses in a future release of Kafka.</li>
+    <li>The Kafka controllers store all of the metadata for the cluster in memory and on disk. We believe that for a typical Kafka cluster 5GB of main memory and 5GB of disk space on the metadata log director is sufficient.</li>
+
+  <h4 class="anchor-heading"><a id="kraft_deployment" class="anchor-link"></a><a href="#kraft_deployment">Missing Features</a></h4>
+
+  <p>The following features are not fullying implemented in KRaft mode:</p>
+
+  <ul>
+    <li>Configuring SCRAM users via the administrative API</li>
+    <li>Supporting JBOD configurations with multiple storage directories</li>
+    <li>Modifying certain dynamic configurations on the standalone KRaft controller</li>
+    <li>Delegation tokens</li>
+    <li>Upgrade from ZooKeeper mode</li>
+  </ul>
+
 </script>
 
 <div class="p-ops"></div>