[ https://issues.apache.org/jira/browse/KAFKA-3260?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15157935#comment-15157935 ]
ASF GitHub Bot commented on KAFKA-3260: --------------------------------------- GitHub user jcustenborder opened a pull request: https://github.com/apache/kafka/pull/950 KAFKA-3260 - Added SourceTask.commitRecord Added commitRecord(SourceRecord record) to SourceTask. This method is called during the callback from producer.send() when the message has been sent successfully. Added commitTaskRecord(SourceRecord record) to WorkerSourceTask to handle calling commitRecord on the SourceTask. Updated tests for calls to commitRecord. You can merge this pull request into a Git repository by running: $ git pull https://github.com/jcustenborder/kafka KAFKA-3260 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/950.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 #950 ---- commit f4e1826f659af99e39189d45af214fd9f030b77b Author: Jeremy Custenborder <jcustenbor...@gmail.com> Date: 2016-02-22T23:22:02Z KAFKA-3260 - Added commitRecord(SourceRecord record) to SourceTask. This method during the callback from producer.send() when the message has been sent successfully. Added commitTaskRecord(SourceRecord record) to WorkerSourceTask to handle calling commitRecord on the SourceTask. Updated tests for calls to commitRecord. ---- > Increase the granularity of commit for SourceTask > ------------------------------------------------- > > Key: KAFKA-3260 > URL: https://issues.apache.org/jira/browse/KAFKA-3260 > Project: Kafka > Issue Type: Improvement > Components: copycat > Affects Versions: 0.9.0.1 > Reporter: Jeremy Custenborder > Assignee: Ewen Cheslack-Postava > > As of right now when commit is called the developer does not know which > messages have been accepted since the last poll. I'm proposing that we extend > the SourceTask class to allow records to be committed individually. > {code} > public void commitRecord(SourceRecord record) throws InterruptedException > { > // This space intentionally left blank. > } > {code} > This method could be overridden to receive a SourceRecord during the callback > of producer.send. This will give us messages that have been successfully > written to Kafka. The developer then has the capability to commit messages to > the source individually or in batch. -- This message was sent by Atlassian JIRA (v6.3.4#6332)