[
https://issues.apache.org/jira/browse/STORM-329?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14166418#comment-14166418
]
ASF GitHub Bot commented on STORM-329:
--------------------------------------
Github user tedxia commented on the pull request:
https://github.com/apache/storm/pull/268#issuecomment-58614296
In this patch, Client have TimeCacheMap for cache new messages that has't
been send out. Worker only need add message to corresponding Client, and Client
will decide when to send the new arrival message. At most time Client will send
new arrival message as soon as possible, but when the connect had been lost or
there are older message. So I removed "send" in worker.clj.
For Client, there is a State machine stand for Client's current state, such
as NEW, CONNECTED, DISCONNECTED and CLOSED. At first Client' state is NEW, then
we will connect remote worker by start a timer that run connect logic (we call
it connector) immediately. If connect success, Client't state will become
CONNECTED, connector will start a periodic timer for send message to remote
worker (we call it flusher). If connect failed, it will retry a few times
until success or Client's state become CLOSED. When Client's state become
CLOSED, Client will be destoried. Flusher will send cached message
periodically, and when flush failed Client's state will become DISCONNECTED and
start connector immediately. For reduce message transfer delay, when some
message arrival, we will start flusher immediately, and of course flusher only
work when Client's state are CONNECTED.
That it the main change for this patch. I am very sorry for all of this. As
a new man for opensource, i will try my best to do better.
Thanks all.
> 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
>
>
> 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)