You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Andrew Otto (JIRA)" <ji...@apache.org> on 2014/09/19 20:16:34 UTC

[jira] [Comment Edited] (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:comment-tabpanel&focusedCommentId=14141017#comment-14141017 ] 

Andrew Otto edited comment on KAFKA-1367 at 9/19/14 6:15 PM:
-------------------------------------------------------------

I just updated the librdkafka issue, pasting it here as well:

I noticed that in my case, only 1 of the 4 brokers was ever missing in the ISRs reported by Kafka Brokers (via librdkafka). This JIRA indicated that a preferred-replica-election should fix the problem. I did this:

controlled-shutdown of offending broker 21. Then actual shutdown of broker 21. Once this was done, librdkafka metadata showed the correct ISRs, since this offending broker really was not in any ISRs. I then restarted broker 21 and let its replicas catch back up. Once it caught up, zookeeper reported that all ISRs were in sync. I then checked librdkafka's metadata, and broker 21 was not listed in any ISR. I then ran a preferred-replica-election. broker 21 was then promoted to leader of some partitions. librdkafka then only showed broker 21 being in the ISRs for which it was also the leader. Any partition that has a replica on broker 21 does not show up in the ISR unless broker 21 is the leader.

  $ kafkacat -L -b analytics1022.eqiad.wmnet  -t webrequest_upload
  Metadata for webrequest_upload (from broker -1: analytics1022.eqiad.wmnet:9092/bootstrap):
   4 brokers:
    broker 12 at analytics1012.eqiad.wmnet:9092
    broker 21 at analytics1021.eqiad.wmnet:9092
    broker 22 at analytics1022.eqiad.wmnet:9092
    broker 18 at analytics1018.eqiad.wmnet:9092
   1 topics:
    topic "webrequest_upload" with 12 partitions:
      partition 11, leader 12, replicas: 12,21,22, isrs: 12,22
      partition 5, leader 21, replicas: 21,22,12, isrs: 22,12,21
      partition 10, leader 22, replicas: 22,18,21, isrs: 18,22
      partition 7, leader 12, replicas: 12,18,21, isrs: 12,18
      partition 8, leader 18, replicas: 18,22,12, isrs: 12,18,22
      partition 3, leader 12, replicas: 12,22,18, isrs: 12,18,22
      partition 4, leader 18, replicas: 18,21,22, isrs: 18,22
      partition 1, leader 21, replicas: 21,18,22, isrs: 18,22,21
      partition 6, leader 22, replicas: 22,12,18, isrs: 12,18,22
      partition 2, leader 22, replicas: 22,21,12, isrs: 12,22
      partition 9, leader 21, replicas: 21,12,18, isrs: 12,18,21
      partition 0, leader 18, replicas: 18,12,21, isrs: 12,18



vs kafka-topic.sh --describe

  Topic:webrequest_upload	PartitionCount:12	ReplicationFactor:3	Configs:
  	Topic: webrequest_upload	Partition: 0	Leader: 18	Replicas: 18,12,21	Isr: 12,18,21
  	Topic: webrequest_upload	Partition: 1	Leader: 21	Replicas: 21,18,22	Isr: 18,22,21
  	Topic: webrequest_upload	Partition: 2	Leader: 22	Replicas: 22,21,12	Isr: 12,22,21
  	Topic: webrequest_upload	Partition: 3	Leader: 12	Replicas: 12,22,18	Isr: 12,18,22
  	Topic: webrequest_upload	Partition: 4	Leader: 18	Replicas: 18,21,22	Isr: 18,22,21
  	Topic: webrequest_upload	Partition: 5	Leader: 21	Replicas: 21,22,12	Isr: 22,12,21
  	Topic: webrequest_upload	Partition: 6	Leader: 22	Replicas: 22,12,18	Isr: 12,18,22
  	Topic: webrequest_upload	Partition: 7	Leader: 12	Replicas: 12,18,21	Isr: 12,18,21
  	Topic: webrequest_upload	Partition: 8	Leader: 18	Replicas: 18,22,12	Isr: 12,18,22
  	Topic: webrequest_upload	Partition: 9	Leader: 21	Replicas: 21,12,18	Isr: 12,18,21
  	Topic: webrequest_upload	Partition: 10	Leader: 22	Replicas: 22,18,21	Isr: 18,22,21
  	Topic: webrequest_upload	Partition: 11	Leader: 12	Replicas: 12,21,22	Isr: 12,22,21


was (Author: ottomata):
I just updated the librdkafka issue, pasting it here as well:

I noticed that in my case, only 1 of the 4 brokers was ever missing in the ISRs reported by Kafka Brokers (via librdkafka). That JIRA indicated that a preferred-replica-election should fix the problem. I did this:

controlled-shutdown of offending broker 21. Then actual shutdown of broker 21. Once this was done, librdkafka metadata showed the correct ISRs, since this offending broker really was not in any ISRs. I then restarted broker 21 and let its replicas catch back up. Once it caught up, zookeeper reported that all ISRs were in sync. I then checked librdkafka's metadata, and broker 21 was not listed in any ISR. I then ran a preferred-replica-election. broker 21 was then promoted to leader of some partitions. librdkafka then only showed broker 21 being in the ISRs for which it was also the leader. Any partition that has a replica on broker 21 does not show up in the ISR unless broker 21 is the leader.

  $ kafkacat -L -b analytics1022.eqiad.wmnet  -t webrequest_upload
  Metadata for webrequest_upload (from broker -1: analytics1022.eqiad.wmnet:9092/bootstrap):
   4 brokers:
    broker 12 at analytics1012.eqiad.wmnet:9092
    broker 21 at analytics1021.eqiad.wmnet:9092
    broker 22 at analytics1022.eqiad.wmnet:9092
    broker 18 at analytics1018.eqiad.wmnet:9092
   1 topics:
    topic "webrequest_upload" with 12 partitions:
      partition 11, leader 12, replicas: 12,21,22, isrs: 12,22
      partition 5, leader 21, replicas: 21,22,12, isrs: 22,12,21
      partition 10, leader 22, replicas: 22,18,21, isrs: 18,22
      partition 7, leader 12, replicas: 12,18,21, isrs: 12,18
      partition 8, leader 18, replicas: 18,22,12, isrs: 12,18,22
      partition 3, leader 12, replicas: 12,22,18, isrs: 12,18,22
      partition 4, leader 18, replicas: 18,21,22, isrs: 18,22
      partition 1, leader 21, replicas: 21,18,22, isrs: 18,22,21
      partition 6, leader 22, replicas: 22,12,18, isrs: 12,18,22
      partition 2, leader 22, replicas: 22,21,12, isrs: 12,22
      partition 9, leader 21, replicas: 21,12,18, isrs: 12,18,21
      partition 0, leader 18, replicas: 18,12,21, isrs: 12,18



vs kafka-topic.sh --describe

  Topic:webrequest_upload	PartitionCount:12	ReplicationFactor:3	Configs:
  	Topic: webrequest_upload	Partition: 0	Leader: 18	Replicas: 18,12,21	Isr: 12,18,21
  	Topic: webrequest_upload	Partition: 1	Leader: 21	Replicas: 21,18,22	Isr: 18,22,21
  	Topic: webrequest_upload	Partition: 2	Leader: 22	Replicas: 22,21,12	Isr: 12,22,21
  	Topic: webrequest_upload	Partition: 3	Leader: 12	Replicas: 12,22,18	Isr: 12,18,22
  	Topic: webrequest_upload	Partition: 4	Leader: 18	Replicas: 18,21,22	Isr: 18,22,21
  	Topic: webrequest_upload	Partition: 5	Leader: 21	Replicas: 21,22,12	Isr: 22,12,21
  	Topic: webrequest_upload	Partition: 6	Leader: 22	Replicas: 22,12,18	Isr: 12,18,22
  	Topic: webrequest_upload	Partition: 7	Leader: 12	Replicas: 12,18,21	Isr: 12,18,21
  	Topic: webrequest_upload	Partition: 8	Leader: 18	Replicas: 18,22,12	Isr: 12,18,22
  	Topic: webrequest_upload	Partition: 9	Leader: 21	Replicas: 21,12,18	Isr: 12,18,21
  	Topic: webrequest_upload	Partition: 10	Leader: 22	Replicas: 22,18,21	Isr: 18,22,21
  	Topic: webrequest_upload	Partition: 11	Leader: 12	Replicas: 12,21,22	Isr: 12,22,21

> 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
>              Labels: newbie++
>         Attachments: KAFKA-1367.txt
>
>
> 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.3.4#6332)