This page is meant as a template for writing a KIP. To create a KIP choose Tools->Copy on this page and modify with your content and replace the heading with the next KIP number and a description of your issue. Replace anything in italics with your own description.

Status

Current state: Accepted (1.0.0)

Discussion thread: here

JIRA: KAFKA-5738

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

Kafka metrics currently group multiple metrics into an MBean with each metric as an Attribute. This is different from other metrics like yammer, where each metric has its own MBean with multiple attributes. For rate metrics, Kafka metrics provides a single attribute that is the rate within a window while yammer metrics has multiple attributes including OneMinuteRate, FiveMinuteRate etc. and cumulative count. Cumulative counts work correctly regardless of metric reporting rate or missing samples and (crucially) do not rely on knowing the window size of the sensor. It makes downstream processing simpler, more accurate, and more flexible. External reporting systems can easily calculate rates and sliding windows based on the count.

For backward compatibility, we need to retain the existing mapping of Kafka metrics to Attributes. To improve processing of rate metrics, this KIP proposes to add an additional attribute for cumulative count corresponding to each rate metric.

Public Interfaces

Every rate metric will be created with two attributes, xxx-rate and xxx-total. xxx-rate will be rate within a window as they are today. xxx-total will be the cumulative count that may be used to calculate rates. xxx-rate is being retained for backward compatibility.

New metrics added by this KIP (right hand column of each table show the new metric, first column is the corresponding rate metric):

Common monitoring metrics for producer/consumer/connect/streams:

connection-creation-rateconnection-creation-total
connection-close-rateconnection-close-total
network-io-ratenetwork-io-total
outgoing-byte-rateoutgoing-byte-total
request-raterequest-total
incoming-byte-rateincoming-byte-total
response-rateresponse-total
select-rateselect-total
io-wait-ratioio-wait-time-total
io-ratioio-time-total

 

Common per-broker metrics for producer/consumer/connect/streams:

outgoing-byte-rateoutgoing-byte-total
request-raterequest-total
incoming-byte-rateincoming-byte-total
response-rateresponse-total


Producer Monitoring

bufferpool-wait-timebufferpool-wait-time-total
record-send-raterecord-send-total
record-retry-raterecord-retry-total
record-error-raterecord-error-total
byte-ratebyte-total
compression-ratecompression-total

Consumer Monitoring

Consumer Group Metrics

commit-ratecommit-total
heartbeat-rateheartbeat-total
join-ratejoin-total
sync-ratesync-total

Consumer Fetch Metrics

bytes-consumed-ratebytes-consumed-total
fetch-ratefetch-total
records-consumed-raterecords-consumed-total


Streams Monitoring

Thread metrics

commit-ratecommit-total
poll-ratepoll-total
process-rateprocess-total
punctuate-ratepunctuate-total
task-created-ratetask-created-total
task-closed-ratetask-closed-total
skipped-records-rateskipped-records-total

Task Metrics

commit-ratecommit-total

Process Node Metrics

[process | punctuate | create | destroy]-rate[process | punctuate | create | destroy]-total
forward-rateforward-total

State Store Metrics

[put | put-if-absent | get | delete | put-all | all | range | flush | restore]-rate[put | put-if-absent | get | delete | put-all | all | range | flush | restore]-total


Proposed Changes

A new Stat named org.apache.kafka.common.metrics.stats.Meter will be added which will be a CompoundStat similar to Percentiles, with two metrics: rate and total (e.g. request-rate and request-total). For consistency, metrics like io-wait-ratio which specify time ratio will also have two attributes (e.g. io-wait-ratio and io-wait-time-total).

Example:

Creating Rate Metrics
rateMetricName = metrics.metricName("request-rate", metricGrpName, "The average number of requests sent per second.", metricTags);
totalMetricName = metrics.metricName("request-total", metricGrpName, "The total number of requests sent.", metricTags);
this.bytesSent.add(new Meter(new Count(), rateMetricName, totalMetricName));

 

Meter would be a CompoundStat that returns both total and the existing rate metrics. This is already handled correctly to support Percentiles. The two metrics appear as two Attributes.

Metrics from Meter as a CompoundStat
 @Override
 public List<NamedMeasurable> stats() {
    List<NamedMeasurable> stats = new ArrayList<NamedMeasurable>(2);
    stats.add(new NamedMeasurable(totalMetricName, total));
    stats.add(new NamedMeasurable(rateMetricName, rate));
    return stats;
}

 

Compatibility, Deprecation, and Migration Plan

  • What impact (if any) will there be on existing users?

Existing attributes for rate metrics are being retained to avoid any impact on existing users.

  • If we are changing behavior how will we phase out the older behavior?

There are no plans to remove the existing metrics attributes that expose rates. When viewing JMX metrics through jconsole, it is useful to view the rate attributes.

Rejected Alternatives

Map each Kafka Metric to an MBean and provide attributes for count, rates etc. 

This is similar to yammer metrics, but would be a breaking change. The proposed change provides the same data without breaking backward compatibility. It is slightly messier to add more attributes to a metric with the current format in future, but since cumulative count is flexible enough for external tools to provide different views, this is a reasonable compromise to retain compatibility.

 

  • No labels