Hi all,

Jason, Colin and I discuss this KIP offline and decided to make the
following changes.

   1. Change the ElectLeadersRequest RPC so that only one election type can
   be specified and it applies to all of the topic partitions enumerated. We
   think that this makes the API easier to use when performing one type of
   election across multiple topic partitions. We think that it is rare that
   they user would like to perform different type of elections in the same
   command (or request).
   2. Change the kafka-leader-election script so that it doesn't default to
   applying the election type to all of the topic partitions. For example
   previously "bin/kafka-preferred-replica-election.sh --bootstrap-server
   $host:$port" would attempt to perform preferred leader election to all of
   the partition. Instead now the user needs to run the following command
   "bin/kafka-leader-election.sh --bootstrap-server $host:@port
   --all-topic-partitions"

The KIP has been updated to includes these changes. The diff is here:
https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=113707931&selectedPageVersions=13&selectedPageVersions=12

Thanks!

On Wed, Apr 24, 2019 at 3:45 PM Jose Armando Garcia Sancio <
jsan...@confluent.io> wrote:

> Hi all,
>
> We would like to extend the "preferred leader election" RPC for the admin
> client to also support unclean leader elections.
>
> The KIP can be found here:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-460%3A+Admin+Leader+Election+RPC
>
> Thanks!
> -Jose
>


-- 
-Jose

Reply via email to