Browse Source

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 <cmccabe@apache.org>, Jose Sancio <jsancio@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Ron Dagostino <rdagostino@confluent.io>
pull/10433/head
Ismael Juma 4 years ago committed by GitHub
parent
commit
16b2d4f3a7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 6
      README.md
  2. 2
      build.gradle
  3. 62
      config/kraft/README.md
  4. 8
      config/kraft/broker.properties
  5. 8
      config/kraft/controller.properties
  6. 8
      config/kraft/server.properties
  7. 2
      core/src/main/scala/kafka/server/ApiVersionManager.scala
  8. 2
      core/src/main/scala/kafka/server/BrokerServer.scala
  9. 2
      core/src/main/scala/kafka/server/ControllerServer.scala
  10. 2
      core/src/main/scala/kafka/server/KafkaApis.scala
  11. 26
      core/src/main/scala/kafka/server/KafkaConfig.scala
  12. 4
      core/src/main/scala/kafka/server/KafkaRaftServer.scala
  13. 2
      core/src/main/scala/kafka/server/KafkaServer.scala
  14. 2
      core/src/main/scala/kafka/server/ReplicaManager.scala
  15. 6
      core/src/main/scala/kafka/tools/StorageTool.scala
  16. 3
      core/src/test/scala/unit/kafka/api/ApiVersionTest.scala
  17. 2
      core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
  18. 4
      core/src/test/scala/unit/kafka/tools/StorageToolTest.scala
  19. 2
      metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
  20. 38
      raft/README.md
  21. 2
      raft/bin/test-kraft-server-start.sh
  22. 0
      raft/config/kraft-log4j.properties
  23. 2
      raft/config/kraft.properties
  24. 12
      tests/kafkatest/services/kafka/quorum.py
  25. 2
      tests/kafkatest/tests/client/client_compatibility_features_test.py

6
README.md

@ -77,14 +77,14 @@ fail due to code changes. You can just run: @@ -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

2
build.gradle

