Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

Once the VotersRecord operation has been committed by the majority of the new voters set, the leader can respond to the RPC. If the removed voters is the leader, the leader will resign from the quorum when the VotersRecord has been committed. To allow this operation to be committed and for the leader to resign the followers will continue to fetch from the leader even if the leader is not part of the new voters set. In KRaft, leader election is triggered when the voter hasn't received a successful response in the fetch timeout.

Bootstrapping

Endpoints information

Replicas need to be able to connect and send RPCs to each others to implement the KRaft protocolThe section describe how the quorum will get bootstrap to support this KIP. There are two configurations that Kafka will support and each will be explain separately. The Kafka cluster can be configured to use the existing controller.quorum.voters and the new property called controller.quorum.bootstrap.servers.important sets of endpoints that are needed to allow the replicas to connect to each other.

Voters set

The voters set is the collection of replicas and their endpoints that are part that are voters and can become leaders. The voters set is stored in the snapshot and log using the VotersRecord control record. If the partition doesn't contain a VotersRecord control record then the value stored in the controller.quorum.voters property will be used.

This is a static quorum configuration where all of the voters' ID, host and port are specified. An example value for this configuration is 1@localhost:9092,2@localhost:9093,3@localhost:9094 .

If the kraft.version is 0, this properties will be used to configure the set of voters for the KRaft cluster metadata partition.

When the kraft.version is upgraded to a version greater 0 and the version is supported by the voters, the leader will write a control record batch that will include the kraft.version record and all of the VotersRecord control record with all of the voters specified. This will allow KRaft to ignore the controller.quorum.voters properties and instead rely solely on the log state when it is upgraded to kraft.version 1.

controller.quorum.bootstrap.servers

This configuration describe the set of hosts and ports that can be queried to discover the cluster metadata partition leader. Observers and to-be-added voters will send Fetch requests to this list of servers until the leader is discovered.

set of endpoints and replicas will be use by the Vote, BeginQuorumEpoch and EndQuorumEpoch RPCs. In other words replicas will use the voters set to establish leadership and to propagate leadership information to all of the voters.

Bootstrap servers

When the leader is known Fetch requests will be sent to the leader's endpoint. The leader's endpoint can be discovered from many of the KRaft RPCs but typically it will be discover from the BeginQuorumEpoch request for voters (controllers) or the Fetch response for observers (brokers).

In the case that the leader is not known the replica (observer) will send the Fetch RPC to the endpoints in the bootstrap servers configuration. The set of endpoints in the bootstrap servers configuration will come from the controller.quorum.bootstrap.servers configuration property. If that property is not specified the endpoints in controller.quorum.voters will be usedWhen using this configuration for a new cluster, the quorum should be started with only one voter. This voter is bootstrapped by running the storage tool format command. This tool will create the cluster metadata partition and append the VotersRecord to it. Additional voters can get added by using the AddVoter RPC as described in this KIP.

Leader election

It is possible for the leader to add a new voter to the voters set, write the VotersRecord to the log and only replicate it to some of the voters in the new configuration. If the leader fails before this record has been replicated to the new voter it is possible that a new leader cannot be elected. This is because voters reject vote request from replicas that are not in the voters set. This check will be removed and replicas will reply to votes Vote request when the candidate is not in the voters set or the voting replica is not in the voters set. The candidate must still have a longer log (offset and epoch) before the voter will grant a vote to itthe candidate.

Once a leader is elected, leader will propagate this information to all of the voters using the BeginQuorumEpoch RPC. The leader will continue to send the BeginQuorumEpoch requests to a voter if the voter doesn't send a Fetch or FetchSnapshot request within the "check quorum" timeout.

First leader

When a KRaft voter becomes leader it will write a KRaftVersionRecord and VotersRecord to the log if the log or the latest snapshot doesn't contain any VotersRecord. This is done to make sure that the voters set in the bootstrap snapshot gets replicated to all of the voters and to not rely on all of the voters being configured with the same bootstrapped voters set.

...

To improve the usability of this feature it would beneficial for the leader of the KRaft cluster metadata leader to automatically rediscover the voters' endpoints. This makes it possible for the operator to update the endpoint of a voter without having to use the kafka-metadata-quorum tool. When a voter becomes a follower and discovers a new leader, it will always send an UpdateVoter RPC to the leader. This request instructs the leader to update the endpoints of the matching replica id and replica uuid. When at voter becomes a leader it will also write an VotersRecord controler control record with the updated endpoints and kraft.version feature.

...

There only two configurations for this feature.

controller.quorum.voters

This is an existing configuration. This configuration describes the state of the quorum and will only be used if the kraft.version feature is 0.

controller.quorum.bootstrap.servers

This is a list of nodes that brokers and new controllers can use to discover the quorum leader. Brokers and new controllers (observers) will send Fetch requests to all of the nodes in this configuration until they discover the quorum leader and the Fetch request succeeds. The quorum voters and their configuration will be learned by fetching and reading the records from the log and snapshot. This includes committed and uncommitted records.

...