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

Powell Molleti commented on ZOOKEEPER-2246:
-------------------------------------------

Plainly removing  sock.setSoTimeout(0) from http://s.apache.org/TfI has the 
unintended consequence of shutting down both the RecvWorker and SendWorker 
threads for all cases. Seems like current code is designed to  keep the socket 
alive (and threads to keep running) so as to reuse this channel to communicate 
again with the the peer node which still alive but needs to redo leader 
election.

I could not reproduce any issue if threads shutdown after the timeout since new 
threads are created for next iteration of leader election. I rather would like 
to reuse the threads and the channel hence I propose the following approach.

The alternative I suggest is to still remove setSoTimeout(0) from here: 
http://s.apache.org/TfI  , also enable SO_KEEPALIVE via setKeepAlive() on this 
socket and do not consider it an error when timeout occurs here: 
http://bit.ly/1JHIdVY but consider it an error when it happens here: 
http://bit.ly/1NTjQ9R

This means that users can play with keep alive timeouts for TCP sockets to 
quicken TCP socket failures propagating to user-space and zookeeper also resets 
the socket if it detects other side is not responding when it knows it needs a 
response within some bounded time.

Ideally I wish there is some userspace pings of every socket channel between 
zookeeper nodes to detect dead channels quickly. Seems like one exists for 
sockets that do Follow/Lead after leader election is done but not for this?. 
Such a feature could be added with care towards making it backward compatible.

> quorum connection manager takes a long time to shut down
> --------------------------------------------------------
>
>                 Key: ZOOKEEPER-2246
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2246
>             Project: ZooKeeper
>          Issue Type: Bug
>          Components: quorum
>            Reporter: Michi Mutsuzaki
>             Fix For: 3.5.2, 3.6.0
>
>
> Receive worker can take a long time to shut down because the socket timeout 
> is set to zero: http://s.apache.org/TfI
> There was a discussion on the mailing list a while back: 
> http://s.apache.org/cYG



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to