[ https://issues.apache.org/jira/browse/ZOOKEEPER-900?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12928011#action_12928011 ]
Flavio Junqueira commented on ZOOKEEPER-900: -------------------------------------------- Hi Vishal, I like your proposal, it seems reasonable and not difficult to implement. On your questions: # I don't think it is necessary to kill a pair SenderWorker/RecvWorker every time, and I'd certainly support changing it; # I'm not sure where you're suggesting to introduce a delay. In the FLE code, a server sends a new batch of notifications if it changes its vote or if it times out waiting for a new notification. This timeout value increases over time. I was actually thinking that we should reset the timeout value upon receiving a notification. I think this is a bug.... Given that it is your proposal, I'd be happy to let you take a stab at it and help you out if you need a hand. Does it make sense for you? > FLE implementation should be improved to use non-blocking sockets > ----------------------------------------------------------------- > > Key: ZOOKEEPER-900 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-900 > Project: Zookeeper > Issue Type: Bug > Reporter: Vishal K > Assignee: Flavio Junqueira > Priority: Critical > > From earlier email exchanges: > 1. Blocking connects and accepts: > a) The first problem is in manager.toSend(). This invokes connectOne(), which > does a blocking connect. While testing, I changed the code so that > connectOne() starts a new thread called AsyncConnct(). AsyncConnect.run() > does a socketChannel.connect(). After starting AsyncConnect, connectOne > starts a timer. connectOne continues with normal operations if the connection > is established before the timer expires, otherwise, when the timer expires it > interrupts AsyncConnect() thread and returns. In this way, I can have an > upper bound on the amount of time we need to wait for connect to succeed. Of > course, this was a quick fix for my testing. Ideally, we should use Selector > to do non-blocking connects/accepts. I am planning to do that later once we > at least have a quick fix for the problem and consensus from others for the > real fix (this problem is big blocker for us). Note that it is OK to do > blocking IO in SenderWorker and RecvWorker threads since they block IO to the > respective ! peer. > b) The blocking IO problem is not just restricted to connectOne(), but also > in receiveConnection(). The Listener thread calls receiveConnection() for > each incoming connection request. receiveConnection does blocking IO to get > peer's info (s.read(msgBuffer)). Worse, it invokes connectOne() back to the > peer that had sent the connection request. All of this is happening from the > Listener. In short, if a peer fails after initiating a connection, the > Listener thread won't be able to accept connections from other peers, because > it would be stuck in read() or connetOne(). Also the code has an inherent > cycle. initiateConnection() and receiveConnection() will have to be very > carefully synchronized otherwise, we could run into deadlocks. This code is > going to be difficult to maintain/modify. > Also see: https://issues.apache.org/jira/browse/ZOOKEEPER-822 -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.