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

Hugo Louro commented on STORM-2914:
-----------------------------------

[~Srdo][~kabhwan][~avermeerbergen] 

Let’s try to quickly agree on the specific action points that will help us move 
past any blockers and release Storm 1.2.0. I suggest the following, which will 
give us a good balance between making this release stable, but at the same time 
will give us the flexibility to accommodate changes that make sense in the 
future.

# (Optional) Rename ProcessingGuarantee.NONE to 
ProcessingGuarantee.NO_GUARANTEES or UNDEFINED (as Spark uses). I think that 
either of these names are clearer and emphasize that this option is likely not 
what the users wants
# Annotate the enum ProcessingGuarantee with *@Unstable* and document that 
ProcessingGuarantee.NO_GUARANTEES may be removed in the near future.
# For now we keep supporting NO_GUARANTEES mode, but there is a good chance 
that we will remove it down the line, unless for performance reasons. We will 
do a benchmark for this.
# NO_GUARANTEES will use *kafkaConsumer.commitAsync(offsetsToCommit, null);* as 
[~Srdo]
#  proposed in the PR.
# Document that ProcessingGuarantee.NO_GUARANTEES has a behavior identical to 
auto.commit.enable=true but disregards this property altogether (if 
auto.commit.enable is set the KafkaSpoutConfig will throw an exception). 
Furthermore, the commits will not be based on a timer as controlled by 
auto.commit.interval.ms, but rather using the following strategy ([~Srdo]
#  perhaps you can add the details in here).
# AT_MOST_ONCE will use *kafkaConsumer.commitSync(offsetsToCommit);* as [~Srdo]
#  proposed in the PR
# AT_LEAST_ONCE will stay as is (in the current code and in the PR) because 
this option requires no change.
# 
Slightly unrelated with this PR but to address [~kabhwan]’s comment on how to 
persist the offsets. The next version of the KafkaSpout (1.2.x and 1.3+.x) 
should have offsets persistence pluggable and abstracted behind an interface. 
The first implementation of this interface will commit offsets to Kafka as it 
is currently doing. In the future we can add support to write offsets to 
arbitrary durable storage.

If we agree on this, let's incorporate these changes in the PRs [~Srdo] created 
and target having them merged by end of Monday.


> 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