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": "01+", "nullableVersions": "0+",
"about": "TheType topicof partitionselections to conduct for electthe leaderspartition.",
A value of '0' elects "fields": [
{ "name": "Topic", "type": "string", "versions": "0+",
"about": "The name of a topicthe 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 partitions of this topic whosepartitions leaderto shouldelect be electedleaders.",
"fields": [
{ "name": "PartitionIdTopic", "type": "int32string", "versions": "0+",
"about": "The partition idname of a topic." },
{ "name": "ElectionTypePartitionId", "type": "int8[]int32", "versions": "10+",
"about": "TypeThe partitions of this electionstopic towhose conductleader forshould thebe partitionelected." A}
value of '0' elects the preferred]
leader. A value of '1' elects an unclean leader if there are no in-sync leaders." }
},
{ "name": "TimeoutMs", "type": "int32", "versions": "0+", "default": "60000",
"about": "The time in ]
ms to wait for the election to complete." }
]
}]
} |
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,
{ "nametype": "TimeoutMsresponse",
"typename": "int32ElectLeadersResponse",
"versionsvalidVersions": "0+-1",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "defaultversions": "600000+",
"about": "The timeduration in msmilliseconds to wait for the election to completefor which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
{ ]
} |
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 ... {"name": "ErrorCode", "type": "int16", "versions": "1+", "ignorable": false,
...
"about": /**
* Attempt to elect a new leader for each of the topic partition in {@code partitionElections}. The type of elections supported are
* document in the {@link TopicPartitionElection} type.
*"The top level response error code." },
{ "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": [
* If {@code partitionElections} is null, then attempt to elect the preferred replica for all of the partitions.
*
* @param partitionElections The partitions and the type of elections to conduct.
* @param options "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", The options to use when electing the leaders.
* @return"versions": "0+",
"about": "The partition id" },
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The ElectLeadersResult.
result error, or zero if */
there was public abstract ElectLeadersResult electLeaders(no error."},
{ "name": "ErrorMessage", Collection<TopicPartitionElection> partitionElections,
"type": "string", "versions": "0+", "nullableVersions": "0+",
ElectLeadersOptions options);
} |
TopicPartitionElection Class
Code Block |
---|
|
package org.apache.kafka.common;
public final class TopicPartitionElection {
public static enum ElectionType { "about": "The result message, or null if there was no error."}
PREFERRED((byte) 0), UNCLEANED((byte) 1);
public final byte value;
ElectionType(byte value) {
this.value = value;
}
}]}
]}
]
} |
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 ... {
...
public/**
final TopicPartition topicPartition;
* ...
public final ElectionType electionType;
* @deprecated Since 2.4..
} |
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.
Option0. Use {@link #electLeaders}.
*/
@Deprecated
public abstract ElectPreferredLeadersResult electPreferredLeaders(Collection<TopicPartition> partitions,
Description
------ ElectPreferredLeadersOptions options);
/**
-----------
--admin.config <String: config file> Admin client config properties file to
* ...
*
* @param electionType The type of election.
* @param partitions The partitions for which to conduct passelections.
to the admin client when* --
@param options The options to use when electing the leaders.
* @return bootstrap-server is given.
--bootstrap-server <String: host:port> A host name and port for the brokerThe toElectLeadersResult.
*/
public abstract ElectLeadersResult electLeaders(
ElectionType electionType,
Set<TopicPartition> partitions,
connect to, in the form host:port.
ElectLeadersOptions options);
} |
ElectLeaderResult Class
Code Block |
---|
final public ElectLeadersResult {
...
public KafkaFuture<Map<TopicPartition, Optional<Throwable>>> partitions() {
...
}
public KafkFuture<Void> all() {
Multiple comma-separated URLs can be...
}
} |
ElectionType Enumeration
Code Block |
---|
|
package org.apache.kafka.common;
public enum ElectionType {
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.
Optiongiven. REQUIRED unless --zookeeper
is given.
--help Print usage information.
Description
---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> -----------
--admin.config <String: config file> Admin client config properties file to pass to the admin
are 0 for preferredclient and 1 for uncleaned.
when --bootstrap-server is given.
--all-topic-partitions Perform election on all of the topic partitions. Not allowed if
If an election is not specified,
--topic, --partition or --path-to-json-file is specified.
--bootstrap-server <String: host:port> A host name and port for the broker to
preferred is the default. This is an
connect to, in the example formatform host:port.
{"partitions":
Multiple comma-separated URLs can be
[{"topic": "foo", "partition": 1},given. REQUIRED unless --zookeeper
{"topic": "foobar", "partition": 2, "election": 1}]
is given.
--election-type <String: election> Type of election to attempt. Possible values are "preferred"
}
for preferred election or "unclean" for unclean election.
Defaults to all existing partitions REQUIRED.
--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 unclean.
If an election is not specified,
preferred is the default. This is an
example format.
{"partitions":
[{"topic": "foo", "partition": 1},
{"topic": "foobar", "partition": 2}]
}
Not allowed if --all-topic-partitions, --topic or --partition
flags 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.