[
https://issues.apache.org/jira/browse/STORM-329?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14176533#comment-14176533
]
ASF GitHub Bot commented on STORM-329:
--------------------------------------
Github user clockfly commented on the pull request:
https://github.com/apache/storm/pull/268#issuecomment-59676137
```
When target worker is down, the data sending to other target worker should
not be blocked.
The approach we currently using is to drop messages when connection to
target worker is not available.
```
This solution may need further discussions:
approach A(adopted in current patch):
If we drop the message, the dropped message may takes up to 30 seconds to
be replayed(depending on config topology.message.timeout.secs).
At the same time it is safer for current worker (no OOM, especially for
unacked topology), and messages dispatching to other workers(no blocking).
approach B:
If we do buffering in the netty client, the latency varies in two case:
case1: target worker is alive, we are doing re-connecting, and the
reconnection will eventually succeed. The latency includes the time to connect
to current worker, and the time interval of flusher.
case 2: target worker is not alive. but the source worker have not be
aware of that. in this case, the latency will be same as approach A(30 seconds
by default)
approach C:
@HeartSaVioR raised that it may be more reasonble to buffer the message
outside of netty client. Better buffered in a map which can be retrived with
task Id, so that we can still recover messages to target taskId, if the mapping
from taskId to worker changes.
For this approach, it will requires the messaging layer user(netty client
user) know the status of connection(possible with new interface
ConnectionWithStatus). And it need larger change in clojure. (For efficiency
and performance, we want to group messages to same target host together).
> 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
> Fix For: 0.9.2-incubating
>
> Attachments: storm-329.patch
>
>
> 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)