[
https://issues.apache.org/jira/browse/STORM-329?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14316771#comment-14316771
]
ASF GitHub Bot commented on STORM-329:
--------------------------------------
GitHub user miguno opened a pull request:
https://github.com/apache/storm/pull/429
STORM-329: fix cascading Storm failure by improving reconnection strategy
and buffering messages
**This PR contains the same code as
https://github.com/apache/storm/pull/428 but as a single commit for a cleaner
commit history of our Storm repo.**
--
This is an improved version of the original pull request discussed at
https://github.com/apache/storm/pull/268. Please refer to the discussion in
the link above.
**Note**: Please give attribution to @tedxia when merging the pull request
as he did a lot (most?) of the work in this pull request.
The changes of this pull request include:
- Most importantly, we fix a bug in Storm that may cause a cascading
failure in a Storm cluster, to the point where the whole cluster becomes
unusable. This is achieved by the work described in the next bullet points.
- We refactor and improve the Netty messaging backend, notably the client.
- During the initial startup of a topology, Storm will now wait until
worker (Netty) connections are ready for operation. See the [original
discussion thread](https://github.com/apache/storm/pull/268) for the detailed
explanation and justification of this change.
@clockfly, @tedxia: Please add any further comments to STORM-329 to this
pull request, if possible.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/miguno/storm 0.10.0-SNAPSHOT-STORM-329-diff
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/storm/pull/429.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #429
----
commit 205eaf4ebe28ab5550a842ea9aabd23b41678743
Author: Michael G. Noll <[email protected]>
Date: 2015-02-11T18:55:53Z
STORM-329: fix cascading Storm failure by improving reconnection strategy
and buffering messages
Thanks to @tedxia for the initial work on this patch, which covered a
lot if not most of the work!
----
> Add Option to Config Message handling strategy when connection timeout
> ----------------------------------------------------------------------
>
> Key: STORM-329
> URL: https://issues.apache.org/jira/browse/STORM-329
> Project: Apache Storm
> Issue Type: Improvement
> Affects Versions: 0.9.2-incubating
> Reporter: Sean Zhong
> Priority: Minor
> Labels: Netty
> Attachments: storm-329.patch, worker-kill-recover3.jpg
>
>
> This is to address a [concern brought
> up|https://github.com/apache/incubator-storm/pull/103#issuecomment-43632986]
> during the work at STORM-297:
> {quote}
> [~revans2] wrote: Your logic makes since to me on why these calls are
> blocking. My biggest concern around the blocking is in the case of a worker
> crashing. If a single worker crashes this can block the entire topology from
> executing until that worker comes back up. In some cases I can see that being
> something that you would want. In other cases I can see speed being the
> primary concern and some users would like to get partial data fast, rather
> then accurate data later.
> Could we make it configurable on a follow up JIRA where we can have a max
> limit to the buffering that is allowed, before we block, or throw data away
> (which is what zeromq does)?
> {quote}
> If some worker crash suddenly, how to handle the message which was supposed
> to be delivered to the worker?
> 1. Should we buffer all message infinitely?
> 2. Should we block the message sending until the connection is resumed?
> 3. Should we config a buffer limit, try to buffer the message first, if the
> limit is met, then block?
> 4. Should we neither block, nor buffer too much, but choose to drop the
> messages, and use the built-in storm failover mechanism?
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)