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

2010-11-19 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-880:


Hi Flavio,

You are right.  We can see RecvWorker leaving but no messages from SendWorker.

2010-09-27 16:02:59,111 WARN 
org.apache.zookeeper.server.quorum.QuorumCnxManager: Connection broken:
java.io.IOException: Channel eof
at 
org.apache.zookeeper.server.quorum.QuorumCnxManager$RecvWorker.run(QuorumCnxManager.java:595)
2010-09-27 16:02:59,162 WARN 
org.apache.zookeeper.server.quorum.QuorumCnxManager: Connection broken:
java.io.IOException: Channel eof
at 
org.apache.zookeeper.server.quorum.QuorumCnxManager$RecvWorker.run(QuorumCnxManager.java:595)
2010-09-27 16:03:14,269 WARN 
org.apache.zookeeper.server.quorum.QuorumCnxManager: Connection broken:
java.io.IOException: Channel eof
at 
org.apache.zookeeper.server.quorum.QuorumCnxManager$RecvWorker.run(QuorumCnxManager.java:595)

I thought that RecvWorker in  3.3.1 called sw.finish() before exiting. Adding 
this call in RecvWorker should fix this problem.


-Vishal

 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-934) Add sanity check for server ID

2010-11-19 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-934:



Nagios might be just sending some 8 byte information to QCM and QCM will accept 
that as a ID and start thread that connection. If we have the above check, we 
will run into this scenario only if nagios sends OBSERVER_ID or a valid server 
ID.

As a first step it might be a good solution to :
1. reject if (sid != OBSERVER_ID  !self.viewContains(sid)
2. interrupt SendWorker When RecvWorker exits
3. Incorporate a sloution for ZOOKEEPER-933. Note with this solution in place, 
Nagios will also have to generate the correct role/peertype string in addition 
to ID.
4. Kill SendWorker and RecvWorker iff leader election has been completed and  
we have no notifications to send. 

In general, this cannot be solved without some form of authentication. 
Essentially, these are forms of DoS attacks.

Another quick solution could be to introduce a cluster password (or a cluster 
identifier string)- We can store this password in zoo.cfg file. A peer can 
include hash of this password in outgoing messages or use the 
f(password,serverid) as a key to hmac outgoing packets. This of course is not 
secure. However, it is good enough to prevent QCM from considering port 
scanners as ZK servers.

 Add sanity check for server ID
 --

 Key: ZOOKEEPER-934
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-934
 Project: Zookeeper
  Issue Type: Sub-task
Reporter: Vishal K
 Fix For: 3.4.0


 2. Should I add a check to reject connections from peers that are not
 listed in the configuration file? Currently, we are not doing any
 sanity check for server IDs. I think this might fix ZOOKEEPER-851.
 The fix is simple. However, I am not sure if anyone in community
 is relying on this ability.

-- 
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-19 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-880:


Leader has the same problem as well. LearnerHandler expects a QuorumPacket to 
be received as the first packet after connection. However, if Nagios was 
monitoring server port as well, then one would expect to see a lot of such 
messages:
LOG.error(First packet  + qp.toString()
+  is not FOLLOWERINFO or OBSERVERINFO!);

Is Nagios not monitoring the server port?


 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] Updated: (ZOOKEEPER-880) QuorumCnxManager$SendWorker grows without bounds

2010-11-19 Thread Vishal K (JIRA)

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

Vishal K updated ZOOKEEPER-880:
---

Attachment: ZOOKEEPER-880.patch



The root cause of frequent disconnect needs to be resolved. In the mean time, I 
have fixed the problem that was causing the leak of every other thread of 
SendWorker.

I tested the patch by connecting to 3888 on one of the servers using telnet.

2010-11-19 14:51:10,081 - INFO  
[/10.17.119.101:3888:quorumcnxmanager$liste...@477] - Received connection 
request /10.16.251.39:2074
2010-11-19 14:51:14,364 - DEBUG 
[/10.17.119.101:3888:quorumcnxmanager$sendwor...@553] - Address of remote peer: 
8103510703875099187
2010-11-19 14:51:19,440 - WARN  [Thread-7:quorumcnxmanager$recvwor...@726] - 
Connection broken for id 8103510703875099187, my id = 1, error = 
java.io.IOException: Received packet with invalid packet: 218824692
at 
org.apache.zookeeper.server.quorum.QuorumCnxManager$RecvWorker.run(QuorumCnxManager.java:711)
2010-11-19 14:51:19,441 - WARN  [Thread-7:quorumcnxmanager$recvwor...@730] - 
Interrupting SendWorker   = SendWorker is getting killed.
2010-11-19 14:51:19,442 - DEBUG [Thread-7:quorumcnxmanager$sendwor...@571] - 
Calling finish for 8103510703875099187
2010-11-19 14:51:19,443 - DEBUG [Thread-7:quorumcnxmanager$sendwor...@591] - 
Removing entry from senderWorkerMap sid=8103510703875099187
2010-11-19 14:51:19,443 - WARN  [Thread-6:quorumcnxmanager$sendwor...@643] - 
Interrupted while waiting for message on queue
java.lang.InterruptedException
at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.reportInterruptAfterWait(AbstractQueuedSynchronizer.java:1899)
at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1976)
at 
java.util.concurrent.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:342)
at 
org.apache.zookeeper.server.quorum.QuorumCnxManager$SendWorker.run(QuorumCnxManager.java:631)
2010-11-19 14:51:19,456 - DEBUG [Thread-6:quorumcnxmanager$sendwor...@571] - 
Calling finish for 8103510703875099187
2010-11-19 14:51:19,457 - WARN  [Thread-6:quorumcnxmanager$sendwor...@652] - 
Send worker leaving thread

Can you see if this fixes the problem?

 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, ZOOKEEPER-880.patch


 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-18 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-880:


Hi Benoit,

May I suggest to see if you can reproduce this problem with 3.3.3
(with patch for ZOOKEEPER-822)? I was going through
QuorumCnxManager.java for 3.2.2. It clearly leaks a SendWorker thread
for every other connection.

After receiving a connection from a peer, it creates a new thread and
inserts its reference in senderWorkerMap.

SendWorker sw = new SendWorker(s, sid);
RecvWorker rw = new RecvWorker(s, sid);
sw.setRecv(rw);

SendWorker vsw = senderWorkerMap.get(sid);
senderWorkerMap.put(sid, sw);

Then it kills the old thread for the peer (created from earlier
connection)

if(vsw != null)
vsw.finish();

However, the SendWorker.finish method removes an entry from
senderWorkerMap. This results in removing a reference for
recently created SendWorker thread.
senderWorkerMap.remove(sid);


Thus, it will end up removing both the entries. As a result, one thread
will be leaked for every other connection.

If you count the number of error messages in
hbase-hadoop-zookeeper-sv4borg9.log, you will see that messages from
RecvWorker is approximately twice of SendWorker. I think this proves
the point.

$:/tmp/hadoop # grep RecvWorker  hbase-hadoop-zookeeper-sv4borg9.log | wc -l
60
$:/tmp/hadoop # grep SendWorker  hbase-hadoop-zookeeper-sv4borg9.log | wc -l
32

-Vishal

 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-934) Add sanity check for server ID

2010-11-18 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-934:



How about we reject connection if (sid != OBSERVER_ID  
!self.viewContains(sid))?

 Add sanity check for server ID
 --

 Key: ZOOKEEPER-934
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-934
 Project: Zookeeper
  Issue Type: Sub-task
Reporter: Vishal K
 Fix For: 3.4.0


 2. Should I add a check to reject connections from peers that are not
 listed in the configuration file? Currently, we are not doing any
 sanity check for server IDs. I think this might fix ZOOKEEPER-851.
 The fix is simple. However, I am not sure if anyone in community
 is relying on this ability.

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



[jira] Commented: (ZOOKEEPER-933) Remove wildcard QuorumPeer.OBSERVER_ID

2010-11-18 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-933:


I looks like we need a way to uniquely identify the observer after we receive a 
connection. One way to do this is to get IP address from the socket. But this 
is not a good idea.

Instead, we can ask the observer to generate a unique id (uuid or crypto hash) 
and send sid, role, uuid after connecting to a peer (instead of just sid in 
the current implementation).
From role, QCM can figure out that the node is observer. It can then ignore 
the sid and use the uuid passed by the observer.

For followers and leader we will use sid as identifier and for observers we 
will use uuid.

How does that sound?


 Remove wildcard  QuorumPeer.OBSERVER_ID
 ---

 Key: ZOOKEEPER-933
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-933
 Project: Zookeeper
  Issue Type: Sub-task
Reporter: Vishal K
 Fix For: 3.4.0


 1. I have a question about the following piece of code in QCM:
 if (remoteSid == QuorumPeer.OBSERVER_ID) {
  /* * Choose identifier at random. We need a value to identify * the 
 connection. */ 
 remoteSid = observerCounter--;
 LOG.info(Setting arbitrary identifier to observer:  + remoteSid); 
 }
 Should we allow this? The problem with this code is that if a peer
 connects twice with QuorumPeer.OBSERVER_ID, we will end up creating
 threads for this peer twice. This could result in redundant
 SendWorker/RecvWorker threads.
 I haven't used observers yet. The documentation
 http://hadoop.apache.org/zookeeper/docs/r3.3.0/zookeeperObservers.html
 says that just like followers, observers should have server IDs. In
 which case, why do we want to provide a wild-card?

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



[jira] Commented: (ZOOKEEPER-902) Fix findbug issue in trunk Malicious code vulnerability

2010-11-17 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-902:


fyi, I had not included test-patch.properties file in my diff.

 Fix findbug issue in trunk Malicious code vulnerability
 -

 Key: ZOOKEEPER-902
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-902
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum, server
Affects Versions: 3.4.0
Reporter: Patrick Hunt
Priority: Minor
 Fix For: 3.4.0


 https://hudson.apache.org/hudson/view/ZooKeeper/job/ZooKeeper-trunk/970/artifact/trunk/findbugs/zookeeper-findbugs-report.html#Warnings_MALICIOUS_CODE
 Malicious code vulnerability Warnings
 Code  Warning
 MSorg.apache.zookeeper.server.quorum.LeaderElection.epochGen isn't final 
 but should be

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



[jira] Created: (ZOOKEEPER-932) Move blocking read/write calls to SendWorker and RecvWorker Threads

2010-11-17 Thread Vishal K (JIRA)
Move blocking read/write calls to SendWorker and RecvWorker Threads
---

 Key: ZOOKEEPER-932
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-932
 Project: Zookeeper
  Issue Type: Sub-task
Reporter: Vishal K
Assignee: Vishal K
 Fix For: 3.4.0


Copying relevant comments:

Vishal K added a comment - 02/Nov/10 02:09 PM
Hi Flavio,

I have a suggestion for changing the blocking IO code in QuorumCnxManager. It 
keeps the current code structure and requires a small amount of changes. I am 
not sure if these comments should go in ZOOKEEPER-901. ZOOKEEPER-901 is 
probably addressing netty as well. Please feel free to close this JIRA if you 
intend to make all the changes as a part of ZOOKEEPER-901.

Basically we jusy need to move parts of initiateConnection and 
receiveConnection to SenderWorker and ReceiveWorker.

A. Current flow for receiving connection:
1. accept connection in Listener.run()
2. receiveConnection()

* Read remote server's ID
* Take action based on my ID and remote server's ID (disconnect and 
reconnect if my ID is  remote server's ID).
* kill current set of SenderWorker and ReciveWorker threads
* Start a new pair

B Current flow for initiating connection:
1. In connectOne(), connect if not already connected. else return.
2. send my ID to the remote server
3. if my ID  remote server disconnect and return
4. if my ID  remote server

* kill current set of SenderWorker and ReceiveWorkter threads for the 
remote server
* Start a new pair

Proposed changes:
Move the code that performs any blocking IO in SenderWorker and ReceiveWorker.

A. Proposed flow for receiving connection:
1. accept connection in Listener.run()
2. receiveConnection()

* kill current set of SenderWorker and ReciveWorker threads
* Start a new pair

Proposed changed to SenderWorker:

* Read remote server's ID
* Take action based on my ID and remote server's ID (disconnect and 
reconnect if my ID is  remote server's ID).
* Proceed to normal operation

B Proposed flow for initiating connection:
1. in connectOne(), return if already connected
2. Start a new SenderWorker and ReceiveWorker pair
2. In SenderWorker

* connect to remote server
* write my ID
* if my ID  remote server disconnect and return (shutdown the pair).
* Proceed to normal operation

Questions:

* In QuorumCnxManager, is it necessary to kill the current pair and restart 
a new one every time we receive a connect request?
* In receiveConnection we may choose to reject an accepted connection if a 
thread in
  SenderWorker is in the process of connecting. Otherwise a server with ID 
  remote server may keep sending frequent connect request that will result 
in the
  remote server closing connections for this peer. But I think we add a 
