[ https://issues.apache.org/jira/browse/STORM-2914?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16342319#comment-16342319 ]
Stig Rohde Døssing commented on STORM-2914: ------------------------------------------- [~kabhwan] It would be good to figure out why Alexandre was seeing that behavior. Just to be clear though, this issue isn't related to fixing that. I simply wanted to know whether people would object to losing the NONE guarantee. To be a little more concrete about the difference between autocommit and how the spout acts with AT_MOST_ONCE, when the consumer is configured to autocommit, it will set an internal timestamp. If we call poll when the timestamp is sufficiently old, the consumer will asynchronously commit the offsets it has returned in earlier poll calls by effectively calling commitAsync on the consumer. When the spout is set to use AT_MOST_ONCE, the spout immediately calls commitSync after calling poll. I don't believe there's any difference between what goes over the network in the two calls, the only differences should be that one call is blocking and one is not, and that the AT_MOST_ONCE setting commits after every poll, while autocommit commits only on polls where the timer has expired. The linked PR removes the option to use autocommit, and makes the spout emulate autocommit's behavior when using NONE. After writing the code I don't think it's a big deal if we want to keep NONE, I thought it would add more complexity. > Remove enable.auto.commit support from storm-kafka-client > --------------------------------------------------------- > > Key: STORM-2914 > URL: https://issues.apache.org/jira/browse/STORM-2914 > Project: Apache Storm > Issue Type: Improvement > Components: storm-kafka-client > Affects Versions: 2.0.0, 1.2.0 > Reporter: Stig Rohde Døssing > Assignee: Stig Rohde Døssing > Priority: Major > Labels: pull-request-available > Time Spent: 10m > Remaining Estimate: 0h > > The enable.auto.commit option causes the KafkaConsumer to periodically commit > the latest offsets it has returned from poll(). It is convenient for use > cases where messages are polled from Kafka and processed synchronously, in a > loop. > Due to https://issues.apache.org/jira/browse/STORM-2913 we'd really like to > store some metadata in Kafka when the spout commits. This is not possible > with enable.auto.commit. I took at look at what that setting actually does, > and it just causes the KafkaConsumer to call commitAsync during poll (and > during a few other operations, e.g. close and assign) with some interval. > Ideally I'd like to get rid of ProcessingGuarantee.NONE, since I think > ProcessingGuarantee.AT_MOST_ONCE covers the same use cases, and is likely > almost as fast. The primary difference between them is that AT_MOST_ONCE > commits synchronously. > If we really want to keep ProcessingGuarantee.NONE, I think we should make > our ProcessingGuarantee.NONE setting cause the spout to call commitAsync > after poll, and never use the enable.auto.commit option. This allows us to > include metadata in the commit. -- This message was sent by Atlassian JIRA (v7.6.3#76005)