You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Jun Rao (JIRA)" <ji...@apache.org> on 2014/08/19 00:29:23 UTC

[jira] [Commented] (KAFKA-1597) New metrics: ResponseQueueSize and BeingSentResponses

    [ https://issues.apache.org/jira/browse/KAFKA-1597?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14101429#comment-14101429 ] 

Jun Rao commented on KAFKA-1597:
--------------------------------

Thanks for the patch. For ResponseQueueSize, I am not sure how intuitive it is to report the total size across queues. Would reporting a max and avg across all response queues be better?

I want to clarify on BeingSentResponses. Is there a particular reason that you want to track this? I was thinking that we can track an InflightRequests on the broker that measures # of requests not completed by the broker.

> New metrics: ResponseQueueSize and BeingSentResponses
> -----------------------------------------------------
>
>                 Key: KAFKA-1597
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1597
>             Project: Kafka
>          Issue Type: New Feature
>          Components: core
>            Reporter: Alexis Midon
>            Priority: Minor
>              Labels: features
>         Attachments: KAFKA-1594_BeingSentResponses.patch, KAFKA-1594_ResponseQueueSize.patch
>
>
> This patch adds two metrics:
> h3. ResponseQueueSize
> As of 0.8.1, the sizes of the response queues are [reported as different metrics|https://github.com/apache/kafka/blob/0.8.1/core/src/main/scala/kafka/network/RequestChannel.scala#L127-L134] - one per processor thread. This is not very ideal for different reasons:
> * charts have to sum the different metrics
> * the metrics collection system might not support 'wild card queries' like {{sum:kafka.network.RequestChannel.Processor_*_ResponseQueueSize}} in which case monitoring now depends on the number of configured network threads
> * monitoring the response by thread is not very valuable. However the global number of responses is useful.
> * proposal*
> So this patch exposes the total number of queued responses as a metric {{ResponseQueueSize}}
> *implementation*
> In {{RequestChannel}}, create a Gauge that adds up the size of the response queues.
> h3. BeingSentResponses
> As of 0.8.1, the processor threads will poll responses from the queues and attach them to the SelectionKey as fast as possible. The consequence of that is that the response queues are not a good indicator of the number of "in-flight" responses. The {{ServerSocketChannel}} acting as another queue of response to be sent.
> The current metrics don't reflect the size of this "buffer", which is an issue.
> *proposal*
> This patch adds a gauge that keeps track of the number of responses being handled by the {{ServerSocketChannel}}.
> That new metric is named "BeingSentResponses" (who said naming was hard?)
> *implementation*
> To calculate that metric, the patch adds up the number of SelectionKeys interested in writing, across processor threads.
> Another approach could be to keep all in-flight responses in a data structure (let's say a map) shared by the processor threads. A response will be added to that map when dequeued from the response queue, and removed when the write is complete. The gauge will simply report the size of that map. I decided against that second approach as it is more intrusive and requires some additional bookkeeping to gather information already available through the {{SelectionKey}}'s



--
This message was sent by Atlassian JIRA
(v6.2#6252)