cmccabe commented on a change in pull request #10227: URL: https://github.com/apache/kafka/pull/10227#discussion_r585094036
########## File path: KIP-500.md ########## @@ -0,0 +1,131 @@ +KIP-500 Early Access Release +============================ + +# Introduction +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. + +When the Kafka cluster is in self-managed 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 Raft quorum of controller nodes. + +Self-managed mode has many benefits-- some obvious, and some not so obvious. Clearly, it is nice to manage and configure one service rather than two services. In addition, you can now run a single process Kafka cluster. Most important of all, self-managed mode is more scalable. We expect to be able to [support many more topics and partitions](https://www.confluent.io/kafka-summit-san-francisco-2019/kafka-needs-no-keeper/) in this mode. + +# 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 without downtime. 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. + +## Generate a cluster ID +The first step is to generate an ID for your new cluster, using the kafka-storage tool: + +~~~~ +$ ./bin/kafka-storage.sh random-uuid +xtzWWN4bTjitpL3kfd9s5g +~~~~ + +## Format Storage Directories +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/raft-combined.properties +Formatting /tmp/raft-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. + +## Start the Kafka Server +Finally, you are ready to start the Kafka server on each node. + +~~~~ +$ ./bin/kafka-server-start.sh ./config/raft-combined.properties +[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/raft-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) +... +~~~~ + +Just like with a ZooKeeper based broker, you can connect to port 9092 (or whatever port you configured) to perform administrative operations or produce or consume data. + +~~~~ +$ ./bin/kafka-topics.sh --create --topic foo --partitions 1 --replication-factor 1 --bootstrap-server localhost:9092 +Created topic foo. +~~~~ + +# Deployment +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. + +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. + +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. + +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. + +# 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 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. + +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 transactions and exactly-once semantics +* Support for adding partitions to existing topics +* Support for partition reassignment +* Support for some configurations, like enabling unclean leader election by default or dynamically changing broker endpoints +* 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. Review comment: I'll link to CONTRIBUTING.md 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org