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

Alexandre Vermeerbergen commented on STORM-2914:
------------------------------------------------

Hello [~Srdo],

We use auto-commit when consuming Kafka messages in our "real-time alerting" 
topology.

This topology doesn't uses acking not anchoring

Indeed, the purpose of our "real-time alerting" topology is to evaluate metrics 
read from Kafka through expressions (called triggers, like in Zabbix) into 
"alerting severities", which we write to both a Redis database (for rendering 
in an alerting web app) and into other Kafka topics (for notifications / 
persistency purposes handled by other topologies).

We observed that sometimes our alerting topology is flooded by an excessive 
rate of metrics = Kafka messages read by kafka spout, because sometimes a 
remote Kafka cluster which was temporarily unable to replicate data to our 
central Kafka cluster (the one plugged to our topologies) comes "back to life" 
and sends all metrics at once, including metrics which are too old to be 
relevant for real-time alerting.

And because if Storm 1.x back-pressure is kind of ... well, say "limited", the 
result of such flooding is that our topology is just stuck and no longer 
consumes any metric.

So we have a simple self-healer "cron" which periodically checks whether or not 
our topology consumes metrics over a sliding windows of 10 minutes, and when 
consumption is stopped for 10 minutes, this cron will restart our topology with 
"LATEST " strategy. 

 

No with this context, after trying many combinations we found out that with 
Storm 1.1.0 and with our own Kafka spout (working in auto-commit mode, but not 
as powerful as storm kafka client), we had to enable auto-commit otherwise our 
topology would get stuck quite too often.

We had been waiting for Storm 1.2.0 to use the official storm-kafka-client  
with auto-commit support, because our own spout was quite limited.

Now, in order to answer your question about storm-kafka-client removal of 
auto-commit support, I am unsure about what I should answer: what would 
guarantee me that my real-time alerting topology's stability won't be worse 
than it was when I used storm-kafka-client before the following commit broke it 
:[https://github.com/apache/storm/commit/a3899b75a79781602fa58b90de6c8aa784af5332#diff-7d7cbc8f5444fa7ada7962033fc31c5e
 
|https://github.com/apache/storm/commit/a3899b75a79781602fa58b90de6c8aa784af5332#diff-7d7cbc8f5444fa7ada7962033fc31c5e]?

Best regards,

Alexandre Vermeerbergen

 

> 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