@ -149,7 +149,7 @@ if (file('.git').exists()) { @@ -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',

62
config/self-managed/README.md → config/kraft/README.md

@ -1,17 +1,21 @@ @@ -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:
@ -25,8 +29,8 @@ xtzWWN4bTjitpL3kfd9s5g @@ -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.
@ -35,10 +39,10 @@ If you are using multiple nodes, then you should run the format command on each @@ -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)
...
@ -54,8 +58,8 @@ Created topic foo. @@ -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
@ -65,16 +69,19 @@ controllers, you can tolerate 1 failure; with 5 controllers, you can tolerate 2 @@ -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:
```
@ -84,21 +91,26 @@ listeners=CONTROLLER://controller1.example.com:9093 @@ -84,21 +91,26 @@ listeners=CONTROLLER://controller1.example.com:9093
controller.quorum.voters=1@controller1.example.com:9093,2@controller2.example.com:9093,3@controller3.example.com: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:
@ -119,8 +131,8 @@ If you encounter an issue, you might want to take a look at the metadata log. @@ -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
@ -145,7 +157,7 @@ baseOffset: 7 lastOffset: 8 count: 2 baseSequence: -1 lastSequence: -1 producerI @@ -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

8
config/self-managed/broker.properties → config/kraft/broker.properties

@ -14,13 +14,13 @@ @@ -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
@ -71,7 +71,7 @@ socket.request.max.bytes=104857600 @@ -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

8
config/self-managed/controller.properties → config/kraft/controller.properties

@ -14,13 +14,13 @@ @@ -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
@ -70,7 +70,7 @@ socket.request.max.bytes=104857600 @@ -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

8
config/self-managed/server.properties → config/kraft/server.properties

@ -14,13 +14,13 @@ @@ -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
@ -71,7 +71,7 @@ socket.request.max.bytes=104857600 @@ -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

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

@ -99,7 +99,7 @@ class DefaultApiVersionManager( @@ -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(

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

@ -51,7 +51,7 @@ import scala.collection.{Map, Seq} @@ -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,

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

@ -46,7 +46,7 @@ import org.apache.kafka.server.authorizer.Authorizer @@ -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,

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

@ -3294,7 +3294,7 @@ class KafkaApis(val requestChannel: RequestChannel, @@ -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 " +

26
core/src/main/scala/kafka/server/KafkaConfig.scala

@ -74,7 +74,7 @@ object Defaults { @@ -74,7 +74,7 @@ object Defaults {
val BrokerHeartbeatIntervalMs = 2000
val BrokerSessionTimeoutMs = 9000
/** Self-managed mode configs */
/** KRaft mode configs */
val EmptyNodeId: Int = -1
/************* Authorizer Configuration ***********/
@ -370,7 +370,7 @@ object KafkaConfig { @@ -370,7 +370,7 @@ object KafkaConfig {
val ConnectionSetupTimeoutMsProp = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG
val ConnectionSetupTimeoutMaxMsProp = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG
/** Self-managed mode configs */
/** KRaft mode configs */
val ProcessRolesProp = "process.roles"
val InitialBrokerRegistrationTimeoutMsProp = "initial.broker.registration.timeout.ms"
val BrokerHeartbeatIntervalMsProp = "broker.heartbeat.interval.ms"
@ -663,18 +663,18 @@ object KafkaConfig { @@ -663,18 +663,18 @@ object KafkaConfig {
val ConnectionSetupTimeoutMsDoc = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_DOC
val ConnectionSetupTimeoutMaxMsDoc = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_DOC
/** Self-managed mode configs */
/** KRaft mode configs */
val ProcessRolesDoc = "The roles that this process plays: 'broker', 'controller', or 'broker,controller' if it is both. " +
"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."
"This configuration is only applicable for clusters in KRaft (Kafka Raft) mode (instead of ZooKeeper). Leave this config undefined or empty for Zookeeper clusters."
val InitialBrokerRegistrationTimeoutMsDoc = "When initially registering with the controller quorum, the number of milliseconds to wait before declaring failure and exiting the broker process."
val BrokerHeartbeatIntervalMsDoc = "The length of time in milliseconds between broker heartbeats. Used when running in self-managed mode."
val BrokerSessionTimeoutMsDoc = "The length of time in milliseconds that a broker lease lasts if no heartbeats are made. Used when running in self-managed mode."
val BrokerHeartbeatIntervalMsDoc = "The length of time in milliseconds between broker heartbeats. Used when running in KRaft mode."
val BrokerSessionTimeoutMsDoc = "The length of time in milliseconds that a broker lease lasts if no heartbeats are made. Used when running in KRaft mode."
val NodeIdDoc = "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."
val MetadataLogDirDoc = "This configuration determines where we put the metadata log for clusters in self-managed mode. " +
"This is required configuration when running in KRaft mode."
val MetadataLogDirDoc = "This configuration determines where we put the metadata log for clusters in KRaft mode. " +
"If it is not set, the metadata log is placed in the first log directory from log.dirs."
val ControllerListenerNamesDoc = "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."
val ControllerListenerNamesDoc = "A comma-separated list of the names of the listeners used by the controller. This is required " +
"if running in KRaft mode. The ZK-based controller will not use this configuration."
val SaslMechanismControllerProtocolDoc = "SASL mechanism used for communication with controllers. Default is GSSAPI."
/************* Authorizer Configuration ***********/
@ -1072,7 +1072,7 @@ object KafkaConfig { @@ -1072,7 +1072,7 @@ object KafkaConfig {
.define(ConnectionSetupTimeoutMaxMsProp, LONG, Defaults.ConnectionSetupTimeoutMaxMs, MEDIUM, ConnectionSetupTimeoutMaxMsDoc)
/*
* Self-managed mode configs. Note that these configs are defined as internal. We will make them public in the 3.0.0 release.
* KRaft mode configs. Note that these configs are defined as internal. We will make them public in the 3.0.0 release.
*/
.defineInternal(ProcessRolesProp, LIST, Collections.emptyList(), ValidList.in("broker", "controller"), HIGH, ProcessRolesDoc)
.defineInternal(NodeIdProp, INT, Defaults.EmptyNodeId, null, HIGH, NodeIdDoc)
@ -1906,10 +1906,10 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean, dynamicConfigO @@ -1906,10 +1906,10 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean, dynamicConfigO
require(brokerId >= 0, "broker.id must be greater than or equal to 0")
}
} else {
// Raft-based metadata quorum
// KRaft-based metadata quorum
if (nodeId < 0) {
throw new ConfigException(s"Missing configuration `${KafkaConfig.NodeIdProp}` which is required " +
s"when `process.roles` is defined (i.e. when using the self-managed quorum).")
s"when `process.roles` is defined (i.e. when running in KRaft mode).")
}
}
require(logRollTimeMillis >= 1, "log.roll.ms must be greater than or equal to 1")

4
core/src/main/scala/kafka/server/KafkaRaftServer.scala

@ -34,8 +34,8 @@ import org.apache.kafka.raft.metadata.{MetaLogRaftShim, MetadataRecordSerde} @@ -34,8 +34,8 @@ import org.apache.kafka.raft.metadata.{MetaLogRaftShim, MetadataRecordSerde}
import scala.collection.Seq
/**
* This class implements the self-managed mode server (aka KIP-500) which relies on a
* Raft quorum for maintaining cluster metadata. It is responsible for
* This class implements the KRaft (Kafka Raft) mode server which relies
* on a KRaft quorum for maintaining cluster metadata. It is responsible for
* constructing the controller and/or broker based on the `process.roles`
* configuration and for managing their basic lifecycle (startup and shutdown).
*

2
core/src/main/scala/kafka/server/KafkaServer.scala

@ -284,7 +284,7 @@ class KafkaServer( @@ -284,7 +284,7 @@ class KafkaServer(
// Delay starting processors until the end of the initialization sequence to ensure
// that credentials have been loaded before processing authentications.
//
// Note that we allow the use of self-managed mode controller APIs when forwarding is enabled
// Note that we allow the use of KRaft mode controller APIs when forwarding is enabled
// so that the Envelope request is exposed. This is only used in testing currently.
socketServer = new SocketServer(config, metrics, time, credentialProvider, apiVersionManager)
socketServer.startup(startProcessingRequests = false)

2
core/src/main/scala/kafka/server/ReplicaManager.scala

@ -1851,7 +1851,7 @@ class ReplicaManager(val config: KafkaConfig, @@ -1851,7 +1851,7 @@ class ReplicaManager(val config: KafkaConfig,
if (sendZkNotification)
if (zkClient.isEmpty) {
warn("Unable to propagate log dir failure via Zookeeper in self-managed mode")
warn("Unable to propagate log dir failure via Zookeeper in KRaft mode")
} else {
zkClient.get.propagateLogDirEvent(localBrokerId)
}

6
core/src/main/scala/kafka/tools/StorageTool.scala

@ -74,7 +74,7 @@ object StorageTool extends Logging { @@ -74,7 +74,7 @@ object StorageTool extends Logging {
val ignoreFormatted = namespace.getBoolean("ignore_formatted")
if (!configToSelfManagedMode(config.get)) {
throw new TerseFailure("The kafka configuration file appears to be for " +
"a legacy cluster. Formatting is only supported for clusters in self-managed mode.")
"a legacy cluster. Formatting is only supported for clusters in KRaft mode.")
}
Exit.exit(formatCommand(System.out, directories, metaProperties, ignoreFormatted ))
@ -144,12 +144,12 @@ object StorageTool extends Logging { @@ -144,12 +144,12 @@ object StorageTool extends Logging {
prevMetadata.foreach { prev =>
if (selfManagedMode) {
if (prev.version == 0) {
problems += "The kafka configuration file appears to be for a cluster in self-managed mode, but " +
problems += "The kafka configuration file appears to be for a cluster in KRaft mode, but " +
"the directories are formatted for legacy mode."
}
} else if (prev.version == 1) {
problems += "The kafka configuration file appears to be for a legacy cluster, but " +
"the directories are formatted for a cluster in self-managed mode."
"the directories are formatted for a cluster in KRaft mode."
}
}

3
core/src/test/scala/unit/kafka/api/ApiVersionTest.scala

@ -253,8 +253,7 @@ class ApiVersionTest { @@ -253,8 +253,7 @@ class ApiVersionTest {
ListenerType.ZK_BROKER
)
// Ensure that APIs needed for the self-managed mode (aka KIP-500)
// are not exposed through ApiVersions until we are ready for them
// Ensure that APIs needed for the KRaft mode are not exposed through ApiVersions until we are ready for them
val exposedApis = apiKeysInResponse(response)
assertFalse(exposedApis.contains(ApiKeys.ENVELOPE))
assertFalse(exposedApis.contains(ApiKeys.VOTE))

2
core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala

@ -645,7 +645,7 @@ class KafkaConfigTest { @@ -645,7 +645,7 @@ class KafkaConfigTest {
case KafkaConfig.ConnectionSetupTimeoutMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.ConnectionSetupTimeoutMaxMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
// Self-managed mode configs
// KRaft mode configs
case KafkaConfig.ProcessRolesProp => // ignore
case KafkaConfig.InitialBrokerRegistrationTimeoutMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.BrokerHeartbeatIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")

4
core/src/test/scala/unit/kafka/tools/StorageToolTest.scala

@ -119,7 +119,7 @@ Found problem: @@ -119,7 +119,7 @@ Found problem:
Found metadata: {cluster.id=XcZZOzUqS4yHOjhMQB6JLQ, version=1}
Found problem:
The kafka configuration file appears to be for a legacy cluster, but the directories are formatted for a cluster in self-managed mode.
The kafka configuration file appears to be for a legacy cluster, but the directories are formatted for a cluster in KRaft mode.
""", stream.toString())
} finally Utils.delete(tempDir)
@ -144,7 +144,7 @@ Found problem: @@ -144,7 +144,7 @@ Found problem:
Found metadata: {broker.id=1, cluster.id=26c36907-4158-4a35-919d-6534229f5241, version=0}
Found problem:
The kafka configuration file appears to be for a cluster in self-managed mode, but the directories are formatted for legacy mode.
The kafka configuration file appears to be for a cluster in KRaft mode, but the directories are formatted for legacy mode.
""", stream.toString())
} finally Utils.delete(tempDir)

2
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java

@ -82,7 +82,7 @@ import static java.util.concurrent.TimeUnit.NANOSECONDS; @@ -82,7 +82,7 @@ import static java.util.concurrent.TimeUnit.NANOSECONDS;
/**
* QuorumController implements the main logic of the self-managed controller (aka KIP-500).
* QuorumController implements the main logic of the KRaft (Kafka Raft Metadata) mode controller.
*
* The node which is the leader of the metadata log becomes the active controller. All
* other nodes remain in standby mode. Standby controllers cannot create new metadata log

38
raft/README.md

@ -1,50 +1,48 @@ @@ -1,50 +1,48 @@
Kafka Raft
=================
Kafka Raft is a sub module of Apache Kafka which features a tailored version of
[Raft Consensus Protocol](https://www.usenix.org/system/files/conference/atc14/atc14-paper-ongaro.pdf).
<p>
KRaft (Kafka Raft)
==================
KRaft (Kafka Raft) is a protocol based on the [Raft Consensus Protocol](https://www.usenix.org/system/files/conference/atc14/atc14-paper-ongaro.pdf)
tailored for Apache Kafka.
Eventually this module will be integrated into the Kafka server. For now,
we have a standalone test server which can be used for performance testing.
Below we describe the details to set this up.
This is used by Apache Kafka in the [KRaft (Kafka Raft Metadata) mode](https://github.com/apache/kafka/blob/trunk/config/kraft/README.md). We
also have a standalone test server which can be used for performance testing. We describe the details to set this up below.
### Run Single Quorum ###
bin/test-raft-server-start.sh --config config/raft.properties
bin/test-kraft-server-start.sh --config config/kraft.properties
### Run Multi Node Quorum ###
Create 3 separate raft quorum properties as the following:
Create 3 separate KRaft quorum properties as the following:
`cat << EOF >> config/raft-quorum-1.properties`
`cat << EOF >> config/kraft-quorum-1.properties`
node.id=1
listeners=PLAINTEXT://localhost:9092
controller.quorum.voters=1@localhost:9092,2@localhost:9093,3@localhost:9094
log.dirs=/tmp/raft-logs-1
log.dirs=/tmp/kraft-logs-1
EOF
`cat << EOF >> config/raft-quorum-2.properties`
`cat << EOF >> config/kraft-quorum-2.properties`
node.id=2
listeners=PLAINTEXT://localhost:9093
controller.quorum.voters=1@localhost:9092,2@localhost:9093,3@localhost:9094
log.dirs=/tmp/raft-logs-2
log.dirs=/tmp/kraft-logs-2
EOF
`cat << EOF >> config/raft-quorum-3.properties`
`cat << EOF >> config/kraft-quorum-3.properties`
node.id=3
listeners=PLAINTEXT://localhost:9094
controller.quorum.voters=1@localhost:9092,2@localhost:9093,3@localhost:9094
log.dirs=/tmp/raft-logs-3
log.dirs=/tmp/kraft-logs-3
EOF
Open up 3 separate terminals, and run individual commands:
bin/test-raft-server-start.sh --config config/raft-quorum-1.properties
bin/test-raft-server-start.sh --config config/raft-quorum-2.properties
bin/test-raft-server-start.sh --config config/raft-quorum-3.properties
bin/test-kraft-server-start.sh --config config/kraft-quorum-1.properties
bin/test-kraft-server-start.sh --config config/kraft-quorum-2.properties
bin/test-kraft-server-start.sh --config config/kraft-quorum-3.properties
Once a leader is elected, it will begin writing to an internal
`__cluster_metadata` topic with a steady workload of random data.
You can control the workload using the `--throughput` and `--record-size`
arguments passed to `test-raft-server-start.sh`.
arguments passed to `test-kraft-server-start.sh`.

2
raft/bin/test-raft-server-start.sh → raft/bin/test-kraft-server-start.sh

@ -17,7 +17,7 @@ @@ -17,7 +17,7 @@
base_dir=$(dirname $0)
if [ "x$KAFKA_LOG4J_OPTS" = "x" ]; then
export KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/../config/raft-log4j.properties"
export KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/../config/kraft-log4j.properties"
fi
if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then

0
raft/config/raft-log4j.properties → raft/config/kraft-log4j.properties

2
raft/config/raft.properties → raft/config/kraft.properties

@ -16,4 +16,4 @@ @@ -16,4 +16,4 @@
node.id=0
listeners=PLAINTEXT://localhost:9092
controller.quorum.voters=0@localhost:9092
log.dirs=/tmp/raft-logs
log.dirs=/tmp/kraft-logs

12
tests/kafkatest/services/kafka/quorum.py

@ -15,14 +15,14 @@ @@ -15,14 +15,14 @@
# the types of metadata quorums we support
zk = 'ZK' # ZooKeeper, used before/during the KIP-500 bridge release(s)
colocated_raft = 'COLOCATED_RAFT' # co-located self-managed Controllers, used during/after the KIP-500 bridge release(s)
remote_raft = 'REMOTE_RAFT' # separate self-managed Controllers, used during/after the KIP-500 bridge release(s)
colocated_raft = 'COLOCATED_RAFT' # co-located Controllers in KRaft mode, used during/after the KIP-500 bridge release(s)
remote_raft = 'REMOTE_RAFT' # separate Controllers in KRaft mode, used during/after the KIP-500 bridge release(s)
# How we will parameterize tests that exercise all quorum styles
# [“ZK”, “REMOTE_RAFT”, "COLOCATED_RAFT"] during the KIP-500 bridge release(s)
# [“REMOTE_RAFT”, "COLOCATED_RAFT”] after the KIP-500 bridge release(s)
all = [zk, remote_raft, colocated_raft]
# How we will parameterize tests that exercise all Raft quorum styles
# How we will parameterize tests that exercise all KRaft quorum styles
all_raft = [remote_raft, colocated_raft]
# How we will parameterize tests that are unrelated to upgrades:
# [“ZK”] before the KIP-500 bridge release(s)
@ -43,12 +43,12 @@ class ServiceQuorumInfo: @@ -43,12 +43,12 @@ class ServiceQuorumInfo:
"""
Exposes quorum-related information for a KafkaService
Kafka can use either ZooKeeper or a Raft Controller quorum for its
metadata. Raft Controllers can either be co-located with Kafka in
Kafka can use either ZooKeeper or a KRaft (Kafka Raft) Controller quorum for
its metadata. KRaft Controllers can either be co-located with Kafka in
the same JVM or remote in separate JVMs. The choice is made via
the 'metadata_quorum' parameter defined for the system test: if it
is not explicitly defined, or if it is set to 'ZK', then ZooKeeper
is used. If it is explicitly set to 'COLOCATED_RAFT' then Raft
is used. If it is explicitly set to 'COLOCATED_RAFT' then KRaft
controllers will be co-located with the brokers; the value
`REMOTE_RAFT` indicates remote controllers.

2
tests/kafkatest/tests/client/client_compatibility_features_test.py

@ -134,6 +134,6 @@ class ClientCompatibilityFeaturesTest(Test): @@ -134,6 +134,6 @@ class ClientCompatibilityFeaturesTest(Test):
self.kafka.start()
features = get_broker_features(broker_version)
if not self.zk:
# The self-managed mode doesn't support acls yet, we should remove this once it does
# The KRaft mode doesn't support acls yet, we should remove this once it does
features["describe-acls-supported"] = False
self.invoke_compatibility_program(features)

Loading…
Cancel
Save