Kafka
  1. Kafka
  2. KAFKA-1367

Broker topic metadata not kept in sync with ZooKeeper

    Details

    • Type: Bug Bug
    • Status: Open
    • Priority: Major Major
    • Resolution: Unresolved
    • Affects Version/s: 0.8.0, 0.8.1
    • Fix Version/s: 0.8.3
    • Component/s: None
    • Labels:

      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

      1. KAFKA-1367.txt
        3 kB
        Ryan Berdeen

        Issue Links

          Activity

          Hide
          Ryan Berdeen added a comment -

          JIRA ate my whitespace. See Attached for a description of the steps.

          Show
          Ryan Berdeen added a comment - JIRA ate my whitespace. See Attached for a description of the steps.
          Hide
          Andrew Otto added a comment -

          This happens to me as well. See: https://github.com/edenhill/librdkafka/issues/147

          Show
          Andrew Otto added a comment - This happens to me as well. See: https://github.com/edenhill/librdkafka/issues/147
          Hide
          Andrew Otto added a comment - - edited

          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

          Show
          Andrew Otto added a comment - - edited 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
          Hide
          Jun Rao added a comment -

          Yes, in the current implementation, ISR returned from metadata requests can be inconsistent with what's stored in ZK. This is because metadata is only propagated from the controller to the brokers when the controller changes the leader or the ISR. However, when a follower catches up, the leader (not the controller) adds it back to ISR and updates ZK. That info is not propagated to all brokers.

          Currently, the ISR part in a metadata response is not really used by the clients. Do you have a usage for this?

          Show
          Jun Rao added a comment - Yes, in the current implementation, ISR returned from metadata requests can be inconsistent with what's stored in ZK. This is because metadata is only propagated from the controller to the brokers when the controller changes the leader or the ISR. However, when a follower catches up, the leader (not the controller) adds it back to ISR and updates ZK. That info is not propagated to all brokers. Currently, the ISR part in a metadata response is not really used by the clients. Do you have a usage for this?
          Hide
          Magnus Edenhill added a comment -

          Apart from supplying the ISR count and list in its metadata API, librdkafka also provides an `enforce.isr.cnt` configuration property that fails
          produce requests locally before transmission if the currently known ISR count is smaller than the configured value.
          This is a workaround for the broker not fully honoring `request.required.acks`, i.e., if `request.required.acks=3` and only one broker is available the produce request will not fail.
          More info in the original issue here: https://github.com/edenhill/librdkafka/issues/91

          Generally I would assume that information provided by the broker is correct, otherwise it should not be included at all since it can't be used (reliably).

          Show
          Magnus Edenhill added a comment - Apart from supplying the ISR count and list in its metadata API, librdkafka also provides an `enforce.isr.cnt` configuration property that fails produce requests locally before transmission if the currently known ISR count is smaller than the configured value. This is a workaround for the broker not fully honoring `request.required.acks`, i.e., if `request.required.acks=3` and only one broker is available the produce request will not fail. More info in the original issue here: https://github.com/edenhill/librdkafka/issues/91 Generally I would assume that information provided by the broker is correct, otherwise it should not be included at all since it can't be used (reliably).
          Hide
          Guozhang Wang added a comment -

          Hello Magnus Edenhill, I think your use case may be supported in a new feature that is currently developed: KAFKA-1555. Would you like to take a look at it and see if your case is really covered?

          Show
          Guozhang Wang added a comment - Hello Magnus Edenhill , I think your use case may be supported in a new feature that is currently developed: KAFKA-1555 . Would you like to take a look at it and see if your case is really covered?
          Hide
          Dan Hoffman added a comment -

          I'd also add that having the broker being able to serve up (accurate) metadata allows client applications to build custom dashboards, etc. As I understand it, there is the idea to move away from zookeeper (or at least for somethings) within the kafka infrastructure - so having the broker be able to provide this would be good.

          Show
          Dan Hoffman added a comment - I'd also add that having the broker being able to serve up (accurate) metadata allows client applications to build custom dashboards, etc. As I understand it, there is the idea to move away from zookeeper (or at least for somethings) within the kafka infrastructure - so having the broker be able to provide this would be good.
          Hide
          Joel Koshy added a comment -

          I definitely agree with Magnus Edenhill that if such a field exists in the response then the information populated in the field should be accurate (or we may as well not include the field) - so we should fix this.

          Show
          Joel Koshy added a comment - I definitely agree with Magnus Edenhill that if such a field exists in the response then the information populated in the field should be accurate (or we may as well not include the field) - so we should fix this.
          Hide
          Neha Narkhede added a comment -

          I definitely agree with Magnus Edenhill that if such a field exists in the response then the information populated in the field should be accurate (or we may as well not include the field) - so we should fix this.

          +1

          Show
          Neha Narkhede added a comment - I definitely agree with Magnus Edenhill that if such a field exists in the response then the information populated in the field should be accurate (or we may as well not include the field) - so we should fix this. +1
          Hide
          Magnus Edenhill added a comment -

          Guozhang Wang Yes, KAFKA-1555 looks like a good match.

          Show
          Magnus Edenhill added a comment - Guozhang Wang Yes, KAFKA-1555 looks like a good match.
          Hide
          Guozhang Wang added a comment -

          Regarding the fix to this issue, we can either 1) remove the ISR field from the metadata response and hence enforce people to use the admin tool (with ZK dependency) for such usages, which would also require a protocol change between client / server; or 2) let the controller to also watch for ISR change and propagate that information to brokers, this will not introduce protocol change to clients but will likely add a lot of burden on controllers since ISR change is more frequent than leader migrations.

          Joel KoshyJun Rao any other thoughts?

          Show
          Guozhang Wang added a comment - Regarding the fix to this issue, we can either 1) remove the ISR field from the metadata response and hence enforce people to use the admin tool (with ZK dependency) for such usages, which would also require a protocol change between client / server; or 2) let the controller to also watch for ISR change and propagate that information to brokers, this will not introduce protocol change to clients but will likely add a lot of burden on controllers since ISR change is more frequent than leader migrations. Joel Koshy Jun Rao any other thoughts?
          Hide
          Guozhang Wang added a comment -

          Just talked to Joel offline. I think since ISR (and also Leader) info in broker is just a cached snapshot and cannot be really used in a scenario like this (i.e. depending on the ISR list to determine if the ack received with -1 setting is reliable or not, since the ISR can shrink while the ack is sent back), we could remove the ISR cache from the brokers and also remove it from the metadata response, unless there is a clear use case of this information.

          Show
          Guozhang Wang added a comment - Just talked to Joel offline. I think since ISR (and also Leader) info in broker is just a cached snapshot and cannot be really used in a scenario like this (i.e. depending on the ISR list to determine if the ack received with -1 setting is reliable or not, since the ISR can shrink while the ack is sent back), we could remove the ISR cache from the brokers and also remove it from the metadata response, unless there is a clear use case of this information.
          Hide
          Neha Narkhede added a comment -

          The ISR cache on the broker was added only because we had to expose that information through the topic metadata response. I don't think we gave a lot of thought, back then, on why the ISR information is useful in the topic metadata response (especially since it's stale and effectively inaccurate). I am not entirely sure if having the controller be aware of all ISR changes is terrible even though it's true that the # of watches it has to add is proportional to the # of partitions in a cluster. But it's not worth doing that if we don't find a use for the ISR information in the topic metadata response. So I'd vote for removing ISR from topic metadata and also from the broker's metadata cache.

          Show
          Neha Narkhede added a comment - The ISR cache on the broker was added only because we had to expose that information through the topic metadata response. I don't think we gave a lot of thought, back then, on why the ISR information is useful in the topic metadata response (especially since it's stale and effectively inaccurate). I am not entirely sure if having the controller be aware of all ISR changes is terrible even though it's true that the # of watches it has to add is proportional to the # of partitions in a cluster. But it's not worth doing that if we don't find a use for the ISR information in the topic metadata response. So I'd vote for removing ISR from topic metadata and also from the broker's metadata cache.
          Hide
          Magnus Edenhill added a comment -

          May I suggest not to change the protocol but to only send an empty ISR vector in the MetadataResponse?

          Show
          Magnus Edenhill added a comment - May I suggest not to change the protocol but to only send an empty ISR vector in the MetadataResponse?
          Hide
          Francois Saint-Jacques added a comment -

          Instead of silently removing the field, could the controller force a cache refresh on a metadata request?

          Show
          Francois Saint-Jacques added a comment - Instead of silently removing the field, could the controller force a cache refresh on a metadata request?

            People

            • Assignee:
              Unassigned
              Reporter:
              Ryan Berdeen
            • Votes:
              4 Vote for this issue
              Watchers:
              15 Start watching this issue

              Dates

              • Created:
                Updated:

                Development