Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Update and publish docs for 3.3 #447

Merged
merged 4 commits into from
Sep 28, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
36 changes: 26 additions & 10 deletions 33/design.html
Original file line number Diff line number Diff line change
Expand Up @@ -322,18 +322,33 @@ <h3 class="anchor-heading"><a id="replication" class="anchor-link"></a><a href="
Followers consume messages from the leader just as a normal Kafka consumer would and apply them to their own log. Having the followers pull from the leader has the nice property of allowing the follower to naturally
batch together log entries they are applying to their log.
<p>
As with most distributed systems automatically handling failures requires having a precise definition of what it means for a node to be "alive". For Kafka node liveness has two conditions
As with most distributed systems, automatically handling failures requires a precise definition of what it means for a node to be "alive." In Kafka, a special node
known as the "controller" is responsible for managing the registration of brokers in the cluster. Broker liveness has two conditions:
<ol>
<li>A node must be able to maintain its session with ZooKeeper (via ZooKeeper's heartbeat mechanism)
<li>If it is a follower it must replicate the writes happening on the leader and not fall "too far" behind
<li>Brokers must maintain an active session with the controller in order to receive regular metadata updates.</li>
<li>Brokers acting as followers must replicate the writes from the leader and not fall "too far" behind.</li>
</ol>
We refer to nodes satisfying these two conditions as being "in sync" to avoid the vagueness of "alive" or "failed". The leader keeps track of the set of "in sync" nodes. If a follower dies, gets stuck, or falls
behind, the leader will remove it from the list of in sync replicas. The determination of stuck and lagging replicas is controlled by the replica.lag.time.max.ms configuration.
<p>
What is meant by an "active session" depends on the cluster configuration. For KRaft clusters, an active session is maintained by
sending periodic heartbeats to the controller. If the controller fails to receive a heartbeat before the timeout configured by
<code>broker.session.timeout.ms</code> expires, then the node is considered offline.
<p>
For clusters using Zookeeper, liveness is determined indirectly through the existence of an ephemeral node which is created by the broker on
initialization of its Zookeeper session. If the broker loses its session after failing to send heartbeats to Zookeeper before expiration of
<code>zookeeper.session.timeout.ms</code>, then the node gets deleted. The controller would then notice the node deletion through a Zookeeper watch
and mark the broker offline.
<p>
We refer to nodes satisfying these two conditions as being "in sync" to avoid the vagueness of "alive" or "failed". The leader keeps track of the set of "in sync" replicas,
which is known as the ISR. If either of these conditions fail to be satisified, then the broker will be removed from the ISR. For example,
if a follower dies, then the controller will notice the failure through the loss of its session, and will remove the broker from the ISR.
On the other hand, if the follower lags too far behind the leader but still has an active session, then the leader can also remove it from the ISR.
The determination of lagging replicas is controlled through the <code>replica.lag.time.max.ms</code> configuration.
Replicas that cannot catch up to the end of the log on the leader within the max time set by this configuration are removed from the ISR.
<p>
In distributed systems terminology we only attempt to handle a "fail/recover" model of failures where nodes suddenly cease working and then later recover (perhaps without knowing that they have died). Kafka does not
handle so-called "Byzantine" failures in which nodes produce arbitrary or malicious responses (perhaps due to bugs or foul play).
<p>
We can now more precisely define that a message is considered committed when all in sync replicas for that partition have applied it to their log.
We can now more precisely define that a message is considered committed when all replicas in the ISR for that partition have applied it to their log.
Only committed messages are ever given out to the consumer. This means that the consumer need not worry about potentially seeing a message that could be lost if the leader fails. Producers, on the other hand,
have the option of either waiting for the message to be committed or not, depending on their preference for tradeoff between latency and durability. This preference is controlled by the acks setting that the
producer uses.
Expand Down Expand Up @@ -381,7 +396,7 @@ <h4><a id="design_replicatedlog" href="#design_replicatedlog">Replicated Logs: Q
expensive approach is not used for the data itself.
<p>
Kafka takes a slightly different approach to choosing its quorum set. Instead of majority vote, Kafka dynamically maintains a set of in-sync replicas (ISR) that are caught-up to the leader. Only members of this set
are eligible for election as leader. A write to a Kafka partition is not considered committed until <i>all</i> in-sync replicas have received the write. This ISR set is persisted to ZooKeeper whenever it changes.
are eligible for election as leader. A write to a Kafka partition is not considered committed until <i>all</i> in-sync replicas have received the write. This ISR set is persisted in the cluster metadata whenever it changes.
Because of this, any replica in the ISR is eligible to be elected leader. This is an important factor for Kafka's usage model where there are many partitions and ensuring leadership balance is important.
With this ISR model and <i>f+1</i> replicas, a Kafka topic can tolerate <i>f</i> failures without losing committed messages.
<p>
Expand Down Expand Up @@ -442,9 +457,10 @@ <h4 class="anchor-heading"><a id="design_replicamanagment" class="anchor-link"><
share of its partitions.
<p>
It is also important to optimize the leadership election process as that is the critical window of unavailability. A naive implementation of leader election would end up running an election per partition for all
partitions a node hosted when that node failed. Instead, we elect one of the brokers as the "controller". This controller detects failures at the broker level and is responsible for changing the leader of all
affected partitions in a failed broker. The result is that we are able to batch together many of the required leadership change notifications which makes the election process far cheaper and faster for a large number
of partitions. If the controller fails, one of the surviving brokers will become the new controller.
partitions a node hosted when that node failed. As discussed above in the section on <a href="#replication">replication</a>, Kafka clusters have a special role known as the "controller" which is
responsible for managing the registration of brokers. If the controller detects the failure of a broker, it is responsible for electing one of the remaining members of the ISR to serve as the new leader.
The result is that we are able to batch together many of the required leadership change notifications which makes the election process far cheaper and faster for a large number
of partitions. If the controller itself fails, then another controller will be elected.

<h3 class="anchor-heading"><a id="compaction" class="anchor-link"></a><a href="#compaction">4.8 Log Compaction</a></h3>

Expand Down
4 changes: 2 additions & 2 deletions 33/generated/connect_metrics.html
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
[2022-09-26 10:18:26,810] INFO Metrics scheduler closed (org.apache.kafka.common.metrics.Metrics:693)
[2022-09-26 10:18:26,812] INFO Metrics reporters closed (org.apache.kafka.common.metrics.Metrics:703)
[2022-09-28 16:37:02,145] INFO Metrics scheduler closed (org.apache.kafka.common.metrics.Metrics:693)
[2022-09-28 16:37:02,148] INFO Metrics reporters closed (org.apache.kafka.common.metrics.Metrics:703)
<table class="data-table"><tbody>
<tr>
<td colspan=3 class="mbeanName" style="background-color:#ccc; font-weight: bold;">kafka.connect:type=connect-worker-metrics</td></tr>
Expand Down
119 changes: 117 additions & 2 deletions 33/ops.html
Original file line number Diff line number Diff line change
Expand Up @@ -1269,11 +1269,11 @@ <h3 class="anchor-heading"><a id="java" class="anchor-link"></a><a href="#java">
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>
Expand Down Expand Up @@ -3401,6 +3401,121 @@ <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.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There were some more changes in this file from the following PR apache/kafka#12679
Have those been pushed to the site already?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

did those changes get backported to the 3.3 branch?

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I do see it on the 3.3 branch -- apache/kafka@2e95280

<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 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 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>