You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Ryan Berdeen (JIRA)" <ji...@apache.org> on 2014/04/08 00:26:16 UTC

[jira] [Updated] (KAFKA-1367) Broker topic metadata not kept in sync with ZooKeeper

     [ https://issues.apache.org/jira/browse/KAFKA-1367?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ryan Berdeen updated KAFKA-1367:
--------------------------------

    Description: 
When a broker is restarted, the topic metadata responses from the brokers will be incorrect (different from ZooKeeper) until a preferred replica leader election.

In the metadata, it looks like leaders are correctly removed from the ISR when a broker disappears, but followers are not. Then, when a broker reappears, the ISR is never updated.


I used a variation of the Vagrant setup created by Joe Stein to reproduce this with latest from the 0.8.1 branch: https://github.com/also/kafka/commit/dba36a503a5e22ea039df0f9852560b4fb1e067c

  was:
When a broker is restarted, the topic metadata responses from the brokers will be incorrect (different from ZooKeeper) until a preferred replica leader election.

In the metadata, it looks like leaders are correctly removed from the ISR when a broker disappears, but followers are not. Then, when a broker reappears, the ISR is never updated.


I used a variation of the Vagrant setup created by Joe Stein to reproduce this with latest from the 0.8.1 branch: https://github.com/also/kafka/commit/dba36a503a5e22ea039df0f9852560b4fb1e067c

To start, the cluster looks like this:

Controller: 1

Brokers:
  1: 192.168.30.10
  2: 192.168.30.20
  3: 192.168.30.30
  4: 192.168.30.40
  5: 192.168.30.50

I create a topic with 5 partitions and 2 replicas:

bin/kafka-topics.sh --zookeeper 192.168.30.5 --create --topic test --replication-factor 2 --partitions 5

The assignment looks like this (L = leader, . = ISR, brokers are columns, partitions are rows):

        1     2     3     4     5
      ---   ---   ---   ---   ---
  0    .     L                   
  1          .     L             
  2                .     L       
  3                      .     L 
  4    L                       . 

After stopping broker 1, the expected state, reflected in ZooKeeper, is that all replicas on broker 1 are missing from their ISRs (! = missing from ISR):

        1     2     3     4     5
      ---   ---   ---   ---   ---
  0    !     L                   
  1          .     L             
  2                .     L       
  3                      .     L 
  4    !                       L 

  The topic metadata, however, still reports that broker 1 is in the ISR for partition 0, despite the fact that broker 1 is not running:

          1     2     3     4     5
      ---   ---   ---   ---   ---
  0    .     L                   
  1          .     L             
  2                .     L       
  3                      .     L 
  4    !                       L 

After starting broker 1 again and waiting for a few seconds, broker 1 should be back in the ISRs. This is reflected in ZooKeeper:

        1     2     3     4     5
      ---   ---   ---   ---   ---
  0    .     L                   
  1          .     L             
  2                .     L       
  3                      .     L 
  4    .                       L 

But the topic metadata response never changes:

        1     2     3     4     5
      ---   ---   ---   ---   ---
  0    .     L                   
  1          .     L             
  2                .     L       
  3                      .     L 
  4    !                       L 

After running

bin/kafka-preferred-replica-election.sh --zookeeper 192.168.30.5
Successfully started preferred replica election for partitions Set([test,1], [test,2], [test,3], [test,4], [test,0])

everything is back to normal. ZooKeeper and topic metadata are in sync, and broker 1 is the leader for partition 4.

        1     2     3     4     5
      ---   ---   ---   ---   ---
  0    .     L                   
  1          .     L             
  2                .     L       
  3                      .     L 
  4    L                       .


> Broker topic metadata not kept in sync with ZooKeeper
> -----------------------------------------------------
>
>                 Key: KAFKA-1367
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1367
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.8.0, 0.8.1
>            Reporter: Ryan Berdeen
>
> When a broker is restarted, the topic metadata responses from the brokers will be incorrect (different from ZooKeeper) until a preferred replica leader election.
> In the metadata, it looks like leaders are correctly removed from the ISR when a broker disappears, but followers are not. Then, when a broker reappears, the ISR is never updated.
> I used a variation of the Vagrant setup created by Joe Stein to reproduce this with latest from the 0.8.1 branch: https://github.com/also/kafka/commit/dba36a503a5e22ea039df0f9852560b4fb1e067c



--
This message was sent by Atlassian JIRA
(v6.2#6252)