[ 
https://issues.apache.org/jira/browse/KAFKA-5368?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16034367#comment-16034367
 ] 
Hamidreza Afzali commented on KAFKA-5368:
-----------------------------------------

*Problem:*

Skipped records sensor is using a {{Rate}} of type {{Count}} for 
skipped-records-rate metric.

In 
{{org.apache.kafka.streams.processor.internals.StreamThread#addRecordsToTasks}} 
the Count value is incremented by one regardless of the number of skipped 
records, i.e. the value increments even if no record is skipped.


{code}
skippedRecordsSensor.add(metrics.metricName("skipped-records-rate", 
this.groupName, "The average per-second number of skipped records.", 
this.tags), new Rate(new Count()));

...

private void addRecordsToTasks(final ConsumerRecords<byte[], byte[]> records) {
    if (records != null && !records.isEmpty()) {
        ...
        streamsMetrics.skippedRecordsSensor.record(records.count() - 
numAddedRecords, timerStartedMs);
    }
}
{code}

{{org.apache.kafka.streams.processor.internals.StreamThread#addRecordsToTasks}} 
is called in 
{{org.apache.kafka.streams.processor.internals.StreamThread#runLoop}} after 
each successful poll request.

{code}

private void runLoop() {
    ...
    while (stillRunning()) {
        ...
        final ConsumerRecords<byte[], byte[]> records = 
pollRequests(pollTimeMs);
        if (records != null && !records.isEmpty() && !activeTasks.isEmpty()) {
            streamsMetrics.pollTimeSensor.record(computeLatency(), 
timerStartedMs);
            addRecordsToTasks(records);
            ...
        }
        ...
    }
    ...
}
{code}

This can explain why skipped-records-rate is equal to poll-rate.

*Solution:*

The sensor should keep a sum of all skipped records.



> Kafka Streams skipped-records-rate sensor produces nonzero values when the 
> timestamps are valid
> -----------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-5368
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5368
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>            Reporter: Hamidreza Afzali
>            Assignee: Hamidreza Afzali
>
> Kafka Streams skipped-records-rate sensor produces nonzero values even when 
> the timestamps are valid and records are processed. The values are equal to 
> poll-rate.
> Related issue: KAFKA-5055 



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to