Skip to content

Commit 16b2d4f

Browse files
authored
MINOR: Self-managed -> KRaft (Kafka Raft) (#10414)
`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]>
1 parent e971d94 commit 16b2d4f

25 files changed

+109
-100
lines changed

README.md

+3-3
Original file line numberDiff line numberDiff line change
@@ -77,14 +77,14 @@ fail due to code changes. You can just run:
7777

7878
./gradlew processMessages processTestMessages
7979

80-
### Running a Kafka broker with ZooKeeper
80+
### Running a Kafka broker in ZooKeeper mode
8181

8282
./bin/zookeeper-server-start.sh config/zookeeper.properties
8383
./bin/kafka-server-start.sh config/server.properties
8484

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

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

8989
### Cleaning the build ###
9090
./gradlew clean

build.gradle

+1-1
Original file line numberDiff line numberDiff line change
@@ -149,7 +149,7 @@ if (file('.git').exists()) {
149149
'gradlew',
150150
'gradlew.bat',
151151
'gradle/wrapper/gradle-wrapper.properties',
152-
'config/self-managed/README.md',
152+
'config/kraft/README.md',
153153
'TROGDOR.md',
154154
'**/README.md',
155155
'**/id_rsa',

config/self-managed/README.md renamed to config/kraft/README.md

+37-25
Original file line numberDiff line numberDiff line change
@@ -1,17 +1,21 @@
1-
Self-managed mode (aka KIP-500 mode) Early Access Release
1+
KRaft (aka KIP-500) mode Early Access Release
22
=========================================================
33

44
# Introduction
5-
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.
5+
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`.
6+
`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
7+
is available for testing in the Kafka 2.8 release.
68

7-
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.
9+
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.
810

9-
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.
11+
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.
12+
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.
1013

1114
# Quickstart
1215

1316
## Warning
14-
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.
17+
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,
18+
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.
1519

1620
## Generate a cluster ID
1721
The first step is to generate an ID for your new cluster, using the kafka-storage tool:
@@ -25,8 +29,8 @@ xtzWWN4bTjitpL3kfd9s5g
2529
The next step is to format your storage directories. If you are running in single-node mode, you can do this with one command:
2630

2731
~~~~
28-
$ ./bin/kafka-storage.sh format -t <uuid> -c ./config/self-managed/server.properties
29-
Formatting /tmp/self-managed-combined-logs
32+
$ ./bin/kafka-storage.sh format -t <uuid> -c ./config/kraft/server.properties
33+
Formatting /tmp/kraft-combined-logs
3034
~~~~
3135

3236
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 +39,10 @@ If you are using multiple nodes, then you should run the format command on each
3539
Finally, you are ready to start the Kafka server on each node.
3640

3741
~~~~
38-
$ ./bin/kafka-server-start.sh ./config/self-managed/server.properties
42+
$ ./bin/kafka-server-start.sh ./config/kraft/server.properties
3943
[2021-02-26 15:37:11,071] INFO Registered kafka:type=kafka.Log4jController MBean (kafka.utils.Log4jControllerRegistration$)
4044
[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)
41-
[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)
45+
[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)
4246
[2021-02-26 15:37:11,509] INFO [raft-expiration-reaper]: Starting (kafka.raft.TimingWheelExpirationService$ExpiredOperationReaper)
4347
[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)
4448
...
@@ -54,8 +58,8 @@ Created topic foo.
5458
# Deployment
5559

5660
## Controller Servers
57-
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
58-
Controller). The specially selected controller servers will participate in the metadata quorum. Each self-managed controller server is either active, or a hot
61+
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
62+
Controller). The specially selected controller servers will participate in the metadata quorum. Each controller server is either active, or a hot
5963
standby for the current active controller server.
6064

6165
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
@@ -65,16 +69,19 @@ controllers, you can tolerate 1 failure; with 5 controllers, you can tolerate 2
6569
## Process Roles
6670
Each Kafka server now has a new configuration key called `process.roles` which can have the following values:
6771

68-
* If `process.roles` is set to `broker`, the server acts as a self-managed broker.
69-
* If `process.roles` is set to `controller`, the server acts as a self-managed controller.
70-
* If `process.roles` is set to `broker,controller`, the server acts as both a self-managed broker and a self-managed controller.
71-
* 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.
72+
* If `process.roles` is set to `broker`, the server acts as a broker in KRaft mode.
73+
* If `process.roles` is set to `controller`, the server acts as a controller in KRaft mode.
74+
* If `process.roles` is set to `broker,controller`, the server acts as both a broker and a controller in KRaft mode.
75+
* 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.
7276

7377
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
74-
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.
78+
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
79+
memory condition, the controller part of the server is not isolated from that OOM condition.
7580

7681
## Quorum Voters
77-
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.
82+
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.
83+
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`
84+
also has IDs for each node. The format is id1@host1:port1,id2@host2:port2, etc.
7885

7986
So if you have 10 brokers and 3 controllers named controller1, controller2, controller3, you might have the following configuration on controller1:
8087
```
@@ -84,21 +91,26 @@ listeners=CONTROLLER://controller1.example.com:9093
8491
8592
```
8693

87-
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.
94+
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.
95+
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
96+
node IDs is probably just to give each server a numeric ID, starting from 0.
8897

8998
Note that clients never need to configure `controller.quorum.voters`; only servers do.
9099

91100
## Kafka Storage Tool
92101
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.
93102

94-
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.
103+
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
104+
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.
95105

96-
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.
106+
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
107+
nothing in the log, which would cause all metadata to be lost.
97108

98109
# Missing Features
99-
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.
110+
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
111+
completing snapshots for the next release.
100112

101-
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.
113+
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.
102114

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

@@ -119,8 +131,8 @@ If you encounter an issue, you might want to take a look at the metadata log.
119131
One way to view the metadata log is with kafka-dump-log.sh tool, like so:
120132

121133
~~~~
122-
[cmccabe@zeratul kafka3]$ ./bin/kafka-dump-log.sh --cluster-metadata-decoder --skip-record-metadata --files /tmp/self-managed-combined-logs/\@metadata-0/*.log
123-
Dumping /tmp/self-managed-combined-logs/@metadata-0/00000000000000000000.log
134+
$ ./bin/kafka-dump-log.sh --cluster-metadata-decoder --skip-record-metadata --files /tmp/kraft-combined-logs/\@metadata-0/*.log
135+
Dumping /tmp/kraft-combined-logs/@metadata-0/00000000000000000000.log
124136
Starting offset: 0
125137
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
126138
@@ -145,7 +157,7 @@ baseOffset: 7 lastOffset: 8 count: 2 baseSequence: -1 lastSequence: -1 producerI
145157
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.
146158

147159
~~~~
148-
$ ./bin/kafka-metadata-shell.sh --snapshot /tmp/self-managed-combined-logs/\@metadata-0/00000000000000000000.log
160+
$ ./bin/kafka-metadata-shell.sh --snapshot /tmp/kraft-combined-logs/\@metadata-0/00000000000000000000.log
149161
>> ls /
150162
brokers local metadataQuorum topicIds topics
151163
>> ls /topics

config/self-managed/broker.properties renamed to config/kraft/broker.properties

+4-4
Original file line numberDiff line numberDiff line change
@@ -14,13 +14,13 @@
1414
# limitations under the License.
1515

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

2121
############################# Server Basics #############################
2222

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

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

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

7676
# The default number of log partitions per topic. More partitions allow greater
7777
# parallelism for consumption, but this will also result in more files across

config/self-managed/controller.properties renamed to config/kraft/controller.properties

+4-4
Original file line numberDiff line numberDiff line change
@@ -14,13 +14,13 @@
1414
# limitations under the License.
1515

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

2121
############################# Server Basics #############################
2222

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

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

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

7575
# The default number of log partitions per topic. More partitions allow greater
7676
# parallelism for consumption, but this will also result in more files across

config/self-managed/server.properties renamed to config/kraft/server.properties

+4-4
Original file line numberDiff line numberDiff line change
@@ -14,13 +14,13 @@
1414
# limitations under the License.
1515

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

2121
############################# Server Basics #############################
2222

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

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

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

7676
# The default number of log partitions per topic. More partitions allow greater
7777
# parallelism for consumption, but this will also result in more files across

core/src/main/scala/kafka/server/ApiVersionManager.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -99,7 +99,7 @@ class DefaultApiVersionManager(
9999
}
100100

101101
// This is a temporary workaround in order to allow testing of forwarding
102-
// in integration tests. We can remove this after the self-managed controller
102+
// in integration tests. We can remove this after the KRaft controller
103103
// is available for integration testing.
104104
if (forwardingManager.isDefined) {
105105
response.data.apiKeys.add(

core/src/main/scala/kafka/server/BrokerServer.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -51,7 +51,7 @@ import scala.collection.{Map, Seq}
5151
import scala.jdk.CollectionConverters._
5252

5353
/**
54-
* A self-managed Kafka broker.
54+
* A Kafka broker that runs in KRaft (Kafka Raft) mode.
5555
*/
5656
class BrokerServer(
5757
val config: KafkaConfig,

core/src/main/scala/kafka/server/ControllerServer.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -46,7 +46,7 @@ import org.apache.kafka.server.authorizer.Authorizer
4646
import scala.jdk.CollectionConverters._
4747

4848
/**
49-
* A self-managed Kafka controller.
49+
* A Kafka controller that runs in KRaft (Kafka Raft) mode.
5050
*/
5151
class ControllerServer(
5252
val metaProperties: MetaProperties,

core/src/main/scala/kafka/server/KafkaApis.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -3294,7 +3294,7 @@ class KafkaApis(val requestChannel: RequestChannel,
32943294
}
32953295

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

0 commit comments

Comments
 (0)