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
          Jun Rao added a comment -

          Well, in that approach, you still have the problem on the very first fetch request. If ISR is not returned in TMR, the first fetch request has to go to the leader. Then the consumer has to switch to another broker on a subsequent request, which seems more complicated.

          I am not sure if we need to rely on periodic metadata refresh to detect whether a replica is out of sync. Basically, as long as the fetch offset is less than HW, the replica can serve the request. If the fetch offset is larger than HW (an indication that the replica is out of sync), the consumer will get an OffsetOutOfRangeException and has to refresh the metadata and pick a new broker to fetch from.

          Show
          Jun Rao added a comment - Well, in that approach, you still have the problem on the very first fetch request. If ISR is not returned in TMR, the first fetch request has to go to the leader. Then the consumer has to switch to another broker on a subsequent request, which seems more complicated. I am not sure if we need to rely on periodic metadata refresh to detect whether a replica is out of sync. Basically, as long as the fetch offset is less than HW, the replica can serve the request. If the fetch offset is larger than HW (an indication that the replica is out of sync), the consumer will get an OffsetOutOfRangeException and has to refresh the metadata and pick a new broker to fetch from.
          Hide
          Joel Koshy added a comment -

          One minor issue with depending on the TMR for KAKFA-2225 even if we fix this is that the consumer would need to periodically refresh its metadata in case the ISR changes after it starts reading from a follower in ISR.

          Another approach for KAFKA-2225 is to add the ISR information to the fetch response. The followers will then have the current ISR information and so will the consumers. There are at least two concerns though: first, it depends on a live replica fetcher thread; second, it's a bit hacky to add ISR to fetch response as it is more associated with metadata.

          Show
          Joel Koshy added a comment - One minor issue with depending on the TMR for KAKFA-2225 even if we fix this is that the consumer would need to periodically refresh its metadata in case the ISR changes after it starts reading from a follower in ISR. Another approach for KAFKA-2225 is to add the ISR information to the fetch response. The followers will then have the current ISR information and so will the consumers. There are at least two concerns though: first, it depends on a live replica fetcher thread; second, it's a bit hacky to add ISR to fetch response as it is more associated with metadata.
          Hide
          Jun Rao added a comment -

          It's probably better to always let the controller propagate metadata changes to the brokers. If the metadata change can be sent from both the controller and other brokers, we need additional logic to reason about ordering.

          Having the broker send the change to the controller is possible. The implication is that there is another thread that can exercise the controller logic, instead of just the ZK watcher thread. So, we may need to deal with more concurrency issues.

          Show
          Jun Rao added a comment - It's probably better to always let the controller propagate metadata changes to the brokers. If the metadata change can be sent from both the controller and other brokers, we need additional logic to reason about ordering. Having the broker send the change to the controller is possible. The implication is that there is another thread that can exercise the controller logic, instead of just the ZK watcher thread. So, we may need to deal with more concurrency issues.
          Hide
          Joel Koshy added a comment -

          Jun - that is a good point. That sounds like a good approach to address the concern with watcher counts. Another way is to just allow brokers to send an equivalent update metadata (or similar) request to the controller to notify it of an ISR change - or even allow leaders to broadcast update metadata requests for ISR changes. We currently don't allow this, but maybe we should consider a generic broker-broker communication component. Given the use-case that Theo raised on the list yesterday, it appears we may want to keep the ISR even in TMR v1. It may make sense to discuss this at an upcoming hangout especially since it affects KIP-4.

          Show
          Joel Koshy added a comment - Jun - that is a good point. That sounds like a good approach to address the concern with watcher counts. Another way is to just allow brokers to send an equivalent update metadata (or similar) request to the controller to notify it of an ISR change - or even allow leaders to broadcast update metadata requests for ISR changes. We currently don't allow this, but maybe we should consider a generic broker-broker communication component. Given the use-case that Theo raised on the list yesterday, it appears we may want to keep the ISR even in TMR v1. It may make sense to discuss this at an upcoming hangout especially since it affects KIP-4.
          Hide
          Jun Rao added a comment -

          There is actually a reasonable use case of ISR in KAFKA-2225. Basically, for economical reasons, we may want to let a consumer fetch from a replica in ISR that's in the same zone. In order to support that, it will be convenient to have TMR return the correct ISR for the consumer to choose.

          Implementation wise, one way to address the concern with too many watchers is to do sth similar to changing topic configs. Basically, when the leader changes the isr, in addition to writing the new isr in the partition state in ZK, it also writes the change as a sequential node under a new isrChangeNotification path in ZK. The controller listens to child changes in the isrChangeNotification path. On child change, the controller reads the new isr and broadcasts it through an UpdateMetadataRequest to every broker.

          Show
          Jun Rao added a comment - There is actually a reasonable use case of ISR in KAFKA-2225 . Basically, for economical reasons, we may want to let a consumer fetch from a replica in ISR that's in the same zone. In order to support that, it will be convenient to have TMR return the correct ISR for the consumer to choose. Implementation wise, one way to address the concern with too many watchers is to do sth similar to changing topic configs. Basically, when the leader changes the isr, in addition to writing the new isr in the partition state in ZK, it also writes the change as a sequential node under a new isrChangeNotification path in ZK. The controller listens to child changes in the isrChangeNotification path. On child change, the controller reads the new isr and broadcasts it through an UpdateMetadataRequest to every broker.
          Hide
          Ashish K Singh added a comment -

          Thanks Joel Koshy! I will update the KIP accordingly.

          Show
          Ashish K Singh added a comment - Thanks Joel Koshy ! I will update the KIP accordingly.
          Hide
          Joel Koshy added a comment -

          Hi Ashish K Singh - sorry I missed your pings. Yes that is the approach we are planning to take. i.e., remove ISR from TMR. As mentioned in KIP-4 the ISR will be removed in v1 of TMR.

          Show
          Joel Koshy added a comment - Hi Ashish K Singh - sorry I missed your pings. Yes that is the approach we are planning to take. i.e., remove ISR from TMR. As mentioned in KIP-4 the ISR will be removed in v1 of TMR.
          Hide
          Ashish K Singh added a comment -

          Joel Koshy pinging you for your confirmation on this.

          Show
          Ashish K Singh added a comment - Joel Koshy pinging you for your confirmation on this.
          Hide
          Ashish K Singh added a comment -

          Neha Narkhede/ Joel Koshy I have put together KIP-24. I will need a bit more information on what we decided for BrokerMetadataRequest, before I can update "Public Interfaces" section of the KIP.

          Is my understanding correct that below is the plan we agreed upon. Below excerpt is actually from the KIP.

          It is proposed to remove ISR information from the TopicMetadataRequest. However, a new request, BrokerMetadataRequest, is proposed to be added. The new request will include ISR information for all the partitions that the broker leads.

          Show
          Ashish K Singh added a comment - Neha Narkhede / Joel Koshy I have put together KIP-24 . I will need a bit more information on what we decided for BrokerMetadataRequest, before I can update "Public Interfaces" section of the KIP. Is my understanding correct that below is the plan we agreed upon. Below excerpt is actually from the KIP. It is proposed to remove ISR information from the TopicMetadataRequest. However, a new request, BrokerMetadataRequest, is proposed to be added. The new request will include ISR information for all the partitions that the broker leads.
          Hide
          Neha Narkhede added a comment -

          Joel Koshy That would work too but looks like Andrii Biletskyi is suggesting that it is not included as part of KIP-4. Maybe we can have whoever picks this JIRA discuss this change as part of a separate KIP?

          Show
          Neha Narkhede added a comment - Joel Koshy That would work too but looks like Andrii Biletskyi is suggesting that it is not included as part of KIP-4. Maybe we can have whoever picks this JIRA discuss this change as part of a separate KIP?
          Hide
          Andrii Biletskyi added a comment -

          Joel Koshy Yes, it appears that topic commands don't require ISR information so it was proposed to remove it at all from the TopicMetadataRequest. There was an idea to create some sort of BrokerMetadataRequest which will include correct topic metadata but since it's not related to KIP-4 directly it won't be a part of it. So everyone is welcome to create a separate KIP for it. Atleast to this conclusion we came last time.

          Show
          Andrii Biletskyi added a comment - Joel Koshy Yes, it appears that topic commands don't require ISR information so it was proposed to remove it at all from the TopicMetadataRequest. There was an idea to create some sort of BrokerMetadataRequest which will include correct topic metadata but since it's not related to KIP-4 directly it won't be a part of it. So everyone is welcome to create a separate KIP for it. Atleast to this conclusion we came last time.
          Hide
          Joel Koshy added a comment -

          I'm a bit wary of the watch approach. I believe at the last google hangout we decided against doing this (as part of the KIP-4 discussion). A number of people had dropped off at that point - I believe we were going with a broker-level metadata request that can return ISR information for partitions that it leads. Andrii Biletskyi can you confirm? I didn't see it in the summary notes so I could be wrong.

          Show
          Joel Koshy added a comment - I'm a bit wary of the watch approach. I believe at the last google hangout we decided against doing this (as part of the KIP-4 discussion). A number of people had dropped off at that point - I believe we were going with a broker-level metadata request that can return ISR information for partitions that it leads. Andrii Biletskyi can you confirm? I didn't see it in the summary notes so I could be wrong.
          Hide
          Ashish K Singh added a comment -

          Neha Narkhede If no one has already started working on this I would like to take a stab at it.

          Show
          Ashish K Singh added a comment - Neha Narkhede If no one has already started working on this I would like to take a stab at it.
          Hide
          Neha Narkhede added a comment -

          Jay Kreps Yup, issue still exists and the solution I still recommend is to have the controller register watches and know the latest ISR for all partitions. This change isn't big if someone wants to take a stab.

          Show
          Neha Narkhede added a comment - Jay Kreps Yup, issue still exists and the solution I still recommend is to have the controller register watches and know the latest ISR for all partitions. This change isn't big if someone wants to take a stab.
          Hide
          Jon Bringhurst added a comment -

          Hey Jay Kreps, I can confirm that the metadata response is still out of sync with ZK in a recent trunk build (about a month old).

          Btw, it's not much effort for me to simply look at ZK or JMX – it was just confusing when I initially ran into this.

          Show
          Jon Bringhurst added a comment - Hey Jay Kreps , I can confirm that the metadata response is still out of sync with ZK in a recent trunk build (about a month old). Btw, it's not much effort for me to simply look at ZK or JMX – it was just confusing when I initially ran into this.
          Hide
          Jun Rao added a comment -

          Michael Graff, for the operational issue that you pointed out, there are jmx such as the underReplicatedCount that one can leverage.

          That said, it's probably useful for an admin client to know the accurate ISR. As Guozhang has suggested, one approach is to register a watcher of the state path for each partition. We probably need to do a bit experiments to see how much overhead this adds. Another approach is to have the controller periodically read the latest ISR for each partition. This is probably a bit simpler to implement, but may not reflect ISR timely and may add unnecessary load to ZK.

          Show
          Jun Rao added a comment - Michael Graff , for the operational issue that you pointed out, there are jmx such as the underReplicatedCount that one can leverage. That said, it's probably useful for an admin client to know the accurate ISR. As Guozhang has suggested, one approach is to register a watcher of the state path for each partition. We probably need to do a bit experiments to see how much overhead this adds. Another approach is to have the controller periodically read the latest ISR for each partition. This is probably a bit simpler to implement, but may not reflect ISR timely and may add unnecessary load to ZK.
          Hide
          Michael Graff added a comment -

          There is another issue that no one seems to be discussing. This ISR data is AFAIK the only way to know when a broker is "Back in service"

          Consider this scenario. I have 5 brokers, and want to upgrade them. I want to know when a broker has caught up so I can take down the next one in sequence to upgrade it. How can I know this if the reported state of the world is different than what is actually in use by the brokers themselves?

          This seems to be very much an operational issue.

          Show
          Michael Graff added a comment - There is another issue that no one seems to be discussing. This ISR data is AFAIK the only way to know when a broker is "Back in service" Consider this scenario. I have 5 brokers, and want to upgrade them. I want to know when a broker has caught up so I can take down the next one in sequence to upgrade it. How can I know this if the reported state of the world is different than what is actually in use by the brokers themselves? This seems to be very much an operational issue.
          Hide
          Jay Kreps added a comment -

          Neha Narkhede Does this problem still exist?

          Show
          Jay Kreps added a comment - Neha Narkhede Does this problem still exist?
          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?
          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
          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
          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
          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
          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
          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
          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
          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
          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
          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
          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
          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
          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
          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.

            People

            • Assignee:
              Ashish K Singh
              Reporter:
              Ryan Berdeen
            • Votes:
              4 Vote for this issue
              Watchers:
              22 Start watching this issue

              Dates

              • Created:
                Updated:

                Development