delay
  before sending notifications, which might be good enough to prevent this
  problem.

Let me know what you think about this. I can also help with the implementation.

Flavio Junqueira added a comment - 03/Nov/10 05:28 PM
Hi Vishal, I like your proposal, it seems reasonable and not difficult to 
implement.

On your questions:

   1. I don't think it is necessary to kill a pair SenderWorker/RecvWorker 
every time, and I'd certainly support changing it;
   2. 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?

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



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

2010-11-17 Thread Vishal K (JIRA)

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

Vishal K reopened ZOOKEEPER-900:



The committed patch fixed one part of the problem (enforced timeout on network 
IO).

Reopening for remaining subtasks.

 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.patch, 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] Updated: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-16 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.patch

Hi Flavio, Pat,

I have attached the patch with suggested modifications. I did some more testing 
and also ran SimpleSysTest.

Other than the changes suggested above and some cosmetic changes to error 
messages, I have added a delay on 1 second in Listener.run() in case we see an 
IOException.

Let me know if you have more comments.

-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.patch, 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] Updated: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-16 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)

patch submitted.

 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.patch, 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-16 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-900:


Hi Flavio,

Currently the QCM  keeps the TCP connections to other peers alive even after 
finishing leader election. How about we shutdown these threads after  a node 
decides to be a leader or a follower?The threads get created on fly.  Do you 
see any problem with that? 

Thanks.
-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.patch, 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-15 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-900:


Hi Flavio, Pat,

I will submit a new patch with suggested changes. For 902 do I just add a file 
./java/test/bin/test-patch.properties with OK_FINDBUGS_WARNINGS=0 in 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

 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] 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 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 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] 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-11 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-900:


Hi Flavio,

I have a question regarding the logic that determines which connection
to retain if peer 1 and peer 2 decide to communicate with each other.

Suppose peer 1 connects to peer 2. It first sends its sid as a
challenge. Peer 2 reads the sid and determines whether to keep the
connection or initiate a connection back to peer 1. Both determine
that peer 2 should be the one initiating the connection to peer 1
since sid of peer2  sid of peer1.  I am concerned that they both 
may not be able to maintain any connection since the handshake is
one-way.

In the current implementation, peer1 disconnects immediately after
writing the challenge to peer 2. It can happen that peer 2 may get a
ClosedChannelException before it reads the challenge from peer 1. As a
result, peer 2 will not initiate a connection to peer 1.

Is this a legitimate problem? If it is, how about we ask peer 2 to
send back a ACK after it reads the challenge. Peer 1 will do a timed
read() after writing a challenge to peer 2. This will hopefully give
peer 2 enough time to read the challenge and take appropriate
action. If peer 2 is really slow, peer 1 will timeout on the read
operation.

-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-928) Follower should stop following and start FLE if it does not receive pings from the leader

2010-11-11 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-928:


Hi Flavio,

Thats correct. I was planning to do this change (in addition to other changes) 
as a part of ZOOKEEPER-900.
But now I think it is better if we make this change first and not wait for 
other changes. So that we don't have to wait till 3.4.0 for this fix.
At least, that will get us around the block forever problem.

-Vishal




 Follower should stop following and start FLE if it does not receive pings 
 from the leader
 -

 Key: ZOOKEEPER-928
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-928
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum, server
Affects Versions: 3.3.2
Reporter: Vishal K
Priority: Critical

 In Follower.followLeader() after syncing with the leader, the follower does:
 while (self.isRunning()) {
 readPacket(qp);
 processPacket(qp);
 }
 It looks like it relies on socket timeout expiry to figure out if the 
 connection with the leader has gone down.  So a follower *with no cilents* 
 may never notice a faulty leader if a Leader has a software hang, but the TCP 
 connections with the peers are still valid. Since it has no cilents, it won't 
 hearbeat with the Leader. If majority of followers are not connected to any 
 clients, then FLE will fail even if other followers attempt to elect a new 
 leader.
 We should keep track of pings received from the leader and see if we havent 
 seen
 a ping packet from the leader for (syncLimit * tickTime) time and give up 
 following the
 leader.

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



[jira] Created: (ZOOKEEPER-928) Follower should stop following and start FLE if it does not receive pings from the leader

2010-11-10 Thread Vishal K (JIRA)
Follower should stop following and start FLE if it does not receive pings from 
the leader
-

 Key: ZOOKEEPER-928
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-928
 Project: Zookeeper
  Issue Type: Bug
Affects Versions: 3.3.2
Reporter: Vishal K
 Fix For: 3.4.0


In Follower.followLeader() after syncing with the leader, the follower does:
while (self.isRunning()) {
readPacket(qp);
processPacket(qp);
}

It looks like it relies on socket timeout expiry to figure out if the 
connection with the leader has gone down.  So a follower *with no cilents* may 
never notice a faulty leader if a Leader has a software hang, but the TCP 
connections with the peers are still valid. Since it has no cilents, it won't 
hearbeat with the Leader. If majority of followers are not connected to any 
clients, then FLE will fail even if other followers attempt to elect a new 
leader.

We should keep track of pings received from the leader and see if we havent seen
a ping packet from the leader for (syncLimit * tickTime) time and give up 
following the
leader.

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



[jira] Commented: (ZOOKEEPER-928) Follower should stop following and start FLE if it does not receive pings from the leader

2010-11-10 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-928:


Hi Flavio,

I was aware of that. However, this is not the case of idefinite TCP IO hang. If 
the leader hangs (e.g., software deadlock in ZooKeeper) its TCP connection will 
remain active. The follower will not see a socket timeout. Now, how can the 
follower determine if the leader is down?

 Follower should stop following and start FLE if it does not receive pings 
 from the leader
 -

 Key: ZOOKEEPER-928
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-928
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum, server
Affects Versions: 3.3.2
Reporter: Vishal K
Priority: Critical
 Fix For: 3.3.3, 3.4.0


 In Follower.followLeader() after syncing with the leader, the follower does:
 while (self.isRunning()) {
 readPacket(qp);
 processPacket(qp);
 }
 It looks like it relies on socket timeout expiry to figure out if the 
 connection with the leader has gone down.  So a follower *with no cilents* 
 may never notice a faulty leader if a Leader has a software hang, but the TCP 
 connections with the peers are still valid. Since it has no cilents, it won't 
 hearbeat with the Leader. If majority of followers are not connected to any 
 clients, then FLE will fail even if other followers attempt to elect a new 
 leader.
 We should keep track of pings received from the leader and see if we havent 
 seen
 a ping packet from the leader for (syncLimit * tickTime) time and give up 
 following the
 leader.

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



[jira] Commented: (ZOOKEEPER-928) Follower should stop following and start FLE if it does not receive pings from the leader

2010-11-10 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-928:


ok, I see your point. I mis-analyzed this part of the code. I will wait for 
Flavio to comment and then close the jira.

 Follower should stop following and start FLE if it does not receive pings 
 from the leader
 -

 Key: ZOOKEEPER-928
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-928
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum, server
Affects Versions: 3.3.2
Reporter: Vishal K
Priority: Critical
 Fix For: 3.3.3, 3.4.0


 In Follower.followLeader() after syncing with the leader, the follower does:
 while (self.isRunning()) {
 readPacket(qp);
 processPacket(qp);
 }
 It looks like it relies on socket timeout expiry to figure out if the 
 connection with the leader has gone down.  So a follower *with no cilents* 
 may never notice a faulty leader if a Leader has a software hang, but the TCP 
 connections with the peers are still valid. Since it has no cilents, it won't 
 hearbeat with the Leader. If majority of followers are not connected to any 
 clients, then FLE will fail even if other followers attempt to elect a new 
 leader.
 We should keep track of pings received from the leader and see if we havent 
 seen
 a ping packet from the leader for (syncLimit * tickTime) time and give up 
 following the
 leader.

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



[jira] Commented: (ZOOKEEPER-928) Follower should stop following and start FLE if it does not receive pings from the leader

2010-11-10 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-928:


sorry for the false alarm. I got confused since SocketChannel is used in 
quorumCnxManager but this part of the code uses Socket and InputArchive.

 Follower should stop following and start FLE if it does not receive pings 
 from the leader
 -

 Key: ZOOKEEPER-928
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-928
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum, server
Affects Versions: 3.3.2
Reporter: Vishal K
Priority: Critical
 Fix For: 3.3.3, 3.4.0


 In Follower.followLeader() after syncing with the leader, the follower does:
 while (self.isRunning()) {
 readPacket(qp);
 processPacket(qp);
 }
 It looks like it relies on socket timeout expiry to figure out if the 
 connection with the leader has gone down.  So a follower *with no cilents* 
 may never notice a faulty leader if a Leader has a software hang, but the TCP 
 connections with the peers are still valid. Since it has no cilents, it won't 
 hearbeat with the Leader. If majority of followers are not connected to any 
 clients, then FLE will fail even if other followers attempt to elect a new 
 leader.
 We should keep track of pings received from the leader and see if we havent 
 seen
 a ping packet from the leader for (syncLimit * tickTime) time and give up 
 following the
 leader.

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



[jira] Commented: (ZOOKEEPER-928) Follower should stop following and start FLE if it does not receive pings from the leader

2010-11-10 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-928:


Hi Flavio,

Can you please try it with SocketChannel and confirm?

 Follower should stop following and start FLE if it does not receive pings 
 from the leader
 -

 Key: ZOOKEEPER-928
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-928
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum, server
Affects Versions: 3.3.2
Reporter: Vishal K
Priority: Critical

 In Follower.followLeader() after syncing with the leader, the follower does:
 while (self.isRunning()) {
 readPacket(qp);
 processPacket(qp);
 }
 It looks like it relies on socket timeout expiry to figure out if the 
 connection with the leader has gone down.  So a follower *with no cilents* 
 may never notice a faulty leader if a Leader has a software hang, but the TCP 
 connections with the peers are still valid. Since it has no cilents, it won't 
 hearbeat with the Leader. If majority of followers are not connected to any 
 clients, then FLE will fail even if other followers attempt to elect a new 
 leader.
 We should keep track of pings received from the leader and see if we havent 
 seen
 a ping packet from the leader for (syncLimit * tickTime) time and give up 
 following the
 leader.

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



[jira] Commented: (ZOOKEEPER-917) Leader election selected incorrect leader

2010-11-09 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-917:


Hi Flavio,

Lets see if I understand this right. Server 2 was replaced and became the 
leader. Server 2 receives old notifications from others and accepts leadership 
even if its epoch, zxid prior to receiving any notifications was 1,0.

Server 2 accepts leadership because 0, 1 vote for 2 and we allow 2 to become 
leader based on point 1. in your comment on 07/Nov/10.

My question with regards to point 1.:
- In your example, it is OK to allow A to join the cluster and become a 
follower (so that A does not remain locked out). But is it OK for A to accept 
leadership even if it has not seen the zxid reported by others (regardless of 
the votes)? Shouldn't it reject leadership?

Am I still misunderstanding the problem?


 Leader election selected incorrect leader
 -

 Key: ZOOKEEPER-917
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-917
 Project: Zookeeper
  Issue Type: Bug
  Components: leaderElection, server
Affects Versions: 3.2.2
 Environment: Cloudera distribution of zookeeper (patched to never 
 cache DNS entries)
 Debian lenny
Reporter: Alexandre Hardy
Priority: Critical
 Fix For: 3.3.3, 3.4.0

 Attachments: zklogs-20101102144159SAST.tar.gz


 We had three nodes running zookeeper:
   * 192.168.130.10
   * 192.168.130.11
   * 192.168.130.14
 192.168.130.11 failed, and was replaced by a new node 192.168.130.13 
 (automated startup). The new node had not participated in any zookeeper 
 quorum previously. The node 192.148.130.11 was permanently removed from 
 service and could not contribute to the quorum any further (powered off).
 DNS entries were updated for the new node to allow all the zookeeper servers 
 to find the new node.
 The new node 192.168.130.13 was selected as the LEADER, despite the fact that 
 it had not seen the latest zxid.
 This particular problem has not been verified with later versions of 
 zookeeper, and no attempt has been made to reproduce this problem as yet.

-- 
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-09 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-900:


Hi Mahadev,

Yes, I will provide a patch (for the approach proposed above). I am working on 
it. I will get in touch with Flavio if I have questions.

-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-914) QuorumCnxManager blocks forever

2010-11-08 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-914:


Hi Flavio,

You are right. Sorry, my comment was not fair.

Regarding SO_TIMEOUT: Per my understanding, SO_TIMEOUT works only when a 
channel is set in non-blocking mode using isConfigureBlocking(). If the channel 
is not configured to work in non-blocking mode, setting SO_TIMEOUT has no 
effect. Please let me know if you think there is a way to set timeout on the 
socket after accepting the connection (without configuring the channel in 
non-blocking mode). The only way I know to use SO_TIMEOUT is by using 
channel.isConfigureBlocking(false). The current code in QuorumCnxManager 
assumes use of blocking IO. We will have to handle partial reads/writes. Please 
refer to my earlier question regarding SO_TIMEOUT for implementing non-blocking 
IO.

