[
https://issues.apache.org/jira/browse/STORM-495?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14157723#comment-14157723
]
ASF GitHub Bot commented on STORM-495:
--------------------------------------
Github user rick-kilgore commented on a diff in the pull request:
https://github.com/apache/storm/pull/254#discussion_r18382648
--- Diff: external/storm-kafka/src/jvm/storm/kafka/SpoutConfig.java ---
@@ -26,8 +26,19 @@
public Integer zkPort = null;
public String zkRoot = null;
public String id = null;
+
+ // setting for how often to save the current kafka offset to ZooKeeper
public long stateUpdateIntervalMs = 2000;
+ // Exponential back-off retry settings. These are used when retrying
messages after a bolt
+ // calls OutputCollector.fail().
+ //
+ // Note: be sure to set backtype.storm.Config.MESSAGE_TIMEOUT_SECS
appropriately to prevent
+ // resubmitting the message while still retrying.
--- End diff --
I created a version of PartitionManager that stops retrying when it would
run up against the TOPOLOGY_MESSAGE_TIMEOUT_SECS setting. But while I was
testing it, I discovered that the Storm framework is not actually failing the
message after this timeout. Did that behavior change recently? I thought I
remembered seeing it do that before.
Since it's not happening, the message ends up getting lost - neither failed
nor acknowledged. So maybe after printing the big nasty error message, I
should still handle the message in some way - either ack() it or just let it
keep retrying?
> Add delayed retries to KafkaSpout
> ---------------------------------
>
> Key: STORM-495
> URL: https://issues.apache.org/jira/browse/STORM-495
> Project: Apache Storm
> Issue Type: Improvement
> Affects Versions: 0.9.3
> Environment: all environments
> Reporter: Rick Kilgore
> Priority: Minor
> Labels: kafka, retry
>
> If a tuple in the topology originates from the KafkaSpout from the
> external/storm-kafka sources, and if a bolt in the topology indicates a
> failure by calling fail() on its OutputCollector, the KafkaSpout will
> immediately retry the message.
> We wish to use this failure and retry behavior in our ingestion system
> whenever we experience a recoverable error from a downstream system, such as
> a 500 or 503 error from a service we depend on. But with the current
> KafkaSpout behavior, doing so results in a tight loop where we retry several
> times over a few seconds and then give up. I want to be able to delay retry
> to give the downstream service some time to recover. Ideally, I would like
> to have configurable, exponential backoff retry.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)