You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Jun Rao (JIRA)" <ji...@apache.org> on 2013/11/25 17:30:36 UTC

[jira] [Commented] (KAFKA-1144) commitOffsets can be passed the offsets to commit

    [ https://issues.apache.org/jira/browse/KAFKA-1144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13831608#comment-13831608 ] 

Jun Rao commented on KAFKA-1144:
--------------------------------

Thanks for the patch. A couple of things.

1. We have a jira that tries to move the storage of offsets off ZK (https://issues.apache.org/jira/browse/KAFKA-1000) since ZK is not really designed for that. So, we may not be able to do conditional updates for offsets in the future.

2. We will be rewriting the consumer client (https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite). In the new api, we will add a callback during consumer rebalances. Do you think that addresses your issue as well?


> commitOffsets can be passed the offsets to commit
> -------------------------------------------------
>
>                 Key: KAFKA-1144
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1144
>             Project: Kafka
>          Issue Type: Improvement
>          Components: consumer
>    Affects Versions: 0.8
>            Reporter: Imran Rashid
>            Assignee: Neha Narkhede
>         Attachments: 0001-allow-committing-of-arbitrary-offsets-to-facilitate-.patch, 0002-add-protection-against-backward-commits.patch
>
>
> This adds another version of commitOffsets that takes the offsets to commit as a parameter.
> Without this change, getting correct user code is very hard. Despite kafka's at-least-once guarantees, most user code doesn't actually have that guarantee, and is almost certainly wrong if doing batch processing. Getting it right requires some very careful synchronization between all consumer threads, which is both:
> 1) painful to get right
> 2) slow b/c of the need to stop all workers during a commit.
> This small change simplifies a lot of this. This was discussed extensively on the user mailing list, on the thread "are kafka consumer apps guaranteed to see msgs at least once?"
> You can also see an example implementation of a user api which makes use of this, to get proper at-least-once guarantees by user code, even for batches:
> https://github.com/quantifind/kafka-utils/pull/1
> I'm open to any suggestions on how to add unit tests for this.



--
This message was sent by Atlassian JIRA
(v6.1#6144)