I thought this fix was supposed to go in for 3.3.3. As I suggested earlier, one 
quick fix to the problem is to use TimerTask(). Before doing blocking IO we can 
start a timer for that channel (in receiveConnect() before read). Once the 
timer expires, check if the read() has finished. If not, interrupt and close 
the channel. I think having such a fix (or some other fix that will get around 
the problem) until the real fix is in is a better approach. Let me what you 
think?

If we decide to go one of the quick fixes, then we can use this JIRA for that 
and use ZOOKEEPER-900 for the real fix.. Otherwise, as you suggested, we can 
close this JIRA and use ZOOKEEPER-900.

-Vishal

 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=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] Commented: (ZOOKEEPER-917) Leader election selected incorrect leader

2010-11-08 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-917:


Hi Flavio,

Thanks for the clarification.

Looks like this scenario is not likely to happen if replace is done carefully. 
For the sake of argument, shouldn't we make the exception that you described 
only if the joining peer is going to be a follower? For the problem reported in 
this jira,  the server that was replaced (server 2) will not have seen the zxid 
received in the notifications from the other two nodes. Therefore, is this 
case, will it make sense for server 2 to start a new round of election?

-Vishal



 Leader election selected incorrect leader
 -

 Key: ZOOKEEPER-917
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-917
 Project: Zookeeper
  Issue Type: Bug
  Components: leaderElection, server
Affects Versions: 3.2.2
 Environment: Cloudera distribution of zookeeper (patched to never 
 cache DNS entries)
 Debian lenny
Reporter: Alexandre Hardy
Priority: Critical
 Fix For: 3.3.3, 3.4.0

 Attachments: zklogs-20101102144159SAST.tar.gz


 We had three nodes running zookeeper:
   * 192.168.130.10
   * 192.168.130.11
   * 192.168.130.14
 192.168.130.11 failed, and was replaced by a new node 192.168.130.13 
 (automated startup). The new node had not participated in any zookeeper 
 quorum previously. The node 192.148.130.11 was permanently removed from 
 service and could not contribute to the quorum any further (powered off).
 DNS entries were updated for the new node to allow all the zookeeper servers 
 to find the new node.
 The new node 192.168.130.13 was selected as the LEADER, despite the fact that 
 it had not seen the latest zxid.
 This particular problem has not been verified with later versions of 
 zookeeper, and no attempt has been made to reproduce this problem as yet.

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



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

2010-11-08 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-914:


Hi Flavio,

The documentation is not clear.
SO_TIMEOUT  has not effect on blocking channels. Non-blocking channels, wait 
for the specified timeout if nothing is available in the buffer. Otherwise, it 
returns whatever bytes are currently available in the buffer. I wrote a test 
the following test to verify this. Let me know if you know about way to make 
SO_TIMEOUT to work.
 
QuorumPeer peerLeader = new QuorumPeer(peers, tmpdir[1], tmpdir[1], 
port[1], 3, 0, 2, 2, 2);
QuorumCnxManager cnxManager = new QuorumCnxManager(peerLeader);
QuorumCnxManager.Listener listener = cnxManager.listener;
SocketChannel channel = SocketChannel.open();
channel.socket().connect(peers.get(new Long(0)).electionAddr, 5000);
channel.configureBlocking(false);
channel.socket().setSoTimeout(1000);
byte[] msgBytes = new byte[8];
ByteBuffer msgBuffer = ByteBuffer.wrap(msgBytes);

/**
 * Don't send any data and call read() and see how long it waits.
 */
long begin = System.currentTimeMillis();
channel.read(msgBuffer);
   long end = System.currentTimeMillis();

Feel to free close duplicate bugs.

 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=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] Commented: (ZOOKEEPER-914) QuorumCnxManager blocks forever

2010-11-08 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-914:


OK, that didn't make sense. My comment about non-blocking channels was 
incorrect. They are non-blocking :-) Doesn't wait till SO_TIMEOUT.

 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=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] Commented: (ZOOKEEPER-900) FLE implementation should be improved to use non-blocking sockets

2010-11-05 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-900:


Hi Flavio,

Thanks for your feedback. I will do the code changes.

For point 2 above, I was referring to the code that deletes the SenderWorker 
and ReceiveWorker pair after receiving a connect request. I was concerned that 
a peer might send frequent connect request before to the remote peer before the 
remote peer can initiate connection back. But I think the 
Notification n = recvqueue.poll(notTimeout,  TimeUnit.MILLISECONDS); in 
lookForLeader will prevent this scenario. Also, this won't be a concern if we 
decide to remove the part that kills the pair for each connect.

I am also thinking of adding a sanity check that will accept connections only 
from peers that are not listed in the zoo.cfg file or OBSERVER_ID.
I have not used observes so far. Can you please explain why a node will use 
OBSERVER_ID instead of its sid? In particular, I am referring to the following 
code in QuorumCnxManager:
// Read server id
sid = Long.valueOf(msgBuffer.getLong());
if(sid == QuorumPeer.OBSERVER_ID){
/*
 * Choose identifier at random. We need a value to identify
 * the connection.
 */

sid = observerCounter--;
LOG.info(Setting arbitrary identifier to observer:  + sid);
}

 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.



[jira] Commented: (ZOOKEEPER-917) Leader election selected incorrect leader

2010-11-05 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-917:


Hi Flavio,

