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

Compare with Current View Page History

« Previous Version 2 Next »

This page is meant as a template for writing a KIP. To create a KIP choose Tools->Copy on this page and modify with your content and replace the heading with the next KIP number and a description of your issue. Replace anything in italics with your own description.

Status

Current state: One of "Under Discussion"

Discussion thread: here [Change the link from the KIP proposal email archive to your own email thread]

JIRA: here [Change the link from KAFKA-1 to your own ticket]

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 on a given set of topic partitions. In addition to preferred replica leader elections the controller also supports three other types of elections. In this proposal we modify the “PreferredLeaderElection” RPC to also support unclean leader election. The new RPC 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 support preferred leader elections 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

  • What impact (if any) will there be on existing users?
  • If we are changing behavior how will we phase out the older behavior?
  • If we need special migration tools, describe them here.
  • When will we remove the existing behavior?

Rejected Alternatives

If there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.

  • No labels