Status
Current state: Under DiscussionAccepted
Discussion thread: here
JIRA:
Jira |
---|
server | ASF JIRA |
---|
columns | key,summary,type,created,updated,due,assignee,reporter,priority,status,resolution |
---|
serverId | 5aa69414-a9e9-3523-82ec-879b028fb15b |
---|
key | KAFKA-8286 |
---|
|
PR: https://github.com/apache/kafka/pull/6686
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
...
Public Interfaces
Network protocol
Request
Added a top level property called ElectionType with supported values of 0 and 1.
Code Block |
---|
|
{
"apiKey": 43,
"type": "request",
"name": "ElectLeadersRequest",
"validVersions": "0-1",
"fields": [
{ "name": "TopicPartitionsElectionType", "type": "[]TopicPartitionsint8", "versions": "0+", "nullableVersions": "01+",
"about": "TheType topicof partitionselections to elect leaders.",
"fields": [
{ "name": "Topic", "type": "string", "versions": "0+",
"about": "The name of a topicconduct for the partition. A value of '0' elects the preferred replica. A value of '1' elects the first live replica if there are no in-sync replica." },
{ "name": "PartitionsTopicPartitions", "type": "[]PartitionsTopicPartitions", "versions": "0+",
"nullableVersions": "0+",
"about": "The topic partitions ofto this topic whose leader should be elected.elect leaders.",
"fields": [
{ "name": "PartitionIdTopic", "type": "int32string", "versions": "0+",
"about": "The name of partitiona idtopic." },
{ "name": "ElectionTypePartitionId", "type": "int8[]int32", "versions": "10+",
"about": "TypeThe partitions of electionsthis totopic whose conductleader forshould thebe partitionelected." A}
value of '0' elects the preferred]
replica. A value of '1' elects the first live replica if there are no in-sync replica." }
},
{ "name": "TimeoutMs", "type": "int32", "versions": "0+", "default": "60000",
"about": "The time in ]
ms to wait for the election to complete." }
]
},
{ "name": "TimeoutMs",} |
Response
Added a top level property called ErrorCode for returning errors that apply to all the topic partitions. The current implementation is using this to return cluster authorization errors.
Code Block |
---|
|
{
"apiKey": 43,
"type": "int32response",
"versionsname": "0+ElectLeadersResponse",
"defaultvalidVersions": "600000-1",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
"about": "The timeduration in msmilliseconds tofor wait forwhich the electionrequest towas complete." }
]
} |
AdminClient Abstract Class
A new method will be added to the AdminClient abstract class to support this new version of the RPC.
Code Block |
---|
|
package org.apache.kafka.clients.admin;
public abstract class AdminClient ... {
...
throttled due to a quota violation, or zero if the request did not violate any quota." },
/**
* ...
* @deprecated Since TBD. Use {@link #electLeaders}.
*/{ "name": "ErrorCode", "type": "int16", "versions": "1+", "ignorable": false,
"about": "The top level response error code." },
@Deprecated
public abstract ElectPreferredLeadersResult electPreferredLeaders(Collection<TopicPartition> partitions{ "name": "ReplicaElectionResults", "type": "[]ReplicaElectionResult", "versions": "0+",
"about": "The election results, or an empty array if the requester did not have permission and the request asks for all partitions.", "fields": [
{ "name": "Topic", "type": "string", "versions": "0+", "entityType": "topicName",
"about": "The topic name" },
{ "name": "PartitionResult", "type": "[]PartitionResult", "versions": "0+",
"about": "The results for each partition", "fields": [
{ "name": "PartitionId", "type": "int32", "versions": "0+",
"about": "The partition id" },
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The result error, or zero if there was no error."},
{ "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The result message, or null if there was no error."}
]}
]}
]
} |
AdminClient Abstract Class
A new method will be added to the AdminClient abstract class to support this new version of the RPC.
When the partition parameter is null the controller will attempt the election type on all of partitions that are legible for election. This means that the RPC will only return success or failure on partition that eligible for election as oppose to all of the partitions in the cluster as the previous implemented behaved.
Code Block |
---|
|
package org.apache.kafka.clients.admin;
public abstract class AdminClient ... {
ElectPreferredLeadersOptions options);
...
/**
* Attempt...
to elect a new leader* for@deprecated each of the topic partition in {@code partitionElections}. The type of elections supported are
* document in theSince 2.4.0. Use {@link TopicPartitionElection#electLeaders} type.
*/
@Deprecated
* Ifpublic {@codeabstract partitionElections} is null, then attempt to elect the preferred replica for all of the partitions.
ElectPreferredLeadersResult electPreferredLeaders(Collection<TopicPartition> partitions,
*
* @param partitionElections The partitions and the type of elections to conduct.
* @param options The options to use when electing the leaders.ElectPreferredLeadersOptions options);
/**
* @return...
*
* @param electionType The type of ElectLeadersResultelection.
*/
@param publicpartitions abstract ElectLeadersResult electLeaders(
The Collection<TopicPartitionElection>partitions partitionElections,
for which to conduct elections.
* ElectLeadersOptions@param options);
} |
TopicPartitionElection Class
Code Block |
---|
|
package org.apache.kafka.common;
public final class TopicPartitionElection {
public static enum ElectionType {
The options to use PREFERRED((byte) 0), UNCLEANED((byte) 1);
when electing the leaders.
* @return public final byte value;
ElectionType(byte value) {
The ElectLeadersResult.
this.value = value;*/
public abstract ElectLeadersResult electLeaders(
}
}
public final TopicPartitionElectionType topicPartition;electionType,
public final ElectionType electionType;
Set<TopicPartition> ...
} |
Admin Command
The command kafka-preferred-replica-election.{sh,bat} will be deprecated and the following command will be added.
partitions,
ElectLeadersOptions options);
} |
ElectLeaderResult Class
Code Block |
---|
final public ElectLeadersResult {
...
public KafkaFuture<Map<TopicPartition, Optional<Throwable>>> partitions() {
|
Code Block |
---|
|
$ bin/kafka-leader-election.sh --help
This tool attempts to elect a new leader for a set of topic partitions. The type of elections supported are preferred replicas and unclean replica.
Option ...
}
public KafkFuture<Void> all() {
...
}
} |
ElectionType Enumeration
Code Block |
---|
|
package org.apache.kafka.common;
public enum ElectionType Description
------{
PREFERRED((byte) 0), UNCLEAN((byte) 1);
public final byte value;
ElectionType(byte value) {
this.value = value;
}
}
|
Admin Command
The command kafka-preferred-replica-election.{sh,bat} will be deprecated and the following command will be added.
Code Block |
---|
|
$ bin/kafka-leader-election.sh --help
This tool attempts to elect a new leader for a set of topic partitions. The type of elections supported are preferred replicas and unclean replica.
Option---------
--admin.config <String: config file> Admin client config properties file to
pass to the admin client when --
Description
------ bootstrap-server is given.
--bootstrap-server <String: host:port> A host name and port for the broker to
-----------
--admin.config <String: config file> Admin client config properties file to pass to the admin
connect to, in the form host:port.
client when --bootstrap-server is given.
--all-topic-partitions Perform election on all of the Multiple comma-separated URLs can betopic partitions. Not allowed if
given. REQUIRED unless --zookeeper
--topic, --partition or --path-to-json-file is specified.
--bootstrap-server <String: host:port> A host name and port for the broker to
is given.
--election-type <String: election> Type of electionconnect to attempt. Possible values are 0
, in the form host:port.
Multiple comma-separated URLs can (or "preferred") for preferred election or 1 (or
be
given. REQUIRED unless --zookeeper
"uncleaned") for uncleaned election. The default value
is given.
--election-type <String: election> isType 0of (or "preferred"), if --topic or --partition iselection to attempt. Possible values are "preferred"
specifiedfor .preferred Notelection allowed if --path-to-json-file isor "unclean" for unclean election.
specifiedREQUIRED.
--topic <String: topic> Name of topic for which to perform an election.
REQUIRED if --partition is specified. Not allowed
if --path-to-json-file or --all-topic-partitions is specified.
--partition <Integer: partition id> Partition id for which to perform an election.
REQUIRED if --topic is specified. Not allowed if
--path-to-json-file or --all-topic-partitions is specified.
--help Print usage information.
--path-to-json-file <String: list of The JSON file with the list of
partitions for which replica partitions for which leader election
leader election needs to be should be done. Supported elections
triggered> are 0 for preferred and 1 for uncleanedunclean.
If an election is not specified,
preferred is the default. This is an
example format.
{"partitions":
[{"topic": "foo", "partition": 1},
{"topic": "foobar", "partition": 2, "election": 1}]
}
Defaults to preferredNot electionallowed toif --all existing partitions if
-topic-partitions, --topic or --partition
flags --topic and --partition flags are not specified.are specified. |
Metrics
The following metrics were added:
- kafka.server:type=DelayedOperationPurgatory,name=NumDelayedOperations,delayedOperation=ElectLeader
- kafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation=ElectLeader
While the following metrics were removed:
- kafka.server:type=DelayedOperationPurgatory,name=NumDelayedOperations,delayedOperation=ElectPreferredLeader
- kafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation=ElectPreferredLeader
Proposed Changes
In addition to the protocol and client changes enumerated above, the Controller will be extended to allow unclean leader election requests to come from the admin client. Currently the controller only supports preferred leader election from the admin client. Unclean leader election can only be enabled through either a topic configuration change or a broker configuration change.
...
Compatibility, Deprecation, and Migration Plan
- The kafka-preferred-replica-elections.{sh,bat} scripts will be deprecated.
- The electPreferredLeaders method in AdminClient will be deprecated.
- The DelayedOperationPurgatory metrics ElectPreferredLeader were replaced with ElectLeader
Rejected Alternatives
Not applicable.