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

Jungtaek Lim commented on STORM-2914:
-------------------------------------

[~hmclouro] [~Srdo]
As I showed the example of Flink and Spark and I believe others are doing it, 
we need to make data source being "adapted" to Storm's nature when implementing 
spout. We are making KafkaSpout being adapted to how Spout is working, and if 
we are going to support stateful exactly-once via checkpoint and need to make 
storm-kafka-client supporting this, we may also need to have a proper strategy 
like Spark and Flink, get rid of committing to kafka or at least not relying on 
that for checkpoint, since it is ideal to store checkpoint into "same" durable 
storage.

We need to take this into account while designing the spout and restrict some 
features if necessary, instead of letting users to use whatever they want to, 
but with the risks we don't guide whichever will be. If we really want to 
provide the feature which doesn't conform to Storm's nature, we should warn 
users, guiding the possible risks are best, but at least guiding that they 
should know what is exactly they're doing now.

That's why I'm not in favor of having NONE processing guarantee semantic only 
for storm-kafka-client, but if it really has enough value to take the risks (if 
there's huge gap between at-most-once and none in point of  performance view) 
we may need to provide the feature, with guiding the risks.

> 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