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

Patrick Hunt commented on ZOOKEEPER-900:
----------------------------------------

Looking at the patch. Quite a bit changed, hard to tell which is important and 
which not. In these situations I've used the -w diff trick to get just the 
important changes, then applied that patch to virgin code, opened the file in 
eclipse and fixed the (relatively) smaller set of formatting issues.

Also, the patch includes log4j.properties change, you don't want to include 
that I'm thinking.


> 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: Vishal K
>            Priority: Critical
>             Fix For: 3.4.0
>
>         Attachments: ZOOKEEPER-900.patch1
>
>
> 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.

Reply via email to