[ https://issues.apache.org/jira/browse/ZOOKEEPER-914?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12926343#action_12926343 ]
Vishal K commented on ZOOKEEPER-914: ------------------------------------ Hi Pat, Flavio, I will begin with admitting that my comment about "bad programming" was not a constructive comment and unwarranted. One can argue that such a comment can be viewed as constructive since it raises a red alert on the quality. But I understand that this is highly subjective, and hence, should be avoided. However, I stand corrected for my comment about lack of testing/tests. I would like to take a moment here to explain our frustrations and then I will get back to this bug and my suggestions to improve QuorumCnxManager and testing. While reading the first part of my comments please ask yourself "why were these issues not uncovered prior to checkins?" rather than "why is this guy complaining?". You may find these to be constructive as well. Finally, I would like to point out that for most of the issues listed below I have tried to help by debugging and/or providing patches. Also, we are interested in and will continue to contribute to ZooKeeper. We wrote an application on top of ZooKeeper. We started testing our application to see how it handles failures. We rebooted the follower and we immediately ran into ZOOKEEPER-335 (zookeeper servers should commit the new leader txn to their logs.). We then tried to reboot the leader and we ran into several bugs reported in ZOOKEEPER-822 (Leader election taking a long time to complete). Once, we misconfigured one of our ZooKeeper servers and we ran into bug ZOOKEEPER-851 (ZK lets any node to become an observer). We made a minor change to our client code and we ran into bug ZOOKEEPER-907 ("KeeperErrorCode = Session moved" messages), which also happens to identify ZOOKEEPER-915 (Errors that happen during sync() processing at the leader do not get propagated back to the client). A few days back we rebooted our follower and ran into ZOOKEEPER-914 (QuorumCnxManager blocks forever). There are a few other issues that I haven't reported yet (still debugging). Looking at the reported bugs, I believe that almost all of them fall under sanity/basic failure testing category. Therefore, if you look at it from our view, clover reports and arguments about the number of tests that cover the code path in question are great, but are not convincing. Anyways, now I will conclude my end of the argument and move forward to look at the real issues at hand. Hopefully, you will find the comments below to be constructive. Moving on... 1. AFAIK, SO_TIMEOUT does not work for blocking channels. Is there a way to set timeout on blocking channels? If not, we will have to use non-blocking channels and then make sure that we handle read/write correctly, because a read/write can return partial results or non-blocking channels. I noticed that Learner.java uses BinaryInputArchive from Jute in non-blocking mode. Should we use that? Also note that QorumCnxManager after accepting connection reads the first 8 bytes from the channel buffer and assumes that it is a server ID. It does not have a tag to indicate packet/request type. 2. We could put a hack to timeout calls in receiveConnection and InitiateConnection using TimerTask (start a timer and interrupt of read hasn't returned after the timer expires) or Threads. But I would rather go for the real fix. 3. Testing failures - Flavio, in addition to handshake protocol, we will need to test failures post handshake (see initiateConnnction) to ensure that a server does not block while writing if the receiver is down. We need a way to introduce faults in the code. At my earlier job, when we implemented a clustered system, we had a way to write some form of assert statements in our code. While writing the code we would put asserts and critical places. We could then enable these asserts (using the assert name) in our tests and trigger faults. Asserts could be used only in debug mode. In addition, we had "assert actions", which could essentially execute a specified method (action). We introduced faults usin these these methods. This was done using propriety library written in C. I am fairly new to the Java world, but I am guessing there is a tool to do something similar (maybe mockito?). Also, in addition, to the failure tests, we should periodically do real failure testing. For example, rebootingnodes. In our experience, such testing introduces unexpected latencies (e.g., exposes code to TCP timeouts). In our application, we have a RMI server that does management of ZooKeeper (start/stop/etc) in addition to other management tasks for our application. We are planning to extend this RMI service for debugging (e.g., add calls to reboot/hang the machine). If such a service seems useful to you as well, then when time permits, I will cleanup the code and submit it to ZK. 4. I have a few suggestions to improve QuorumCnxManager. I will post them later for ZOOKEEPER-901. > 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 > Fix For: 3.3.3, 3.4.0 > > > 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=0x00007fa920005800 nid=0x11bb runnable > [0x00007fa9275ed000] > 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 <0x00007fa93315f988> (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.