Sorry for not making much progress on 
(http://wiki.apache.org/hadoop/ZooKeeper/ClusterMembership). I have spent some 
time to understand the code. But It is a bit difficult to focus on development 
without dedicated development time. I am pushing to get dedicated development 
time at work for this so that I don't have to rely on my spare time. 

Few questions related to your comments:
1. Can you please elaborate on : At the same time, a server A decides to 
follow another server B if it receives a message from B saying that B is 
leading and from a quorum saying that they are following, even if A is in a 
later election epoch. This mechanism is there to avoid A being locked out of 
the ensemble in the case it partitions away and comes back later.

2. Why is it not OK for B to give up leadership when it sees that its 
epoch,zxid is lower than others?

Thanks.


 Leader election selected incorrect leader
 -

 Key: ZOOKEEPER-917
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-917
 Project: Zookeeper
  Issue Type: Bug
  Components: leaderElection, server
Affects Versions: 3.2.2
 Environment: Cloudera distribution of zookeeper (patched to never 
 cache DNS entries)
 Debian lenny
Reporter: Alexandre Hardy
Priority: Critical
 Fix For: 3.3.3, 3.4.0

 Attachments: zklogs-20101102144159SAST.tar.gz


 We had three nodes running zookeeper:
   * 192.168.130.10
   * 192.168.130.11
   * 192.168.130.14
 192.168.130.11 failed, and was replaced by a new node 192.168.130.13 
 (automated startup). The new node had not participated in any zookeeper 
 quorum previously. The node 192.148.130.11 was permanently removed from 
 service and could not contribute to the quorum any further (powered off).
 DNS entries were updated for the new node to allow all the zookeeper servers 
 to find the new node.
 The new node 192.168.130.13 was selected as the LEADER, despite the fact that 
 it had not seen the latest zxid.
 This particular problem has not been verified with later versions of 
 zookeeper, and no attempt has been made to reproduce this problem as yet.

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



[jira] Commented: (ZOOKEEPER-872) Small fixes to PurgeTxnLog

2010-11-01 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-872:


Hi Pat,

The current code prints to stdout.  We have a RMI service that has ZK server 
embedded in it. We do this so that we can run/start/stop ZK across platforms 
without having to write platform specific scripts. In this server, we  start a 
thread that periodically calls PurgeTxnlog.purge(). As you pointed out, we 
should have a -q flag to direct to log instead stdout to statisfy both the 
approaches. I will make that change.

We chose number 2 here because we think having only one backup will be enough. 
It is not clear to us under what conditions the additional backup will be 
useful.

Backups are useful under the following scenario (correct me if I am wrong):
1. The current ZooKeeper transaction log and/or snapshot is corrupted, but the 
past snapshots and transaction logs are ok. Corrupting can mean either disk 
file corruption or corrupting of transaction entries in the log. We store 
ZooKeeper data on mirrored disks.
2. The application itself made some errors that requires reverting back to the 
older version.

For the first point, having one additional backup would suffice. The second 
point is really tricky. I am not sure how the application can decide which 
snapshot to revert to. I think in most cases it will be trial and error. It is 
not clear to me how to estimate the number of backups needed. Also, it is not 
clear how one would go about going back in time. I looked at LogFormatter 
utility and that utility does not help much in undoing the erroneous 
transactions for case 2 above. In general, I think it is good to enforce users 
to have a minimum of one backup.

Related question: Is there hash on the log files (or internal tree structures) 
that can tell the ZooKeeper server if the logs are corrupted. If yes, the 
zookeeper server can verify the hash during startup and take some action based 
on that. For example, make sure that it never becomes a leader until it gets 
the correct snapshot from the existing leader (otherwise it may endup 
corrupting other server's log). Corrupting here refers to the case where the 
file is readable, but one or more transactions in the log are bad.

I am not sure if there is a test for this. If I remember correctly, there is a 
bug that causes the purge() function to leave behind one addition log file. 
Please refer to my question above about findNRecentSnapshots(). I can add a 
test or modify the pruge utlity once we have concluded this discussion.

 Small fixes to PurgeTxnLog 
 ---

 Key: ZOOKEEPER-872
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-872
 Project: Zookeeper
  Issue Type: Bug
Affects Versions: 3.3.1
Reporter: Vishal K
Assignee: Vishal K
Priority: Minor
 Fix For: 3.4.0

 Attachments: ZOOKEEPER-872


 PurgeTxnLog forces us to have at least 2 backups (by having count = 3. Also, 
 it prints to stdout instead of using Logger.

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



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

2010-10-29 Thread Vishal K (JIRA)

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

[jira] Commented: (ZOOKEEPER-851) ZK lets any node to become an observer

2010-10-28 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-851:


I have not used observers yet. Looks like then I need to verify if this 
follower can become a leader.  I think I had tried this. If I remember, the 4th 
node did not become a leader. But this was a while back. I will try it again 
and update the jira.

 ZK lets any node to become an observer
 --

 Key: ZOOKEEPER-851
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-851
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum, server
Affects Versions: 3.3.1
Reporter: Vishal K
Priority: Critical
 Fix For: 3.4.0


 I had a 3 node cluster running. The zoo.cfg on each contained 3 entries as 
 show below:
 tickTime=2000
 dataDir=/var/zookeeper
 clientPort=2181
 initLimit=5
 syncLimit=2
 server.0=10.150.27.61:2888:3888
 server.1=10.150.27.62:2888:3888
 server.2=10.150.27.63:2888:3888
 I wanted to add another node to the cluster. In fourth node's zoo.cfg, I 
 created another entry for that node and started zk server. The zoo.cfg on the 
 first 3 nodes was left unchanged. The fourth node was able to join the 
 cluster even though the 3 nodes had no idea about the fourth node.
 zoo.cfg on fourth node:
 tickTime=2000
 dataDir=/var/zookeeper
 clientPort=2181
 initLimit=5
 syncLimit=2
 server.0=10.150.27.61:2888:3888
 server.1=10.150.27.62:2888:3888
 server.2=10.150.27.63:2888:3888
 server.3=10.17.117.71:2888:3888
 It looks like 10.17.117.71 is becoming an observer in this case. I was 
 expecting that the leader will reject 10.17.117.71.
 # telnet 10.17.117.71 2181
 Trying 10.17.117.71...
 Connected to 10.17.117.71.
 Escape character is '^]'.
 stat
 Zookeeper version: 3.3.0--1, built on 04/02/2010 22:40 GMT
 Clients:
  /10.17.117.71:37297[1](queued=0,recved=1,sent=0)
 Latency min/avg/max: 0/0/0
 Received: 3
 Sent: 2
 Outstanding: 0
 Zxid: 0x20065
 Mode: follower
 Node count: 288

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



[jira] Commented: (ZOOKEEPER-907) Spurious KeeperErrorCode = Session moved messages

2010-10-27 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-907:


Which return code are you referring to? You will see this message in the log 
file of the reader. It is not passed on to the caller anywhere.

 Spurious KeeperErrorCode = Session moved messages
 ---

 Key: ZOOKEEPER-907
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-907
 Project: Zookeeper
  Issue Type: Bug
Affects Versions: 3.3.1
Reporter: Vishal K
Assignee: Vishal K
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: ZOOKEEPER-907.patch, ZOOKEEPER-907.patch_v2


 The sync request does not set the session owner in Request.
 As a result, the leader keeps printing:
 2010-07-01 10:55:36,733 - INFO  [ProcessThread:-1:preprequestproces...@405] - 
 Got user-level KeeperException when processing sessionid:0x298d3b1fa9 
 type:sync: cxid:0x6 zxid:0xfffe txntype:unknown reqpath:/ Error 
 Path:null Error:KeeperErrorCode = Session moved

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



[jira] Commented: (ZOOKEEPER-907) Spurious KeeperErrorCode = Session moved messages

2010-10-27 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-907:


It did occur to me. I thought this was by design for sync? sync() is an async 
call. So the caller never gets any exceptions unless a callback is specified.
I might be worng here though, I am still reading the code to understand how 
sync works.

 Spurious KeeperErrorCode = Session moved messages
 ---

 Key: ZOOKEEPER-907
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-907
 Project: Zookeeper
  Issue Type: Bug
Affects Versions: 3.3.1
Reporter: Vishal K
Assignee: Vishal K
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: ZOOKEEPER-907.patch, ZOOKEEPER-907.patch_v2


 The sync request does not set the session owner in Request.
 As a result, the leader keeps printing:
 2010-07-01 10:55:36,733 - INFO  [ProcessThread:-1:preprequestproces...@405] - 
 Got user-level KeeperException when processing sessionid:0x298d3b1fa9 
 type:sync: cxid:0x6 zxid:0xfffe txntype:unknown reqpath:/ Error 
 Path:null Error:KeeperErrorCode = Session moved

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



[jira] Updated: (ZOOKEEPER-907) Spurious KeeperErrorCode = Session moved messages

2010-10-25 Thread Vishal K (JIRA)

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

Vishal K updated ZOOKEEPER-907:
---

Attachment: ZOOKEEPER-907.patch_v2

Attaching cleaned-up patch.
Ben, let me know what you think about the test that I suggested.

 Spurious KeeperErrorCode = Session moved messages
 ---

 Key: ZOOKEEPER-907
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-907
 Project: Zookeeper
  Issue Type: Bug
Affects Versions: 3.3.1
Reporter: Vishal K
Assignee: Vishal K
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: ZOOKEEPER-907.patch, ZOOKEEPER-907.patch_v2


 The sync request does not set the session owner in Request.
 As a result, the leader keeps printing:
 2010-07-01 10:55:36,733 - INFO  [ProcessThread:-1:preprequestproces...@405] - 
 Got user-level KeeperException when processing sessionid:0x298d3b1fa9 
 type:sync: cxid:0x6 zxid:0xfffe txntype:unknown reqpath:/ Error 
 Path:null Error:KeeperErrorCode = Session moved

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



[jira] Commented: (ZOOKEEPER-907) Spurious KeeperErrorCode = Session moved messages

2010-10-23 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-907:


Let the client connect to the follower. Make sure that sync is your first 
request.

To catch this bug you need a test that for each zk client command connects to 
the follower, issues the request, and disconnects.

 Spurious KeeperErrorCode = Session moved messages
 ---

 Key: ZOOKEEPER-907
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-907
 Project: Zookeeper
  Issue Type: Bug
Affects Versions: 3.3.1
Reporter: Vishal K
Assignee: Vishal K
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: ZOOKEEPER-907.patch


 The sync request does not set the session owner in Request.
 As a result, the leader keeps printing:
 2010-07-01 10:55:36,733 - INFO  [ProcessThread:-1:preprequestproces...@405] - 
 Got user-level KeeperException when processing sessionid:0x298d3b1fa9 
 type:sync: cxid:0x6 zxid:0xfffe txntype:unknown reqpath:/ Error 
 Path:null Error:KeeperErrorCode = Session moved

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



[jira] Commented: (ZOOKEEPER-907) Spurious KeeperErrorCode = Session moved messages

2010-10-21 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-907:


Ben,

What do you mean by partial sync? Will the follower still sync with the leader?

Can you explain how sync works?  I was going throught the code flow. I traced 
the sync call, but I could find the place that performs the data transfer 
between the leader and follwer.

I will cleanup the patch. As per the test goes,  I am not sure what is the best 
way to check if a server has received a request with null owner. I was planning 
to add a test that will initiate all zk client requests. In 
SessionTrackerImpl.checkesession(), set a flag if it sees a null owner. At the 
end of the test, fail the test if the flag is set. Any suggestions?

-Vishal

 Spurious KeeperErrorCode = Session moved messages
 ---

 Key: ZOOKEEPER-907
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-907
 Project: Zookeeper
  Issue Type: Bug
Affects Versions: 3.3.1
Reporter: Vishal K
Assignee: Vishal K
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: ZOOKEEPER-907.patch


 The sync request does not set the session owner in Request.
 As a result, the leader keeps printing:
 2010-07-01 10:55:36,733 - INFO  [ProcessThread:-1:preprequestproces...@405] - 
 Got user-level KeeperException when processing sessionid:0x298d3b1fa9 
 type:sync: cxid:0x6 zxid:0xfffe txntype:unknown reqpath:/ Error 
 Path:null Error:KeeperErrorCode = Session moved

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



[jira] Created: (ZOOKEEPER-907) Spurious KeeperErrorCode = Session moved messages

2010-10-20 Thread Vishal K (JIRA)
Spurious KeeperErrorCode = Session moved messages
---

 Key: ZOOKEEPER-907
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-907
 Project: Zookeeper
  Issue Type: Bug
Affects Versions: 3.3.1
Reporter: Vishal K


The sync request does not set the session owner in Request.

As a result, the leader keeps printing:
2010-07-01 10:55:36,733 - INFO  [ProcessThread:-1:preprequestproces...@405] - 
Got user-level KeeperException when processing sessionid:0x298d3b1fa9 
type:sync: cxid:0x6 zxid:0xfffe txntype:unknown reqpath:/ Error 
Path:null Error:KeeperErrorCode = Session moved


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



[jira] Updated: (ZOOKEEPER-907) Spurious KeeperErrorCode = Session moved messages

2010-10-20 Thread Vishal K (JIRA)

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

Vishal K updated ZOOKEEPER-907:
---

Attachment: ZOOKEEPER-907.patch

attaching patch.

 Spurious KeeperErrorCode = Session moved messages
 ---

 Key: ZOOKEEPER-907
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-907
 Project: Zookeeper
  Issue Type: Bug
Affects Versions: 3.3.1
Reporter: Vishal K
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: ZOOKEEPER-907.patch


 The sync request does not set the session owner in Request.
 As a result, the leader keeps printing:
 2010-07-01 10:55:36,733 - INFO  [ProcessThread:-1:preprequestproces...@405] - 
 Got user-level KeeperException when processing sessionid:0x298d3b1fa9 
 type:sync: cxid:0x6 zxid:0xfffe txntype:unknown reqpath:/ Error 
 Path:null Error:KeeperErrorCode = Session moved

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



[jira] Commented: (ZOOKEEPER-907) Spurious KeeperErrorCode = Session moved messages

2010-10-20 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-907:


sure, I will write a test.

What do you think is the effect of this bug? In 
PrepRequestProcessor.pRequest(), the leader will not pass sync request to 
nextProcessor. Does that mean that sync did not succeed?

 Spurious KeeperErrorCode = Session moved messages
 ---

 Key: ZOOKEEPER-907
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-907
 Project: Zookeeper
  Issue Type: Bug
Affects Versions: 3.3.1
Reporter: Vishal K
Assignee: Vishal K
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: ZOOKEEPER-907.patch


 The sync request does not set the session owner in Request.
 As a result, the leader keeps printing:
 2010-07-01 10:55:36,733 - INFO  [ProcessThread:-1:preprequestproces...@405] - 
 Got user-level KeeperException when processing sessionid:0x298d3b1fa9 
 type:sync: cxid:0x6 zxid:0xfffe txntype:unknown reqpath:/ Error 
 Path:null Error:KeeperErrorCode = Session moved

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



[jira] Commented: (ZOOKEEPER-851) ZK lets any node to become an observer

2010-10-15 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-851:


Hi Henry ,

Any comments?

Thanks.
-Vishal

 ZK lets any node to become an observer
 --

 Key: ZOOKEEPER-851
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-851
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum, server
Affects Versions: 3.3.1
Reporter: Vishal K
Priority: Critical
 Fix For: 3.4.0


 I had a 3 node cluster running. The zoo.cfg on each contained 3 entries as 
 show below:
 tickTime=2000
 dataDir=/var/zookeeper
 clientPort=2181
 initLimit=5
 syncLimit=2
 server.0=10.150.27.61:2888:3888
 server.1=10.150.27.62:2888:3888
 server.2=10.150.27.63:2888:3888
 I wanted to add another node to the cluster. In fourth node's zoo.cfg, I 
 created another entry for that node and started zk server. The zoo.cfg on the 
 first 3 nodes was left unchanged. The fourth node was able to join the 
 cluster even though the 3 nodes had no idea about the fourth node.
 zoo.cfg on fourth node:
 tickTime=2000
 dataDir=/var/zookeeper
 clientPort=2181
 initLimit=5
 syncLimit=2
 server.0=10.150.27.61:2888:3888
 server.1=10.150.27.62:2888:3888
 server.2=10.150.27.63:2888:3888
 server.3=10.17.117.71:2888:3888
 It looks like 10.17.117.71 is becoming an observer in this case. I was 
 expecting that the leader will reject 10.17.117.71.
 # telnet 10.17.117.71 2181
 Trying 10.17.117.71...
 Connected to 10.17.117.71.
 Escape character is '^]'.
 stat
 Zookeeper version: 3.3.0--1, built on 04/02/2010 22:40 GMT
 Clients:
  /10.17.117.71:37297[1](queued=0,recved=1,sent=0)
 Latency min/avg/max: 0/0/0
 Received: 3
 Sent: 2
 Outstanding: 0
 Zxid: 0x20065
 Mode: follower
 Node count: 288

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



[jira] Commented: (ZOOKEEPER-445) Potential bug in leader code

2010-10-15 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-445:


Is this a bug? If not, we should just close this one.

 Potential bug in leader code
 

 Key: ZOOKEEPER-445
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-445
 Project: Zookeeper
  Issue Type: Bug
  Components: server
 Environment: Linux fortiz-desktop 2.6.27-7-generic #1 SMP Fri Oct 24 
 06:42:44 UTC 2008 i686 GNU/Linux
 java version 1.6.0_10
 Java(TM) SE Runtime Environment (build 1.6.0_10-b33)
 Java HotSpot(TM) Client VM (build 11.0-b15, mixed mode, sharing)
Reporter: Manos Kapritsos
Priority: Minor
 Fix For: 3.4.0

   Original Estimate: 0.33h
  Remaining Estimate: 0.33h

 There is a suspicious line in server/quorum/Leader.java:226. It reads
 if (stop) {
 LOG.info(exception while shutting down acceptor:  + e);
 stop = true;
 }

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



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

2010-10-15 Thread Vishal K (JIRA)
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 pe!
 er.

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-10-04 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-880:


While debugging for https://issues.apache.org/jira/browse/ZOOKEEPER-822 I found 
that senderWorkerMap would not have an entry for a server, but there will be a 
RecvWorker and SendWorker thread running for the server. In my case, this was 
seen when the leader died (i.e., during leader election). However, I think this 
can happen when a peer disconnects from another peer. The cause was incorrect 
handling of add/remove of entries from senderWorkerMap, which is exposed due to 
race conditions in QuorumCnxManager. There is a patch available for 
ZOOKEEPER-822.

I am not sure if the ZOOKEEPER-822 is causing trouble here as well. I just 
wanted to point out the possibility.


 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
 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-822) Leader election taking a long time to complete

2010-09-21 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-822:


Hi Flavio,

+1. Looks good. I remember looking at the socket.connect() method, but I don't 
remember why I ruled it out in the favor of thread.
Minor point - missing space before error in LOG.warn(Connection broken: for 
id  + sid + my id =  + self.getId() + error..).

Thank you.

-Vishal



 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Assignee: Vishal K
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: 822.tar.gz, rhel.tar.gz, test_zookeeper_1.log, 
 test_zookeeper_2.log, zk_leader_election.tar.gz, zookeeper-3.4.0.tar.gz, 
 ZOOKEEPER-822-3.3.2.patch, ZOOKEEPER-822-3.3.2.patch, ZOOKEEPER-822.patch, 
 ZOOKEEPER-822.patch, ZOOKEEPER-822.patch, ZOOKEEPER-822.patch, 
 ZOOKEEPER-822.patch_v1


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Commented: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-09-20 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-822:


Hi Flavio,

Thanks. I will take a look at the patches.

-Vishal

 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Assignee: Vishal K
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: 822.tar.gz, rhel.tar.gz, test_zookeeper_1.log, 
 test_zookeeper_2.log, zk_leader_election.tar.gz, zookeeper-3.4.0.tar.gz, 
 ZOOKEEPER-822-3.3.2.patch, ZOOKEEPER-822.patch, ZOOKEEPER-822.patch, 
 ZOOKEEPER-822.patch, ZOOKEEPER-822.patch_v1


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Commented: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-09-14 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-822:


Hi Flavio,

As I mentioned earlier, this is a temporary patch until the selector based 
approach (non-blocking IO)  is ready.

In general, what is the concern with the current fix? There will be only one 
thread running at a time. The thread just makes sure that we can bound the 
connection time. This patch is working well for us as a temporary fix. Apart 
from the overhead of starting a thread I don't see anything wrong with the fix.

Again, given that this bug is a blocker for us, we certainly cannot wait until 
the non-blocking implementation  is done and released.

Thanks.
-Vishal

 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Assignee: Vishal K
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: 822.tar.gz, rhel.tar.gz, test_zookeeper_1.log, 
 test_zookeeper_2.log, zk_leader_election.tar.gz, zookeeper-3.4.0.tar.gz, 
 ZOOKEEPER-822.patch_v1


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Commented: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-09-14 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-822:


I agree with Mahadev. 

 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Assignee: Vishal K
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: 822.tar.gz, rhel.tar.gz, test_zookeeper_1.log, 
 test_zookeeper_2.log, zk_leader_election.tar.gz, zookeeper-3.4.0.tar.gz, 
 ZOOKEEPER-822.patch_v1


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Created: (ZOOKEEPER-872) Small fixes to PurgeTxnLog

2010-09-14 Thread Vishal K (JIRA)
Small fixes to PurgeTxnLog 
---

 Key: ZOOKEEPER-872
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-872
 Project: Zookeeper
  Issue Type: Bug
Affects Versions: 3.3.1
Reporter: Vishal K
Priority: Minor


PurgeTxnLog forces us to have at least 2 backups (by having count = 3. Also, 
it prints to stdout instead of using Logger.

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



[jira] Updated: (ZOOKEEPER-872) Small fixes to PurgeTxnLog

2010-09-14 Thread Vishal K (JIRA)

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

Vishal K updated ZOOKEEPER-872:
---

Attachment: ZOOKEEPER-872

patch attached.


 Small fixes to PurgeTxnLog 
 ---

 Key: ZOOKEEPER-872
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-872
 Project: Zookeeper
  Issue Type: Bug
Affects Versions: 3.3.1
Reporter: Vishal K
Priority: Minor
 Attachments: ZOOKEEPER-872


 PurgeTxnLog forces us to have at least 2 backups (by having count = 3. Also, 
 it prints to stdout instead of using Logger.

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



[jira] Commented: (ZOOKEEPER-872) Small fixes to PurgeTxnLog

2010-09-14 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-872:


Is there any reason why we dont have a findNRecentLogs(int n) method to return 
the n most recent logs (similar to findNRecentSnapshots)?
While testing I noticed that it can happen that a log file is left undeleted 
depending on the transaction id of the nth snapshot file.  Thus, we will have n 
snapshots, but n+1 log file left behind instead of n. This file will be deleted 
after the next snapshot is taken.

We won't have this problem if we have a function that keeps the n most recent 
logs and removes the rest.

 Small fixes to PurgeTxnLog 
 ---

 Key: ZOOKEEPER-872
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-872
 Project: Zookeeper
  Issue Type: Bug
Affects Versions: 3.3.1
Reporter: Vishal K
Priority: Minor
 Attachments: ZOOKEEPER-872


 PurgeTxnLog forces us to have at least 2 backups (by having count = 3. Also, 
 it prints to stdout instead of using Logger.

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



[jira] Updated: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-09-07 Thread Vishal K (JIRA)

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

Vishal K updated ZOOKEEPER-822:
---

Attachment: ZOOKEEPER-822.patch_v1

Hi,

I have attached the diff of my changes. It is a pretty simple fix. My  
intention was to have a fix with minimal code changes.

connectOne waits for 2 seconds for a connection to establish. Right now the 
value is hard-coded. I was planning to add a new property that would specify 
the amount of time to wait for a connect request. If you think that the 
proposed changes are good enough, I will make the change for using the property 
value and resubmit a second patch. 

Let me know.

-Vishal

 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Assignee: Vishal K
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: 822.tar.gz, rhel.tar.gz, test_zookeeper_1.log, 
 test_zookeeper_2.log, zk_leader_election.tar.gz, zookeeper-3.4.0.tar.gz, 
 ZOOKEEPER-822.patch_v1


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Commented: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-09-02 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-822:


Hi Flavio,

I was planning to send out a mail explaining the problems in the FLE 
implementation that I have found so far. For now, I will put the info here. We 
can create new JIRAs if needed. I am waiting to hear back from our legal 
department to resolve copyright issues so that I can share my fixes as well.

1. Blocking connects and accepts:
You are right, when the node is down TCP timeouts rule.

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 pe!
 er.

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.

2. Buggy senderWorkerMap handling:
The code that manages senderWorkerMap is very buggy. It is causing multiple 
election rounds. While debugging I found that sometimes after FLE a node will 
have its sendWorkerMap empty even if it has SenderWorker and RecvWorker threads 
for each peer.

a) The receiveConnection() method calls the finish() method, which removes an 
entry from the map. Additionally, the thread itself calls finish() which could 
remove the newly added entry from the map. In short, receiveConnection is 
causing the exact condition that you mentioned above.

b) Apart from the bug in finish(), receiveConnection is making an entry in 
senderWorkerMap at the wrong place. Here's the buggy code:
SendWorker vsw = senderWorkerMap.get(sid);
senderWorkerMap.put(sid, sw);
if(vsw != null)
vsw.finish();
It makes an entry for the new thread and then calls finish, which causes the 
new thread to be removed from the Map. The old thread will also get terminated 
since finish() will interrupt the thread.

3. Race condition in receiveConnection and initiateConnection:

*In theory*, two peers can keep disconnecting each other's connection.

Example:
T0: Peer 0 initiates a connection (request 1)

   T1: Peer 1 receives connection from peer 0

   T2: Peer 1 calls receiveConnection()
T2: Peer 0 closes connection to Peer 1 because its ID is lower.
T3: Peer 0 re-initiates connection to Peer 1 from manger.toSend() (request 2)
T3: Peer 1 terminates older connection to peer 0
T4: Peer 1 calls connectOne() which starts new sendWorker threads for peer 0
T5: Peer 1 kills connection created in T3 because it receives another (request 
2) connect request from 0

The problem here is that while Peer 0 is accepting a connection from Peer 1 it 
can also be initiating a connection to Peer 1. So if they hit the right 
frequencies they could sit in a connect/disconnect loop and cause multiple 
rounds of leader election.

I think the cause here is again blocking connects()/accepts(). A peer starts to 
take action (to kill existing threads and start new threads) as soon as a 
connection is established at the *TCP level*. That is, it does not give us any 
control to synchronized connect and accepts. We could use non-blocking connects 
and accepts. This will allow us to a) tell a 

[jira] Commented: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-08-26 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-822:


The fix for problem 1 and 2 above eliminates the bug. I will have a patch out 
soon.

 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Priority: Blocker
 Attachments: 822.tar.gz, rhel.tar.gz, test_zookeeper_1.log, 
 test_zookeeper_2.log, zk_leader_election.tar.gz, zookeeper-3.4.0.tar.gz


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Updated: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-08-20 Thread Vishal K (JIRA)

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

Vishal K updated ZOOKEEPER-822:
---

Attachment: zookeeper-3.4.0.tar.gz

Hi,

To rule out any setup specific issues (VM/ESX/ect), I tried to reproduce this 
problem with 3 physical machines (no VMs). I was able to reproduce the same 
problem after 15 reboots or so. It took a little more number of reboots on the 
physical setup. I used the latest code this time. I  have attached the logs. 

I am suspecting 3 problems:
1. blocking connect in WorkerSender.process as described in my earlier comments.
2. SendWorker.run() calls finish at the end. This could result in finish() 
getting called twice (e.g., finish called from receiveConnection), thus, 
causing  senderWorkerMap.remove(sid) called twice and removing an entry that 
should be removed.
3. Race condition that causes one of the peers to disconnect other peer's 
connection (receiveConnection/initiateConnection issue). I am still 
verifying/investigating the this.

In logs of server id 0 and 1, search for ***REBOOTING LEADER*** from the 
bottom. This line marks the start of faulty FLE.

-Vishal

 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Priority: Blocker
 Attachments: 822.tar.gz, rhel.tar.gz, test_zookeeper_1.log, 
 test_zookeeper_2.log, zk_leader_election.tar.gz, zookeeper-3.4.0.tar.gz


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Commented: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-08-20 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-822:


Correction:
2. SendWorker.run() calls finish at the end. This could result in finish() 
getting called twice (e.g., finish called from receiveConnection), thus, 
causing senderWorkerMap.remove(sid) called twice and removing an entry that 
should *not* be removed.

 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Priority: Blocker
 Attachments: 822.tar.gz, rhel.tar.gz, test_zookeeper_1.log, 
 test_zookeeper_2.log, zk_leader_election.tar.gz, zookeeper-3.4.0.tar.gz


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Commented: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-08-19 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-822:


Hi Flavio,

Ah! my trunk is quite old then. But I don't think it is necessary to run with 
the latest code for debugging this issue.

I have identified one problem in WorkerSender.process(). This function calls 
manager.toSend() whicih calls connectOne. connectOne does a blocking connect 
(which takes order of minutes to return if a node is down). Thus, 
WorkerSender.run() will  block and not send any successive notifications to 
other nodes.

Let met know what you think

I tired with adding timeouts to connectOne, but I am running into similar issue 
somewhere else. So that didnt fix the problem





 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Priority: Blocker
 Attachments: 822.tar.gz, rhel.tar.gz, test_zookeeper_1.log, 
 test_zookeeper_2.log, zk_leader_election.tar.gz


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Commented: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-08-18 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-822:


While going through the code yesterday,  I found two potential problems that I 
though might be worth reporting in the context of this bug.

1. In FastLeaderElection.java
/**
 * Check if all queues are empty, indicating that all messages have been 
delivered.
 */
boolean haveDelivered() {
for (ArrayBlockingQueueByteBuffer queue : queueSendMap.values()) {
LOG.debug(Queue size:  + queue.size());
if (queue.size() == 0)
return true;
}

return false;
}

the haveDelivered()  function returns true without checking if rest of the 
queus are empty.

2. QuorumCnxManager.connectAll() function connects to one peer at a time and it 
uses a blocking connect (SocketChannle.open). I added a timeout to the 
SocketChannel.open and that did not fix the problem. 

 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Priority: Blocker
 Attachments: 822.tar.gz, rhel.tar.gz, test_zookeeper_1.log, 
 test_zookeeper_2.log, zk_leader_election.tar.gz


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Commented: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-08-18 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-822:


I am suspecting that one of the node (10.17.119.101) is not sending the 
notification to the other node. sendNotifications() is called to send 
notification to all peers. This functions enteres the notification in 
sendqueue. However, either the entry was not put in the queue (sendqueue.offer 
failed) or the thread that polls sendqueue did not wake up. I am not sure what 
the cause is yet.

I had added extra debug messages. Three messages are of main interest:

1. in sendNotifications():  Print IN FLE sending notification to server id = 
1 for each server. Also print  proposedLeader, proposedZxid, logicalclock
2. In FastLeaderElection.lookForLeader() print Updating proposa before 
calling upgradeProposal if (totalOrderPredicate(n.leader, n.zxid, proposedZxid) 
is true
3. in WorkerSender.process(), log -  LOG.info(WorkSender.process() QUEUEING 
m.state=  + m.state +  m.leader= + m.leader +  m.sid= + m.sid);

Suppporting log entries from 10.17.119.101-zookeeper.log.  I  have added 
description inline..
--
2010-08-18 14:53:56,451 - INFO  
[QuorumPeer:/0:0:0:0:0:0:0:0:2181:fastleaderelect...@475] - IN FLE sending 
notification to server id = 1
2010-08-18 14:53:56,451 - INFO  
[QuorumPeer:/0:0:0:0:0:0:0:0:2181:fastleaderelect...@476] - proposedLeader, 
proposedZxid, logicalclock 0343597383684
2010-08-18 14:53:56,452 - INFO  [WorkerSender 
Thread:fastleaderelection$messenger$workersen...@352] - WorkSender.process() 
QUEUEING m.state= LOOKING m.leader=0 m.sid=1
2010-08-18 14:53:56,452 - DEBUG [WorkerSender Thread:quorumcnxmana...@347] - 
Opening channel to server 1
2010-08-18 14:53:56,453 - INFO  
[QuorumPeer:/0:0:0:0:0:0:0:0:2181:fastleaderelect...@475] - IN FLE sending 
notification to server id = 2
2010-08-18 14:53:56,453 - INFO  
[QuorumPeer:/0:0:0:0:0:0:0:0:2181:fastleaderelect...@476] - proposedLeader, 
proposedZxid, logicalclock 0343597383684
2010-08-18 14:53:56,453 - INFO  
[QuorumPeer:/0:0:0:0:0:0:0:0:2181:fastleaderelect...@690] - Notification: 1, 
34359738368, 4, 0, LOOKING, LOOKING, 1
2010-08-18 14:53:56,454 - DEBUG 
[QuorumPeer:/0:0:0:0:0:0:0:0:2181:fastleaderelect...@496] - id: 1, proposed id: 
0, zxid: 34359738368, proposed zxid: 34359738368
2010-08-18 14:53:56,454 - INFO  
[QuorumPeer:/0:0:0:0:0:0:0:0:2181:fastleaderelect...@710] - Updating proposal
2010-08-18 14:53:56,454 - INFO  [WorkerSender Thread:quorumcnxmana...@162] - 
Have smaller server identifier, so dropping the connection: (1, 0)
2010-08-18 14:53:56,455 - INFO  [WorkerSender 
Thread:fastleaderelection$messenger$workersen...@352] - WorkSender.process() 
QUEUEING m.state= LOOKING m.leader=0 m.sid=2
2010-08-18 14:53:56,458 - DEBUG [WorkerSender Thread:quorumcnxmana...@347] - 
Opening channel to server 2
2010-08-18 14:53:56,458 - WARN  [Thread-19:quorumcnxmanager$recvwor...@659] - 
Connection broken:
java.io.IOException: Channel eof
at 
org.apache.zookeeper.server.quorum.QuorumCnxManager$RecvWorker.run(QuorumCnxManager.java:631)
2010-08-18 14:53:56,459 - INFO  
[QuorumPeer:/0:0:0:0:0:0:0:0:2181:fastleaderelect...@475] - IN FLE sending 
notification to server id = 0

2010-08-18 14:53:56,460 - INFO  
[QuorumPeer:/0:0:0:0:0:0:0:0:2181:fastleaderelect...@476] - proposedLeader, 
proposedZxid, logicalclock 1343597383684  

* The above line shows that this node (server 0) is going to vote for 1. 
see - proposedLeader, proposedZxid, logicalclock 1 34359738368 4  Forgot to add 
spaces in the message  :-) 


2010-08-18 14:53:56,460 - DEBUG [Thread-1:quorumcnxmanager$liste...@446] - 
Connection request /10.17.119.102:41597
2010-08-18 14:53:56,461 - DEBUG [Thread-1:quorumcnxmanager$liste...@449] - 
Connection request: 0
2010-08-18 14:53:56,461 - DEBUG [Thread-1:quorumcnxmanager$sendwor...@505] - 
Address of remote peer: 1
2010-08-18 14:53:56,461 - INFO  
[QuorumPeer:/0:0:0:0:0:0:0:0:2181:fastleaderelect...@475] - IN FLE sending 
notification to server id = 1
2010-08-18 14:53:56,462 - INFO  
[QuorumPeer:/0:0:0:0:0:0:0:0:2181:fastleaderelect...@476] - proposedLeader, 
proposedZxid, logicalclock 1343597383684 

* Above, server 0 queued a notification to be sent to server 1. The 
notfication is saying that it accepts 1 as the leader. But the notification 
never got sent. process() was not called at all from WorkerSender. Its almost 
as if the notification was never entered in sendqueue (in sendNotifications). 
*

2010-08-18 14:53:56,462 - INFO  
[QuorumPeer:/0:0:0:0:0:0:0:0:2181:fastleaderelect...@475] - IN FLE sending 
notification to server id = 2
2010-08-18 14:53:56,462 - INFO  
[QuorumPeer:/0:0:0:0:0:0:0:0:2181:fastleaderelect...@476] - proposedLeader, 
proposedZxid, logicalclock 1343597383684
2010-08-18 14:53:56,463 - DEBUG 

[jira] Commented: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-08-18 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-822:


at line 852 in 10.17.119.101-zookeeper.log WorkSender finally finds something 
in sendqueue and starts sending the notification to server 1.

 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Priority: Blocker
 Attachments: 822.tar.gz, rhel.tar.gz, test_zookeeper_1.log, 
 test_zookeeper_2.log, zk_leader_election.tar.gz


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Created: (ZOOKEEPER-851) ZK lets any node to become an observer

2010-08-16 Thread Vishal K (JIRA)
ZK lets any node to become an observer
--

 Key: ZOOKEEPER-851
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-851
 Project: Zookeeper
  Issue Type: Bug
Reporter: Vishal K
 Fix For: 3.3.1


I had a 3 node cluster running. The zoo.cfg on each contained 3 entries as show 
below:

tickTime=2000
dataDir=/var/zookeeper
clientPort=2181
initLimit=5
syncLimit=2
server.0=10.150.27.61:2888:3888
server.1=10.150.27.62:2888:3888
server.2=10.150.27.63:2888:3888

I wanted to add another node to the cluster. In fourth node's zoo.cfg, I 
created another entry for that node and started zk server. The zoo.cfg on the 
first 3 nodes was left unchanged. The fourth node was able to join the cluster 
even though the 3 nodes had no idea about the fourth node.

zoo.cfg on fourth node:
tickTime=2000
dataDir=/var/zookeeper
clientPort=2181
initLimit=5
syncLimit=2
server.0=10.150.27.61:2888:3888
server.1=10.150.27.62:2888:3888
server.2=10.150.27.63:2888:3888
server.3=10.17.117.71:2888:3888

It looks like 10.17.117.71 is becoming an observer in this case. I was 
expecting that the leader will reject 10.17.117.71.

# telnet 10.17.117.71 2181
Trying 10.17.117.71...
Connected to 10.17.117.71.
Escape character is '^]'.
stat
Zookeeper version: 3.3.0--1, built on 04/02/2010 22:40 GMT
Clients:
 /10.17.117.71:37297[1](queued=0,recved=1,sent=0)

Latency min/avg/max: 0/0/0
Received: 3
Sent: 2
Outstanding: 0
Zxid: 0x20065
Mode: follower
Node count: 288

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



[jira] Commented: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-08-16 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-822:


Hi Ivan,

Can you describe me your setup?

My setup info:
- 3 ESX boxes
- 1 SLES 11 VMs on each
- Cluster of 3 nodes

I hit this problem consistently after rebooting the leader.

Thanks.
-Vishal


 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Priority: Blocker
 Attachments: 822.tar.gz, test_zookeeper_1.log, test_zookeeper_2.log, 
 zk_leader_election.tar.gz


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Commented: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-08-06 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-822:


Hi Ivan.

Were the logs of any help? I might be worth having 3 VMs and rebooting the 
leader instead of shutting down the interface. We have seen this on all of our 
dev cluster. Al tough all the dev clusters are based on same VM images. So it 
won't be fair to claim that the problem was seen on different set of machines. 
I will try with the latest trunk and let you know the result. What FLE changes 
do you think would have fixed this problem?

Thanks.

-Vishal




 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Priority: Blocker
 Attachments: 822.tar.gz, test_zookeeper_1.log, test_zookeeper_2.log, 
 zk_leader_election.tar.gz


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Updated: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-08-02 Thread Vishal K (JIRA)

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

Vishal K updated ZOOKEEPER-822:
---

Attachment: 822.tar.gz

attaching new logs.

 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Priority: Blocker
 Attachments: 822.tar.gz, test_zookeeper_1.log, test_zookeeper_2.log, 
 zk_leader_election.tar.gz


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Commented: (ZOOKEEPER-790) Last processed zxid set prematurely while establishing leadership

2010-07-29 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-790:


Hi,

I tested the patch and it works really well. I have a test that fails and 
restarts zookeeper server in a loop. I kept it going for 150 rounds and found 
no issues.
Thanks a lot.

-Vishal

 Last processed zxid set prematurely while establishing leadership
 -

 Key: ZOOKEEPER-790
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-790
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.1
Reporter: Flavio Junqueira
Assignee: Flavio Junqueira
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: ZOOKEEPER-790-3.3.patch, ZOOKEEPER-790-3.3.patch, 
 ZOOKEEPER-790-3.3.v2.patch, ZOOKEEPER-790-follower-request-NPE.log, 
 ZOOKEEPER-790-test.patch, ZOOKEEPER-790.patch, ZOOKEEPER-790.patch, 
 ZOOKEEPER-790.patch, ZOOKEEPER-790.patch, ZOOKEEPER-790.patch, 
 ZOOKEEPER-790.travis.log.bz2, ZOOKEEPER-790.v2.patch, ZOOKEEPER-790.v2.patch, 
 ZOOKEEPER-790.v2.patch


 The leader code is setting the last processed zxid to the first of the new 
 epoch even before connecting to a quorum of followers. Because the leader 
 code sets this value before connecting to a quorum of followers 
 (Leader.java:281) and the follower code throws an IOException 
 (Follower.java:73) if the leader epoch is smaller, we have that when the 
 false leader drops leadership and becomes a follower, it finds a smaller 
 epoch and kills itself.

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



[jira] Updated: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-07-22 Thread Vishal K (JIRA)

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

Vishal K updated ZOOKEEPER-822:
---

Attachment: zk_leader_election.tar.gz

Hi Flavio,

I have attached the logs. Hope this helps. Do you have more info on loggraph 
(doc, etc)?

Thanks.

 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Priority: Blocker
 Attachments: test_zookeeper_1.log, test_zookeeper_2.log, 
 zk_leader_election.tar.gz


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Updated: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-07-19 Thread Vishal K (JIRA)

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

Vishal K updated ZOOKEEPER-822:
---

Attachment: test_zookeeper_2.log
test_zookeeper_1.log

Attaching logs from two nodes that took too long to complete leader election. 

 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Priority: Blocker
 Attachments: test_zookeeper_1.log, test_zookeeper_2.log


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Created: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-07-19 Thread Vishal K (JIRA)
Leader election taking a long time  to complete
---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Priority: Blocker


Created a 3 node cluster.

1 Fail the ZK leader
2. Let leader election finish. Restart the leader and let it join the 
3. Repeat 

After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
Note- we didn't have any ZK clients and no new znodes were created.

zoo.cfg is shown below:

#Mon Jul 19 12:15:10 UTC 2010
server.1=192.168.4.12\:2888\:3888
server.0=192.168.4.11\:2888\:3888
clientPort=2181
dataDir=/var/zookeeper
syncLimit=2
server.2=192.168.4.13\:2888\:3888
initLimit=5
tickTime=2000

I have attached logs from two nodes that took a long time to form the cluster 
after failing the leader. The leader was down anyways so logs from that node 
shouldn't matter.
Look for START HERE. Logs after that point should be of our interest.

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



[jira] Commented: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-07-19 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-822:


I would like that add that the problem is highly reproducible.

 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Priority: Blocker
 Attachments: test_zookeeper_1.log, test_zookeeper_2.log


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Commented: (ZOOKEEPER-822) Leader election taking a long time to complete

2010-07-19 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-822:


Hi Flavio,

I have Zookeeper servers running in a VM. To kill ZK server, I power off a VM. 
On the other hand, I tried several times killing ZK process and restarting it 
and I did not see any issues.
So there is something about the reboot that is causing this problem (TCP 
session not getting cleaned-up?).

I don't see many connection exceptions in the log.

Once the leader election starts  we start seeing Notification time out 
messages.

However, before this we do see that the connection was established (show below):

2010-07-19 14:40:52,562 - DEBUG [WorkerSender Thread:quorumcnxmana...@366] - 
There is a connection already for server 0
2010-07-19 14:40:52,563 - DEBUG [WorkerSender Thread:quorumcnxmana...@346] - 
Opening channel to server 2

Do you still think this is a communication problem?

 Leader election taking a long time  to complete
 ---

 Key: ZOOKEEPER-822
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-822
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.0
Reporter: Vishal K
Priority: Blocker
 Attachments: test_zookeeper_1.log, test_zookeeper_2.log


 Created a 3 node cluster.
 1 Fail the ZK leader
 2. Let leader election finish. Restart the leader and let it join the 
 3. Repeat 
 After a few rounds leader election takes anywhere 25- 60 seconds to finish. 
 Note- we didn't have any ZK clients and no new znodes were created.
 zoo.cfg is shown below:
 #Mon Jul 19 12:15:10 UTC 2010
 server.1=192.168.4.12\:2888\:3888
 server.0=192.168.4.11\:2888\:3888
 clientPort=2181
 dataDir=/var/zookeeper
 syncLimit=2
 server.2=192.168.4.13\:2888\:3888
 initLimit=5
 tickTime=2000
 I have attached logs from two nodes that took a long time to form the cluster 
 after failing the leader. The leader was down anyways so logs from that node 
 shouldn't matter.
 Look for START HERE. Logs after that point should be of our interest.

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



[jira] Commented: (ZOOKEEPER-790) Last processed zxid set prematurely while establishing leadership

2010-07-16 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-790:


great! I will give it a try. Thanks.

 Last processed zxid set prematurely while establishing leadership
 -

 Key: ZOOKEEPER-790
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-790
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.1
Reporter: Flavio Paiva Junqueira
Assignee: Flavio Paiva Junqueira
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: ZOOKEEPER-790.patch, ZOOKEEPER-790.patch, 
 ZOOKEEPER-790.travis.log.bz2


 The leader code is setting the last processed zxid to the first of the new 
 epoch even before connecting to a quorum of followers. Because the leader 
 code sets this value before connecting to a quorum of followers 
 (Leader.java:281) and the follower code throws an IOException 
 (Follower.java:73) if the leader epoch is smaller, we have that when the 
 false leader drops leadership and becomes a follower, it finds a smaller 
 epoch and kills itself.

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



[jira] Commented: (ZOOKEEPER-790) Last processed zxid set prematurely while establishing leadership

2010-07-15 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-790:


Likewise.

 Last processed zxid set prematurely while establishing leadership
 -

 Key: ZOOKEEPER-790
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-790
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.1
Reporter: Flavio Paiva Junqueira
Assignee: Flavio Paiva Junqueira
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: ZOOKEEPER-790.patch, ZOOKEEPER-790.travis.log.bz2


 The leader code is setting the last processed zxid to the first of the new 
 epoch even before connecting to a quorum of followers. Because the leader 
 code sets this value before connecting to a quorum of followers 
 (Leader.java:281) and the follower code throws an IOException 
 (Follower.java:73) if the leader epoch is smaller, we have that when the 
 false leader drops leadership and becomes a follower, it finds a smaller 
 epoch and kills itself.

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



[jira] Commented: (ZOOKEEPER-790) Last processed zxid set prematurely while establishing leadership

2010-07-14 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-790:


Hi Flavio,

I am going to retry again with the patch. Thanks.

-Vishal

 Last processed zxid set prematurely while establishing leadership
 -

 Key: ZOOKEEPER-790
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-790
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.1
Reporter: Flavio Paiva Junqueira
Assignee: Flavio Paiva Junqueira
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: ZOOKEEPER-790.patch


 The leader code is setting the last processed zxid to the first of the new 
 epoch even before connecting to a quorum of followers. Because the leader 
 code sets this value before connecting to a quorum of followers 
 (Leader.java:281) and the follower code throws an IOException 
 (Follower.java:73) if the leader epoch is smaller, we have that when the 
 false leader drops leadership and becomes a follower, it finds a smaller 
 epoch and kills itself.

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



[jira] Commented: (ZOOKEEPER-790) Last processed zxid set prematurely while establishing leadership

2010-07-14 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-790:


Hi Patrick,

I had done that earlier. I will do it again to reconfirm. I would also help if 
Travis or Charity  can also try to verify the patch, if possible.

-Vishal

 Last processed zxid set prematurely while establishing leadership
 -

 Key: ZOOKEEPER-790
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-790
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.1
Reporter: Flavio Paiva Junqueira
Assignee: Flavio Paiva Junqueira
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: ZOOKEEPER-790.patch


 The leader code is setting the last processed zxid to the first of the new 
 epoch even before connecting to a quorum of followers. Because the leader 
 code sets this value before connecting to a quorum of followers 
 (Leader.java:281) and the follower code throws an IOException 
 (Follower.java:73) if the leader epoch is smaller, we have that when the 
 false leader drops leadership and becomes a follower, it finds a smaller 
 epoch and kills itself.

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



[jira] Commented: (ZOOKEEPER-790) Last processed zxid set prematurely while establishing leadership

2010-07-13 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-790:


Folks,

Sorry to the delay. The patch did not work. Any other ideas? Thanks.

-Vishal

 Last processed zxid set prematurely while establishing leadership
 -

 Key: ZOOKEEPER-790
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-790
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.1
Reporter: Flavio Paiva Junqueira
Assignee: Flavio Paiva Junqueira
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: ZOOKEEPER-790.patch


 The leader code is setting the last processed zxid to the first of the new 
 epoch even before connecting to a quorum of followers. Because the leader 
 code sets this value before connecting to a quorum of followers 
 (Leader.java:281) and the follower code throws an IOException 
 (Follower.java:73) if the leader epoch is smaller, we have that when the 
 false leader drops leadership and becomes a follower, it finds a smaller 
 epoch and kills itself.

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



[jira] Commented: (ZOOKEEPER-790) Last processed zxid set prematurely while establishing leadership

2010-07-13 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-790:


copying comments from email to jira.

Hi Flavio ,

I got the same error messages. I can reproduce this quite easily. I will 
capture the logs again. Is there anything else you would like me to provide? 
Thanks.

-Vishal

 Last processed zxid set prematurely while establishing leadership
 -

 Key: ZOOKEEPER-790
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-790
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.1
Reporter: Flavio Paiva Junqueira
Assignee: Flavio Paiva Junqueira
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: ZOOKEEPER-790.patch


 The leader code is setting the last processed zxid to the first of the new 
 epoch even before connecting to a quorum of followers. Because the leader 
 code sets this value before connecting to a quorum of followers 
 (Leader.java:281) and the follower code throws an IOException 
 (Follower.java:73) if the leader epoch is smaller, we have that when the 
 false leader drops leadership and becomes a follower, it finds a smaller 
 epoch and kills itself.

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



[jira] Commented: (ZOOKEEPER-790) Last processed zxid set prematurely while establishing leadership

2010-07-13 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-790:


copying comments from email to jira. 

I forgot if you have provided already a description of how you reproduce it. If 
you could point me out to that, I would appreciate.

-Flavio



 Last processed zxid set prematurely while establishing leadership
 -

 Key: ZOOKEEPER-790
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-790
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.1
Reporter: Flavio Paiva Junqueira
Assignee: Flavio Paiva Junqueira
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: ZOOKEEPER-790.patch


 The leader code is setting the last processed zxid to the first of the new 
 epoch even before connecting to a quorum of followers. Because the leader 
 code sets this value before connecting to a quorum of followers 
 (Leader.java:281) and the follower code throws an IOException 
 (Follower.java:73) if the leader epoch is smaller, we have that when the 
 false leader drops leadership and becomes a follower, it finds a smaller 
 epoch and kills itself.

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



[jira] Commented: (ZOOKEEPER-790) Last processed zxid set prematurely while establishing leadership

2010-07-13 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-790:


From ZOOKEEPER-335..

Hi,

I enabled tracing and did some more debugging. Looks like the restarted peer 
(and trying to join the cluster) determines that it is a leader and increments 
its epoch. However, rest of the nodes don't acknowledge this node as the 
leader, and hence, have an older epoch. I will attache the log. Unfortunately, 
I don't have traces from other nodes. I will repeat the experiment later and 
attache logs from other nodes.

Scenario:

* Form a 3 node cluster. This is not just ZK cluster. It also involves our 
application cluster that uses ZK.
* Kill one of the follower
* After a minute or so restart follower
* Follower rejects leader with Leader epoch y is less than our epoch y + 1

From logs:

a) Peer X restarts and starts leader election.
a) For a small window of time, X thinks that it is the new leader! During this 
window, for some reason, rest of the nodes tell X that they are also trying to 
find a leader. I.e., all 3 nodes are in LOOKING state. After seeing that all 3 
nodes are in LOOKING state, X decides to be a leader?

