[jira] [Commented] (KAFKA-4942) Kafka Connect: Offset committing times out before expected
[ https://issues.apache.org/jira/browse/KAFKA-4942?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16039681#comment-16039681 ] ASF GitHub Bot commented on KAFKA-4942: --- Github user simplesteph closed the pull request at: https://github.com/apache/kafka/pull/2730 > Kafka Connect: Offset committing times out before expected > -- > > Key: KAFKA-4942 > URL: https://issues.apache.org/jira/browse/KAFKA-4942 > Project: Kafka > Issue Type: Bug > Components: KafkaConnect >Affects Versions: 0.10.2.0 >Reporter: Stephane Maarek > Fix For: 0.11.0.0, 0.11.1.0 > > > On Kafka 0.10.2.0 > I run a connector that deals with a lot of data, in a kafka connect cluster > When the offsets are getting committed, I get the following: > {code} > [2017-03-23 03:56:25,134] INFO WorkerSinkTask{id=MyConnector-1} Committing > offsets (org.apache.kafka.connect.runtime.WorkerSinkTask) > [2017-03-23 03:56:25,135] WARN Commit of WorkerSinkTask{id=MyConnector-1} > offsets timed out (org.apache.kafka.connect.runtime.WorkerSinkTask) > {code} > If you look at the timestamps, they're 1 ms apart. My settings are the > following: > {code} > offset.flush.interval.ms = 12 > offset.flush.timeout.ms = 6 > offset.storage.topic = _connect_offsets > {code} > It seems the offset flush timeout setting is completely ignored for the look > of the logs. I would expect the timeout message to happen 60 seconds after > the commit offset INFO message, not 1 millisecond later. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (KAFKA-4942) Kafka Connect: Offset committing times out before expected
[ https://issues.apache.org/jira/browse/KAFKA-4942?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16039671#comment-16039671 ] ASF GitHub Bot commented on KAFKA-4942: --- Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2912 > Kafka Connect: Offset committing times out before expected > -- > > Key: KAFKA-4942 > URL: https://issues.apache.org/jira/browse/KAFKA-4942 > Project: Kafka > Issue Type: Bug > Components: KafkaConnect >Affects Versions: 0.10.2.0 >Reporter: Stephane Maarek > Fix For: 0.11.0.0, 0.11.1.0 > > > On Kafka 0.10.2.0 > I run a connector that deals with a lot of data, in a kafka connect cluster > When the offsets are getting committed, I get the following: > {code} > [2017-03-23 03:56:25,134] INFO WorkerSinkTask{id=MyConnector-1} Committing > offsets (org.apache.kafka.connect.runtime.WorkerSinkTask) > [2017-03-23 03:56:25,135] WARN Commit of WorkerSinkTask{id=MyConnector-1} > offsets timed out (org.apache.kafka.connect.runtime.WorkerSinkTask) > {code} > If you look at the timestamps, they're 1 ms apart. My settings are the > following: > {code} > offset.flush.interval.ms = 12 > offset.flush.timeout.ms = 6 > offset.storage.topic = _connect_offsets > {code} > It seems the offset flush timeout setting is completely ignored for the look > of the logs. I would expect the timeout message to happen 60 seconds after > the commit offset INFO message, not 1 millisecond later. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (KAFKA-4942) Kafka Connect: Offset committing times out before expected
[ https://issues.apache.org/jira/browse/KAFKA-4942?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15983038#comment-15983038 ] ASF GitHub Bot commented on KAFKA-4942: --- GitHub user 56quarters opened a pull request: https://github.com/apache/kafka/pull/2912 KAFKA-4942 Fix commitTimeoutMs being set before the commit actually started This fixes KAFKA-4942 This supersededs #2730 /cc @simplesteph @gwenshap @ewencp You can merge this pull request into a Git repository by running: $ git pull https://github.com/smarter-travel-media/kafka fix-connect-offset-commit Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/2912.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 #2912 commit f93bd001a723c5e1402bf92474f43fb0b991a44c Author: simplesteph Date: 2017-03-24T00:03:07Z Fixed commitTimeoutMs being set before the commit actually started Fixes KAFKA-4942 commit e7b704d97b8de35384f6d24ba48f050a0b20be01 Author: Nick Pillitteri Date: 2017-04-21T19:49:37Z Test for commitTimeoutMs being set before commit started See KAFKA-4942 > Kafka Connect: Offset committing times out before expected > -- > > Key: KAFKA-4942 > URL: https://issues.apache.org/jira/browse/KAFKA-4942 > Project: Kafka > Issue Type: Bug > Components: KafkaConnect >Affects Versions: 0.10.2.0 >Reporter: Stephane Maarek > > On Kafka 0.10.2.0 > I run a connector that deals with a lot of data, in a kafka connect cluster > When the offsets are getting committed, I get the following: > {code} > [2017-03-23 03:56:25,134] INFO WorkerSinkTask{id=MyConnector-1} Committing > offsets (org.apache.kafka.connect.runtime.WorkerSinkTask) > [2017-03-23 03:56:25,135] WARN Commit of WorkerSinkTask{id=MyConnector-1} > offsets timed out (org.apache.kafka.connect.runtime.WorkerSinkTask) > {code} > If you look at the timestamps, they're 1 ms apart. My settings are the > following: > {code} > offset.flush.interval.ms = 12 > offset.flush.timeout.ms = 6 > offset.storage.topic = _connect_offsets > {code} > It seems the offset flush timeout setting is completely ignored for the look > of the logs. I would expect the timeout message to happen 60 seconds after > the commit offset INFO message, not 1 millisecond later. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (KAFKA-4942) Kafka Connect: Offset committing times out before expected
[ https://issues.apache.org/jira/browse/KAFKA-4942?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15939464#comment-15939464 ] ASF GitHub Bot commented on KAFKA-4942: --- GitHub user simplesteph opened a pull request: https://github.com/apache/kafka/pull/2730 KAFKA-4942 fixed the commitTimeoutMs being set before the commit actually started this fixes KAFKA-4942 You can merge this pull request into a Git repository by running: $ git pull https://github.com/simplesteph/kafka fix-connect-offset-commit Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/2730.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 #2730 commit 4d1db91ec8276da3dcca6d85b66c70497cf0815d Author: simplesteph Date: 2017-03-24T00:03:07Z fixed the commitTimeoutMs being set before the commit actually started this fixes KAFKA-4942 > Kafka Connect: Offset committing times out before expected > -- > > Key: KAFKA-4942 > URL: https://issues.apache.org/jira/browse/KAFKA-4942 > Project: Kafka > Issue Type: Bug > Components: KafkaConnect >Affects Versions: 0.10.2.0 >Reporter: Stephane Maarek > > On Kafka 0.10.2.0 > I run a connector that deals with a lot of data, in a kafka connect cluster > When the offsets are getting committed, I get the following: > {code} > [2017-03-23 03:56:25,134] INFO WorkerSinkTask{id=MyConnector-1} Committing > offsets (org.apache.kafka.connect.runtime.WorkerSinkTask) > [2017-03-23 03:56:25,135] WARN Commit of WorkerSinkTask{id=MyConnector-1} > offsets timed out (org.apache.kafka.connect.runtime.WorkerSinkTask) > {code} > If you look at the timestamps, they're 1 ms apart. My settings are the > following: > {code} > offset.flush.interval.ms = 12 > offset.flush.timeout.ms = 6 > offset.storage.topic = _connect_offsets > {code} > It seems the offset flush timeout setting is completely ignored for the look > of the logs. I would expect the timeout message to happen 60 seconds after > the commit offset INFO message, not 1 millisecond later. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (KAFKA-4942) Kafka Connect: Offset committing times out before expected
[ https://issues.apache.org/jira/browse/KAFKA-4942?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15939453#comment-15939453 ] Stephane Maarek commented on KAFKA-4942: [~ewencp] nothing weird from the log, except that EVERY offset commit gives me that behaviour. You know what's even weirder? Offsets still are committed. It's because the line you linked me doesn't interrupt the offset commit, it just throws a warning. Now the reason I would see that line being triggered right away is when now >= commitTimeoutMs now >= commitStarted + workerConfig.getLong(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG) I believe the setting workerConfig.getLong should return something 6 ms, so it hints towards commitStarted being corrupted. And I think I just found your issue. commitTimeoutMs happens before the try block. so no matter what happens in the commitOffsets(now, false); , it won't impact the comparison: if (committing && now >= commitTimeoutMs) And that's the bug :) > Kafka Connect: Offset committing times out before expected > -- > > Key: KAFKA-4942 > URL: https://issues.apache.org/jira/browse/KAFKA-4942 > Project: Kafka > Issue Type: Bug > Components: KafkaConnect >Affects Versions: 0.10.2.0 >Reporter: Stephane Maarek > > On Kafka 0.10.2.0 > I run a connector that deals with a lot of data, in a kafka connect cluster > When the offsets are getting committed, I get the following: > {code} > [2017-03-23 03:56:25,134] INFO WorkerSinkTask{id=MyConnector-1} Committing > offsets (org.apache.kafka.connect.runtime.WorkerSinkTask) > [2017-03-23 03:56:25,135] WARN Commit of WorkerSinkTask{id=MyConnector-1} > offsets timed out (org.apache.kafka.connect.runtime.WorkerSinkTask) > {code} > If you look at the timestamps, they're 1 ms apart. My settings are the > following: > {code} > offset.flush.interval.ms = 12 > offset.flush.timeout.ms = 6 > offset.storage.topic = _connect_offsets > {code} > It seems the offset flush timeout setting is completely ignored for the look > of the logs. I would expect the timeout message to happen 60 seconds after > the commit offset INFO message, not 1 millisecond later. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (KAFKA-4942) Kafka Connect: Offset committing times out before expected
[ https://issues.apache.org/jira/browse/KAFKA-4942?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15938892#comment-15938892 ] Ewen Cheslack-Postava commented on KAFKA-4942: -- [~stephane.maa...@gmail.com] Any more context from earlier in the logs? Definitely looks fishy, and I don't see a way that could happen where the log message would just be misleading (it happens here https://github.com/apache/kafka/blob/trunk/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java#L171 after a condition, so it's not anything like an unrelated exception). > Kafka Connect: Offset committing times out before expected > -- > > Key: KAFKA-4942 > URL: https://issues.apache.org/jira/browse/KAFKA-4942 > Project: Kafka > Issue Type: Bug > Components: KafkaConnect >Affects Versions: 0.10.2.0 >Reporter: Stephane Maarek > > On Kafka 0.10.2.0 > I run a connector that deals with a lot of data, in a kafka connect cluster > When the offsets are getting committed, I get the following: > {code} > [2017-03-23 03:56:25,134] INFO WorkerSinkTask{id=MyConnector-1} Committing > offsets (org.apache.kafka.connect.runtime.WorkerSinkTask) > [2017-03-23 03:56:25,135] WARN Commit of WorkerSinkTask{id=MyConnector-1} > offsets timed out (org.apache.kafka.connect.runtime.WorkerSinkTask) > {code} > If you look at the timestamps, they're 1 ms apart. My settings are the > following: > {code} > offset.flush.interval.ms = 12 > offset.flush.timeout.ms = 6 > offset.storage.topic = _connect_offsets > {code} > It seems the offset flush timeout setting is completely ignored for the look > of the logs. I would expect the timeout message to happen 60 seconds after > the commit offset INFO message, not 1 millisecond later. -- This message was sent by Atlassian JIRA (v6.3.15#6346)