Skip to content

Commit

Permalink
MINOR: Self-managed -> KRaft (Kafka Raft) (apache#10414)
Browse files Browse the repository at this point in the history
`Self-managed` is also used in the context of Cloud vs on-prem and it can
be confusing.

`KRaft` is a cute combination of `Kafka Raft` and it's pronounced like `craft`
(as in `craftsmanship`).

Reviewers: Colin P. McCabe <[email protected]>, Jose Sancio <[email protected]>, Chia-Ping Tsai <[email protected]>, Ron Dagostino <[email protected]>
  • Loading branch information
ijuma authored Mar 29, 2021
1 parent e971d94 commit 16b2d4f
Show file tree
Hide file tree
Showing 25 changed files with 109 additions and 100 deletions.
6 changes: 3 additions & 3 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -77,14 +77,14 @@ fail due to code changes. You can just run:

./gradlew processMessages processTestMessages

### Running a Kafka broker with ZooKeeper
### Running a Kafka broker in ZooKeeper mode

./bin/zookeeper-server-start.sh config/zookeeper.properties
./bin/kafka-server-start.sh config/server.properties

### Running a Kafka broker in self-managed mode
### Running a Kafka broker in KRaft (Kafka Raft metadata) mode

See [config/self-managed/README.md](https://github.com/apache/kafka/blob/trunk/config/self-managed/README.md).
See [config/kraft/README.md](https://github.com/apache/kafka/blob/trunk/config/kraft/README.md).

### Cleaning the build ###
./gradlew clean
Expand Down
2 changes: 1 addition & 1 deletion build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -149,7 +149,7 @@ if (file('.git').exists()) {
'gradlew',
'gradlew.bat',
'gradle/wrapper/gradle-wrapper.properties',
'config/self-managed/README.md',
'config/kraft/README.md',
'TROGDOR.md',
'**/README.md',
'**/id_rsa',
Expand Down
62 changes: 37 additions & 25 deletions config/self-managed/README.md → config/kraft/README.md
Original file line number Diff line number Diff line change
@@ -1,17 +1,21 @@
Self-managed mode (aka KIP-500 mode) Early Access Release
KRaft (aka KIP-500) mode 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.
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 *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.
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.

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.
KRaft 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, KRaft 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 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.
KRaft 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 KRaft 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 KRaft mode.

## Generate a cluster ID
The first step is to generate an ID for your new cluster, using the kafka-storage tool:
Expand All @@ -25,8 +29,8 @@ xtzWWN4bTjitpL3kfd9s5g
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 <uuid> -c ./config/self-managed/server.properties
Formatting /tmp/self-managed-combined-logs
$ ./bin/kafka-storage.sh format -t <uuid> -c ./config/kraft/server.properties
Formatting /tmp/kraft-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.
Expand All @@ -35,10 +39,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.

~~~~
$ ./bin/kafka-server-start.sh ./config/self-managed/server.properties
$ ./bin/kafka-server-start.sh ./config/kraft/server.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/self-managed-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/kraft-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)
...
Expand All @@ -54,8 +58,8 @@ Created topic foo.
# Deployment

## Controller Servers
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
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.

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
Expand All @@ -65,16 +69,19 @@ controllers, you can tolerate 1 failure; with 5 controllers, you can tolerate 2
## 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-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.
* If `process.roles` is set to `broker`, the server acts as a broker in KRaft mode.
* If `process.roles` is set to `controller`, the server acts as a controller in KRaft mode.
* If `process.roles` is set to `broker,controller`, the server acts as both a broker and a controller in KRaft mode.
* 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 ZooKeeper mode and KRaft mode without reformatting.

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.
some 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.
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 ZooKeeper config, however, `controller.quorum.voters`
also has IDs for each node. The format is id1@host1:port1,id2@host2:port2, etc.

So if you have 10 brokers and 3 controllers named controller1, controller2, controller3, you might have the following configuration on controller1:
```
Expand All @@ -84,21 +91,26 @@ listeners=CONTROLLER://controller1.example.com:9093
[email protected]:9093,[email protected]:9093,[email protected]:9093
```

Each broker and each controller must set `controller.quorum.voters`. Note that the node ID supplied in the controller.quorum.voters configuration must match that supplied to the server. So on controller1, node.id must be set to 1, and so forth. Note that there is no requirement for controller IDs to start at 0 or 1. However, the easiest and least confusing way to allocate node IDs is probably just to give each server a numeric ID, starting from 0.
Each broker and each controller must set `controller.quorum.voters`. Note that the node ID supplied in the `controller.quorum.voters` configuration must match that supplied to the server.
So on controller1, node.id must be set to 1, and so forth. Note that there is no requirement for controller IDs to start at 0 or 1. However, the easiest and least confusing way to allocate
node IDs is probably just to give each server a numeric ID, starting from 0.

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.

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 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.
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 completing 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 important gap that we are working on.
We also don't support any kind of upgrade right now, either to or from KRaft mode. This is another important gap that we are working on.

Finally, the following Kafka features have not yet been fully implemented:

Expand All @@ -119,8 +131,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:

~~~~
[cmccabe@zeratul kafka3]$ ./bin/kafka-dump-log.sh --cluster-metadata-decoder --skip-record-metadata --files /tmp/self-managed-combined-logs/\@metadata-0/*.log
Dumping /tmp/self-managed-combined-logs/@metadata-0/00000000000000000000.log
$ ./bin/kafka-dump-log.sh --cluster-metadata-decoder --skip-record-metadata --files /tmp/kraft-combined-logs/\@metadata-0/*.log
Dumping /tmp/kraft-combined-logs/@metadata-0/00000000000000000000.log
Starting offset: 0
baseOffset: 0 lastOffset: 0 count: 1 baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 1 isTransactional: false isControl: true position: 0 CreateTime: 1614382631640 size: 89 magic: 2 compresscodec: NONE crc: 1438115474 isvalid: true
Expand All @@ -145,7 +157,7 @@ baseOffset: 7 lastOffset: 8 count: 2 baseSequence: -1 lastSequence: -1 producerI
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.

~~~~
$ ./bin/kafka-metadata-shell.sh --snapshot /tmp/self-managed-combined-logs/\@metadata-0/00000000000000000000.log
$ ./bin/kafka-metadata-shell.sh --snapshot /tmp/kraft-combined-logs/\@metadata-0/00000000000000000000.log
>> ls /
brokers local metadataQuorum topicIds topics
>> ls /topics
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,13 +14,13 @@
# limitations under the License.

#
# This configuration file is intended for use in self-managed mode, where
# Apache ZooKeeper is not present. See config/self-managed/README.md for details.
# This configuration file is intended for use in KRaft mode, where
# Apache ZooKeeper is not present. See config/kraft/README.md for details.
#

############################# Server Basics #############################

# The role of this server. Setting this puts us in self-managed mode
# The role of this server. Setting this puts us in KRaft mode
process.roles=broker

# The node id associated with this instance's roles
Expand Down Expand Up @@ -71,7 +71,7 @@ socket.request.max.bytes=104857600
############################# Log Basics #############################

# A comma separated list of directories under which to store log files
log.dirs=/tmp/self-managed-broker-logs
log.dirs=/tmp/kraft-broker-logs

# The default number of log partitions per topic. More partitions allow greater
# parallelism for consumption, but this will also result in more files across
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,13 +14,13 @@
# limitations under the License.

#
# This configuration file is intended for use in self-managed mode, where
# Apache ZooKeeper is not present. See config/self-managed/README.md for details.
# This configuration file is intended for use in KRaft mode, where
# Apache ZooKeeper is not present. See config/kraft/README.md for details.
#

############################# Server Basics #############################

# The role of this server. Setting this puts us in self-managed mode
# The role of this server. Setting this puts us in KRaft mode
process.roles=controller

# The node id associated with this instance's roles
Expand Down Expand Up @@ -70,7 +70,7 @@ socket.request.max.bytes=104857600
############################# Log Basics #############################

# A comma separated list of directories under which to store log files
log.dirs=/tmp/self-managed-controller-logs
log.dirs=/tmp/raft-controller-logs

# The default number of log partitions per topic. More partitions allow greater
# parallelism for consumption, but this will also result in more files across
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,13 +14,13 @@
# limitations under the License.

#
# This configuration file is intended for use in self-managed mode, where
# Apache ZooKeeper is not present. See config/self-managed/README.md for details.
# This configuration file is intended for use in KRaft mode, where
# Apache ZooKeeper is not present. See config/kraft/README.md for details.
#

############################# Server Basics #############################

# The role of this server. Setting this puts us in self-managed mode
# The role of this server. Setting this puts us in KRaft mode
process.roles=broker,controller

# The node id associated with this instance's roles
Expand Down Expand Up @@ -71,7 +71,7 @@ socket.request.max.bytes=104857600
############################# Log Basics #############################

# A comma separated list of directories under which to store log files
log.dirs=/tmp/self-managed-combined-logs
log.dirs=/tmp/kraft-combined-logs

# The default number of log partitions per topic. More partitions allow greater
# parallelism for consumption, but this will also result in more files across
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/kafka/server/ApiVersionManager.scala
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ class DefaultApiVersionManager(
}

// This is a temporary workaround in order to allow testing of forwarding
// in integration tests. We can remove this after the self-managed controller
// in integration tests. We can remove this after the KRaft controller
// is available for integration testing.
if (forwardingManager.isDefined) {
response.data.apiKeys.add(
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/kafka/server/BrokerServer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ import scala.collection.{Map, Seq}
import scala.jdk.CollectionConverters._

/**
* A self-managed Kafka broker.
* A Kafka broker that runs in KRaft (Kafka Raft) mode.
*/
class BrokerServer(
val config: KafkaConfig,
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/kafka/server/ControllerServer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ import org.apache.kafka.server.authorizer.Authorizer
import scala.jdk.CollectionConverters._

/**
* A self-managed Kafka controller.
* A Kafka controller that runs in KRaft (Kafka Raft) mode.
*/
class ControllerServer(
val metaProperties: MetaProperties,
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/kafka/server/KafkaApis.scala
Original file line number Diff line number Diff line change
Expand Up @@ -3294,7 +3294,7 @@ class KafkaApis(val requestChannel: RequestChannel,
}

def handleUnregisterBrokerRequest(request: RequestChannel.Request): Unit = {
// This function will not be called when in self-managed quorum mode, since the
// This function will not be called when in KRaft mode, since the
// UNREGISTER_BROKER API is marked as forwardable and we will always have a forwarding
// manager.
throw new UnsupportedVersionException("The broker unregistration API is not available when using " +
Expand Down
Loading

0 comments on commit 16b2d4f

Please sign in to comment.