155 2010-06-20 23:22:46,421 - DEBUG [WorkerSender Thread:quorumcnxmana...@346] 
- Opening channel to server 1
156 2010-06-20 23:22:46,423 - DEBUG [WorkerReceiver 
Thread:fastleaderelection$messenger$workerrecei...@214] - Receive new 
notification message. My id = 0
157 2010-06-20 23:22:46,424 - INFO 
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@689] - Notification: 0, 
77309411393, 1, 0, LOOKING, LOOKING, 0
158 2010-06-20 23:22:46,424 - DEBUG 
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@495] - id: 0, proposed id: 0, 
zxid: 77309411393, proposed zxid: 77309411393
159 2010-06-20 23:22:46,424 - DEBUG 
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@717] - Adding vote: From = 0, 
Proposed leader = 0, Porposed zxid = 77309411393, Proposed epoch = 1
160 2010-06-20 23:22:46,426 - INFO [WorkerSender Thread:quorumcnxmana...@162] - 
Have smaller server identifier, so dropping the connection: (1, 0)
161 2010-06-20 23:22:46,426 - DEBUG [WorkerSender Thread:quorumcnxmana...@346] 
- Opening channel to server 2
162 2010-06-20 23:22:46,427 - DEBUG [Thread-1:quorumcnxmanager$liste...@445] - 
Connection request /192.168.1.182:46701
163 2010-06-20 23:22:46,427 - DEBUG [Thread-1:quorumcnxmanager$liste...@448] - 
Connection request: 0
164 2010-06-20 23:22:46,428 - DEBUG [Thread-1:quorumcnxmanager$sendwor...@504] 
- Address of remote peer: 1
165 2010-06-20 23:22:46,428 - INFO [WorkerSender Thread:quorumcnxmana...@162] - 
Have smaller server identifier, so dropping the connection: (2, 0)
166 2010-06-20 23:22:46,431 - DEBUG [WorkerReceiver 
Thread:fastleaderelection$messenger$workerrecei...@214] - Receive new 
notification message. My id = 0
167 2010-06-20 23:22:46,432 - INFO 
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@689] - Notification: 1, 
77309411372, 1, 0, LOOKING, LOOKING, 1
168 2010-06-20 23:22:46,432 - DEBUG 
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@495] - id: 1, proposed id: 0, 
zxid: 77309411372, proposed zxid: 77309411393
169 2010-06-20 23:22:46,432 - DEBUG 
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@717] - Adding vote: From = 1, 
Proposed leader = 1, Porposed zxid = 77309411372, Proposed epoch = 1
170 2010-06-20 23:22:46,436 - DEBUG [Thread-1:quorumcnxmanager$liste...@445] - 
Connection request /192.168.1.183:44310
171 2010-06-20 23:22:46,436 - DEBUG [Thread-1:quorumcnxmanager$liste...@448] - 
Connection request: 0
172 2010-06-20 23:22:46,436 - DEBUG [Thread-1:quorumcnxmanager$sendwor...@504] 
- Address of remote peer: 2
173 2010-06-20 23:22:46,440 - DEBUG [WorkerReceiver 
Thread:fastleaderelection$messenger$workerrecei...@214] - Receive new 
notification message. My id = 0
174 2010-06-20 23:22:46,440 - INFO 
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@689] - Notification: 2, 
7301097, 1, 0, LOOKING, LOOKING, 2
175 2010-06-20 23:22:46,440 - DEBUG 
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@495] - id: 2, proposed id: 0, 
zxid: 7301097, proposed zxid: 77309411393
176 2010-06-20 23:22:46,441 - DEBUG 
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@717] - Adding vote: From = 2, 
Proposed leader = 2, Porposed zxid = 7301097, Proposed epoch = 1
177 2010-06-20 23:22:46,441 - INFO [QuorumPeer:/0.0.0.0:2181:quorump...@647] - 
LEADING

