[ https://issues.apache.org/jira/browse/KAFKA-1367?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14569484#comment-14569484 ]
Jun Rao commented on KAFKA-1367: -------------------------------- [~joel koshy], yes, that's a good point. If we want to switch back to the closest replica for consumption, we do need to refresh the metadata periodically to check if the closest replica is back in ISR. We also need to handle OffsetOutOfRangeException a bit differently. If the consumer gets OffsetOutOfRangeException because the replica is out of sync, we want to switch to another in-sync replica instead of resetting the offset. One way to do that is the following protocol. 1. Get topic metadata. 2. Pick the "closest" in-sync replica to issue fetch requests. 3. On an OffsetOutOfRangeException, get the smallest/largest offset. If the fetch offset is within the range, go back to step 1 to switch to a different in-sync replica. Otherwise, go through the offset reset logic. 4. Periodically refresh the metadata. Switch to the "closest" in-sync replica for fetching, if needed. > 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 > Assignee: Ashish K Singh > Labels: newbie++ > Fix For: 0.8.3 > > 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)