[jira] Commented: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-12 Thread Flavio Junqueira (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-900?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12931353#action_12931353
 ] 

Flavio Junqueira commented on ZOOKEEPER-900:


Hi Vishal, This is a good question. I'm actually assuming that the behavior of 
TCP is such that if I send a message and then close the channel properly 
(calling close()), due to the reliability and order guarantees of the 
connection, the message will get through before the connection closes. 
Essentially, I'm relying upon the TCP ACK to do exactly what you're proposing. 
However, it might be a good idea to make sure that the assumption is correct or 
if you know the answer already, just let me know. Overall I do agree that 
having an ACK is important.  




 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


 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.



[jira] Commented: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-12 Thread Patrick Hunt (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-900?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12931444#action_12931444
 ] 

Patrick Hunt commented on ZOOKEEPER-900:


Flavio, I'd be worried that different tcp stacks might (inter)operate 
differently in practice vs theory.


In general it's pretty tough to get this right - look at all the problems we've 
been having with netcat behavior
https://issues.apache.org/jira/secure/IssueNavigator.jspa?reset=truequery=netcatsummary=truedescription=truebody=truepid=12310801

Ubuntu recently moved from traditional to the newish bsd flavor (supports 
ipv6 natively) of nc and we are back to having issues after having made 
significant changes in 3.3 to fix this (incl a number of tests that simulated 
the nc behavior as closely as we could understand it).

 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


 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.



[jira] Updated: (ZOOKEEPER-880) QuorumCnxManager$SendWorker grows without bounds

