[ https://issues.apache.org/jira/browse/KAFKA-3068?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15084481#comment-15084481 ]
Jason Gustafson commented on KAFKA-3068: ---------------------------------------- [~junrao] It does seem like an issue to me (can't believe we all missed it). From memory, I think the problem that the nodesEverSeen collection was trying to solve was what to do when the last node in the cluster becomes unreachable. In this case, the only thing the client can do is keep trying to reconnect to the one node indefinitely (since we have no other ways to discover other nodes in the cluster). By keeping track of the history of nodes, we have an out in this case and we can retry against one of the other nodes that we connected to before. But, as you say, this can lead to other problems. It seems like what we should do in this case is maybe revert to the bootstrap brokers in configuration. > NetworkClient may connect to a different Kafka cluster than originally > configured > --------------------------------------------------------------------------------- > > Key: KAFKA-3068 > URL: https://issues.apache.org/jira/browse/KAFKA-3068 > Project: Kafka > Issue Type: Bug > Components: clients > Affects Versions: 0.9.0.0 > Reporter: Jun Rao > > In https://github.com/apache/kafka/pull/290, we added the logic to cache all > brokers (id and ip) that the client has ever seen. If we can't find an > available broker from the current Metadata, we will pick a broker that we > have ever seen (in NetworkClient.leastLoadedNode()). > One potential problem this logic can introduce is the following. Suppose that > we have a broker with id 1 in a Kafka cluster. A producer client remembers > this broker in nodesEverSeen. At some point, we bring down this broker and > use the host in a different Kafka cluster. Then, the producer client uses > this broker from nodesEverSeen to refresh metadata. It will find the metadata > in a different Kafka cluster and start producing data there. -- This message was sent by Atlassian JIRA (v6.3.4#6332)