MINOR: Use self-managed mode instead of KIP-500 and nozk (#10362)
KIP-500 is not particularly descriptive. I also tweaked the readme text a bit.
Tested that the readme for self-managed still works after these changes.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ron Dagostino <rdagostino@confluent.io>, Jason Gustafson <jason@confluent.io>
It is now possible to run Apache Kafka without Apache ZooKeeper! We call this mode [self-managed mode](https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum). It is currently *EARLY ACCESS AND SHOULD NOT BE USED IN PRODUCTION*, but it is available for testing in the Kafka 2.8 release.
@ -11,7 +11,7 @@ Self-managed mode has many benefits -- some obvious, and some not so obvious. C
@@ -11,7 +11,7 @@ Self-managed mode has many benefits -- some obvious, and some not so obvious. C
# Quickstart
## Warning
Self-managed mode in Kafka 2.8 is provided for testing only, *NOT* for production. We do not yet support upgrading existing ZooKeeper-based Kafka clusters into this mode. In fact, when Kafka 3.0 is released, it may not even be possible to upgrade your self-managed clusters from 2.8 to 3.0. There may be bugs, including serious ones. You should *assume that your data could be lost at any time* if you try the early access release of KIP-500.
Self-managed mode in Kafka 2.8 is provided for testing only, *NOT* for production. We do not yet support upgrading existing ZooKeeper-based Kafka clusters into this mode. In fact, when Kafka 3.0 is released, it will not be possible to upgrade your self-managed clusters from 2.8 to 3.0. There may be bugs, including serious ones. You should *assume that your data could be lost at any time* if you try the early access release of self-managed mode.
## Generate a cluster ID
The first step is to generate an ID for your new cluster, using the kafka-storage tool:
The next step is to format your storage directories. If you are running in single-node mode, you can do this with one command:
~~~~
$ ./bin/kafka-storage.sh format -t xtzWWN4bTjitpL3kfd9s5g -c ./config/nozk-combined.properties
Formatting /tmp/nozk-combined-logs
$ ./bin/kafka-storage.sh format -t <uuid> -c ./config/self-managed/server.properties
Formatting /tmp/self-managed-combined-logs
~~~~
If you are using multiple nodes, then you should run the format command on each node. Be sure to use the same cluster ID for each one.
@ -35,10 +35,10 @@ If you are using multiple nodes, then you should run the format command on each
@@ -35,10 +35,10 @@ If you are using multiple nodes, then you should run the format command on each
Finally, you are ready to start the Kafka server on each node.
[2021-02-26 15:37:11,071] INFO Registered kafka:type=kafka.Log4jController MBean (kafka.utils.Log4jControllerRegistration$)
[2021-02-26 15:37:11,294] INFO Setting -D jdk.tls.rejectClientInitiatedRenegotiation=true to disable client-initiated TLS renegotiation (org.apache.zookeeper.common.X509Util)
[2021-02-26 15:37:11,466] INFO [Log partition=@metadata-0, dir=/tmp/nozk-combined-logs] Loading producer state till offset 0 with message format version 2 (kafka.log.Log)
[2021-02-26 15:37:11,466] INFO [Log partition=@metadata-0, dir=/tmp/self-managed-combined-logs] Loading producer state till offset 0 with message format version 2 (kafka.log.Log)
[2021-02-26 15:37:11,509] INFO [raft-expiration-reaper]: Starting (kafka.raft.TimingWheelExpirationService$ExpiredOperationReaper)
[2021-02-26 15:37:11,640] INFO [RaftManager nodeId=1] Completed transition to Unattached(epoch=0, voters=[1], electionTimeoutMs=9037) (org.apache.kafka.raft.QuorumState)
...
@ -54,19 +54,24 @@ Created topic foo.
@@ -54,19 +54,24 @@ Created topic foo.
# Deployment
## Controller Servers
Unlike in ZooKeeper-based mode, where any server can become the controller, in self-managed mode, only a small group of specially selected servers can act as controllers. The specially selected controller servers will participate in the metadata quorum. Each KIP-500 controller server is either active, or a hot standby for the current active controller server.
In self-managed 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 self-managed controller server is either active, or a hot
standby for the current active controller server.
Typically you will select either 3 or 5 servers for this role, depending on the size of your cluster. Just like with ZooKeeper, you must keep a majority of the controllers alive in order to maintain availability. So if you have 3 controllers, you can tolerate 1 failure; with 5 controllers, you can tolerate 2 failures.
You 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. Just like with ZooKeeper, you must keep a majority of the controllers alive in order to maintain availability. So if you have 3
controllers, you can tolerate 1 failure; with 5 controllers, you can tolerate 2 failures.
## Process Roles
Each Kafka server now has a new configuration key called `process.roles` which can have the following values:
* If `process.roles` is set to `broker`, the server acts as a self-managed broker.
* If `process.roles` is set to `controller`, the server acts as a self-managed controller.
* If `process.roles` is set to `broker,controller`, the server acts as both a self-managed broker and a self-managd controller.
* If `process.roles` is not set at all then we are assumed to be in ZooKeeper mode. As mentioned earlier, you can't yet transition back and forth between ZK mode and self-managed mode without reformatting.
* If `process.roles` is set to `broker,controller`, the server acts as both a self-managed broker and a self-managed controller.
* If `process.roles` is not set at all then we are assumed to be in ZooKeeper mode. As mentioned earlier, you can't currently transition back and forth between ZK mode and self-managed mode without reformatting.
Nodes that act as both brokers and controllers are referred to as "combined" nodes. The advantage of using combined nodes you will have uses fewer Java Virtual Machines (JVMs). This will allow you to avoid some of the fixed memory overheads associated with JVMs. The disdavantage is that the controller will be less isolated from the rest of the system. For example, if activity on the broker causes an out of memory condition, the controller part of the server is not isolated from that OOM condition.
Nodes that act as both brokers and controllers are referred to as "combined" nodes. Combined nodes are simpler to operate for simple use cases and allow you to avoid
some of the fixed memory overheads associated with JVMs. The key disadvantage is that the controller will be less isolated from the rest of the system. For example, if activity on the broker causes an out of memory condition, the controller part of the server is not isolated from that OOM condition.
## Quorum Voters
All nodes in the system must set the `controller.quorum.voters` configuration. This identifies the quorum controller servers that should be used. All the controllers must be enumerated. This is similar to how, when using ZooKeeper, the `zookeeper.connect` configuration must contain all the ZooKeeper servers. Unlike with the ZK config, however, `controller.quorum.voters` also has IDs for each node. The format is id1@host1:port1,id2@host2:port2, etc.
@ -84,20 +89,20 @@ Each broker and each controller must set `controller.quorum.voters`. Note that
@@ -84,20 +89,20 @@ Each broker and each controller must set `controller.quorum.voters`. Note that
Note that clients never need to configure `controller.quorum.voters`; only servers do.
## Kafka Storage Tool
As described above in the QuickStart section, you must use the kafka-storage.sh tool to generate a cluster ID for your new cluster, and then run the format command on each node before starting the node.
As described above in the QuickStart section, you must use the `kafka-storage.sh` tool to generate a cluster ID for your new cluster, and then run the format command on each node before starting the node.
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 UUID automatically. One reason for the change is that auto-formatting can sometimes obscure an error condition. For example, under UNIX, if a data directory can't be mounted, it may show up as blank. In this case, auto-formatting would be the wrong thing to do.
This is particularly important for the metadata log maintained by the controller servers. If two controllers out of three controllers were able to start with blank logs, a leader might be able to be elected with nothing in the log, which would cause all metadata to be lost.
# Missing Features
We do not yet support generating or loading KIP-630 metadata snapshots. This means that after a while, the time required to restart a broker will become very large. This is a known issue and we are working on implementing snapshots for the next release.
We do not yet support generating or loading KIP-630 metadata snapshots. This means that after a while, the time required to restart a broker will become very large. This is a known issue and we are working on completing snapshots for the next release.
We also don't support any kind of upgrade right now, either to or from self-managed mode. This is another big gap that we are working on.
We also don't support any kind of upgrade right now, either to or from self-managed mode. This is another important gap that we are working on.
Finally, the following Kafka features have not yet been fully implemented:
* Support for security (configuring an Authorizer, setting up SCRAM, delegation tokens, and so forth)
* Support for certain security features: configuring an Authorizer, setting up SCRAM, delegation tokens, and so forth
* Support for transactions and exactly-once semantics
* Support for adding partitions to existing topics
* Support for partition reassignment
@ -105,7 +110,7 @@ Finally, the following Kafka features have not yet been fully implemented:
@@ -105,7 +110,7 @@ Finally, the following Kafka features have not yet been fully implemented:
* Support for KIP-112 "JBOD" modes
* Support for KIP-631 controller metrics
We've tried to make it clear when a feature is not supported in the early access release, but you may encounter some rough edges.
We've tried to make it clear when a feature is not supported in the early access 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.
@ -114,8 +119,8 @@ If you encounter an issue, you might want to take a look at the metadata log.
@@ -114,8 +119,8 @@ If you encounter an issue, you might want to take a look at the metadata log.
One way to view the metadata log is with kafka-dump-log.sh tool, like so:
Another tool for examining the metadata logs is the Kafka metadata shell. Just like the ZooKeeper shell, this allows you to inspect the metadata of the cluster.
valProcessRolesDoc="The roles that this process plays: 'broker', 'controller', or 'broker,controller' if it is both. "+
"This configuration is only for clusters upgraded for KIP-500, which replaces the dependence on Zookeeper with "+
"a self-managed Raft quorum. Leave this config undefined or empty for Zookeeper clusters."
"This configuration is only for clusters in self-managed mode, which rely on a Raft quorum instead of ZooKeeper. Leave this config undefined or empty for Zookeeper clusters."
valInitialBrokerRegistrationTimeoutMsDoc="When initially registering with the controller quorum, the number of milliseconds to wait before declaring failure and exiting the broker process."
valBrokerHeartbeatIntervalMsDoc="The length of time in milliseconds between broker heartbeats. Used when running in KIP-500 mode."
valBrokerSessionTimeoutMsDoc="The length of time in milliseconds that a broker lease lasts if no heartbeats are made. Used when running in KIP-500 mode."
valBrokerHeartbeatIntervalMsDoc="The length of time in milliseconds between broker heartbeats. Used when running in self-managed mode."
valBrokerSessionTimeoutMsDoc="The length of time in milliseconds that a broker lease lasts if no heartbeats are made. Used when running in self-managed mode."
valNodeIdDoc="The node ID associated with the roles this process is playing when `process.roles` is non-empty. "+
"This is required configuration when the self-managed quorum is enabled."
valMetadataLogDirDoc="This configuration determines where we put the metadata log for clusters upgraded to "+
"KIP-500. If it is not set, the metadata log is placed in the first log directory from log.dirs."
valControllerListenerNamesDoc="A comma-separated list of the names of the listeners used by the KIP-500 controller. This is required "+
"if this process is a KIP-500 controller. The ZK-based controller will not use this configuration."
valMetadataLogDirDoc="This configuration determines where we put the metadata log for clusters in self-managed mode. "+
"If it is not set, the metadata log is placed in the first log directory from log.dirs."
valControllerListenerNamesDoc="A comma-separated list of the names of the listeners used by the self-managed controller. This is required "+
"if the process is part of a self-managed cluster. The ZK-based controller will not use this configuration."
valSaslMechanismControllerProtocolDoc="SASL mechanism used for communication with controllers. Default is GSSAPI."