[ 
https://issues.apache.org/jira/browse/STORM-329?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14197418#comment-14197418
 ] 

ASF GitHub Bot commented on STORM-329:
--------------------------------------

Github user tedxia commented on the pull request:

    https://github.com/apache/storm/pull/268#issuecomment-61752674
  
    I test this patch on our product cluster, with five machine, each with 6 
workers as max;
    
    The topology based on trident run about 5 hours without fails.
    
    
    Then I kill one worker called A, then I found the log below on worker 
B.Worker B don't exit as worker A died. 
    ```
    2014-11-04 17:18:08 b.s.m.n.Client [INFO] Reconnect started for 
Netty-Client-A/xxx.xxx.xxx.xxx:21812... [47]
    2014-11-04 17:18:12 b.s.m.n.Client [INFO] Reconnect started for 
Netty-Client-A/xxx.xxx.xxx.xxx:21812... [48]
    2014-11-04 17:18:16 b.s.m.n.Client [INFO] Reconnect started for 
Netty-Client-A/xxx.xxx.xxx.xxx:21812... [49]
    2014-11-04 17:18:20 b.s.m.n.Client [INFO] Reconnect started for 
Netty-Client-A/xxx.xxx.xxx.xxx:21812... [50]
    2014-11-04 17:18:24 b.s.m.n.Client [INFO] Closing Netty Client 
Netty-Client-A/xxx.xxx.xxx.xxx:21812
    2014-11-04 17:18:24 b.s.m.n.Client [INFO] Waiting for pending batchs to be 
sent with Netty-Client-A/xxx.xxx.xxx.xxx:21812..., timeout: 600000ms, pendings: 0
    2014-11-04 17:18:24 b.s.m.n.Client [INFO] Client is being closed, and does 
not take requests any more, drop the messages...
    2014-11-04 17:18:24 b.s.m.n.Client [INFO] Client is being closed, and does 
not take requests any more, drop the messages...
    ```
    
    As worker A died, nimbus reschedule a new worker F, then worker B connect 
to worker F.
    ```
    2014-11-04 17:16:53 b.s.m.n.Client [INFO] Reconnect started for 
Netty-Client-A/xxx.xxx.xxx.xxx:21812... [21]
    2014-11-04 17:16:54 b.s.m.n.Client [INFO] Reconnect started for 
Netty-Client-F/xxx.xxx.xxx.xxx:21813... [17]
    2014-11-04 17:16:54 b.s.m.n.Client [INFO] connection established to a 
remote host Netty-Client-F/xxx.xxx.xxx.xxx:21813, [id: 0xbf721a18, 
/xxx.xxx.xxx.xxx:63811 => F/xxx.xxx.xxx.xxx:21813]
    2014-11-04 17:16:55 b.s.m.n.Client [INFO] Reconnect started for 
Netty-Client-A/10.2.201.65:21812... [22]
    ```
    worker B connect to worker F successful before worker B close connection 
with Worker A.
    
    Because this is our product cluster, I rewrite the hostname and ip in the 
log.
    



> 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, 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)

Reply via email to