[
https://issues.apache.org/jira/browse/KAFKA-1367?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14571268#comment-14571268
]
Joel Koshy commented on KAFKA-1367:
-----------------------------------
Follow-up from the KIP hangout. Side-note: this and most of the above comments
are actually implementation details for KAFKA-2225. This is relevant here only
because we are considering keeping vs. removing the ISR field.
I do think it is possible to implement KAFKA-2225 without ISR support either in
metadata or the fetch response. The fetch response already contains HW. So the
consumer can watch its fetch offset and the current HW (from the last fetch
response). If the fetchOffset << HW but if the fetch response size is smaller
than the requested bytes and the highest offset in the response is << HW then
the consumer knows that the follower that it is fetching from is lagging behind
(especially if this difference increases in successive fetches). The main
caveat with this is that it depends on the replica having a live replica
fetcher. The other issue is that the consumer needs to have its own definition
of what it takes to deem a replica as out of sync (since the replica lag time
config is server-side). The other observation is that ISR is a highly relevant
and useful field in the topic metadata response. I would be in favor of keeping
it in the TMR and just having the consumer refresh the topic metadata
periodically to keep itself informed of ISR changes.
> 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)