b) As a result X increments its epoch. Worse, since this node decided to be a 
leader, it starts doing transactions. The first set of transactions start 
removing all ephemeral nodes. But these transactions are only done locally. 
Other peers do not ack these transactions since they know that this peer is not 
the leader.

c) After a few seconds (8 secs), X relinquishes leadership since it does not 
receive any ack from rest of the peers
d) It starts leader election 

[jira] Commented: (ZOOKEEPER-335) zookeeper servers should commit the new leader txn to their logs.

2010-06-22 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-335:


I will try out the patch. FYI I am using 3.3.0.

 zookeeper servers should commit the new leader txn to their logs.
 -

 Key: ZOOKEEPER-335
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-335
 Project: Zookeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.1.0
Reporter: Mahadev konar
Assignee: Mahadev konar
Priority: Blocker
 Fix For: 3.4.0

 Attachments: faultynode-vishal.txt, zk.log.gz, zklogs.tar.gz


 currently the zookeeper followers do not commit the new leader election. This 
 will cause problems in a failure scenarios with a follower acking to the same 
 leader txn id twice, which might be two different intermittent leaders and 
 allowing them to propose two different txn's of the same zxid.

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



[jira] Commented: (ZOOKEEPER-790) Last processed zxid set prematurely while establishing leadership

