[
https://issues.apache.org/jira/browse/KAFKA-901?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Neha Narkhede updated KAFKA-901:
--------------------------------
Attachment: kafka-901.patch
I like Joel's suggestion for 2 reasons -
1. Like he mentioned, if the controller pushes metadata updates to brokers, it
will avoid the herd effect when multiple clients need to update metadata.
2. It is not a good idea to overload LeaderAndIsrRequest with UpdateMetadata
since those 2 requests are fired under different circumstances. Part of this
complication is also due to the fact that LeaderAndIsrRequest is also
overloaded by start replica state change.
The latest patch includes Joel's suggestion. It includes the following changes -
1. A new controller state change request and response is defined -
UpdateMetadataRequest and UpdateMetadataResponse. UpdateMetadataRequest has the
partition state information like leader, isr, replicas for a list of
partitions. In addition to this, it also has a list of live brokers and the
broker id -> host:port mapping for all brokers in the cluster. The live brokers
information is used when the broker handles metadata request to figure out if
the leader is alive or not. UpdateMetadataResponse is similar to
LeaderAndIsrResponse, in the sense that it has an error code per partition and
a top level error code just like any other state change request
2. Every kafka broker maintains 3 data structures - leader cache, the list of
alive brokers, the broker id-> host:port mapping for all brokers. These data
structures are updated by the UpdateMetadataRequest and queried by the
TopicMetadataRequest. So those accesses need to be synchronized
3. The controller fires the update metadata request -
3.1 When a new broker is started up. The newly restarted brokers are sent the
partition state info for all partitions in the cluster.
3.2 When a broker fails, since leaders for some partitions would've changed
3.3 On a controller failover, since there could've been leader changes during
the failover
3.4 On preferred replica election, since leaders for many partitions could've
changed
3.5 On partition reassignment, since leader could've changed for the reassigned
partitions
3.6 On controlled shutdown, since leaders move for the partitions hosted on the
broker being shut down
4. Unit tests have changed to wait until the update metadata request has
trickled to all servers. The best way I could think of is to make the KafkaApis
object and the leaderCache accessible from KafkaServer.
> Kafka server can become unavailable if clients send several metadata requests
> -----------------------------------------------------------------------------
>
> Key: KAFKA-901
> URL: https://issues.apache.org/jira/browse/KAFKA-901
> Project: Kafka
> Issue Type: Bug
> Components: replication
> Affects Versions: 0.8
> Reporter: Neha Narkhede
> Assignee: Neha Narkhede
> Priority: Blocker
> Attachments: kafka-901.patch, metadata-request-improvement.patch
>
>
> Currently, if a broker is bounced without controlled shutdown and there are
> several clients talking to the Kafka cluster, each of the clients realize the
> unavailability of leaders for some partitions. This leads to several metadata
> requests sent to the Kafka brokers. Since metadata requests are pretty slow,
> all the I/O threads quickly become busy serving the metadata requests. This
> leads to a full request queue, that stalls handling of finished responses
> since the same network thread handles requests as well as responses. In this
> situation, clients timeout on metadata requests and send more metadata
> requests. This quickly makes the Kafka cluster unavailable.
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira