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

Stig Rohde Døssing commented on STORM-2914:
-------------------------------------------

[~avermeerbergen]

Yes, those logs are expected to occur, but only once per spout instance. Once 
the spout commits, the log should not be shown again.

[~kabhwan]

Sure, if we're moving to a stateful checkpointing model we might want to use 
that mechanism's storage to commit Kafka offsets as well. I don't think there's 
a reason to switch off of Kafka storage yet though.

If I'm understanding you correctly, you're saying that it's fine to restrict 
how users can configure the spout and consumer, because some configurations 
don't make sense for Storm (right?) If so, I agree and I think it's what we're 
trying to do with e.g. the code around enable.auto.commit already 
([https://github.com/apache/storm/blob/master/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java#L461),]
 or by not offering an option to use the subscribe API. This issue has largely 
the same kind of intent.

I'm hoping to have a bit of time to compare NONE and AT_MOST_ONCE this weekend. 
I'm wondering if [~avermeerbergen] has any interest in comparing these two 
settings as well, since you have a real workload?

Regarding the spout only committing when nextTuple is called, you have a point. 
I'm not sure why we couldn't move commit to happen during acks for the 
at-least-once case (the only one where acks matter). I don't have any insight 
into whether the case you describe would be likely to happen, but I'd think 
that it would often be possible to "luck into" a commit during the tuple 
sequence that leads up to triggering backpressure.

 

> 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: 20m
>  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)

Reply via email to