2010-06-22 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-790:



I will try out the patch. I will try it out on 3.3.0 since that is the version 
we are currently using.

-Vishal

 Last processed zxid set prematurely while establishing leadership
 -

 Key: ZOOKEEPER-790
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-790
 Project: Zookeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.3.1
Reporter: Flavio Paiva Junqueira
Assignee: Flavio Paiva Junqueira
Priority: Blocker
 Fix For: 3.3.2, 3.4.0

 Attachments: ZOOKEEPER-790.patch


 The leader code is setting the last processed zxid to the first of the new 
 epoch even before connecting to a quorum of followers. Because the leader 
 code sets this value before connecting to a quorum of followers 
 (Leader.java:281) and the follower code throws an IOException 
 (Follower.java:73) if the leader epoch is smaller, we have that when the 
 false leader drops leadership and becomes a follower, it finds a smaller 
 epoch and kills itself.

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



[jira] Commented: (ZOOKEEPER-335) zookeeper servers should commit the new leader txn to their logs.

2010-06-21 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-335:


Hi,

I enabled tracing and did some more debugging. Looks like the restarted peer 
(and trying to join the cluster) determines that it is a leader and increments 
its epoch. However, rest of the nodes don't acknowledge this node as the 
leader, and hence, have an older epoch. I will attache the log. Unfortunately, 
I don't have traces from other nodes. I will repeat the experiment later and 
attache logs from other nodes. 

Scenario:
- Form a 3 node cluster. This is not just ZK cluster. It also involves our 
application cluster that uses ZK.
- Kill one of the follower
- After a minute or so restart follower
- Follower rejects leader with Leader epoch y is less than our epoch y + 1

From logs:

a) Peer X restarts and starts leader election.
a) For a small window of time, X thinks that it is the new leader! During this 
window, for some reason, rest of the nodes tell X that they are also trying to 
find a leader. I.e., all 3 nodes are in LOOKING state. After seeing that all 3 
nodes are in LOOKING state, X decides to be a leader?

   155 2010-06-20 23:22:46,421 - DEBUG [WorkerSender 
Thread:quorumcnxmana...@346] - Opening channel to server 1
   156 2010-06-20 23:22:46,423 - DEBUG [WorkerReceiver 
Thread:fastleaderelection$messenger$workerrecei...@214] - Receive new 
notification message. My id = 0
   157 2010-06-20 23:22:46,424 - INFO  
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@689] - Notification: 0, 
77309411393, 1, 0, LOOKING, LOOKING, 0
   158 2010-06-20 23:22:46,424 - DEBUG 
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@495] - id: 0, proposed id: 0, 
zxid: 77309411393, proposed zxid: 77309411393
   159 2010-06-20 23:22:46,424 - DEBUG 
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@717] - Adding vote: From = 0, 
Proposed leader = 0, Porposed zxid = 77309411393, Proposed epoch = 1
   160 2010-06-20 23:22:46,426 - INFO  [WorkerSender 
Thread:quorumcnxmana...@162] - Have smaller server identifier, so dropping the 
connection: (1, 0)
   161 2010-06-20 23:22:46,426 - DEBUG [WorkerSender 
Thread:quorumcnxmana...@346] - Opening channel to server 2
   162 2010-06-20 23:22:46,427 - DEBUG [Thread-1:quorumcnxmanager$liste...@445] 
- Connection request /192.168.1.182:46701
   163 2010-06-20 23:22:46,427 - DEBUG [Thread-1:quorumcnxmanager$liste...@448] 
- Connection request: 0
   164 2010-06-20 23:22:46,428 - DEBUG 
[Thread-1:quorumcnxmanager$sendwor...@504] - Address of remote peer: 1
   165 2010-06-20 23:22:46,428 - INFO  [WorkerSender 
Thread:quorumcnxmana...@162] - Have smaller server identifier, so dropping the 
connection: (2, 0)
   166 2010-06-20 23:22:46,431 - DEBUG [WorkerReceiver 
Thread:fastleaderelection$messenger$workerrecei...@214] - Receive new 
notification message. My id = 0
   167 2010-06-20 23:22:46,432 - INFO  
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@689] - Notification: 1, 
77309411372, 1, 0, LOOKING, LOOKING, 1
   168 2010-06-20 23:22:46,432 - DEBUG 
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@495] - id: 1, proposed id: 0, 
zxid: 77309411372, proposed zxid: 77309411393
   169 2010-06-20 23:22:46,432 - DEBUG 
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@717] - Adding vote: From = 1, 
Proposed leader = 1, Porposed zxid = 77309411372, Proposed epoch = 1
   170 2010-06-20 23:22:46,436 - DEBUG [Thread-1:quorumcnxmanager$liste...@445] 
- Connection request /192.168.1.183:44310
   171 2010-06-20 23:22:46,436 - DEBUG [Thread-1:quorumcnxmanager$liste...@448] 
- Connection request: 0
   172 2010-06-20 23:22:46,436 - DEBUG 
[Thread-1:quorumcnxmanager$sendwor...@504] - Address of remote peer: 2
   173 2010-06-20 23:22:46,440 - DEBUG [WorkerReceiver 
Thread:fastleaderelection$messenger$workerrecei...@214] - Receive new 
notification message. My id = 0
   174 2010-06-20 23:22:46,440 - INFO  
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@689] - Notification: 2, 
7301097, 1, 0, LOOKING, LOOKING, 2
   175 2010-06-20 23:22:46,440 - DEBUG 
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@495] - id: 2, proposed id: 0, 
zxid: 7301097, proposed zxid: 77309411393
   176 2010-06-20 23:22:46,441 - DEBUG 
[QuorumPeer:/0.0.0.0:2181:fastleaderelect...@717] - Adding vote: From = 2, 
Proposed leader = 2, Porposed zxid = 7301097, Proposed epoch = 1
   177 2010-06-20 23:22:46,441 - INFO  
[QuorumPeer:/0.0.0.0:2181:quorump...@647] - LEADING

b) As a result X  increments its epoch. Worse, since this node decided to be a 
leader, it starts doing transactions. The first set of transactions start 
removing all ephemeral nodes. But these transactions are only done locally. 
Other peers do not ack these transactions since they know that this peer is not 
the leader.

c) After a few seconds (8 secs), X relinquishes leadership since it does not 
receive any ack from rest of 

[jira] Updated: (ZOOKEEPER-335) zookeeper servers should commit the new leader txn to their logs.

2010-06-21 Thread Vishal K (JIRA)

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

Vishal K updated ZOOKEEPER-335:
---

Attachment: faultynode-vishal.txt

Apologies for multiple attachments.

 zookeeper servers should commit the new leader txn to their logs.
 -

 Key: ZOOKEEPER-335
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-335
 Project: Zookeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.1.0
Reporter: Mahadev konar
Assignee: Mahadev konar
Priority: Blocker
 Fix For: 3.4.0

 Attachments: faultynode-vishal.txt, zk.log.gz, zklogs.tar.gz


 currently the zookeeper followers do not commit the new leader election. This 
 will cause problems in a failure scenarios with a follower acking to the same 
 leader txn id twice, which might be two different intermittent leaders and 
 allowing them to propose two different txn's of the same zxid.

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



[jira] Updated: (ZOOKEEPER-335) zookeeper servers should commit the new leader txn to their logs.

2010-06-18 Thread Vishal K (JIRA)

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

Vishal K updated ZOOKEEPER-335:
---

Attachment: zklogs.tar.gz

Hi,

I have attached the logs. The log entries are similar to those reported by 
others.

We are testing with 3 nodes. Each node is run in a VM running SLES 11. All 3 
VMs are run on the same host. VMs are sharing the same disk. cpuinfo and 
meminfo for VM is in the attached file.
I have also tried to collect more info with -verbose:gc -Xloggc:/../ -Xprof 
options to java.

gc.lg contains the gc output
rest of the info should be in msgs.log
Default java heap size was used. java version  1.6.0_18 was used.

One point to note - In my case, on the misbehaving node could not joing the 
cluster. Rest of the cluster was stable (except for the flood of log messages 
on the leader because the misbehaving follower kept terminating session).

Hope this helps.
Thanks.

 zookeeper servers should commit the new leader txn to their logs.
 -

 Key: ZOOKEEPER-335
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-335
 Project: Zookeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.1.0
Reporter: Mahadev konar
Assignee: Mahadev konar
Priority: Blocker
 Fix For: 3.4.0

 Attachments: zk.log.gz, zklogs.tar.gz


 currently the zookeeper followers do not commit the new leader election. This 
 will cause problems in a failure scenarios with a follower acking to the same 
 leader txn id twice, which might be two different intermittent leaders and 
 allowing them to propose two different txn's of the same zxid.

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



[jira] Commented: (ZOOKEEPER-335) zookeeper servers should commit the new leader txn to their logs.

2010-06-16 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-335:


Hi,

We are running into this bug very often (almost 60-75% hit rate) while testing 
our newly developed application over ZK.
This is almost a blocker for us. Will the fix be simplified if backward 
compatibility was not an issue?

Thanks.

 zookeeper servers should commit the new leader txn to their logs.
 -

 Key: ZOOKEEPER-335
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-335
 Project: Zookeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.1.0
Reporter: Mahadev konar
Assignee: Mahadev konar
Priority: Blocker
 Fix For: 3.4.0


 currently the zookeeper followers do not commit the new leader election. This 
 will cause problems in a failure scenarios with a follower acking to the same 
 leader txn id twice, which might be two different intermittent leaders and 
 allowing them to propose two different txn's of the same zxid.

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



[jira] Commented: (ZOOKEEPER-107) Allow dynamic changes to server cluster membership

2010-05-03 Thread Vishal K (JIRA)

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

Vishal K commented on ZOOKEEPER-107:


Hi Henry,

We are using ZK for one the projects at VMware. We are very much interested in 
having dynamic membership managment. I went through the dev mailing list above 
. I would like to contribute and develop this feature. It sounds like a fun 
project.

Can you please provide an update regarding how far we are with this and any 
documentation that you may have? I will start off a separte discussion thread 
regarding this on the dev mailing list instead of having it over the jira.

Thanks.

Regards,
-Vishal

 Allow dynamic changes to server cluster membership
 --

 Key: ZOOKEEPER-107
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-107
 Project: Zookeeper
  Issue Type: Improvement
  Components: server
Reporter: Patrick Hunt
Assignee: Henry Robinson
 Attachments: SimpleAddition.rtf


 Currently cluster membership is statically defined, adding/removing hosts 
 to/from the server cluster dynamically needs to be supported.

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



[jira] Updated: (ZOOKEEPER-734) QuorumPeerTestBase.java and ZooKeeperServerMainTest.java do not handle windows path correctly

2010-04-22 Thread Vishal K (JIRA)

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

Vishal K updated ZOOKEEPER-734:
---

Attachment: ZOOKEEPER-734.patch

Request for review.

There is probably a better way to handle backlashes in Windows. I tried a use 
properties (file.separator etc), but didn't work. So finally decided to do it 
the dumb way. Replace \\ with /. If we done replace backslash, then java 
considers it as escape sequence and essentially the backslash gets removed.

Please, feel free to drop the patch If there is a better way to handle it.

 QuorumPeerTestBase.java and ZooKeeperServerMainTest.java do not handle 
 windows path correctly
 -

 Key: ZOOKEEPER-734
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-734
 Project: Zookeeper
  Issue Type: Bug
  Components: tests
Affects Versions: 3.3.0
 Environment: Windows 32-bit
Reporter: Vishal K
 Fix For: 3.3.1, 3.4.0

 Attachments: ZOOKEEPER-734.patch


 While runniing ant test-core-java QuorumPeerTestBase.java and 
 ZooKeeperServerMainTest.java fail. The problem seems to be in 
 ZookeeperserverMainTest.java:MainThread():66 and in 
 QuorumPeerBaseTest.java:MainThread:76.
 FileWriter.write() writes windows path to the conf file. Java does not like 
 windows path. Therefore, the test complains that it cannot find myid and 
 fails. 
 Solution - convert windows path to UNIX path. This worked for me on windows.  
 Diffs are attached below. Solution not tested on Linux since for some reason 
 build is failing (due to problems not related to this change).
 vmc-floorb-dhcp116-114:/opt/zksrc/zookeeper-3.3.0/src/java/test/org/apache/zookeeper/server
  # svn diff
 Index: ZooKeeperServerMainTest.java
 ===
 --- ZooKeeperServerMainTest.java  (revision 931240)
 +++ ZooKeeperServerMainTest.java  (working copy)
 @@ -61,7 +61,8 @@
  if (!dataDir.mkdir()) {
  throw new IOException(unable to mkdir  + dataDir);
  }
 -fwriter.write(dataDir= + dataDir.toString() + \n);
 +String data = dataDir.toString().replace('\\', '/');
 +fwriter.write(dataDir= + data + \n);
  
  fwriter.write(clientPort= + clientPort + \n);
  fwriter.flush();
 Index: quorum/QuorumPeerTestBase.java
 ===
 --- quorum/QuorumPeerTestBase.java(revision 931240)
 +++ quorum/QuorumPeerTestBase.java(working copy)
 @@ -73,7 +73,8 @@
  if (!dataDir.mkdir()) {
  throw new IOException(Unable to mkdir  + dataDir);
  }
 -fwriter.write(dataDir= + dataDir.toString() + \n);
 +String data = dataDir.toString().replace('\\', '/');
 +fwriter.write(dataDir= + data + \n);
  
  fwriter.write(clientPort= + clientPort + \n);
  fwriter.write(quorumCfgSection + \n);

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