You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 8 Next »

Status

Current state: Under Discussion

Discussion thread: here

JIRA:

KAFKA-8286 - Getting issue details... STATUS

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

The admin client allows users to instruct the controller to attempt to elect the preferred replica as leader on a given set of topic partitions. In addition to preferred replica leader elections the controller also supports three other types of elections. One of those other election types is what we call unclean leader election. For the user to enable and trigger unclean leader election they need to either modify  the topic configuration or the broker configuration. We believe that this is potentially a dangerous configuration that could lead to data loss. Specially if the user forgets to disable unclean leader election after it completes. Being able to trigger unclean leader election once for a given partition is a much safer mechanism. 

In this proposal we modify the “PreferredLeaderElection” RPC to support unclean leader election in addition to preferred leader election. The new RPC definition also makes it possible to easily add new type of elections in the future.

Public Interfaces

Network protocol

{
  "apiKey": 43,
  "type": "request",
  "name": "ElectLeadersRequest",
  "validVersions": "0-1",
  "fields": [
    { "name": "TopicPartitions", "type": "[]TopicPartitions", "versions": "0+", "nullableVersions": "0+",
      "about": "The topic partitions to elect leaders.",
      "fields": [
        { "name": "Topic", "type": "string", "versions": "0+",
          "about": "The name of a topic." },
        { "name": "PartitionId", "type": "[]int32", "versions": "0",
          "about": "The partitions of this topic whose preferred leader should be elected." },
        { "name": "Partitions", "type": "[]Partitions", "versions": "1+",
          "about": "The partitions of this topic whose leader should be elected.",
          "fields": [
              { "name": "PartitionId", "type": "int32", "versions": "1+",
                "about": "The partition id." },
              { "name": "ElectionType", "type": "int8", "versions": "1+",
                "about": "Type of elections to conduct for the partition. 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." }
  ]
}

AdminClient Abstract Class

A new method will be added to the AdminClient abstract class to support this new version of the RPC.

package org.apache.kafka.clients.admin;

public abstract class AdminClient ... {
    ...

    /**
     * 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.
     *
     * 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                 The options to use when electing the leaders.
     * @return                        The ElectLeadersResult.
     */
    public abstract ElectLeadersResult electLeaders(
            Collection<TopicPartitionElection> partitionElections,
            ElectLeadersOptions options);
}

TopicPartitionElection Class

package org.apache.kafka.common;

public final class TopicPartitionElection {
    public static enum ElectionType {
        PREFERRED((byte) 0), UNCLEANED((byte) 1);

        public final byte value;

        ElectionType(byte value) {
            this.value = value;
        }
    }

    public final TopicPartition topicPartition;
    public final ElectionType electionType;

    ...
}

Admin Command

The command kafka-preferred-replica-election.{sh,bat} will be deprecated and the following command will be added.

$ 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                                  Description
------                                  -----------

--admin.config <String: config file>    Admin client config properties file to
                                          pass to the admin client when --
                                          bootstrap-server is given.
--bootstrap-server <String: host:port>  A host name and port for the broker to
                                          connect to, in the form host:port.
                                          Multiple comma-separated URLs can be
                                          given. REQUIRED unless --zookeeper
                                          is given.
--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 uncleaned.
                                          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 all existing partitions

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.

When performing unclean leader election on a topic partition through the admin client the "unclean leader election" topic configuration and broker configuration will be ignored. In the code this will result is code similar to the one below:

uncleanElectionFromAdminClient || logConfigs(partition.topic).uncleanLeaderElectionEnable.booleanValue()


Compatibility, Deprecation, and Migration Plan

The kafka-preferred-replica-elections.{sh,bat} scripts will be deprecated.

Rejected Alternatives

Not applicable.

  • No labels