[
https://issues.apache.org/jira/browse/STORM-329?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14312356#comment-14312356
]
ASF GitHub Bot commented on STORM-329:
--------------------------------------
Github user miguno commented on the pull request:
https://github.com/apache/storm/pull/268#issuecomment-73527901
I have improved the patch in this pull request, which particularly meant
modifications to
The updated patch is availabe at:
*
https://github.com/miguno/storm/commits/0.10.0-SNAPSHOT-STORM-392-miguno-merge
* The relevant commit is
https://github.com/miguno/storm/commit/8ebaaf8dbc63df3c2691e0cc3ac5102af7721ec3.
Code lineage: This improved patch includes the latest Storm 0.10.0-SNAPSHOT
code as of today, Feb 09. I used the patch in this pull request (#268) as the
basis, added my changes, and then pulled in the latest upstream Storm changes
for 0.10.0-SNAPSHOT via `git merge upstream/master`.
High-level changes:
- The great majority was refactoring the
[Client.java](https://github.com/apache/storm/blob/master/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java)
in the Netty messaging backend. For instance, I removed unused imports,
obsolete code, clarified variables/methods, consolidated duplicate code, added
documentation, improved logging (which also helped me a lot while I was
bug-hunting), etc.
- I also introduced a few functional changes to the code in order to fix
the issue at hand. Notably, the new code can properly detect disconnects /
channel loss and trigger reconnect events. This also meant that the new code
is now more consistently verifying whether a channel from the client to the
server is truly `CONNECTED`. Also, we're now using a `ListenableFuture` instead
of a `Runnable` to handle the connection/reconnection logic, which leads to
cleaner and more readable code IMHO.
- I updated
[netty_unit_test.clj](https://github.com/apache/storm/blob/master/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj)
so that all tests also wait until the Netty server and client instances are
ready (i.e. all of their connections are ready). This change was needed
because the original+improved STORM-329 patches modify Storm so that it will
ensure the readiness of server and client instances during the initial startup
of a topology, but the Netty unit tests are test-driving servers and clients
directly, so the tests couldn't rely on Storm's help.
Testing:
- The improved patch passes the full Storm's test suite.
- I have successfully tested a patched version of Storm 0.10.0-SNAPSHOT
(code as of today) in a Storm cluster via the aforementioned
storm-bolt-of-death topology.
- With the original patch, restarted bolt instances would not receive
new incoming data. With the improved patch the bolt instances start processing
new data immediately (as expected).
Next steps:
- I would appreciate any feedback of other developers on this improved
patch. I'd particularly appreciate any performance tests in addition to
functional tests. (I do not expect a performance degradation but of course I
might be wrong.)
- The storm-bolt-of-death topology to trigger the cascading failure should
be available by tomorrow for your testing convenience.
Many thanks again to @tedxia and @clockfly for the original patch, which
covered a lot (most?) of work already.
> 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)