cmccabe commented on code in PR #12642:
URL: https://github.com/apache/kafka/pull/12642#discussion_r971205378


##########
docs/ops.html:
##########
@@ -3180,6 +3180,119 @@ <h4 class="anchor-heading"><a id="zkops" 
class="anchor-link"></a><a href="#zkops
     <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 way to word it.
+
+  <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, as a 
broker or as 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 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>Nodes that act as both brokers and controllers are referred to as 
"combined" nodes. Combined nodes are simpler to operate for simple use cases 
like a development environment. The key disadvantage is that the controller 
will be less isolated from the rest of the system. Combined mode is not 
recommended is 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, only a small group of specially selected servers can act 
as controllers (unlike the ZooKeeper-based mode, where any server can become 
the Controller). The specially selected controller servers will participate in 
the metadata quorum. Each controller server is either active, or a hot standby 
for the current active controller server.</p>
+
+  <p>A Kafka cluster 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. This is an example configuration: 
<code>controller.quorum.voters=id1@host1:port1,id2@host2:port2,id3@host3:port3</code></p>
+
+  <p>If the Kafka cluster has 3 controllers named controller1, controller2 and 
controller3 then controller3 may have the following:</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 nodes in a 
particular cluster. No two nodes 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 node 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 elected with missing committed data.</p>
+
+  <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 display 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 --skip-record-metadat --files 
metadata_log_dir/__cluster_metadata-0/00000000000000000000.log</code></pre>

Review Comment:
   can we leave off `--skip-record-metadata`? I recall it making the output a 
bit weird. also, it's misspelled here.



##########
docs/ops.html:
##########
@@ -3180,6 +3180,119 @@ <h4 class="anchor-heading"><a id="zkops" 
class="anchor-link"></a><a href="#zkops
     <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 way to word it.
+
+  <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, as a 
broker or as 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 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>Nodes that act as both brokers and controllers are referred to as 
"combined" nodes. Combined nodes are simpler to operate for simple use cases 
like a development environment. The key disadvantage is that the controller 
will be less isolated from the rest of the system. Combined mode is not 
recommended is 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, only a small group of specially selected servers can act 
as controllers (unlike the ZooKeeper-based mode, where any server can become 
the Controller). The specially selected controller servers will participate in 
the metadata quorum. Each controller server is either active, or a hot standby 
for the current active controller server.</p>
+
+  <p>A Kafka cluster 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. This is an example configuration: 
<code>controller.quorum.voters=id1@host1:port1,id2@host2:port2,id3@host3:port3</code></p>
+
+  <p>If the Kafka cluster has 3 controllers named controller1, controller2 and 
controller3 then controller3 may have the following:</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 nodes in a 
particular cluster. No two nodes 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 node 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 elected with missing committed data.</p>
+
+  <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 display 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 --skip-record-metadat --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 --skip-record-metadat --files 
metadata_log_dir/__cluster_metadata-0/00000000000000000100-0000000001.checkpoint</code></pre>

Review Comment:
   same.
   
   can we leave off `--skip-record-metadata`? I recall it making the output a 
bit weird. also, it's misspelled here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to