[jira] [Commented] (KAFKA-5368) Kafka Streams skipped-records-rate sensor produces nonzero values when the timestamps are valid
[ https://issues.apache.org/jira/browse/KAFKA-5368?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16036169#comment-16036169 ] Matthias J. Sax commented on KAFKA-5368: I actually think, we should add an integration test here. We can use a custom TS extractor to return -1 timestamps in a controlled manner. > 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 > Fix For: 0.11.0.0 > > > 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)
[jira] [Commented] (KAFKA-5368) Kafka Streams skipped-records-rate sensor produces nonzero values when the timestamps are valid
[ https://issues.apache.org/jira/browse/KAFKA-5368?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16035429#comment-16035429 ] Guozhang Wang commented on KAFKA-5368: -- Sounds good to me. Maybe we can add one in StreamsMetricsImplTest? Also I think the newly added `Sum` could subsume the original `Total` class as well, but removing `Total` is not backward compatible, neither does changing `Total` to extend from SampledStat and replace `Sum`... Any ideas [~ijuma]? > 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 > Fix For: 0.11.0.0 > > > 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)
[jira] [Commented] (KAFKA-5368) Kafka Streams skipped-records-rate sensor produces nonzero values when the timestamps are valid
[ https://issues.apache.org/jira/browse/KAFKA-5368?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16035344#comment-16035344 ] Matthias J. Sax commented on KAFKA-5368: Could we add a test for this? I think this issue was introduce by code refactoring and slipped because of a missing test... > 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 > Fix For: 0.11.0.0 > > > 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)
[jira] [Commented] (KAFKA-5368) Kafka Streams skipped-records-rate sensor produces nonzero values when the timestamps are valid
[ https://issues.apache.org/jira/browse/KAFKA-5368?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16035281#comment-16035281 ] ASF GitHub Bot commented on KAFKA-5368: --- Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3206 > 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 > Fix For: 0.11.0.0 > > > 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)
[jira] [Commented] (KAFKA-5368) Kafka Streams skipped-records-rate sensor produces nonzero values when the timestamps are valid
[ https://issues.apache.org/jira/browse/KAFKA-5368?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16034461#comment-16034461 ] ASF GitHub Bot commented on KAFKA-5368: --- GitHub user hrafzali opened a pull request: https://github.com/apache/kafka/pull/3206 KAFKA-5368 Kafka Streams skipped-records-rate sensor bug This resolved the issue with Kafka Streams skipped records sensor reporting wrong values. Jira ticket: https://issues.apache.org/jira/browse/KAFKA-5368 The contribution is my original work and I license the work to the project under the project's open source license. You can merge this pull request into a Git repository by running: $ git pull https://github.com/hrafzali/kafka KAFKA-5368_skipped-records-sensor-bug Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/3206.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #3206 commit db4e424c55d63b9172de563fb559002f435b7f01 Author: Hamidreza Afzali Date: 2017-06-02T09:43:32Z KAFKA-5368 Kafka Streams skipped-records-rate sensor bug > 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)
[jira] [Commented] (KAFKA-5368) Kafka Streams skipped-records-rate sensor produces nonzero values when the timestamps are valid
[ 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 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 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)