2010-11-12 Thread Benoit Sigoure (JIRA)

 [ 
https://issues.apache.org/jira/browse/ZOOKEEPER-880?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Benoit Sigoure updated ZOOKEEPER-880:
-

Priority: Critical  (was: Major)

 QuorumCnxManager$SendWorker grows without bounds
 

 Key: ZOOKEEPER-880
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-880
 Project: Zookeeper
  Issue Type: Bug
Affects Versions: 3.2.2
Reporter: Jean-Daniel Cryans
Priority: Critical
 Attachments: hbase-hadoop-zookeeper-sv4borg12.log.gz, 
 hbase-hadoop-zookeeper-sv4borg9.log.gz, jstack, 
 TRACE-hbase-hadoop-zookeeper-sv4borg9.log.gz


 We're seeing an issue where one server in the ensemble has a steady growing 
 number of QuorumCnxManager$SendWorker threads up to a point where the OS runs 
 out of native threads, and at the same time we see a lot of exceptions in the 
 logs.  This is on 3.2.2 and our config looks like:
 {noformat}
 tickTime=3000
 dataDir=/somewhere_thats_not_tmp
 clientPort=2181
 initLimit=10
 syncLimit=5
 server.0=sv4borg9:2888:3888
 server.1=sv4borg10:2888:3888
 server.2=sv4borg11:2888:3888
 server.3=sv4borg12:2888:3888
 server.4=sv4borg13:2888:3888
 {noformat}
 The issue is on the first server. I'm going to attach threads dumps and logs 
 in moment.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Commented: (ZOOKEEPER-880) QuorumCnxManager$SendWorker grows without bounds

2010-11-12 Thread Benoit Sigoure (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-880?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12931457#action_12931457
 ] 

Benoit Sigoure commented on ZOOKEEPER-880:
--

Bumping up the severity.  This took down one of our clusters again.

 QuorumCnxManager$SendWorker grows without bounds
 

 Key: ZOOKEEPER-880
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-880
 Project: Zookeeper
  Issue Type: Bug
Affects Versions: 3.2.2
Reporter: Jean-Daniel Cryans
Priority: Critical
 Attachments: hbase-hadoop-zookeeper-sv4borg12.log.gz, 
 hbase-hadoop-zookeeper-sv4borg9.log.gz, jstack, 
 TRACE-hbase-hadoop-zookeeper-sv4borg9.log.gz


 We're seeing an issue where one server in the ensemble has a steady growing 
 number of QuorumCnxManager$SendWorker threads up to a point where the OS runs 
 out of native threads, and at the same time we see a lot of exceptions in the 
 logs.  This is on 3.2.2 and our config looks like:
 {noformat}
 tickTime=3000
 dataDir=/somewhere_thats_not_tmp
 clientPort=2181
 initLimit=10
 syncLimit=5
 server.0=sv4borg9:2888:3888
 server.1=sv4borg10:2888:3888
 server.2=sv4borg11:2888:3888
 server.3=sv4borg12:2888:3888
 server.4=sv4borg13:2888:3888
 {noformat}
 The issue is on the first server. I'm going to attach threads dumps and logs 
 in moment.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Commented: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-12 Thread Flavio Junqueira (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-900?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12931460#action_12931460
 ] 

Flavio Junqueira commented on ZOOKEEPER-900:


That's a pretty strong statement. You're essentially suggesting that we 
shouldn't rely upon TCP to implement even its basic functionality. Also, my 
understanding is that Vishal is just reasoning about the code and he hasn't 
been able to reproduce that situation. Please correct me if I'm mistaken, 
Vishal.

 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


 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.



[jira] Commented: (ZOOKEEPER-880) QuorumCnxManager$SendWorker grows without bounds

2010-11-12 Thread Flavio Junqueira (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-880?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12931464#action_12931464
 ] 

Flavio Junqueira commented on ZOOKEEPER-880:


Benoit, just to clarify, is this also due to monitoring or scanning?

 QuorumCnxManager$SendWorker grows without bounds
 

 Key: ZOOKEEPER-880
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-880
 Project: Zookeeper
  Issue Type: Bug
Affects Versions: 3.2.2
Reporter: Jean-Daniel Cryans
Priority: Critical
 Attachments: hbase-hadoop-zookeeper-sv4borg12.log.gz, 
 hbase-hadoop-zookeeper-sv4borg9.log.gz, jstack, 
 TRACE-hbase-hadoop-zookeeper-sv4borg9.log.gz


 We're seeing an issue where one server in the ensemble has a steady growing 
 number of QuorumCnxManager$SendWorker threads up to a point where the OS runs 
 out of native threads, and at the same time we see a lot of exceptions in the 
 logs.  This is on 3.2.2 and our config looks like:
 {noformat}
 tickTime=3000
 dataDir=/somewhere_thats_not_tmp
 clientPort=2181
 initLimit=10
 syncLimit=5
 server.0=sv4borg9:2888:3888
 server.1=sv4borg10:2888:3888
 server.2=sv4borg11:2888:3888
 server.3=sv4borg12:2888:3888
 server.4=sv4borg13:2888:3888
 {noformat}
 The issue is on the first server. I'm going to attach threads dumps and logs 
 in moment.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Commented: (ZOOKEEPER-860) Add alternative search-provider to ZK site

2010-11-12 Thread Patrick Hunt (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-860?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12931462#action_12931462
 ] 

Patrick Hunt commented on ZOOKEEPER-860:


Hi Alex, Otis. Take a look at ZOOKEEPER-925. I think this is a good time (new 
site gen and new site once/if we get approved as TLP) to introduce this change. 
Perhaps you could update the sitegen to include this? It would give ppl a 
change to try it out. Regards.

 Add alternative search-provider to ZK site
 --

 Key: ZOOKEEPER-860
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-860
 Project: Zookeeper
  Issue Type: Improvement
  Components: documentation
Reporter: Alex Baranau
Assignee: Alex Baranau
Priority: Minor
 Fix For: 3.4.0

 Attachments: ZOOKEEPER-860.patch


 Use search-hadoop.com service to make available search in ZK sources, MLs, 
 wiki, etc.
 This was initially proposed on user mailing list 
 (http://search-hadoop.com/m/sTZ4Y1BVKWg1). The search service was already 
 added in site's skin (common for all Hadoop related projects) before (as a 
 part of [AVRO-626|https://issues.apache.org/jira/browse/AVRO-626]) so this 
 issue is about enabling it for ZK. The ultimate goal is to use it at all 
 Hadoop's sub-projects' sites.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Commented: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-12 Thread Patrick Hunt (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-900?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12931466#action_12931466
 ] 

Patrick Hunt commented on ZOOKEEPER-900:


I don't know for this specific case, but the corners I've looked at (tearing 
down a connection) there have been issues. Perhaps they are issues on our side, 
I'm not certain, but I do know that we fail with this version of nc (default in 
ubuntu maverick) even after significant work was done to address the original 
problem:
OpenBSD netcat (Debian patchlevel 1.89-3ubuntu2)

Let's assume what you say is correct -- we'd want to test this carefully to 
assure ourselves.


 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


 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.



[ANNOUNCE] Apache ZooKeeper 3.3.2

2010-11-12 Thread Patrick Hunt
The Apache ZooKeeper team is proud to announce Apache ZooKeeper version 3.3.2

ZooKeeper is a high-performance coordination service for distributed
applications. It exposes common services - such as naming,
configuration management, synchronization, and group services - in a
simple interface so you don't have to write them from scratch. You can
use it off-the-shelf to implement consensus, group management, leader
election, and presence protocols. And you can build on it for your
own, specific needs.

For ZooKeeper release details and downloads, visit:
http://hadoop.apache.org/zookeeper/releases.html

ZooKeeper 3.3.2 Release Notes are at:
http://hadoop.apache.org/zookeeper/docs/r3.3.2/releasenotes.html

Regards,

The ZooKeeper Team


[jira] Commented: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-12 Thread Flavio Junqueira (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-900?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12931470#action_12931470
 ] 

Flavio Junqueira commented on ZOOKEEPER-900:


Sure, I can investigate a little further, and Vishal let us know if you find 
anything.

 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


 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.



FYI: Netty is forking

2010-11-12 Thread Patrick Hunt
http://www.jboss.org/netty/community.html#nabble-td5730963

Patrick


[jira] Updated: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-12 Thread Vishal K (JIRA)

 [ 
https://issues.apache.org/jira/browse/ZOOKEEPER-900?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Vishal K updated ZOOKEEPER-900:
---

Attachment: ZOOKEEPER-900.patch1

There are two enhancements that I am working on for QuorumCnxManager.

1. QCM uses blocking IO for communicating with other peers. It does
not set a timeout for network read/write operations. SO_TIMEOUT does
not work with SocketChannel.

2. Incoming requests are processed one at a time. As a result, if QCM is
processing a connection from a peer and that peer fails, then requests
from other peers won't be processed. Even if we add timeout to
read/write calls, other peers will be blocked for that amount of
time. I had proposed a change in my earlier post for this part (see
above). I am working on a fix.

The attached patch addresses the first problem. Earlier, QCM used
SocketChannels. Now it uses DataInputStream/DataOutputStream, which
will blocki only until SO_TIMEOUT expires.

There are also some formatting changes done automatically by my editor
according to Java coding standards. So some of the changes are just
cosmetic.

I have tested this change by creating a 3 node cluster and rebooting
leader/follower several times. The patch also includes a simple test.

Please let me know your comments.

 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.



[jira] Commented: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-12 Thread Patrick Hunt (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-900?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12931487#action_12931487
 ] 

Patrick Hunt commented on ZOOKEEPER-900:


please try to keep the reformatting changes to a minimum unless it's code 
directly being worked on. otw it makes it harder to review (svn -x -w diff does 
help, but still) and blame detail is lost.

 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.



[jira] Commented: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-12 Thread Vishal K (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-900?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12931489#action_12931489
 ] 

Vishal K commented on ZOOKEEPER-900:


ok. how about making an exception for formatting for this patch? I would have 
to spend some time reapplying  the changes (which I would like to avoid ;-)).

 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.



[jira] Commented: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-12 Thread Patrick Hunt (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-900?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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.



[jira] Commented: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-12 Thread Vishal K (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-900?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12931502#action_12931502
 ] 

Vishal K commented on ZOOKEEPER-900:


Diff of log4j file was included by mistake. I will post a patch without 
formatting changes.

 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.



[jira] Commented: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-12 Thread Vishal K (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-900?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12931522#action_12931522
 ] 

Vishal K commented on ZOOKEEPER-900:


Hi Flavio,

Regarding your comment:
I was reasoning about the code. I had not tried to reproduce the problem when I 
posted the question. 

I tried a simple test and I am not able to reproduce the problem on Suse. I 
closed the connection after writing the server ID but before the receiving 
server issued a  read. The receiver was able to read the ID and on the 
following read it got a socket closed exception.

I am not sure if all TCP implementations would behave this way.

-Vishal

 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.



[jira] Updated: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-12 Thread Vishal K (JIRA)

 [ 
https://issues.apache.org/jira/browse/ZOOKEEPER-900?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Vishal K updated ZOOKEEPER-900:
---

Attachment: ZOOKEEPER-900.patch2

Attaching the patch without formatting changes.

 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, ZOOKEEPER-900.patch2


 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.



[jira] Commented: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-12 Thread Patrick Hunt (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-900?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12931554#action_12931554
 ] 

Patrick Hunt commented on ZOOKEEPER-900:


fyi, if a patch is ready for review/commit then click the submit patch link 
-- will trigger the workflow.
Also if you use the same patch name (ZOOKEEPER-###.patch) and re-attach with 
the same name jira will handle this correctly, more detail here:
http://wiki.apache.org/hadoop/ZooKeeper/HowToContribute
thanks!

 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, ZOOKEEPER-900.patch2


 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.



[jira] Resolved: (ZOOKEEPER-914) QuorumCnxManager blocks forever

2010-11-12 Thread Vishal K (JIRA)

 [ 
https://issues.apache.org/jira/browse/ZOOKEEPER-914?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Vishal K resolved ZOOKEEPER-914.


   Resolution: Duplicate
Fix Version/s: (was: 3.3.3)
   (was: 3.4.0)

Dup of ZOOKEEPER-900.

 QuorumCnxManager blocks forever 
 

 Key: ZOOKEEPER-914
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-914
 Project: Zookeeper
  Issue Type: Bug
  Components: leaderElection
Reporter: Vishal K
Assignee: Vishal K
Priority: Blocker

 This was a disaster. While testing our application we ran into a scenario 
 where a rebooted follower could not join the cluster. Further debugging 
 showed that the follower could not join because the QuorumCnxManager on the 
 leader was blocked for indefinite amount of time in receiveConnect()
 Thread-3 prio=10 tid=0x7fa920005800 nid=0x11bb runnable 
 [0x7fa9275ed000]
java.lang.Thread.State: RUNNABLE
 at sun.nio.ch.FileDispatcher.read0(Native Method)
 at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:21)
 at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:233)
 at sun.nio.ch.IOUtil.read(IOUtil.java:206)
 at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:236)
 - locked 0x7fa93315f988 (a java.lang.Object)
 at 
 org.apache.zookeeper.server.quorum.QuorumCnxManager.receiveConnection(QuorumCnxManager.java:210)
 at 
 org.apache.zookeeper.server.quorum.QuorumCnxManager$Listener.run(QuorumCnxManager.java:501)
 I had pointed out this bug along with several other problems in 
 QuorumCnxManager earlier in 
 https://issues.apache.org/jira/browse/ZOOKEEPER-900 and 
 https://issues.apache.org/jira/browse/ZOOKEEPER-822.
 I forgot to patch this one as a part of ZOOKEEPER-822. I am working on a fix 
 and a patch will be out soon. 
 The problem is that QuorumCnxManager is using SocketChannel in blocking mode. 
 It does a read() in receiveConnection() and a write() in initiateConnection().
 Sorry, but this is really bad programming. Also, points out to lack of 
 failure tests for QuorumCnxManager.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Updated: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-12 Thread Vishal K (JIRA)

 [ 
https://issues.apache.org/jira/browse/ZOOKEEPER-900?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Vishal K updated ZOOKEEPER-900:
---

Status: Patch Available  (was: Open)

submitted patch ZOOKEEPER-900.patch1

 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, ZOOKEEPER-900.patch2


 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.



[jira] Commented: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-12 Thread Hadoop QA (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-900?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12931565#action_12931565
 ] 

Hadoop QA commented on ZOOKEEPER-900:
-

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12459485/ZOOKEEPER-900.patch2
  against trunk revision 1034003.

+1 @author.  The patch does not contain any @author tags.

+1 tests included.  The patch appears to include 3 new or modified tests.

-1 javadoc.  The javadoc tool appears to have generated 1 warning messages.

+1 javac.  The applied patch does not increase the total number of javac 
compiler warnings.

-1 findbugs.  The patch appears to introduce 2 new Findbugs warnings.

+1 release audit.  The applied patch does not increase the total number of 
release audit warnings.

+1 core tests.  The patch passed core unit tests.

+1 contrib tests.  The patch passed contrib unit tests.

Test results: 
https://hudson.apache.org/hudson/job/PreCommit-ZOOKEEPER-Build/26//testReport/
Findbugs warnings: 
https://hudson.apache.org/hudson/job/PreCommit-ZOOKEEPER-Build/26//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Console output: 
https://hudson.apache.org/hudson/job/PreCommit-ZOOKEEPER-Build/26//console

This message is automatically generated.

 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, ZOOKEEPER-900.patch2


 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.