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

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

[~avermeerbergen]

Regarding your use case, you might try at-least-once mode plus a filter to 
discard old metrics, plus the topology.max.spout.pending parameter to limit how 
many messages go into your topology at a time. That said, I'll continue on with 
the assumption that your current configuration is what you want to keep.

I think it's likely that use cases like yours where you don't need 
at-least-once would be equally well supported by using the AT_MOST_ONCE 
processing guarantee 
https://github.com/apache/storm/blob/master/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java#L165.
 

The main difference between enable.auto.commit and AT_MOST_ONCE is that 
AT_MOST_ONCE synchronously commits offsets to Kafka once it has polled, but 
before the tuples are emitted, while enable.auto.commit commits every once in a 
while before the consumer polls. What this means is that enable.auto.commit may 
emit a tuple 0, 1, 2, etc. times, while AT_MOST_ONCE will emit a tuple 0 or 1 
times. Since users of enable.offset.commit don't care how many times a message 
is processed, they should be fine with the AT_MOST_ONCE guarantee.

My only concern with dropping support for enable.auto.commit and asking people 
to use AT_MOST_ONCE instead, would be the switch to synchronous committing, 
which may have some impact on performance. I would be surprised if the impact 
were meaningful though. 

For now I think we should try to make the spout emulate enable.auto.commit when 
the processing guarantee is NONE, without actually setting enable.auto.commit 
on the consumer. That way we can specify metadata when committing, which allows 
us to fix STORM-2913 properly. If we decide later that enable.auto.commit is 
unnecessary, we can remove NONE as an option.

> 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
>
> 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