[
https://issues.apache.org/jira/browse/KAFKA-901?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13660835#comment-13660835
]
Jun Rao commented on KAFKA-901:
-------------------------------
Thanks for patch v4. A few more comments:
40. KafkaController.ControllerContext: This is not introduced in this patch,
but serveOrShuttingDownBrokerIds should just be liveBrokerIdUnderlying.
41.ControllerBrokerRequestBatch: Instead of maintaining aliveBrokers, could we
just get it from controllerContext?
42. KafkaApis.handleTopicMetadataRequest:
42.1 We can rewrite the following statement
val partitionMetadata = sortedPartitions.map { partitionReplicaMap =>
to
val partitionMetadata = sortedPartitions.map { case(topicAndPartition,
partitionState) =>
Then, we don't have to redefine topicAndPartition and partitionState.
42.2 The val partitionReplicaAssignment seems unintuitive. Should we rename it
to partitionStateInfo?
42.3 The outermost try/catch is unnecessary since it should be handled by the
caller handle().
42.4 Not sure if we need to log the following error since it's either due to
LeaderNotAvailable or ReplicaNotAvailable, both are expected.
error("Error while fetching topic metadata for topic %s due to %s
".format(topicMetadata.topic,
43. UpdateMetadataRequest: This class needs to define handleError(). This
method is actually required to be defined in every request. So we should remove
the empty body of handleError() in RequestOrResponse.
44. UpdateMetadataResponse: Do we really need the per partition level error
code? It seems that a global error code is enough.
45. ConsumerFetcherManager: We should put the following statement under
logger.isDebugEnabled().
topicsMetadata.foreach(topicMetadata =>
debug(topicMetadata.toString()))
46. TopicMetadata.toString(): It only prints the leader. We need to print other
fields in PartitionMetadata too.
47. BrokerPartitionInfo: If the metadata response has no error, it seems that
we show throw an UnknownTopicOrPartitionException, instead of a KafkaException.
Alternatively, should we not throw exception at all in this case since the
caller already has to deal with the case when there is no metadata?
48. AsyncProducerTest.testInvalidPartition(): The message in the following
statement is a bit missing leading. It's probably better to say sth like
"Should not thrown any exception". Actually, instead of catching just
UnknownTopicOrPartitionException, we should catch and fail any exception.
case e: UnknownTopicOrPartitionException => fail("Should fail with
UnknownTopicOrPartitionException")
> 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, kafka-901-v2.patch, kafka-901-v4.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