[jira] [Commented] (ZOOKEEPER-1549) Data inconsistency when follower is receiving a DIFF with a dirty snapshot

2014-07-01 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1549:


Here is the recap on the issue, for those who just found this JIRA

Problem:
When the leader started, it will treat every txn in its txnlog as committed and 
apply all of them into its in-memory data tree even though some of them was 
only acked by the leader (or the minority). 

If there is a follow that need to synchronize with the leader via snapshot.  
The follower will get a snapshot with uncommitted txns in it and take dirty 
snapshot to disk. If there is a leader failure, it is possible that uncommitted 
txn is discarded in the next leader election round so this follower will have 
dirty snapshot on disk and there is no way it can recovered from this. 

The solution so far:
The fix on the follower side is to simply not taking snapshot until the quorum 
switch to broadcast phase. The follower can have dirty snapshot in memory but 
as long as it doesn’t write to disk, we are ok and part of the issue is 
addressed

On the leader side, the proposed patch is to change server startup and 
synchronization sequence.  Uncommitted txn (any txn after the last snapshot) 
should never get applied to the data tree until synchronization phase is done.  
We use synchronization phase to catchup all follower and imply that all of the 
follower accepted the txn. Then, we apply these txns before starting broadcast 
phase.  
 
I will try to find someone on my team to help on this. 


 Data inconsistency when follower is receiving a DIFF with a dirty snapshot
 --

 Key: ZOOKEEPER-1549
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1549
 Project: ZooKeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.4.3
Reporter: Jacky007
Assignee: Thawan Kooburat
Priority: Blocker
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1549-3.4.patch, ZOOKEEPER-1549-learner.patch, 
 case.patch


 the trunc code (from ZOOKEEPER-1154?) cannot work correct if the snapshot is 
 not correct.
 here is scenario(similar to 1154):
 Initial Condition
 1.Lets say there are three nodes in the ensemble A,B,C with A being the 
 leader
 2.The current epoch is 7. 
 3.For simplicity of the example, lets say zxid is a two digit number, 
 with epoch being the first digit.
 4.The zxid is 73
 5.All the nodes have seen the change 73 and have persistently logged it.
 Step 1
 Request with zxid 74 is issued. The leader A writes it to the log but there 
 is a crash of the entire ensemble and B,C never write the change 74 to their 
 log.
 Step 2
 A,B restart, A is elected as the new leader,  and A will load data and take a 
 clean snapshot(change 74 is in it), then send diff to B, but B died before 
 sync with A. A died later.
 Step 3
 B,C restart, A is still down
 B,C form the quorum
 B is the new leader. Lets say B minCommitLog is 71 and maxCommitLog is 73
 epoch is now 8, zxid is 80
 Request with zxid 81 is successful. On B, minCommitLog is now 71, 
 maxCommitLog is 81
 Step 4
 A starts up. It applies the change in request with zxid 74 to its in-memory 
 data tree
 A contacts B to registerAsFollower and provides 74 as its ZxId
 Since 71=74=81, B decides to send A the diff. 
 Problem:
 The problem with the above sequence is that after truncate the log, A will 
 load the snapshot again which is not correct.
 In 3.3 branch, FileTxnSnapLog.restore does not call listener(ZOOKEEPER-874), 
 the leader will send a snapshot to follower, it will not be a problem.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Assigned] (ZOOKEEPER-1416) Persistent Recursive Watch

2014-04-22 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat reassigned ZOOKEEPER-1416:
--

Assignee: Thawan Kooburat

 Persistent Recursive Watch
 --

 Key: ZOOKEEPER-1416
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1416
 Project: ZooKeeper
  Issue Type: Improvement
  Components: c client, documentation, java client, server
Reporter: Phillip Liu
Assignee: Thawan Kooburat
   Original Estimate: 504h
  Remaining Estimate: 504h

 h4. The Problem
 A ZooKeeper Watch can be placed on a single znode and when the znode changes 
 a Watch event is sent to the client. If there are thousands of znodes being 
 watched, when a client (re)connect, it would have to send thousands of watch 
 requests. At Facebook, we have this problem storing information for thousands 
 of db shards. Consequently a naming service that consumes the db shard 
 definition issues thousands of watch requests each time the service starts 
 and changes client watcher.
 h4. Proposed Solution
 We add the notion of a Persistent Recursive Watch in ZooKeeper. Persistent 
 means no Watch reset is necessary after a watch-fire. Recursive means the 
 Watch applies to the node and descendant nodes. A Persistent Recursive Watch 
 behaves as follows:
 # Recursive Watch supports all Watch semantics: CHILDREN, DATA, and EXISTS.
 # CHILDREN and DATA Recursive Watches can be placed on any znode.
 # EXISTS Recursive Watches can be placed on any path.
 # A Recursive Watch behaves like a auto-watch registrar on the server side. 
 Setting a  Recursive Watch means to set watches on all descendant znodes.
 # When a watch on a descendant fires, no subsequent event is fired until a 
 corresponding getData(..) on the znode is called, then Recursive Watch 
 automically apply the watch on the znode. This maintains the existing Watch 
 semantic on an individual znode.
 # A Recursive Watch overrides any watches placed on a descendant znode. 
 Practically this means the Recursive Watch Watcher callback is the one 
 receiving the event and event is delivered exactly once.
 A goal here is to reduce the number of semantic changes. The guarantee of no 
 intermediate watch event until data is read will be maintained. The only 
 difference is we will automatically re-add the watch after read. At the same 
 time we add the convience of reducing the need to add multiple watches for 
 sibling znodes and in turn reduce the number of watch messages sent from the 
 client to the server.
 There are some implementation details that needs to be hashed out. Initial 
 thinking is to have the Recursive Watch create per-node watches. This will 
 cause a lot of watches to be created on the server side. Currently, each 
 watch is stored as a single bit in a bit set relative to a session - up to 3 
 bits per client per znode. If there are 100m znodes with 100k clients, each 
 watching all nodes, then this strategy will consume approximately 3.75TB of 
 ram distributed across all Observers. Seems expensive.
 Alternatively, a blacklist of paths to not send Watches regardless of Watch 
 setting can be set each time a watch event from a Recursive Watch is fired. 
 The memory utilization is relative to the number of outstanding reads and at 
 worst case it's 1/3 * 3.75TB using the parameters given above.
 Otherwise, a relaxation of no intermediate watch event until read guarantee 
 is required. If the server can send watch events regardless of one has 
 already been fired without corresponding read, then the server can simply 
 fire watch events without tracking.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (ZOOKEEPER-1460) IPv6 literal address not supported for quorum members

2014-03-28 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1460:
---

Assignee: Thawan Kooburat

 IPv6 literal address not supported for quorum members
 -

 Key: ZOOKEEPER-1460
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1460
 Project: ZooKeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.4.3
Reporter: Chris Dolan
Assignee: Thawan Kooburat

 Via code inspection, I see that the server.nnn configuration key does not 
 support literal IPv6 addresses because the property value is split on :. In 
 v3.4.3, the problem is in QuorumPeerConfig:
 {noformat}
 String parts[] = value.split(:);
 InetSocketAddress addr = new InetSocketAddress(parts[0],
 Integer.parseInt(parts[1]));
 {noformat}
 In the current trunk 
 (http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java?view=markup)
  this code has been refactored into QuorumPeer.QuorumServer, but the bug 
 remains:
 {noformat}
 String serverClientParts[] = addressStr.split(;);
 String serverParts[] = serverClientParts[0].split(:);
 addr = new InetSocketAddress(serverParts[0],
 Integer.parseInt(serverParts[1]));
 {noformat}
 This bug probably affects very few users because most will naturally use a 
 hostname rather than a literal IP address. But given that IPv6 addresses are 
 supported for clients via ZOOKEEPER-667 it seems that server support should 
 be fixed too.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (ZOOKEEPER-1887) C implementation of removeWatches

2014-03-04 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1887:


I don't think we need to support both, but we need to support (watcher_fn, 
context) to make it equivalent to Java client. 

 C implementation of removeWatches
 -

 Key: ZOOKEEPER-1887
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1887
 Project: ZooKeeper
  Issue Type: New Feature
  Components: c client
Reporter: Raul Gutierrez Segales
Assignee: Raul Gutierrez Segales
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1887.patch, ZOOKEEPER-1887.patch


 This is equivalent for ZOOKEEPER-442's Java impl. 



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (ZOOKEEPER-1887) C implementation of removeWatches

2014-03-03 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1887:


This feature is a bit more complicated that what I have thought. I will try to 
spend more time understanding Java implementation so I can review this one more 
thoroughly. 

Here is some of comment on my first pass on the patch
- There is startServer() and stopserver() facility in TestClient.cc.  I 
wondering if you can use this to expand the test coverage for the feature 
especially for local=true. Since you can remove the watch even if you are in 
disconnected state
- In c-client a watch is represented by (watcher_fn, context) tuple or 
watcher_object_t internally.  To make the c-client has the equivalent 
functionality with Java client, we have to actually remove a watch based on 
(watcher_fn, context) tuple instead.  What do you think?
 

 C implementation of removeWatches
 -

 Key: ZOOKEEPER-1887
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1887
 Project: ZooKeeper
  Issue Type: New Feature
  Components: c client
Reporter: Raul Gutierrez Segales
Assignee: Raul Gutierrez Segales
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1887.patch, ZOOKEEPER-1887.patch


 This is equivalent for ZOOKEEPER-442's Java impl. 



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (ZOOKEEPER-1879) improve the correctness checking of txn log replay

2014-02-10 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1879:


We can contribute our consistency checker  as a contrib module. It is 
essentially a program that read the entire data tree from 2 servers using 
normal client API and compare it.   It has heuristic to ignore in-flight 
changes so it never report false positive.   We use this program to make 
pair-wise comparison between servers in each production ensemble.   

  

 improve the correctness checking of txn log replay
 --

 Key: ZOOKEEPER-1879
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1879
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.4.6, 3.5.0
Reporter: Patrick Hunt
 Fix For: 3.4.7, 3.5.0


 In ZOOKEEPER-1573 we decided to fix an issue by relaxing some of the 
 checking. Specifically when the sequence of txns is as follows:
 * zxid 1: create /prefix/a
 * zxid 2: create /prefix/a/b
 * zxid 3: delete /prefix/a/b
 * zxid 4: delete /prefix/a
 the log may fail to replay.
 We addressed this by relaxing a check, which is essentially invalid for this 
 case, but is important in finding corruptions of the datastore.
 We should add this check back with proper validation of correctness.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (ZOOKEEPER-1863) Race condition in commit processor leading to out of order request completion, xid mismatch on client.

2014-02-03 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1863:


Yeah, we still need to think about test case (if it is possible to test) 

Also, I will try to allocate sometime to do perf test using my test cases that 
we don't see significant perf regression with this change. 

 Race condition in commit processor leading to out of order request 
 completion, xid mismatch on client.
 --

 Key: ZOOKEEPER-1863
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1863
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.5.0
Reporter: Dutch T. Meyer
Priority: Blocker
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1863.patch, stack.17512


 In CommitProcessor.java processor, if we are at the primary request handler 
 on line 167:
 {noformat}
 while (!stopped  !isWaitingForCommit() 
!isProcessingCommit() 
(request = queuedRequests.poll()) != null) {
 if (needCommit(request)) {
 nextPending.set(request);
 } else {
 sendToNextProcessor(request);
 }
 }
 {noformat}
 A request can be handled in this block and be quickly processed and completed 
 on another thread. If queuedRequests is empty, we then exit the block. Next, 
 before this thread makes any more progress, we can get 2 more requests, one 
 get_children(say), and a sync placed on queuedRequests for the processor. 
 Then, if we are very unlucky, the sync request can complete and this object's 
 commit() routine is called (from FollowerZookeeperServer), which places the 
 sync request on the previously empty committedRequests queue. At that point, 
 this thread continues.
 We reach line 182, which is a check on sync requests.
 {noformat}
 if (!stopped  !isProcessingRequest() 
 (request = committedRequests.poll()) != null) {
 {noformat}
 Here we are not processing any requests, because the original request has 
 completed. We haven't dequeued either the read or the sync request in this 
 processor. Next, the poll above will pull the sync request off the queue, and 
 in the following block, the sync will get forwarded to the next processor.
 This is a problem because the read request hasn't been forwarded yet, so 
 requests are now out of order.
 I've been able to reproduce this bug reliably by injecting a 
 Thread.sleep(5000) between the two blocks above to make the race condition 
 far more likely, then in a client program.
 {noformat}
 zoo_aget_children(zh, /, 0, getchildren_cb, NULL);
 //Wait long enough for queuedRequests to drain
 sleep(1);
 zoo_aget_children(zh, /, 0, getchildren_cb, th_ctx[0]);
 zoo_async(zh, /, sync_cb, th_ctx[0]);
 {noformat}
 When this bug is triggered, 3 things can happen:
 1) Clients will see requests complete out of order and fail on xid mismatches.
 2) Kazoo in particular doesn't handle this runtime exception well, and can 
 orphan outstanding requests.
 3) I've seen zookeeper servers deadlock, likely because the commit cannot be 
 completed, which can wedge the commit processor.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (ZOOKEEPER-1875) NullPointerException in ClientCnxn$EventThread.processEvent

2014-02-03 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1875:


Patch need to be in svn format

 NullPointerException in ClientCnxn$EventThread.processEvent
 ---

 Key: ZOOKEEPER-1875
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1875
 Project: ZooKeeper
  Issue Type: Bug
  Components: java client
Affects Versions: 3.4.5
Reporter: Jerry He
Priority: Minor
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1875-trunk.patch


 We've been seeing NullPointerException while working on HBase:
 {code}
 14/01/30 22:15:25 INFO zookeeper.ZooKeeper: Client 
 environment:user.dir=/home/biadmin/hbase-trunk
 14/01/30 22:15:25 INFO zookeeper.ZooKeeper: Initiating client connection, 
 connectString=hdtest009:2181 sessionTimeout=9 watcher=null
 14/01/30 22:15:25 INFO zookeeper.ClientCnxn: Opening socket connection to 
 server hdtest009/9.30.194.18:2181. Will not attempt to authenticate using 
 SASL (Unable to locate a login configuration)
 14/01/30 22:15:25 INFO zookeeper.ClientCnxn: Socket connection established to 
 hdtest009/9.30.194.18:2181, initiating session
 14/01/30 22:15:25 INFO zookeeper.ClientCnxn: Session establishment complete 
 on server hdtest009/9.30.194.18:2181, sessionid = 0x143986213e67e48, 
 negotiated timeout = 6
 14/01/30 22:15:25 ERROR zookeeper.ClientCnxn: Error while calling watcher
 java.lang.NullPointerException
 at 
 org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:519)
 at 
 org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:495)
 {code}
 The reason is the watcher is null in this part of the code:
 {code}
private void processEvent(Object event) {
   try {
   if (event instanceof WatcherSetEventPair) {
   // each watcher will process the event
   WatcherSetEventPair pair = (WatcherSetEventPair) event;
   for (Watcher watcher : pair.watchers) {
   try {
   watcher.process(pair.event);
   } catch (Throwable t) {
   LOG.error(Error while calling watcher , t);
   }
   }
 {code}



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (ZOOKEEPER-1863) Race condition in commit processor leading to out of order request completion, xid mismatch on client.

2014-01-30 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1863:


This is how that how the patch may looks like.  Some of the if statement can be 
merge but this is easier to understand for now
{code}
/*
 * Processing committedRequests: check and see if the commit
 * came in for the pending request. We can only commit a
 * request when there is no other request being processed.
 */
if (!stopped  !isProcessingRequest() 
(request = committedRequests.peak()) != null) {

// Abort the loop if there a new request waiting in 
queuedRequests
if ( ! isWaitingForCommit() and ! queuedRequests.isEmpty()) 
{ 
   continue;
}   

request = committedRequests.poll();

/*
 * We match with nextPending so that we can move to the
 * next request when it is committed. We also want to
 * use nextPending because it has the cnxn member set
 * properly.
 */
 .
{code}

Regarding your concern about this block.  There is not change to it, we still 
execute this one.  However, it will only get executed when queuedRequests is 
empty.  
{code}
// this request came from someone else so just  
 
// send the commit packet   
 
currentlyCommitting.set(request);
sendToNextProcessor(request);
{code}
This means that there is a potential starvation for committedRequests queue 
instead. Since we give a higher priority for queuedRequests  but the existing 
code already do that. My understanding is that ZK is supposed to be 
notification driven system. If there is no update (committedRequest) going 
through client would stop issuing read request and queuedRequests will be empty 
eventually and allow committed requests to go through. 


 Race condition in commit processor leading to out of order request 
 completion, xid mismatch on client.
 --

 Key: ZOOKEEPER-1863
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1863
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.5.0
Reporter: Dutch T. Meyer
Priority: Blocker
 Attachments: ZOOKEEPER-1863.patch, stack.17512


 In CommitProcessor.java processor, if we are at the primary request handler 
 on line 167:
 {noformat}
 while (!stopped  !isWaitingForCommit() 
!isProcessingCommit() 
(request = queuedRequests.poll()) != null) {
 if (needCommit(request)) {
 nextPending.set(request);
 } else {
 sendToNextProcessor(request);
 }
 }
 {noformat}
 A request can be handled in this block and be quickly processed and completed 
 on another thread. If queuedRequests is empty, we then exit the block. Next, 
 before this thread makes any more progress, we can get 2 more requests, one 
 get_children(say), and a sync placed on queuedRequests for the processor. 
 Then, if we are very unlucky, the sync request can complete and this object's 
 commit() routine is called (from FollowerZookeeperServer), which places the 
 sync request on the previously empty committedRequests queue. At that point, 
 this thread continues.
 We reach line 182, which is a check on sync requests.
 {noformat}
 if (!stopped  !isProcessingRequest() 
 (request = committedRequests.poll()) != null) {
 {noformat}
 Here we are not processing any requests, because the original request has 
 completed. We haven't dequeued either the read or the sync request in this 
 processor. Next, the poll above will pull the sync request off the queue, and 
 in the following block, the sync will get forwarded to the next processor.
 This is a problem because the read request hasn't been forwarded yet, so 
 requests are now out of order.
 I've been able to reproduce this bug reliably by injecting a 
 Thread.sleep(5000) between the two blocks above to make the race condition 
 far more likely, then in a client program.
 {noformat}
 zoo_aget_children(zh, /, 0, getchildren_cb, NULL);
 //Wait long enough for queuedRequests 

[jira] [Commented] (ZOOKEEPER-1863) Race condition in commit processor leading to out of order request completion, xid mismatch on client.

2014-01-29 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1863:


Thanks for proposing a fix.  Here is my understanding of your proposed 
solution. (Assuming that the race is based on Camille's repro) 

1. It is only safe to pop {{committedRequests}} and pass it down to next 
process when there is an outstanding write request (nextPending != null). Since 
we get to compare it with the outstanding write request. 

2. When nextPending == null,   a request can be in {{commitedRequests}} only 
when it is already added to {{queuedRequests}}. If there is a request in 
{{commitedRequests}} it just need to check {{queuedRequests}} again to check 
for potential matching request.

I believe traversing over {{queuedRequests}} is expensive and not needed. So I 
am thinking of doing this instead of that loop
{code}
if ( ! isWaitingForCommit() and ! queuedRequests.isEmpty()) { 
  continue;
} 
{code}

The idea is that we abort the current attempt of trying to process the current 
committed request. Then repeat the loop of processing {{queuedRequests}}. 
Eventually, if there is a matching request (or other write request) it will be 
assigned to {{nextPending}} which allow us to process {{committedRequests}} 
correctly




 Race condition in commit processor leading to out of order request 
 completion, xid mismatch on client.
 --

 Key: ZOOKEEPER-1863
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1863
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.5.0
Reporter: Dutch T. Meyer
Priority: Blocker
 Attachments: ZOOKEEPER-1863.patch, stack.17512


 In CommitProcessor.java processor, if we are at the primary request handler 
 on line 167:
 {noformat}
 while (!stopped  !isWaitingForCommit() 
!isProcessingCommit() 
(request = queuedRequests.poll()) != null) {
 if (needCommit(request)) {
 nextPending.set(request);
 } else {
 sendToNextProcessor(request);
 }
 }
 {noformat}
 A request can be handled in this block and be quickly processed and completed 
 on another thread. If queuedRequests is empty, we then exit the block. Next, 
 before this thread makes any more progress, we can get 2 more requests, one 
 get_children(say), and a sync placed on queuedRequests for the processor. 
 Then, if we are very unlucky, the sync request can complete and this object's 
 commit() routine is called (from FollowerZookeeperServer), which places the 
 sync request on the previously empty committedRequests queue. At that point, 
 this thread continues.
 We reach line 182, which is a check on sync requests.
 {noformat}
 if (!stopped  !isProcessingRequest() 
 (request = committedRequests.poll()) != null) {
 {noformat}
 Here we are not processing any requests, because the original request has 
 completed. We haven't dequeued either the read or the sync request in this 
 processor. Next, the poll above will pull the sync request off the queue, and 
 in the following block, the sync will get forwarded to the next processor.
 This is a problem because the read request hasn't been forwarded yet, so 
 requests are now out of order.
 I've been able to reproduce this bug reliably by injecting a 
 Thread.sleep(5000) between the two blocks above to make the race condition 
 far more likely, then in a client program.
 {noformat}
 zoo_aget_children(zh, /, 0, getchildren_cb, NULL);
 //Wait long enough for queuedRequests to drain
 sleep(1);
 zoo_aget_children(zh, /, 0, getchildren_cb, th_ctx[0]);
 zoo_async(zh, /, sync_cb, th_ctx[0]);
 {noformat}
 When this bug is triggered, 3 things can happen:
 1) Clients will see requests complete out of order and fail on xid mismatches.
 2) Kazoo in particular doesn't handle this runtime exception well, and can 
 orphan outstanding requests.
 3) I've seen zookeeper servers deadlock, likely because the commit cannot be 
 completed, which can wedge the commit processor.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (ZOOKEEPER-1863) Race condition in commit processor leading to out of order request completion, xid mismatch on client.

2014-01-28 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1863:


Commit processor is critical to the perf of ZK so additional synchronization 
must be done very carefully. I agree that correctness is definitely more 
important. The commit processor main's loop is the bottleneck since it is 
executed by a single thread. So it would be best to avoid expensive locking 
there. 

I will see log from my prod if the background thread that abort the server had 
done its job or not.

 Race condition in commit processor leading to out of order request 
 completion, xid mismatch on client.
 --

 Key: ZOOKEEPER-1863
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1863
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.5.0
Reporter: Dutch T. Meyer
Priority: Blocker
 Attachments: stack.17512


 In CommitProcessor.java processor, if we are at the primary request handler 
 on line 167:
 {noformat}
 while (!stopped  !isWaitingForCommit() 
!isProcessingCommit() 
(request = queuedRequests.poll()) != null) {
 if (needCommit(request)) {
 nextPending.set(request);
 } else {
 sendToNextProcessor(request);
 }
 }
 {noformat}
 A request can be handled in this block and be quickly processed and completed 
 on another thread. If queuedRequests is empty, we then exit the block. Next, 
 before this thread makes any more progress, we can get 2 more requests, one 
 get_children(say), and a sync placed on queuedRequests for the processor. 
 Then, if we are very unlucky, the sync request can complete and this object's 
 commit() routine is called (from FollowerZookeeperServer), which places the 
 sync request on the previously empty committedRequests queue. At that point, 
 this thread continues.
 We reach line 182, which is a check on sync requests.
 {noformat}
 if (!stopped  !isProcessingRequest() 
 (request = committedRequests.poll()) != null) {
 {noformat}
 Here we are not processing any requests, because the original request has 
 completed. We haven't dequeued either the read or the sync request in this 
 processor. Next, the poll above will pull the sync request off the queue, and 
 in the following block, the sync will get forwarded to the next processor.
 This is a problem because the read request hasn't been forwarded yet, so 
 requests are now out of order.
 I've been able to reproduce this bug reliably by injecting a 
 Thread.sleep(5000) between the two blocks above to make the race condition 
 far more likely, then in a client program.
 {noformat}
 zoo_aget_children(zh, /, 0, getchildren_cb, NULL);
 //Wait long enough for queuedRequests to drain
 sleep(1);
 zoo_aget_children(zh, /, 0, getchildren_cb, th_ctx[0]);
 zoo_async(zh, /, sync_cb, th_ctx[0]);
 {noformat}
 When this bug is triggered, 3 things can happen:
 1) Clients will see requests complete out of order and fail on xid mismatches.
 2) Kazoo in particular doesn't handle this runtime exception well, and can 
 orphan outstanding requests.
 3) I've seen zookeeper servers deadlock, likely because the commit cannot be 
 completed, which can wedge the commit processor.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (ZOOKEEPER-1863) Race condition in commit processor leading to out of order request completion, xid mismatch on client.

2014-01-24 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1863:


I have seen a Commit Processor getting stuck in our prod (which run our 
internal branch) I spent a few days digging into the problem but couldn't 
locate the root cause.  

The sequence of action that you put in description is very unlikely to occur in 
quorum mode.  First, the Follower/ObserverReuestProcessor which is in the front 
of the CommitProcessor put a request into queuedRequests even before sending it 
out to the leader.   It need at least a network round trip ( or a full quorum 
vote) before the same request will comeback from a leader and get put into 
commitRequest.  This is the assumption that even the original CommitProcessor 
(prior to ZOOKEEPER-1505) rely on. However, a combination of bad thread 
scheduling and long GC pause might break this assumption.

Sync request is special unlike other write request because it doesn't require 
quorum voting, but I still don't think it matter in this case. 

Again, since I saw this in prod but I am unable to repro it. I did add a 
background thread to detect a request stuck in nextPending for extended period 
of time and kill the server if it is the case.  I can post the patch if we are 
able unable find the root cause. 

You can also capture a heap dump of server to inspect which request get stuck 
(at nextPending) and correlated the possible event.  




 Race condition in commit processor leading to out of order request 
 completion, xid mismatch on client.
 --

 Key: ZOOKEEPER-1863
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1863
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.5.0
Reporter: Dutch T. Meyer
Priority: Blocker
 Attachments: stack.17512


 In CommitProcessor.java processor, if we are at the primary request handler 
 on line 167:
 {noformat}
 while (!stopped  !isWaitingForCommit() 
!isProcessingCommit() 
(request = queuedRequests.poll()) != null) {
 if (needCommit(request)) {
 nextPending.set(request);
 } else {
 sendToNextProcessor(request);
 }
 }
 {noformat}
 A request can be handled in this block and be quickly processed and completed 
 on another thread. If queuedRequests is empty, we then exit the block. Next, 
 before this thread makes any more progress, we can get 2 more requests, one 
 get_children(say), and a sync placed on queuedRequests for the processor. 
 Then, if we are very unlucky, the sync request can complete and this object's 
 commit() routine is called (from FollowerZookeeperServer), which places the 
 sync request on the previously empty committedRequests queue. At that point, 
 this thread continues.
 We reach line 182, which is a check on sync requests.
 {noformat}
 if (!stopped  !isProcessingRequest() 
 (request = committedRequests.poll()) != null) {
 {noformat}
 Here we are not processing any requests, because the original request has 
 completed. We haven't dequeued either the read or the sync request in this 
 processor. Next, the poll above will pull the sync request off the queue, and 
 in the following block, the sync will get forwarded to the next processor.
 This is a problem because the read request hasn't been forwarded yet, so 
 requests are now out of order.
 I've been able to reproduce this bug reliably by injecting a 
 Thread.sleep(5000) between the two blocks above to make the race condition 
 far more likely, then in a client program.
 {noformat}
 zoo_aget_children(zh, /, 0, getchildren_cb, NULL);
 //Wait long enough for queuedRequests to drain
 sleep(1);
 zoo_aget_children(zh, /, 0, getchildren_cb, th_ctx[0]);
 zoo_async(zh, /, sync_cb, th_ctx[0]);
 {noformat}
 When this bug is triggered, 3 things can happen:
 1) Clients will see requests complete out of order and fail on xid mismatches.
 2) Kazoo in particular doesn't handle this runtime exception well, and can 
 orphan outstanding requests.
 3) I've seen zookeeper servers deadlock, likely because the commit cannot be 
 completed, which can wedge the commit processor.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (ZOOKEEPER-1504) Multi-thread NIOServerCnxn

2014-01-24 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1504:


I don't have access to solaris platform.   It is possible to get access to 
build machine so I have dev/test environment?

 Multi-thread NIOServerCnxn
 --

 Key: ZOOKEEPER-1504
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1504
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.4.3, 3.4.4, 3.5.0
Reporter: Jay Shrauner
Assignee: Jay Shrauner
  Labels: performance, scaling
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1504.patch, ZOOKEEPER-1504.patch, 
 ZOOKEEPER-1504.patch, ZOOKEEPER-1504.patch, ZOOKEEPER-1504.patch, 
 ZOOKEEPER-1504.patch


 NIOServerCnxnFactory is single threaded, which doesn't scale well to large 
 numbers of clients. This is particularly noticeable when thousands of clients 
 connect. I propose multi-threading this code as follows:
 - 1   acceptor thread, for accepting new connections
 - 1-N selector threads
 - 0-M I/O worker threads
 Numbers of threads are configurable, with defaults scaling according to 
 number of cores. Communication with the selector threads is handled via 
 LinkedBlockingQueues, and connections are permanently assigned to a 
 particular selector thread so that all potentially blocking SelectionKey 
 operations can be performed solely by the selector thread. An ExecutorService 
 is used for the worker threads.
 On a 32 core machine running Linux 2.6.38, achieved best performance with 4 
 selector threads and 64 worker threads for a 70% +/- 5% improvement in 
 throughput.
 This patch incorporates and supersedes the patches for
 https://issues.apache.org/jira/browse/ZOOKEEPER-517
 https://issues.apache.org/jira/browse/ZOOKEEPER-1444
 New classes introduced in this patch are:
   - ExpiryQueue (from ZOOKEEPER-1444): factor out the logic from 
 SessionTrackerImpl used to expire sessions so that the same logic can be used 
 to expire connections
   - RateLogger (from ZOOKEEPER-517): rate limit error message logging, 
 currently only used to throttle rate of logging out of file descriptors 
 errors
   - WorkerService (also in ZOOKEEPER-1505): ExecutorService wrapper that 
 makes worker threads daemon threads and names then in an easily debuggable 
 manner. Supports assignable threads (as used by CommitProcessor) and 
 non-assignable threads (as used here).



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (ZOOKEEPER-1851) Follower and Observer Request Processors Do Not Forward create2 Requests

2014-01-24 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1851:


Thanks for the fix, this is my bad. I will take a look and commit it.  It is 
quite strange that none of the existing unit test catch this

 Follower and Observer Request Processors Do Not Forward create2 Requests
 

 Key: ZOOKEEPER-1851
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1851
 Project: ZooKeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.5.0
Reporter: Chris Chen
  Labels: patch
 Attachments: ZOOKEEPER-1851.patch, ZOOKEEPER-1851.patch


 Recent changes to the Observer and Follower Request Processors switch on the 
 request opcode, but create2 is left out. This leads to a condition where the 
 create2 request is passed to the CommitProcessor, but the leader never gets 
 the request, the CommitProcessor can't find a matching request, so the client 
 gets disconnected.
 Added tests as well.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Assigned] (ZOOKEEPER-1851) Follower and Observer Request Processors Do Not Forward create2 Requests

2014-01-24 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat reassigned ZOOKEEPER-1851:
--

Assignee: Thawan Kooburat

 Follower and Observer Request Processors Do Not Forward create2 Requests
 

 Key: ZOOKEEPER-1851
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1851
 Project: ZooKeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.5.0
Reporter: Chris Chen
Assignee: Thawan Kooburat
  Labels: patch
 Attachments: ZOOKEEPER-1851.patch, ZOOKEEPER-1851.patch


 Recent changes to the Observer and Follower Request Processors switch on the 
 request opcode, but create2 is left out. This leads to a condition where the 
 create2 request is passed to the CommitProcessor, but the leader never gets 
 the request, the CommitProcessor can't find a matching request, so the client 
 gets disconnected.
 Added tests as well.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Assigned] (ZOOKEEPER-1504) Multi-thread NIOServerCnxn

2014-01-24 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat reassigned ZOOKEEPER-1504:
--

Assignee: Thawan Kooburat  (was: Jay Shrauner)

 Multi-thread NIOServerCnxn
 --

 Key: ZOOKEEPER-1504
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1504
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.4.3, 3.4.4, 3.5.0
Reporter: Jay Shrauner
Assignee: Thawan Kooburat
  Labels: performance, scaling
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1504.patch, ZOOKEEPER-1504.patch, 
 ZOOKEEPER-1504.patch, ZOOKEEPER-1504.patch, ZOOKEEPER-1504.patch, 
 ZOOKEEPER-1504.patch


 NIOServerCnxnFactory is single threaded, which doesn't scale well to large 
 numbers of clients. This is particularly noticeable when thousands of clients 
 connect. I propose multi-threading this code as follows:
 - 1   acceptor thread, for accepting new connections
 - 1-N selector threads
 - 0-M I/O worker threads
 Numbers of threads are configurable, with defaults scaling according to 
 number of cores. Communication with the selector threads is handled via 
 LinkedBlockingQueues, and connections are permanently assigned to a 
 particular selector thread so that all potentially blocking SelectionKey 
 operations can be performed solely by the selector thread. An ExecutorService 
 is used for the worker threads.
 On a 32 core machine running Linux 2.6.38, achieved best performance with 4 
 selector threads and 64 worker threads for a 70% +/- 5% improvement in 
 throughput.
 This patch incorporates and supersedes the patches for
 https://issues.apache.org/jira/browse/ZOOKEEPER-517
 https://issues.apache.org/jira/browse/ZOOKEEPER-1444
 New classes introduced in this patch are:
   - ExpiryQueue (from ZOOKEEPER-1444): factor out the logic from 
 SessionTrackerImpl used to expire sessions so that the same logic can be used 
 to expire connections
   - RateLogger (from ZOOKEEPER-517): rate limit error message logging, 
 currently only used to throttle rate of logging out of file descriptors 
 errors
   - WorkerService (also in ZOOKEEPER-1505): ExecutorService wrapper that 
 makes worker threads daemon threads and names then in an easily debuggable 
 manner. Supports assignable threads (as used by CommitProcessor) and 
 non-assignable threads (as used here).



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (ZOOKEEPER-1847) Normalize line endings in repository

2014-01-24 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1847:


I would love to fix this as well. However, it is going to get reintroduce as 
new patch being accepted. 

It would be better to invest effort in enforcing this on a new patch which I 
don't know how to do that with the current workflow

 Normalize line endings in repository
 

 Key: ZOOKEEPER-1847
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1847
 Project: ZooKeeper
  Issue Type: Bug
Reporter: Enis Soztutar
Assignee: Enis Soztutar
 Fix For: 3.5.0

 Attachments: zookeeper-1847_v1.patch


 It is good practice to have all the code in the repository use the same line 
 endings (LF) so that patches can be applied normally. We can add a 
 gitattributes file so that checked out code can still have platform dependent 
 line endings. 
 More readings: 
 https://help.github.com/articles/dealing-with-line-endings
 http://stackoverflow.com/questions/170961/whats-the-best-crlf-handling-strategy-with-git



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Created] (ZOOKEEPER-1865) Fix retry logic in Learner.connectToLeader()

2014-01-21 Thread Thawan Kooburat (JIRA)
Thawan Kooburat created ZOOKEEPER-1865:
--

 Summary: Fix retry logic in Learner.connectToLeader() 
 Key: ZOOKEEPER-1865
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1865
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.4.3, 3.5.0
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat


We discovered a long leader election time today in one of our prod ensemble.

Here is the description of the event. 

Before the old leader goes down, it is able to announce notification message. 
So 3 out 5 (including the old leader) elected the old leader to be a new leader 
for the next epoch. While, the old leader is being rebooted, 2 other machines 
are trying to connect to the old leader.  So the quorum couldn't form until 
those 2 machines give up and move to the next round of leader election.

This is because Learner.connectToLeader() use a simple retry logic. The 
contract for this method is that it should never spend longer that initLimit 
trying to connect to the leader.  In our outage, each sock.connect() is 
probably blocked for initLimit and it is called 5 times.




--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (ZOOKEEPER-1573) Unable to load database due to missing parent node

2013-11-15 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1573:


Probably need a comment from other people as well.  We disable this check in 
our prod system because we have some other way of detecting data inconsistency. 
 This check has shown to catch a real bug but it can also raise false possible 
in certain usage pattern.

 Unable to load database due to missing parent node
 --

 Key: ZOOKEEPER-1573
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1573
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.4.3, 3.5.0
Reporter: Thawan Kooburat
 Attachments: ZOOKEEPER-1573.patch


 While replaying txnlog on data tree, the server has a code to detect missing 
 parent node. This code block was last modified as part of ZOOKEEPER-1333. In 
 our production, we found a case where this check is return false positive.
 The sequence of txns is as follows:
 zxid 1:  create /prefix/a
 zxid 2:  create /prefix/a/b
 zxid 3:  delete /prefix/a/b
 zxid 4:  delete /prefix/a
 The server start capturing snapshot at zxid 1. However, by the time it 
 traversing the data tree down to /prefix, txn 4 is already applied and 
 /prefix have no children. 
 When the server restore from snapshot, it process txnlog starting from zxid 
 2. This txn generate missing parent error and the server refuse to start up.
 The same check allow me to discover bug in ZOOKEEPER-1551, but I don't know 
 if we have any option beside removing this check to solve this issue.  



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1798) Fix race condition in testNormalObserverRun

2013-11-13 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1798:


Committed
trunk: 1541810
3.4: 1541814

 Fix race condition in testNormalObserverRun
 ---

 Key: ZOOKEEPER-1798
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1798
 Project: ZooKeeper
  Issue Type: Bug
Reporter: Flavio Junqueira
Assignee: Thawan Kooburat
Priority: Blocker
 Fix For: 3.4.6, 3.5.0

 Attachments: TEST-org.apache.zookeeper.server.quorum.Zab1_0Test.txt, 
 ZOOKEEPER-1798-2.patch, ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798-b3.4.patch, 
 ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798.patch, ZOOKEEPER-1798.patch


 This is the output messges:
 noformat
 Testcase: testNormalObserverRun took 4.221 sec
 FAILED
 expected:data[2] but was:data[1]
 junit.framework.AssertionFailedError: expected:data[2] but was:data[1]
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test$8.converseWithObserver(Zab1_0Test.java:1118)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testObserverConversation(Zab1_0Test.java:546)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testNormalObserverRun(Zab1_0Test.java:994)
 noformat



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (ZOOKEEPER-1798) Fix race condition in testNormalObserverRun

2013-11-07 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1798:
---

Attachment: ZOOKEEPER-1798-2.patch

Attach a patch that also shutdown sync request processor as part of observer 
shutdown

 Fix race condition in testNormalObserverRun
 ---

 Key: ZOOKEEPER-1798
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1798
 Project: ZooKeeper
  Issue Type: Bug
Reporter: Flavio Junqueira
Assignee: Thawan Kooburat
Priority: Blocker
 Fix For: 3.4.6, 3.5.0

 Attachments: TEST-org.apache.zookeeper.server.quorum.Zab1_0Test.txt, 
 ZOOKEEPER-1798-2.patch, ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798-b3.4.patch, 
 ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798.patch, ZOOKEEPER-1798.patch


 This is the output messges:
 noformat
 Testcase: testNormalObserverRun took 4.221 sec
 FAILED
 expected:data[2] but was:data[1]
 junit.framework.AssertionFailedError: expected:data[2] but was:data[1]
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test$8.converseWithObserver(Zab1_0Test.java:1118)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testObserverConversation(Zab1_0Test.java:546)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testNormalObserverRun(Zab1_0Test.java:994)
 noformat



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1807) Observers spam each other creating connections to the election addr

2013-11-04 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1807:


I believe we have a much different concern using large number of observers. In 
our internal deployment, we did a few hacks which essentially kill all 
observer-to-observer communication. Observers only observe the result of 
election algorithm. We also add random delay when observer try to reconnect, so 
that participants has a chance to synchronize with the leader and form the 
quorum before the observers take away the leader's bandwidth. 

My understanding is that with our leader election algorithm, you need to 
broadcast your vote whenever your current vote change, so this will generate a 
lot of message during the initial phase of the algorithm. Also, N x N 
communication needed by LE is not going to scale for large deployment.  For me, 
I don't think promoting observer to participant is going to be a common case 
(only needed for DR purpose), it would be acceptable to have optional flag to 
disable that feature in order to reduce LE overhead with large number of 
observers.

 Observers spam each other creating connections to the election addr
 ---

 Key: ZOOKEEPER-1807
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1807
 Project: ZooKeeper
  Issue Type: Bug
Reporter: Raul Gutierrez Segales
Assignee: Germán Blanco
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1807.patch, notifications-loop.png


 Hey [~shralex],
 I noticed today that my Observers are spamming each other trying to open 
 connections to the election port. I've got tons of these:
 {noformat}
 2013-11-01 22:19:45,819 - DEBUG [WorkerSender[myid=13]] - There is a 
 connection already for server 9
 2013-11-01 22:19:45,819 - DEBUG [WorkerSender[myid=13]] - There is a 
 connection already for server 10
 2013-11-01 22:19:45,819 - DEBUG [WorkerSender[myid=13]] - There is a 
 connection already for server 6
 2013-11-01 22:19:45,819 - DEBUG [WorkerSender[myid=13]] - There is a 
 connection already for server 12
 2013-11-01 22:19:45,819 - DEBUG [WorkerSender[myid=13]] - There is a 
 connection already for server 14
 {noformat}
 and so and so on ad nauseam. 
 Now, looking around I found this inside FastLeaderElection.java from when you 
 committed ZOOKEEPER-107:
 {noformat}
  private void sendNotifications() {
 -for (QuorumServer server : self.getVotingView().values()) {
 -long sid = server.id;
 -
 +for (long sid : self.getAllKnownServerIds()) {
 +QuorumVerifier qv = self.getQuorumVerifier();
 {noformat}
 Is that really desired? I suspect that is what's causing Observers to try to 
 connect to each other (as opposed as just connecting to participants). I'll 
 give it a try now and let you know. (Also, we use observer ids that are  0, 
 and I saw some parts of the code that might not deal with that assumption - 
 so it could be that too..). 



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1798) Fix race condition in testNormalObserverRun

2013-11-01 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1798:


Just for the record, this test is not known to be flaky in our internal Jenkins 
(that test our internal branch).  

I am able to repro this on my mac.  (Java 1.7.0_15, OSX 10.7.5).  When this 
happen, it looks txnlog doesn't have any valid content in it.  So the zkdb that 
we loaded after shutting down the observer never have txn that its znodes to 
data2.   I also modified the test to leave the data files around and try to 
load it manually after the test fail. The txnlog is loaded successfully with 
the right content. 

I am thinking that the data flushed to disk by one thread is not visible by the 
other thread even after thread.join() is called in between. However, this 
really seem unlikely. But I ran the same test in our production host, I cannot 
repro the issue (yet)

In Patrick log, this is slightly different. The test failed at line 1105, this 
means that the first txn in txnlog is read correctly, but not the second one. 

 Fix race condition in testNormalObserverRun
 ---

 Key: ZOOKEEPER-1798
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1798
 Project: ZooKeeper
  Issue Type: Bug
Reporter: Flavio Junqueira
Assignee: Thawan Kooburat
Priority: Blocker
 Fix For: 3.4.6, 3.5.0

 Attachments: TEST-org.apache.zookeeper.server.quorum.Zab1_0Test.txt, 
 ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798-b3.4.patch, 
 ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798.patch, ZOOKEEPER-1798.patch


 This is the output messges:
 noformat
 Testcase: testNormalObserverRun took 4.221 sec
 FAILED
 expected:data[2] but was:data[1]
 junit.framework.AssertionFailedError: expected:data[2] but was:data[1]
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test$8.converseWithObserver(Zab1_0Test.java:1118)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testObserverConversation(Zab1_0Test.java:546)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testNormalObserverRun(Zab1_0Test.java:994)
 noformat



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1790) Deal with special ObserverId in QuorumCnxManager.receiveConnection

2013-11-01 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1790:


In our internal deployment, we use negative sid to for observers  (actually all 
of them is -1) . This is probably not the intended usage but it worked so far.  
It would be nice to add to release note of 3.5 if there is a change in the 
valid sid range. 

 Deal with special ObserverId in QuorumCnxManager.receiveConnection
 --

 Key: ZOOKEEPER-1790
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1790
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.4.6, 3.5.0
Reporter: Alexander Shraer
Assignee: Alexander Shraer
 Fix For: 3.4.6, 3.5.0


 QuorumCnxManager.receiveConnection assumes that a negative sid means that 
 this is a 3.5.0 server, which has a different communication protocol. This 
 doesn't account for the fact that ObserverId = -1 is a special id that may be 
 used by observers and is also negative. 
 This requires a fix to trunk and a separate fix to 3.4 branch, where this 
 function is different (see ZOOKEEPER-1633)



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1807) Observers spam each other creating connections to the election addr

2013-11-01 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1807:


In our internal deployment, the host list in zoo.cfg for each observer only 
have the participants and itself.  This helps address this issue a bit but 
obviously, in 3.5 world, this won't work if you want to promote an observer to 
a participant. 

 Observers spam each other creating connections to the election addr
 ---

 Key: ZOOKEEPER-1807
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1807
 Project: ZooKeeper
  Issue Type: Bug
Reporter: Raul Gutierrez Segales
Assignee: Raul Gutierrez Segales
 Fix For: 3.5.0


 Hey [~shralex],
 I noticed today that my Observers are spamming each other trying to open 
 connections to the election port. I've got tons of these:
 {noformat}
 2013-11-01 22:19:45,819 - DEBUG [WorkerSender[myid=13]] - There is a 
 connection already for server 9
 2013-11-01 22:19:45,819 - DEBUG [WorkerSender[myid=13]] - There is a 
 connection already for server 10
 2013-11-01 22:19:45,819 - DEBUG [WorkerSender[myid=13]] - There is a 
 connection already for server 6
 2013-11-01 22:19:45,819 - DEBUG [WorkerSender[myid=13]] - There is a 
 connection already for server 12
 2013-11-01 22:19:45,819 - DEBUG [WorkerSender[myid=13]] - There is a 
 connection already for server 14
 {noformat}
 and so and so on ad nauseam. 
 Now, looking around I found this inside FastLeaderElection.java from when you 
 committed ZOOKEEPER-107:
 {noformat}
  private void sendNotifications() {
 -for (QuorumServer server : self.getVotingView().values()) {
 -long sid = server.id;
 -
 +for (long sid : self.getAllKnownServerIds()) {
 +QuorumVerifier qv = self.getQuorumVerifier();
 {noformat}
 Is that really desired? I suspect that is what's causing Observers to try to 
 connect to each other (as opposed as just connecting to participants). I'll 
 give it a try now and let you know. (Also, we use observer ids that are  0, 
 and I saw some parts of the code that might not deal with that assumption - 
 so it could be that too..). 



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1732) ZooKeeper server unable to join established ensemble

2013-10-28 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1732:


May be we should start considering automate rolling upgrade test?.  In jenkins 
we might be able to continuously grab 3.4 branch and perform rolling upgrade to 
3.5 and verify that quorum come up


 ZooKeeper server unable to join established ensemble
 

 Key: ZOOKEEPER-1732
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1732
 Project: ZooKeeper
  Issue Type: Bug
  Components: leaderElection
Affects Versions: 3.4.5
 Environment: Windows 7, Java 1.7
Reporter: Germán Blanco
Assignee: Germán Blanco
Priority: Blocker
 Fix For: 3.4.6, 3.5.0

 Attachments: CREATE_INCONSISTENCIES_patch.txt, zklog.tar.gz, 
 ZOOKEEPER-1732-3.4.patch, ZOOKEEPER-1732-3.4.patch, ZOOKEEPER-1732-3.4.patch, 
 ZOOKEEPER-1732-3.4.patch, ZOOKEEPER-1732-b3.4.patch, 
 ZOOKEEPER-1732-b3.4.patch, ZOOKEEPER-1732.patch, ZOOKEEPER-1732.patch, 
 ZOOKEEPER-1732.patch, ZOOKEEPER-1732.patch, ZOOKEEPER-1732.patch


 I have a test in which I do a rolling restart of three ZooKeeper servers and 
 it was failing from time to time.
 I ran the tests in a loop until the failure came out and it seems that at 
 some point one of the servers is unable to join the enssemble formed by the 
 other two.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1800) jenkins failure in testGetProposalFromTxn

2013-10-28 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1800:


Yeah, fsync time on these boxes is unbelievable.  

2013-10-24 10:43:32,575 [myid:] - WARN  [SyncThread:0:FileTxnLog@322] - 
fsync-ing the write ahead log in SyncThread:0 took 7333ms which will adversely 
effect operation latency. See the ZooKeeper troubleshooting guide
2013-10-24 10:43:33,900 [myid:] - WARN  [SyncThread:0:FileTxnLog@322] - 
fsync-ing the write ahead log in SyncThread:0 took 1324ms which will adversely 
effect operation latency. See the ZooKeeper troubleshooting guide
2013-10-24 10:43:33,902 [myid:] - INFO  
[main:JUnit4ZKTestRunner$LoggedInvokeMethod@54] - TEST METHOD FAILED 
testGetProposalFromTxn
org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = 
ConnectionLoss for /invalidsnap-129

 jenkins failure in testGetProposalFromTxn
 -

 Key: ZOOKEEPER-1800
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1800
 Project: ZooKeeper
  Issue Type: Bug
  Components: tests
Affects Versions: 3.5.0
Reporter: Patrick Hunt
Assignee: Thawan Kooburat
 Fix For: 3.5.0


 https://builds.apache.org/view/S-Z/view/ZooKeeper/job/ZooKeeper-trunk-jdk7/691/testReport/junit/org.apache.zookeeper.test/GetProposalFromTxnTest/testGetProposalFromTxn/
 test was introduced in ZOOKEEPER-1413, seems to have failed twice so far this 
 month.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1802) flakey test testResyncByTxnlogThenDiffAfterFollowerCrashes

2013-10-28 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1802:


As part of fixing duplicate NEWLEADER packet (ZOOKEEPER-1324), 
lastProcessedZxid on each server can be different when the quorum start up and 
there is no new request (They may point to the last txn from the previous 
epoch). As shown in the log here

2013-10-24 10:42:07,301 [myid:] - INFO  
[main:FollowerResyncConcurrencyTest@588] - Timeout waiting for zxid to sync: 
leader 0x13ecc clean 0x2 restarted 0x13ecc

I can switch to rely on other method to verify that all server has up-to-date 
data instead of checking lastProcessedZxid



 flakey test testResyncByTxnlogThenDiffAfterFollowerCrashes
 --

 Key: ZOOKEEPER-1802
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1802
 Project: ZooKeeper
  Issue Type: Bug
  Components: tests
Affects Versions: 3.5.0
Reporter: Patrick Hunt
Assignee: Thawan Kooburat

 This test fails intermittently on trunk:
 https://builds.apache.org/view/S-Z/view/ZooKeeper/job/ZooKeeper-trunk-jdk7/691/testReport/junit/org.apache.zookeeper.test/FollowerResyncConcurrencyTest/testResyncByTxnlogThenDiffAfterFollowerCrashes/



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1798) testNormalObserverRun failed on branch 3.4

2013-10-23 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1798:


Find bug warning is due to exit() call when IO exception is caught.  I added 
this because sync request processor already invoke exit() if exception is 
thrown during its normal operation.

 testNormalObserverRun failed on branch 3.4
 --

 Key: ZOOKEEPER-1798
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1798
 Project: ZooKeeper
  Issue Type: Bug
Reporter: Flavio Junqueira
Assignee: Thawan Kooburat
Priority: Blocker
 Fix For: 3.4.6, 3.5.0

 Attachments: TEST-org.apache.zookeeper.server.quorum.Zab1_0Test.txt, 
 ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798.patch


 This is the output messges:
 noformat
 Testcase: testNormalObserverRun took 4.221 sec
 FAILED
 expected:data[2] but was:data[1]
 junit.framework.AssertionFailedError: expected:data[2] but was:data[1]
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test$8.converseWithObserver(Zab1_0Test.java:1118)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testObserverConversation(Zab1_0Test.java:546)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testNormalObserverRun(Zab1_0Test.java:994)
 noformat



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (ZOOKEEPER-1798) Fix race condition in testNormalObserverRun

2013-10-23 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1798:
---

Attachment: ZOOKEEPER-1798-b3.4.patch

 Fix race condition in testNormalObserverRun
 ---

 Key: ZOOKEEPER-1798
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1798
 Project: ZooKeeper
  Issue Type: Bug
Reporter: Flavio Junqueira
Assignee: Thawan Kooburat
Priority: Blocker
 Fix For: 3.4.6, 3.5.0

 Attachments: TEST-org.apache.zookeeper.server.quorum.Zab1_0Test.txt, 
 ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798-b3.4.patch, 
 ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798.patch


 This is the output messges:
 noformat
 Testcase: testNormalObserverRun took 4.221 sec
 FAILED
 expected:data[2] but was:data[1]
 junit.framework.AssertionFailedError: expected:data[2] but was:data[1]
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test$8.converseWithObserver(Zab1_0Test.java:1118)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testObserverConversation(Zab1_0Test.java:546)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testNormalObserverRun(Zab1_0Test.java:994)
 noformat



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (ZOOKEEPER-1798) Fix race condition in testNormalObserverRun

2013-10-23 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1798:
---

Summary: Fix race condition in testNormalObserverRun  (was: 
testNormalObserverRun failed on branch 3.4)

 Fix race condition in testNormalObserverRun
 ---

 Key: ZOOKEEPER-1798
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1798
 Project: ZooKeeper
  Issue Type: Bug
Reporter: Flavio Junqueira
Assignee: Thawan Kooburat
Priority: Blocker
 Fix For: 3.4.6, 3.5.0

 Attachments: TEST-org.apache.zookeeper.server.quorum.Zab1_0Test.txt, 
 ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798-b3.4.patch, 
 ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798.patch


 This is the output messges:
 noformat
 Testcase: testNormalObserverRun took 4.221 sec
 FAILED
 expected:data[2] but was:data[1]
 junit.framework.AssertionFailedError: expected:data[2] but was:data[1]
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test$8.converseWithObserver(Zab1_0Test.java:1118)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testObserverConversation(Zab1_0Test.java:546)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testNormalObserverRun(Zab1_0Test.java:994)
 noformat



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (ZOOKEEPER-1798) Fix race condition in testNormalObserverRun

2013-10-23 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1798:
---

Attachment: ZOOKEEPER-1798.patch

Remove exit call and add 3.4 version of the patch

 Fix race condition in testNormalObserverRun
 ---

 Key: ZOOKEEPER-1798
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1798
 Project: ZooKeeper
  Issue Type: Bug
Reporter: Flavio Junqueira
Assignee: Thawan Kooburat
Priority: Blocker
 Fix For: 3.4.6, 3.5.0

 Attachments: TEST-org.apache.zookeeper.server.quorum.Zab1_0Test.txt, 
 ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798-b3.4.patch, 
 ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798.patch, ZOOKEEPER-1798.patch


 This is the output messges:
 noformat
 Testcase: testNormalObserverRun took 4.221 sec
 FAILED
 expected:data[2] but was:data[1]
 junit.framework.AssertionFailedError: expected:data[2] but was:data[1]
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test$8.converseWithObserver(Zab1_0Test.java:1118)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testObserverConversation(Zab1_0Test.java:546)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testNormalObserverRun(Zab1_0Test.java:994)
 noformat



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (ZOOKEEPER-1798) testNormalObserverRun failed on branch 3.4

2013-10-22 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1798:
---

Attachment: ZOOKEEPER-1798.patch

Make shutdown sequence flush all pending txn to disk

 testNormalObserverRun failed on branch 3.4
 --

 Key: ZOOKEEPER-1798
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1798
 Project: ZooKeeper
  Issue Type: Bug
Reporter: Flavio Junqueira
Assignee: Flavio Junqueira
Priority: Blocker
 Fix For: 3.4.6, 3.5.0

 Attachments: TEST-org.apache.zookeeper.server.quorum.Zab1_0Test.txt, 
 ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798.patch


 This is the output messges:
 noformat
 Testcase: testNormalObserverRun took 4.221 sec
 FAILED
 expected:data[2] but was:data[1]
 junit.framework.AssertionFailedError: expected:data[2] but was:data[1]
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test$8.converseWithObserver(Zab1_0Test.java:1118)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testObserverConversation(Zab1_0Test.java:546)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testNormalObserverRun(Zab1_0Test.java:994)
 noformat



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1798) testNormalObserverRun failed on branch 3.4

2013-10-22 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1798:


[~fpj] I am planing to apply this patch to both trunk and 3.4.  If you are OK 
with the current implementation, I will also create a 3.4 patch as well and 
commit them.

 testNormalObserverRun failed on branch 3.4
 --

 Key: ZOOKEEPER-1798
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1798
 Project: ZooKeeper
  Issue Type: Bug
Reporter: Flavio Junqueira
Assignee: Thawan Kooburat
Priority: Blocker
 Fix For: 3.4.6, 3.5.0

 Attachments: TEST-org.apache.zookeeper.server.quorum.Zab1_0Test.txt, 
 ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798.patch


 This is the output messges:
 noformat
 Testcase: testNormalObserverRun took 4.221 sec
 FAILED
 expected:data[2] but was:data[1]
 junit.framework.AssertionFailedError: expected:data[2] but was:data[1]
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test$8.converseWithObserver(Zab1_0Test.java:1118)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testObserverConversation(Zab1_0Test.java:546)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testNormalObserverRun(Zab1_0Test.java:994)
 noformat



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1557) jenkins jdk7 test failure in testBadSaslAuthNotifiesWatch

2013-10-22 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1557:


If JDK7 test fail intermittently (but not in JDK6), it is probably due to 
interference between each unit test in the same file when they are running in 
different order.  For AutoResetWithPending, it was due to JVM flag leakage from 
one test to another. A simple fix is to separate the test as Eugene suggest or 
actually track down what is the interference and clean up each test properly.

 jenkins jdk7 test failure in testBadSaslAuthNotifiesWatch
 -

 Key: ZOOKEEPER-1557
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1557
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.4.5, 3.5.0
Reporter: Patrick Hunt
Assignee: Eugene Koontz
 Fix For: 3.4.6, 3.5.0

 Attachments: jstack.out, SaslAuthFailTest.log, ZOOKEEPER-1557.patch


 Failure of testBadSaslAuthNotifiesWatch on the jenkins jdk7 job:
 https://builds.apache.org/job/ZooKeeper-trunk-jdk7/407/
 haven't seen this before.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1798) testNormalObserverRun failed on branch 3.4

2013-10-21 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1798:


The observer don't need to wait for a request to be persisted on disk before 
applying to the db right?   

Currently, the test is failing because of timing issue but this shouldn't 
affect the correctness of the system.  So we just add some delay in the test or 
some mechanism to make sure that request get flushed before we do the verify 
step instead.  So we don't have to pay for the cost of extra latency on running 
system. 



 testNormalObserverRun failed on branch 3.4
 --

 Key: ZOOKEEPER-1798
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1798
 Project: ZooKeeper
  Issue Type: Bug
Reporter: Flavio Junqueira
Assignee: Flavio Junqueira
Priority: Blocker
 Fix For: 3.4.6

 Attachments: TEST-org.apache.zookeeper.server.quorum.Zab1_0Test.txt, 
 ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798-b3.4.patch


 This is the output messges:
 noformat
 Testcase: testNormalObserverRun took 4.221 sec
 FAILED
 expected:data[2] but was:data[1]
 junit.framework.AssertionFailedError: expected:data[2] but was:data[1]
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test$8.converseWithObserver(Zab1_0Test.java:1118)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testObserverConversation(Zab1_0Test.java:546)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testNormalObserverRun(Zab1_0Test.java:994)
 noformat



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1798) testNormalObserverRun failed on branch 3.4

2013-10-21 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1798:


If you are ok with approach that fixing just the test itself (may be checking 
with the sync processor that all request is flushed) I will come up with a 
diff. 



 testNormalObserverRun failed on branch 3.4
 --

 Key: ZOOKEEPER-1798
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1798
 Project: ZooKeeper
  Issue Type: Bug
Reporter: Flavio Junqueira
Assignee: Flavio Junqueira
Priority: Blocker
 Fix For: 3.4.6

 Attachments: TEST-org.apache.zookeeper.server.quorum.Zab1_0Test.txt, 
 ZOOKEEPER-1798-b3.4.patch, ZOOKEEPER-1798-b3.4.patch


 This is the output messges:
 noformat
 Testcase: testNormalObserverRun took 4.221 sec
 FAILED
 expected:data[2] but was:data[1]
 junit.framework.AssertionFailedError: expected:data[2] but was:data[1]
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test$8.converseWithObserver(Zab1_0Test.java:1118)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testObserverConversation(Zab1_0Test.java:546)
 at 
 org.apache.zookeeper.server.quorum.Zab1_0Test.testNormalObserverRun(Zab1_0Test.java:994)
 noformat



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1558) Leader should not snapshot uncommitted state

2013-10-18 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1558:


Thanks Flavio, the patch looks good to me

 Leader should not snapshot uncommitted state
 

 Key: ZOOKEEPER-1558
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1558
 Project: ZooKeeper
  Issue Type: Sub-task
  Components: quorum
Affects Versions: 3.4.6
Reporter: Flavio Junqueira
Assignee: Flavio Junqueira
Priority: Blocker
 Fix For: 3.4.6

 Attachments: ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch, 
 ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch, 
 ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch


 Leader currently takes a snapshot when it calls loadData in the beginning of 
 the lead() method. The loaded data, however, may contain uncommitted state.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1558) Leader should not snapshot uncommitted state

2013-10-16 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1558:


Minor style change request: please use  private static int randRoll ... and 
move variable declaration to be on top of constructor (ideally below snapcount 
since it is a related variable) 

Other parts look good for me  so  +1 once style change is made 



 Leader should not snapshot uncommitted state
 

 Key: ZOOKEEPER-1558
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1558
 Project: ZooKeeper
  Issue Type: Sub-task
  Components: quorum
Affects Versions: 3.4.6
Reporter: Flavio Junqueira
Assignee: Flavio Junqueira
Priority: Blocker
 Fix For: 3.4.6

 Attachments: ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch, 
 ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch, 
 ZOOKEEPER-1558.patch


 Leader currently takes a snapshot when it calls loadData in the beginning of 
 the lead() method. The loaded data, however, may contain uncommitted state.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1793) Zab1_0Test.testNormalObserverRun() is flaky

2013-10-16 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1793:


I wasn't able to repro this on my mac.  The jenkins builds look very stable to 
me. I also don't have issue with this test in our internal branch as well.

When this assert fail, it means that observer didn't flush file to disk. So 
data read from disk by a separate thread did have the latest value.  

 Zab1_0Test.testNormalObserverRun() is flaky
 ---

 Key: ZOOKEEPER-1793
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1793
 Project: ZooKeeper
  Issue Type: Bug
  Components: quorum, server, tests
Reporter: Alexander Shraer
Assignee: Thawan Kooburat

 not sure if this is due to a known issue or not.
 // check and make sure the change is persisted
 zkDb2 = new ZKDatabase(new FileTxnSnapLog(logDir, snapDir));
 lastZxid = zkDb2.loadDataBase();
 Assert.assertEquals(data2, new String(zkDb2.getData(/foo, stat, null)));
 this assert periodically (once every 3 runs of the test or so) fails saying 
 that  getData returns data1 and not data2.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (ZOOKEEPER-1793) Zab1_0Test.testNormalObserverRun() is flaky

2013-10-16 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1793:
---

Assignee: (was: Thawan Kooburat)

 Zab1_0Test.testNormalObserverRun() is flaky
 ---

 Key: ZOOKEEPER-1793
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1793
 Project: ZooKeeper
  Issue Type: Bug
  Components: quorum, server, tests
Reporter: Alexander Shraer

 not sure if this is due to a known issue or not.
 // check and make sure the change is persisted
 zkDb2 = new ZKDatabase(new FileTxnSnapLog(logDir, snapDir));
 lastZxid = zkDb2.loadDataBase();
 Assert.assertEquals(data2, new String(zkDb2.getData(/foo, stat, null)));
 this assert periodically (once every 3 runs of the test or so) fails saying 
 that  getData returns data1 and not data2.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (ZOOKEEPER-1793) Zab1_0Test.testNormalObserverRun() is flaky

2013-10-16 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1793:
---

Assignee: (was: Thawan Kooburat)

 Zab1_0Test.testNormalObserverRun() is flaky
 ---

 Key: ZOOKEEPER-1793
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1793
 Project: ZooKeeper
  Issue Type: Bug
  Components: quorum, server, tests
Reporter: Alexander Shraer

 not sure if this is due to a known issue or not.
 // check and make sure the change is persisted
 zkDb2 = new ZKDatabase(new FileTxnSnapLog(logDir, snapDir));
 lastZxid = zkDb2.loadDataBase();
 Assert.assertEquals(data2, new String(zkDb2.getData(/foo, stat, null)));
 this assert periodically (once every 3 runs of the test or so) fails saying 
 that  getData returns data1 and not data2.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1793) Zab1_0Test.testNormalObserverRun() is flaky

2013-10-11 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1793:


Alex,  where do you see that test is flaky? These builds look ok to me 
(https://builds.apache.org/job/ZooKeeper-trunk/, 
https://builds.apache.org/job/ZooKeeper_branch34/)

 Zab1_0Test.testNormalObserverRun() is flaky
 ---

 Key: ZOOKEEPER-1793
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1793
 Project: ZooKeeper
  Issue Type: Bug
  Components: quorum, server, tests
Reporter: Alexander Shraer
Assignee: Thawan Kooburat

 not sure if this is due to a known issue or not.
 // check and make sure the change is persisted
 zkDb2 = new ZKDatabase(new FileTxnSnapLog(logDir, snapDir));
 lastZxid = zkDb2.loadDataBase();
 Assert.assertEquals(data2, new String(zkDb2.getData(/foo, stat, null)));
 this assert periodically (once every 3 runs of the test or so) fails saying 
 that  getData returns data1 and not data2.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1558) Leader should not snapshot uncommitted state

2013-10-10 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1558:


Yeah, I think that should work as well.  Seem like a new txnlog file is 
produced on a new epoch so that should be fine. 

I am wondering if we should apply this to 3.5 so at least the problem in 1549 
is partially fixed in trunk as well before 1549 land and we also get more 
testing.

 Leader should not snapshot uncommitted state
 

 Key: ZOOKEEPER-1558
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1558
 Project: ZooKeeper
  Issue Type: Sub-task
  Components: quorum
Affects Versions: 3.4.6
Reporter: Flavio Junqueira
Assignee: Flavio Junqueira
Priority: Blocker
 Fix For: 3.4.6

 Attachments: ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch, 
 ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch


 Leader currently takes a snapshot when it calls loadData in the beginning of 
 the lead() method. The loaded data, however, may contain uncommitted state.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Created] (ZOOKEEPER-1787) Add support enabling local session in rolling upgrade

2013-10-09 Thread Thawan Kooburat (JIRA)
Thawan Kooburat created ZOOKEEPER-1787:
--

 Summary: Add support enabling local session in rolling upgrade
 Key: ZOOKEEPER-1787
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1787
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.5.0
Reporter: Thawan Kooburat
Priority: Minor


Currently, local session need to be enable by stopping the entire ensemble. If 
a rolling upgrade is used, all write request from a local session will fail 
with session move until the local session is enabled on the leader.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1787) Add support enabling local session in rolling upgrade

2013-10-09 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1787:


Your original patch is to add 4lw comment to disable session validation on the 
leader right? If you could try my approach, I think the patch should be much 
smaller. 

 Add support enabling local session in rolling upgrade
 -

 Key: ZOOKEEPER-1787
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1787
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.5.0
Reporter: Thawan Kooburat
Priority: Minor

 Currently, local session need to be enable by stopping the entire ensemble. 
 If a rolling upgrade is used, all write request from a local session will 
 fail with session move until the local session is enabled on the leader.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1147) Add support for local sessions

2013-10-09 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1147:


Committed to trunk. Thanks a lot everyone.

 Add support for local sessions
 --

 Key: ZOOKEEPER-1147
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1147
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.3.3
Reporter: Vishal Kathuria
Assignee: Thawan Kooburat
  Labels: api-change, scaling
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1147.patch, ZOOKEEPER-1147.patch, 
 ZOOKEEPER-1147.patch, ZOOKEEPER-1147.patch, ZOOKEEPER-1147.patch, 
 ZOOKEEPER-1147.patch, ZOOKEEPER-1147.patch, ZOOKEEPER-1147.patch, 
 ZOOKEEPER-1147.patch

   Original Estimate: 840h
  Remaining Estimate: 840h

 This improvement is in the bucket of making ZooKeeper work at a large scale. 
 We are planning on having about a 1 million clients connect to a ZooKeeper 
 ensemble through a set of 50-100 observers. Majority of these clients are 
 read only - ie they do not do any updates or create ephemeral nodes.
 In ZooKeeper today, the client creates a session and the session creation is 
 handled like any other update. In the above use case, the session create/drop 
 workload can easily overwhelm an ensemble. The following is a proposal for a 
 local session, to support a larger number of connections.
 1.   The idea is to introduce a new type of session - local session. A 
 local session doesn't have a full functionality of a normal session.
 2.   Local sessions cannot create ephemeral nodes.
 3.   Once a local session is lost, you cannot re-establish it using the 
 session-id/password. The session and its watches are gone for good.
 4.   When a local session connects, the session info is only maintained 
 on the zookeeper server (in this case, an observer) that it is connected to. 
 The leader is not aware of the creation of such a session and there is no 
 state written to disk.
 5.   The pings and expiration is handled by the server that the session 
 is connected to.
 With the above changes, we can make ZooKeeper scale to a much larger number 
 of clients without making the core ensemble a bottleneck.
 In terms of API, there are two options that are being considered
 1. Let the client specify at the connect time which kind of session do they 
 want.
 2. All sessions connect as local sessions and automatically get promoted to 
 global sessions when they do an operation that requires a global session 
 (e.g. creating an ephemeral node)
 Chubby took the approach of lazily promoting all sessions to global, but I 
 don't think that would work in our case, where we want to keep sessions which 
 never create ephemeral nodes as always local. Option 2 would make it more 
 broadly usable but option 1 would be easier to implement.
 We are thinking of implementing option 1 as the first cut. There would be a 
 client flag, IsLocalSession (much like the current readOnly flag) that would 
 be used to determine whether to create a local session or a global session.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1558) Leader should not snapshot uncommitted state

2013-10-09 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1558:


Again, my concern is that the current solution would cause leader to be blocked 
taking the snapshot before starting to send ping to quorum members. If the 
snapshot taking time is larger than syncLimit, the quorum will tear down. You 
can simply simulate this situation by adding sleep which is longer that 
syncLimit into the new code where takeSnaphot() is called. 

If we agree that this is any issue, a simple fix is to create a method that 
take snapshot asynchronously (spin up a thread on demand similar to 
SyncRequestProcessor).  Some refactoring and additional locking may be needed 
as well in order to do this cleanly.  

 Leader should not snapshot uncommitted state
 

 Key: ZOOKEEPER-1558
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1558
 Project: ZooKeeper
  Issue Type: Sub-task
  Components: quorum
Affects Versions: 3.4.6
Reporter: Flavio Junqueira
Assignee: Flavio Junqueira
Priority: Blocker
 Fix For: 3.4.6

 Attachments: ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch, 
 ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch


 Leader currently takes a snapshot when it calls loadData in the beginning of 
 the lead() method. The loaded data, however, may contain uncommitted state.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (ZOOKEEPER-1558) Leader should not snapshot uncommitted state

2013-10-09 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1558:
---

Attachment: ZOOKEEPER-1558.patch

Update a patch to make it apply cleanly with 3.4 branch

 Leader should not snapshot uncommitted state
 

 Key: ZOOKEEPER-1558
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1558
 Project: ZooKeeper
  Issue Type: Sub-task
  Components: quorum
Affects Versions: 3.4.6
Reporter: Flavio Junqueira
Assignee: Flavio Junqueira
Priority: Blocker
 Fix For: 3.4.6

 Attachments: ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch, 
 ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch


 Leader currently takes a snapshot when it calls loadData in the beginning of 
 the lead() method. The loaded data, however, may contain uncommitted state.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1624) PrepRequestProcessor abort multi-operation incorrectly

2013-10-08 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1624:


As I already comment earlier, the current Java test doesn't actually catch the 
bug due to timing issue.  I guess, I will have to rewrite it to test 
PrepRequestProcessor directly (which is probably not going to rely on 
ZOOKEEPER-1572)

If you want to commit this now,  the patch itself has a proper and reliable (at 
least on my box) unit test in C.  Our test infrastructure do run C unit test 
and report the result right?  I agree with Camile that it would be nice to have 
Java test for server-side functionality but it isn't strictly needed right? 

 PrepRequestProcessor abort multi-operation incorrectly
 --

 Key: ZOOKEEPER-1624
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1624
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Critical
  Labels: zk-review
 Fix For: 3.4.6, 3.5.0

 Attachments: ZOOKEEPER-1624.patch, ZOOKEEPER-1624.patch, 
 ZOOKEEPER-1624.patch, ZOOKEEPER-1624.patch, ZOOKEEPER-1624.patch


 We found this issue when trying to issue multiple instances of the following 
 multi-op concurrently
 multi {
 1. create sequential node /a- 
 2. create node /b
 }
 The expected result is that only the first multi-op request should success 
 and the rest of request should fail because /b is already exist
 However, the reported result is that the subsequence multi-op failed because 
 of sequential node creation failed which is not possible.
 Below is the return code for each sub-op when issuing 3 instances of the 
 above multi-op asynchronously
 1. ZOK, ZOK
 2. ZOK, ZNODEEXISTS,
 3. ZNODEEXISTS, ZRUNTIMEINCONSISTENCY,
 When I added more debug log. The cause is that PrepRequestProcessor rollback 
 outstandingChanges of the second multi-op incorrectly causing sequential node 
 name generation to be incorrect. Below is the sequential node name generated 
 by PrepRequestProcessor
 1. create /a-0001
 2. create /a-0003
 3. create /a-0001
 The bug is getPendingChanges() method. In failed to copied ChangeRecord for 
 the parent node (/).  So rollbackPendingChanges() cannot restore the right 
 previous change record of the parent node when aborting the second multi-op
 The impact of this bug is that sequential node creation on the same parent 
 node may fail until the previous one is committed. I am not sure if there is 
 other implication or not.  



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (ZOOKEEPER-1551) Observers ignore txns that come after snapshot and UPTODATE

2013-10-07 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1551:
---

Summary: Observers ignore txns that come after snapshot and UPTODATE   
(was: Observer ignore txns that comes after snapshot and UPTODATE )

 Observers ignore txns that come after snapshot and UPTODATE 
 

 Key: ZOOKEEPER-1551
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1551
 Project: ZooKeeper
  Issue Type: Bug
  Components: quorum, server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Blocker
 Fix For: 3.4.6, 3.5.0

 Attachments: ZOOKEEPER-1551-3.4.patch, ZOOKEEPER-1551-b3.4.patch, 
 ZOOKEEPER-1551.patch, ZOOKEEPER-1551.patch, ZOOKEEPER-1551-trunk.patch, 
 ZOOKEEPER-1551-trunk.patch, ZOOKEEPER-1551-trunk.patch


 In Learner.java, txns which comes after the learner has taken the snapshot 
 (after NEWLEADER packet) are stored in packetsNotCommitted. The follower has 
 special logic to apply these txns at the end of syncWithLeader() method. 
 However, the observer will ignore these txns completely, causing data 
 inconsistency. 



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1551) Observer ignore txns that comes after snapshot and UPTODATE

2013-10-04 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1551:


b3.4 patch looks good for me (+1). If the trunk patch also looks good to you, I 
will commit the patch   

 Observer ignore txns that comes after snapshot and UPTODATE 
 

 Key: ZOOKEEPER-1551
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1551
 Project: ZooKeeper
  Issue Type: Bug
  Components: quorum, server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Blocker
 Fix For: 3.4.6, 3.5.0

 Attachments: ZOOKEEPER-1551-3.4.patch, ZOOKEEPER-1551-b3.4.patch, 
 ZOOKEEPER-1551.patch, ZOOKEEPER-1551.patch, ZOOKEEPER-1551-trunk.patch, 
 ZOOKEEPER-1551-trunk.patch, ZOOKEEPER-1551-trunk.patch


 In Learner.java, txns which comes after the learner has taken the snapshot 
 (after NEWLEADER packet) are stored in packetsNotCommitted. The follower has 
 special logic to apply these txns at the end of syncWithLeader() method. 
 However, the observer will ignore these txns completely, causing data 
 inconsistency. 



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1777) Missing ephemeral nodes in one of the members of the ensemble

2013-10-03 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1777:


Haven't have time to go over your log yet, it might be possible that current 
release don't handle all the cases correctly.  I believe ZOOKEEPER-1413 should 
already fixed this issue. If you can repro the problem using trunk, I will fix 
it. 

With ZOOKEEPER-1413, the leader knows that  (1,7c) to (1,a9)  don't belong to 
its history, so it is going to send TRUNC. If the leader don't have sufficient 
history, it will send SNAP to A. 

 Missing ephemeral nodes in one of the members of the ensemble
 -

 Key: ZOOKEEPER-1777
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1777
 Project: ZooKeeper
  Issue Type: Bug
  Components: quorum
Affects Versions: 3.4.5
 Environment: Linux, Java 1.7
Reporter: Germán Blanco
Assignee: Germán Blanco
Priority: Blocker
 Fix For: 3.4.6, 3.5.0

 Attachments: snaps.tar, ZOOKEEPER-1777.tar.gz


 In a 3-servers ensemble, one of the followers doesn't see part of the 
 ephemeral nodes that are present in the leader and the other follower. 
 The 8 missing nodes in the follower that is not ok were created in the end 
 of epoch 1, the ensemble is running in epoch 2.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1551) Observer ignore txns that comes after snapshot and UPTODATE

2013-10-01 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1551:


yeah, I will look into it and update the patch

 Observer ignore txns that comes after snapshot and UPTODATE 
 

 Key: ZOOKEEPER-1551
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1551
 Project: ZooKeeper
  Issue Type: Bug
  Components: quorum, server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Blocker
 Fix For: 3.4.6, 3.5.0

 Attachments: ZOOKEEPER-1551-3.4.patch, ZOOKEEPER-1551.patch, 
 ZOOKEEPER-1551.patch, ZOOKEEPER-1551-trunk.patch, ZOOKEEPER-1551-trunk.patch


 In Learner.java, txns which comes after the learner has taken the snapshot 
 (after NEWLEADER packet) are stored in packetsNotCommitted. The follower has 
 special logic to apply these txns at the end of syncWithLeader() method. 
 However, the observer will ignore these txns completely, causing data 
 inconsistency. 



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (ZOOKEEPER-1551) Observer ignore txns that comes after snapshot and UPTODATE

2013-10-01 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1551:
---

Attachment: ZOOKEEPER-1551-trunk.patch

Testing is failing because syncEnabled is not enable by default when QuorumPeer 
object is created directly.

 Observer ignore txns that comes after snapshot and UPTODATE 
 

 Key: ZOOKEEPER-1551
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1551
 Project: ZooKeeper
  Issue Type: Bug
  Components: quorum, server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Blocker
 Fix For: 3.4.6, 3.5.0

 Attachments: ZOOKEEPER-1551-3.4.patch, ZOOKEEPER-1551.patch, 
 ZOOKEEPER-1551.patch, ZOOKEEPER-1551-trunk.patch, ZOOKEEPER-1551-trunk.patch, 
 ZOOKEEPER-1551-trunk.patch


 In Learner.java, txns which comes after the learner has taken the snapshot 
 (after NEWLEADER packet) are stored in packetsNotCommitted. The follower has 
 special logic to apply these txns at the end of syncWithLeader() method. 
 However, the observer will ignore these txns completely, causing data 
 inconsistency. 



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1758) Add documentation for zookeeper.observer.syncRequestProcessorEnabled flag

2013-09-30 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1758:


+1

 Add documentation for zookeeper.observer.syncRequestProcessorEnabled flag
 -

 Key: ZOOKEEPER-1758
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1758
 Project: ZooKeeper
  Issue Type: Improvement
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Minor
 Fix For: 3.5.0, 3.4.6

 Attachments: ZOOKEEPER-1758.patch, ZOOKEEPER-1758.patch






--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1552) Enable sync request processor in Observer

2013-09-30 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1552:


+1   Thanks a lot Flavio

 Enable sync request processor in Observer
 -

 Key: ZOOKEEPER-1552
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1552
 Project: ZooKeeper
  Issue Type: Improvement
  Components: quorum, server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Flavio Junqueira
 Fix For: 3.5.0, 3.4.6

 Attachments: ZOOKEEPER-1552-3.4.patch, ZOOKEEPER-1552-b3.4.patch, 
 ZOOKEEPER-1552-b3.4.patch, ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, 
 ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, 
 ZOOKEEPER-1552-trunk.patch


 Observer doesn't forward its txns to SyncRequestProcessor. So it never 
 persists the txns onto disk or periodically creates snapshots. This increases 
 the start-up time since it will get the entire snapshot if the observer has 
 be running for a long time. 



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1558) Leader should not snapshot uncommitted state

2013-09-30 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1558:


I think the patch need to be rebased in order to apply. 

With the recent issue of ZOOKEEPER-1697, I am wondering if we need to revisit 
the solution.  If I understand correctly, this patch move the snapshot step 
from pre-leader election to after synchronization and before broadcast phase. I 
believe we already transition to use syncLimit by the time we take this 
snapshot. I believe we should be taking snapshot while using initLimit  

In any case, we should add more comment about timing consideration into the 
patch as well.   

 Leader should not snapshot uncommitted state
 

 Key: ZOOKEEPER-1558
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1558
 Project: ZooKeeper
  Issue Type: Sub-task
  Components: quorum
Reporter: Flavio Junqueira
Assignee: Flavio Junqueira
Priority: Blocker
 Fix For: 3.4.6

 Attachments: ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch, 
 ZOOKEEPER-1558.patch, ZOOKEEPER-1558.patch


 Leader currently takes a snapshot when it calls loadData in the beginning of 
 the lead() method. The loaded data, however, may contain uncommitted state.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (ZOOKEEPER-1551) Observer ignore txns that comes after snapshot and UPTODATE

2013-09-30 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1551:
---

Attachment: ZOOKEEPER-1551-trunk.patch

Rebase and enable test that depends on ZOOKEEPER-1552

 Observer ignore txns that comes after snapshot and UPTODATE 
 

 Key: ZOOKEEPER-1551
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1551
 Project: ZooKeeper
  Issue Type: Bug
  Components: quorum, server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Blocker
 Fix For: 3.4.6, 3.5.0

 Attachments: ZOOKEEPER-1551-3.4.patch, ZOOKEEPER-1551.patch, 
 ZOOKEEPER-1551.patch, ZOOKEEPER-1551-trunk.patch, ZOOKEEPER-1551-trunk.patch


 In Learner.java, txns which comes after the learner has taken the snapshot 
 (after NEWLEADER packet) are stored in packetsNotCommitted. The follower has 
 special logic to apply these txns at the end of syncWithLeader() method. 
 However, the observer will ignore these txns completely, causing data 
 inconsistency. 



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (ZOOKEEPER-1674) There is no need to clear load the database across leader election

2013-09-26 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1674:


Internally, we are working on this one at the moment. We are trying to fix this 
issue without relying on or fixing ZOOKEEPER-1549, so this patch won't be too 
large or complicate. 

 There is no need to clear  load the database across leader election
 

 Key: ZOOKEEPER-1674
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1674
 Project: ZooKeeper
  Issue Type: Improvement
Reporter: Jacky007

 It is interesting to notice the piece of codes in QuorumPeer.java
  /* ZKDatabase is a top level member of quorumpeer 
  * which will be used in all the zookeeperservers
  * instantiated later. Also, it is created once on 
  * bootup and only thrown away in case of a truncate
  * message from the leader
  */
 private ZKDatabase zkDb;
 It is introduced by ZOOKEEPER-596. Now, we just drop the database every 
 leader election.
 We can keep it safely with ZOOKEEPER-1549.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1638) Redundant zk.getZKDatabase().clear();

2013-09-25 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1638:


Can you add a comment into a patch saying that db is clear as part of 
deserializeSnapshot()?

I will +1 and commit the patch after the change



 Redundant zk.getZKDatabase().clear();
 -

 Key: ZOOKEEPER-1638
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1638
 Project: ZooKeeper
  Issue Type: Improvement
Reporter: Alexander Shraer
Assignee: neil bhakta
Priority: Trivial
  Labels: newbie
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1638.patch


 Learner.syncWithLeader calls zk.getZKDatabase().clear() right before 
 zk.getZKDatabase().deserializeSnapshot(leaderIs); Then the first thing 
 deserializeSnapshot does is another clear(). 
 Suggest to remove the clear() in syncWithLeader.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1552) Enable sync request processor in Observer

2013-09-25 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1552:


LTGM  +1

 Enable sync request processor in Observer
 -

 Key: ZOOKEEPER-1552
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1552
 Project: ZooKeeper
  Issue Type: Improvement
  Components: quorum, server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Flavio Junqueira
 Fix For: 3.5.0, 3.4.6

 Attachments: ZOOKEEPER-1552-3.4.patch, ZOOKEEPER-1552-b3.4.patch, 
 ZOOKEEPER-1552-b3.4.patch, ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, 
 ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch


 Observer doesn't forward its txns to SyncRequestProcessor. So it never 
 persists the txns onto disk or periodically creates snapshots. This increases 
 the start-up time since it will get the entire snapshot if the observer has 
 be running for a long time. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1552) Enable sync request processor in Observer

2013-09-20 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1552:


Is it possible that we also keep the JVM flag? 

All of the patch from our internal branch have a coding convention of 
introducing a JVM flag into a file that it affect the functionality. I can 
debate why it has some merit but I do agree that configuration file is easier 
to use for user. If this coding convention is not accept by upstream, I will 
have to change our internal coding convention as well because all 
pending/future patch has this style. 



 Enable sync request processor in Observer
 -

 Key: ZOOKEEPER-1552
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1552
 Project: ZooKeeper
  Issue Type: Improvement
  Components: quorum, server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Flavio Junqueira
 Fix For: 3.5.0, 3.4.6

 Attachments: ZOOKEEPER-1552-3.4.patch, ZOOKEEPER-1552-b3.4.patch, 
 ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, 
 ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch


 Observer doesn't forward its txns to SyncRequestProcessor. So it never 
 persists the txns onto disk or periodically creates snapshots. This increases 
 the start-up time since it will get the entire snapshot if the observer has 
 be running for a long time. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1462) Read-only server does not initialize database properly

2013-09-17 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1462:


ZOOKEEPER-1552 make sure that observer has up-to-date information on disk. So 
it partially fix the problem that German has with the observer. 

This patch fixed when any type of server restarted and transition into 
Read-only server. It will use on-disk state to populate its data-tree.  

When thinking about this, it doesn't seem like it is safe to use read-only 
server with participants. Since it may have uncommitted txn in its txnlog.

Internally, we use ZOOKEEPER-1607, so I didn't put more effort on read-only 
server related features. 

 Read-only server does not initialize database properly
 --

 Key: ZOOKEEPER-1462
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1462
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Critical
 Fix For: 3.4.6

 Attachments: ZOOKEEPER-1462.patch


 Brief Description:
 When a participant or observer get partitioned and restart as Read-only 
 server. ZkDb doesn't get reinitialized. This causes the RO server to drop any 
 incoming request with zxid  0 
 Error message:
 Refusing session request for client /xx.xx.xx.xx:39875 
 as it has seen zxid 0x2e00405fd9 our last zxid is 0x0 client must try another 
 server
 Steps to reproduce:
 Start an RO-enabled observer connecting to an ensemble. Kill the ensemble and 
 wait until the observer restart in RO mode. Zxid of this observer should be 0.
 Description:
 Before a server transition into LOOKING state, its database get closed as 
 part of shutdown sequence. The database of leader, follower and observer get 
 initialized as a side effect of participating in leader election protocol. 
 (eg. observer will call registerWithLeader() and call getLastLoggedZxid() 
 which initialize the db if not already).
 However, RO server does not participate in this protocol so its DB doesn't 
 get initialized properly
  

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1552) Enable sync request processor in Observer

2013-09-16 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1552:


[~fpj] If this looks good to you, I can commit the patch (and the documentation 
patch ZOOKEEPER-1758), so I can continue to work on ZOOKEEPER-1551

 Enable sync request processor in Observer
 -

 Key: ZOOKEEPER-1552
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1552
 Project: ZooKeeper
  Issue Type: Improvement
  Components: quorum, server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
 Fix For: 3.5.0, 3.4.6

 Attachments: ZOOKEEPER-1552-3.4.patch, ZOOKEEPER-1552.patch, 
 ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, 
 ZOOKEEPER-1552.patch


 Observer doesn't forward its txns to SyncRequestProcessor. So it never 
 persists the txns onto disk or periodically creates snapshots. This increases 
 the start-up time since it will get the entire snapshot if the observer has 
 be running for a long time. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1552) Enable sync request processor in Observer

2013-09-16 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1552:


I haven't start working on it and has no plan to work on that in a near future. 
If we really need a config file, we can just modify the start up script to read 
a file and set up JVM property on start up. This is essentially what we do on 
our internal branch.  

 Enable sync request processor in Observer
 -

 Key: ZOOKEEPER-1552
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1552
 Project: ZooKeeper
  Issue Type: Improvement
  Components: quorum, server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
 Fix For: 3.5.0, 3.4.6

 Attachments: ZOOKEEPER-1552-3.4.patch, ZOOKEEPER-1552.patch, 
 ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, 
 ZOOKEEPER-1552.patch


 Observer doesn't forward its txns to SyncRequestProcessor. So it never 
 persists the txns onto disk or periodically creates snapshots. This increases 
 the start-up time since it will get the entire snapshot if the observer has 
 be running for a long time. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1624) PrepRequestProcessor abort multi-operation incorrectly

2013-09-16 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1624:


For Java, I think the best approach is to unit test the PrepRequestProcessor 
itself directly (and mock the rest of system). With this approach, we won't 
need async interface.  I haven't have time to rewrite the test yet but if we 
want this in 3.4.6, I will find some time to work on it after I am done with 
ZK-1551,1552

 PrepRequestProcessor abort multi-operation incorrectly
 --

 Key: ZOOKEEPER-1624
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1624
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Critical
  Labels: zk-review
 Fix For: 3.5.0, 3.4.6

 Attachments: ZOOKEEPER-1624.patch, ZOOKEEPER-1624.patch, 
 ZOOKEEPER-1624.patch, ZOOKEEPER-1624.patch, ZOOKEEPER-1624.patch


 We found this issue when trying to issue multiple instances of the following 
 multi-op concurrently
 multi {
 1. create sequential node /a- 
 2. create node /b
 }
 The expected result is that only the first multi-op request should success 
 and the rest of request should fail because /b is already exist
 However, the reported result is that the subsequence multi-op failed because 
 of sequential node creation failed which is not possible.
 Below is the return code for each sub-op when issuing 3 instances of the 
 above multi-op asynchronously
 1. ZOK, ZOK
 2. ZOK, ZNODEEXISTS,
 3. ZNODEEXISTS, ZRUNTIMEINCONSISTENCY,
 When I added more debug log. The cause is that PrepRequestProcessor rollback 
 outstandingChanges of the second multi-op incorrectly causing sequential node 
 name generation to be incorrect. Below is the sequential node name generated 
 by PrepRequestProcessor
 1. create /a-0001
 2. create /a-0003
 3. create /a-0001
 The bug is getPendingChanges() method. In failed to copied ChangeRecord for 
 the parent node (/).  So rollbackPendingChanges() cannot restore the right 
 previous change record of the parent node when aborting the second multi-op
 The impact of this bug is that sequential node creation on the same parent 
 node may fail until the previous one is committed. I am not sure if there is 
 other implication or not.  

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (ZOOKEEPER-1757) Adler32 may not be sufficient to protect against data corruption

2013-09-13 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1757:
---

Summary: Adler32 may not be sufficient to protect against data corruption  
(was: Adler32 may not be sufficient to protect aginst data corruption)

 Adler32 may not be sufficient to protect against data corruption
 

 Key: ZOOKEEPER-1757
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1757
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
 Environment: Linux.  Oracle JDK6/7
Reporter: Thawan Kooburat
 Attachments: ZOOKEEPER.1757.patch, ZOOKEEPER.1757.patch


 I was investigating data inconsistency bug in our internal branch. One 
 possible area is snapshot/txnlog corruption. So I wrote a more robust 
 corruption test and found that it is easy to break our checksum algorithm 
 which is Adler32.
 When this happen, it is more likely that corrupted data will fail other 
 sanity check during deserialization phase, but it is still scary that it can 
 pass the checksum.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (ZOOKEEPER-1552) Enable sync request processor in Observer

2013-09-13 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1552:
---

Attachment: ZOOKEEPER-1552.patch

Attach a trunk patch

 Enable sync request processor in Observer
 -

 Key: ZOOKEEPER-1552
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1552
 Project: ZooKeeper
  Issue Type: Improvement
  Components: quorum, server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
 Fix For: 3.5.0, 3.4.6

 Attachments: ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, 
 ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch


 Observer doesn't forward its txns to SyncRequestProcessor. So it never 
 persists the txns onto disk or periodically creates snapshots. This increases 
 the start-up time since it will get the entire snapshot if the observer has 
 be running for a long time. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (ZOOKEEPER-1552) Enable sync request processor in Observer

2013-09-13 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1552:
---

Attachment: ZOOKEEPER-1552-3.4.patch

Re-upload German's patch 

 Enable sync request processor in Observer
 -

 Key: ZOOKEEPER-1552
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1552
 Project: ZooKeeper
  Issue Type: Improvement
  Components: quorum, server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
 Fix For: 3.5.0, 3.4.6

 Attachments: ZOOKEEPER-1552-3.4.patch, ZOOKEEPER-1552.patch, 
 ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, 
 ZOOKEEPER-1552.patch


 Observer doesn't forward its txns to SyncRequestProcessor. So it never 
 persists the txns onto disk or periodically creates snapshots. This increases 
 the start-up time since it will get the entire snapshot if the observer has 
 be running for a long time. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Created] (ZOOKEEPER-1758) Add documentation for zookeeper.observer.syncRequestProcessorEnabled flag

2013-09-13 Thread Thawan Kooburat (JIRA)
Thawan Kooburat created ZOOKEEPER-1758:
--

 Summary: Add documentation for 
zookeeper.observer.syncRequestProcessorEnabled flag
 Key: ZOOKEEPER-1758
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1758
 Project: ZooKeeper
  Issue Type: Improvement
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Minor
 Fix For: 3.5.0, 3.4.6




--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (ZOOKEEPER-1758) Add documentation for zookeeper.observer.syncRequestProcessorEnabled flag

2013-09-13 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1758:
---

Attachment: ZOOKEEPER-1758.patch

 Add documentation for zookeeper.observer.syncRequestProcessorEnabled flag
 -

 Key: ZOOKEEPER-1758
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1758
 Project: ZooKeeper
  Issue Type: Improvement
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Minor
 Fix For: 3.5.0, 3.4.6

 Attachments: ZOOKEEPER-1758.patch




--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1552) Enable sync request processor in Observer

2013-09-13 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1552:


Not sure how to tell Hadoop QA to pickup a certain file to run.   Also upload a 
documentation patch

 Enable sync request processor in Observer
 -

 Key: ZOOKEEPER-1552
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1552
 Project: ZooKeeper
  Issue Type: Improvement
  Components: quorum, server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
 Fix For: 3.5.0, 3.4.6

 Attachments: ZOOKEEPER-1552-3.4.patch, ZOOKEEPER-1552.patch, 
 ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, ZOOKEEPER-1552.patch, 
 ZOOKEEPER-1552.patch


 Observer doesn't forward its txns to SyncRequestProcessor. So it never 
 persists the txns onto disk or periodically creates snapshots. This increases 
 the start-up time since it will get the entire snapshot if the observer has 
 be running for a long time. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (ZOOKEEPER-1551) Observer ignore txns that comes after snapshot and UPTODATE

2013-09-13 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1551:
---

Attachment: ZOOKEEPER-1551-3.4.patch

Upload 3.4 patch

 Observer ignore txns that comes after snapshot and UPTODATE 
 

 Key: ZOOKEEPER-1551
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1551
 Project: ZooKeeper
  Issue Type: Bug
  Components: quorum, server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Blocker
 Fix For: 3.5.0, 3.4.6

 Attachments: ZOOKEEPER-1551-3.4.patch, ZOOKEEPER-1551.patch, 
 ZOOKEEPER-1551.patch, ZOOKEEPER-1551-trunk.patch


 In Learner.java, txns which comes after the learner has taken the snapshot 
 (after NEWLEADER packet) are stored in packetsNotCommitted. The follower has 
 special logic to apply these txns at the end of syncWithLeader() method. 
 However, the observer will ignore these txns completely, causing data 
 inconsistency. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (ZOOKEEPER-1757) Adler32 may not be sufficient to protect against data corruption

2013-09-13 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1757:
---

Priority: Minor  (was: Major)

 Adler32 may not be sufficient to protect against data corruption
 

 Key: ZOOKEEPER-1757
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1757
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
 Environment: Linux.  Oracle JDK6/7
Reporter: Thawan Kooburat
Priority: Minor
 Attachments: ZOOKEEPER.1757.patch, ZOOKEEPER.1757.patch


 I was investigating data inconsistency bug in our internal branch. One 
 possible area is snapshot/txnlog corruption. So I wrote a more robust 
 corruption test and found that it is easy to break our checksum algorithm 
 which is Adler32.
 When this happen, it is more likely that corrupted data will fail other 
 sanity check during deserialization phase, but it is still scary that it can 
 pass the checksum.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1757) Adler32 may not be sufficient to protect against data corruption

2013-09-13 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1757:


Reducing the severity because it is unlikely the server will load the snapshot 
cleanly.  It would fail other sanity check while deserializing the snapshot.

However, one thing to note is that we are using just 8-byte Aler32 checksum for 
the entire snapshot.  For snapshot, i think it would be better to go with 
md5sum.  In our internal branch, we compress the snapshot before writing, so we 
get additional checksum through compression   

 Adler32 may not be sufficient to protect against data corruption
 

 Key: ZOOKEEPER-1757
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1757
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
 Environment: Linux.  Oracle JDK6/7
Reporter: Thawan Kooburat
Priority: Minor
 Attachments: ZOOKEEPER.1757.patch, ZOOKEEPER.1757.patch


 I was investigating data inconsistency bug in our internal branch. One 
 possible area is snapshot/txnlog corruption. So I wrote a more robust 
 corruption test and found that it is easy to break our checksum algorithm 
 which is Adler32.
 When this happen, it is more likely that corrupted data will fail other 
 sanity check during deserialization phase, but it is still scary that it can 
 pass the checksum.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Created] (ZOOKEEPER-1757) Adler32 may not be sufficient to protect aginst data corruption

2013-09-12 Thread Thawan Kooburat (JIRA)
Thawan Kooburat created ZOOKEEPER-1757:
--

 Summary: Adler32 may not be sufficient to protect aginst data 
corruption
 Key: ZOOKEEPER-1757
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1757
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
 Environment: Linux.  Oracle JDK6/7
Reporter: Thawan Kooburat


I was investigating data inconsistency bug in our internal branch. One possible 
area is snapshot/txnlog corruption. So I wrote a more robust corruption test 
and found that it is easy to break our checksum algorithm which is Adler32.

When this happen, it is more likely that corrupted data will fail other sanity 
check during deserialization phase, but it is still scary that it can pass the 
checksum.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (ZOOKEEPER-1757) Adler32 may not be sufficient to protect aginst data corruption

2013-09-12 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1757:
---

Attachment: TEST-org.apache.zookeeper.server.persistence.FileSnapTest.txt
ZOOKEEPER.1757.patch

Attached a test case that show the example of corrupted snapshot passed Adler32 
check. 

I hard coded a seek value trigger this condition. On other environment, the 
value may have to change by using the random seek. 

In my case, I only need to do a few run to trigger a case that can bypass 
checksum.

Also attached a sample log file from this unit test run.

 Adler32 may not be sufficient to protect aginst data corruption
 ---

 Key: ZOOKEEPER-1757
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1757
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
 Environment: Linux.  Oracle JDK6/7
Reporter: Thawan Kooburat
 Attachments: 
 TEST-org.apache.zookeeper.server.persistence.FileSnapTest.txt, 
 ZOOKEEPER.1757.patch


 I was investigating data inconsistency bug in our internal branch. One 
 possible area is snapshot/txnlog corruption. So I wrote a more robust 
 corruption test and found that it is easy to break our checksum algorithm 
 which is Adler32.
 When this happen, it is more likely that corrupted data will fail other 
 sanity check during deserialization phase, but it is still scary that it can 
 pass the checksum.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (ZOOKEEPER-1757) Adler32 may not be sufficient to protect aginst data corruption

2013-09-12 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1757:
---

Attachment: (was: 
TEST-org.apache.zookeeper.server.persistence.FileSnapTest.txt)

 Adler32 may not be sufficient to protect aginst data corruption
 ---

 Key: ZOOKEEPER-1757
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1757
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
 Environment: Linux.  Oracle JDK6/7
Reporter: Thawan Kooburat
 Attachments: ZOOKEEPER.1757.patch, ZOOKEEPER.1757.patch


 I was investigating data inconsistency bug in our internal branch. One 
 possible area is snapshot/txnlog corruption. So I wrote a more robust 
 corruption test and found that it is easy to break our checksum algorithm 
 which is Adler32.
 When this happen, it is more likely that corrupted data will fail other 
 sanity check during deserialization phase, but it is still scary that it can 
 pass the checksum.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (ZOOKEEPER-1757) Adler32 may not be sufficient to protect aginst data corruption

2013-09-12 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1757:
---

Attachment: ZOOKEEPER.1757.patch

Re-upload the test case

 Adler32 may not be sufficient to protect aginst data corruption
 ---

 Key: ZOOKEEPER-1757
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1757
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
 Environment: Linux.  Oracle JDK6/7
Reporter: Thawan Kooburat
 Attachments: ZOOKEEPER.1757.patch, ZOOKEEPER.1757.patch


 I was investigating data inconsistency bug in our internal branch. One 
 possible area is snapshot/txnlog corruption. So I wrote a more robust 
 corruption test and found that it is easy to break our checksum algorithm 
 which is Adler32.
 When this happen, it is more likely that corrupted data will fail other 
 sanity check during deserialization phase, but it is still scary that it can 
 pass the checksum.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1392) Should not allow to read ACL when not authorized to read node

2013-08-05 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1392:


Sorry for not getting into this JIRA earlier. Can you justify why we should 
prevent client from reading ACL value? What is the security concern if you 
allow unauthorized access to ACL list? 

I think ACL can be considered as metadata. In unix file system, you can still 
read file permission event if you don't have read permission. Additionally, 
exists() call don't perform any ACL check as well. I don't know if that is 
intentional or it is just another bug. 



 Should not allow to read ACL when not authorized to read node
 -

 Key: ZOOKEEPER-1392
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1392
 Project: ZooKeeper
  Issue Type: Bug
  Components: server
Affects Versions: 3.4.2
Reporter: Thomas Weise
Assignee: Bruce Gao
 Attachments: ZOOKEEPER-1392.patch


 Not authorized to read, yet still able to list ACL:
 [zk: localhost:2181(CONNECTED) 0] getAcl /sasltest/n4
 'sasl,'no...@example.com
 : cdrwa
 [zk: localhost:2181(CONNECTED) 1] get /sasltest/n4
 Exception in thread main 
 org.apache.zookeeper.KeeperException$NoAuthException: KeeperErrorCode = 
 NoAuth for /sasltest/n4
   at org.apache.zookeeper.KeeperException.create(KeeperException.java:113)
   at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
   at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:1131)
   at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:1160)
   at 
 org.apache.zookeeper.ZooKeeperMain.processZKCmd(ZooKeeperMain.java:711)
   at org.apache.zookeeper.ZooKeeperMain.processCmd(ZooKeeperMain.java:593)
   at 
 org.apache.zookeeper.ZooKeeperMain.executeLine(ZooKeeperMain.java:365)
   at org.apache.zookeeper.ZooKeeperMain.run(ZooKeeperMain.java:323)
   at org.apache.zookeeper.ZooKeeperMain.main(ZooKeeperMain.java:282)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1730) Make ZooKeeper easier to test - support simulating a session expiration

2013-07-23 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1730:


Sorry, I just tested your patch manually.  Here is what i see

In this case, when you caused the SendThread to exit, it will be the same as 
trigger connection loss. On exit, SendThread will queue connection loss event 
if the state.isAlive() is true (but it isn't in the current patch)

So if you call sendThread.join() before queuing session expire event and modify 
the SendThread exit path a bit we should get  disconnect event and then 
followed by  session expire.   
 
So this is why I think it is a just a bit more effort to add 
injectDiscconnect() into this patch.  

 Make ZooKeeper easier to test - support simulating a session expiration
 ---

 Key: ZOOKEEPER-1730
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1730
 Project: ZooKeeper
  Issue Type: Sub-task
  Components: java client
Reporter: Jordan Zimmerman
Assignee: Jordan Zimmerman
 Fix For: 3.5.0

 Attachments: gitdiff, ZOOKEEPER-1730-2.patch, ZOOKEEPER-1730-3.patch


 As part of making ZooKeeper clients more test friendly, it would be useful to 
 easily simulate a session loss event

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (ZOOKEEPER-1730) Make ZooKeeper easier to test - support simulating a session expiration

2013-07-22 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1730:
---

Attachment: gitdiff

I used to script to convert git patch to svn patch. This only works if you are 
using git to clone from SVN repo.  

If you are cloning from git repo, you might be able to find the revision number 
and hard code it in the script.

 Make ZooKeeper easier to test - support simulating a session expiration
 ---

 Key: ZOOKEEPER-1730
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1730
 Project: ZooKeeper
  Issue Type: Sub-task
  Components: java client
Reporter: Jordan Zimmerman
Assignee: Jordan Zimmerman
 Fix For: 3.5.0

 Attachments: gitdiff, ZOOKEEPER-1730-2.patch, ZOOKEEPER-1730-3.patch


 As part of making ZooKeeper clients more test friendly, it would be useful to 
 easily simulate a session loss event

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1730) Make ZooKeeper easier to test - support simulating a session expiration

2013-07-22 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1730:


Comments:
1. instead of using debugSendThreadExitLatch.  You should be able to just call 
clientCnxn.sendThread.join() to wait for sendThread to exit inside 
injectSessionExpiration()

2. Can you rename TestTestable to be something like ZooKeeperTestableTest.  
This seem to be a naming conversion for the rest of the test cases

3. in unit test, it will be great if we you can also check that we get 
disconnected and then followed by session expired event.

The rest looks good to me. 

 Make ZooKeeper easier to test - support simulating a session expiration
 ---

 Key: ZOOKEEPER-1730
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1730
 Project: ZooKeeper
  Issue Type: Sub-task
  Components: java client
Reporter: Jordan Zimmerman
Assignee: Jordan Zimmerman
 Fix For: 3.5.0

 Attachments: gitdiff, ZOOKEEPER-1730-2.patch, ZOOKEEPER-1730-3.patch


 As part of making ZooKeeper clients more test friendly, it would be useful to 
 easily simulate a session loss event

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1730) Make ZooKeeper easier to test - support simulating a session expiration

2013-07-18 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1730:


This looks good.  I think we can put a bit more effort to make this mimic a 
real behavior.

I think the important part is that we need to fail all pending request and 
callback correctly (SendThread.cleanup()). Additionally, the client should see 
the state transition in order  disconnected - session expire. So user can 
write unit test based on what ZooKeeeper client guarantee.

Here are my suggestions

1. We can add injectConnectionLoss() method.  Essentially, we need SendThread 
to call cleanup() and queue Disconnected event into the queue. We might be able 
to do this by causing SendThread to exit

2. In injectSessionExpire() method.  I think the current approach is good, but 
we should wait until SendThread exit before queuing SessionExpire event. So it 
is like invoking injectConnectionLoss() if it isn't already in disconnected 
state. 

3. Unit test should make sure that ZooKeeperTestable behave as expected 
regarding state transition and pending operations/callbacks.  




 Make ZooKeeper easier to test - support simulating a session expiration
 ---

 Key: ZOOKEEPER-1730
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1730
 Project: ZooKeeper
  Issue Type: Sub-task
  Components: java client
Reporter: Jordan Zimmerman
 Attachments: ZOOKEEPER-1730-2.patch


 As part of making ZooKeeper clients more test friendly, it would be useful to 
 easily simulate a session loss event

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1730) Make ZooKeeper easier to test - support simulating a session expiration

2013-07-16 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1730:


Did you for got to add new files to the patch? 

 Make ZooKeeper easier to test - support simulating a session expiration
 ---

 Key: ZOOKEEPER-1730
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1730
 Project: ZooKeeper
  Issue Type: Sub-task
  Components: java client
Reporter: Jordan Zimmerman
 Attachments: ZOOKEEPER-1730.patch


 As part of making ZooKeeper clients more test friendly, it would be useful to 
 easily simulate a session loss event

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Created] (ZOOKEEPER-1729) Add l4w command snap to trigger log rotation and snapshotting

2013-07-15 Thread Thawan Kooburat (JIRA)
Thawan Kooburat created ZOOKEEPER-1729:
--

 Summary: Add l4w command snap to trigger log rotation and 
snapshotting 
 Key: ZOOKEEPER-1729
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1729
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Minor


snap command can be used to trigger log rotate and snapshotting on each 
server. 

One use case for this command is to make server restart faster by issuing snap 
command before restarting the server. This help when txnlog is large (due to 
txn size or number of txn)

snap is a blocking command, it will return when snapshot is written to disk. So 
it is safe to call this prior to restarting the server.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Created] (ZOOKEEPER-1722) Make ZooKeeper clients more test friendly

2013-07-08 Thread Thawan Kooburat (JIRA)
Thawan Kooburat created ZOOKEEPER-1722:
--

 Summary: Make ZooKeeper clients more test friendly
 Key: ZOOKEEPER-1722
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1722
 Project: ZooKeeper
  Issue Type: Improvement
  Components: c client, java client
Reporter: Thawan Kooburat


We should be able to expose a few more API calls that allow user write unit 
tests that cover various failure scenarios (similar to the TestableZookeer in 
zookeeper test). This should also minimize the effort on setting the test 
framework for application developer

Here is some example calls that we should provide.
1. zookeeper_close() that don't actually send close request to the server: 
This can be used to simulate a client crash without actually crashing the test 
program.
2. Allow client to trigger CONNECTION_LOSS or SESSSION_EXPIRE event: This will 
allow the user to test their watchers and callback (and possible race condition)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1722) Make ZooKeeper clients more test friendly

2013-07-08 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1722:


I initially thought that we can do this by add a few hooks into client library, 
so we can easily provide testing for all languages. However, this route won't 
provide full test coverage. 

My only concern with mock server is the effort of maintaining it to make its 
behavior match the actual server-side. I don't have strong objection about 
this, see if others have any suggestions.  

 Make ZooKeeper clients more test friendly
 -

 Key: ZOOKEEPER-1722
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1722
 Project: ZooKeeper
  Issue Type: Improvement
  Components: c client, java client
Reporter: Thawan Kooburat

 We should be able to expose a few more API calls that allow user write unit 
 tests that cover various failure scenarios (similar to the TestableZookeer in 
 zookeeper test). This should also minimize the effort on setting the test 
 framework for application developer
 Here is some example calls that we should provide.
 1. zookeeper_close() that don't actually send close request to the server:
  This can be used to simulate a client crash without actually crashing the 
 test program.
 2. Allow client to trigger CONNECTION_LOSS or SESSSION_EXPIRE event: This 
 will allow the user to test their watchers and callback (and possible race 
 condition)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1722) Make ZooKeeper clients more test friendly

2013-07-08 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1722:


No problem, I think we need at least 2 sub-tickets anyway (for Java and C 
client)  

 Make ZooKeeper clients more test friendly
 -

 Key: ZOOKEEPER-1722
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1722
 Project: ZooKeeper
  Issue Type: Improvement
  Components: c client, java client
Reporter: Thawan Kooburat

 We should be able to expose a few more API calls that allow user write unit 
 tests that cover various failure scenarios (similar to the TestableZookeer in 
 zookeeper test). This should also minimize the effort on setting the test 
 framework for application developer
 Here is some example calls that we should provide.
 1. zookeeper_close() that don't actually send close request to the server:
  This can be used to simulate a client crash without actually crashing the 
 test program.
 2. Allow client to trigger CONNECTION_LOSS or SESSSION_EXPIRE event: This 
 will allow the user to test their watchers and callback (and possible race 
 condition)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Resolved] (ZOOKEEPER-876) Unnecessary snapshot transfers between new leader and followers

2013-07-01 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat resolved ZOOKEEPER-876.
---

Resolution: Fixed

ZOOKEEPER-1413 address all of the issues mentioned in the description. However, 
there is one corner case (used by the unit test in this patch) which is not 
covered by ZOOKEEPER-1413 where the learner tries to sync with leader with no 
txnlog. 

 Unnecessary snapshot transfers between new leader and followers
 ---

 Key: ZOOKEEPER-876
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-876
 Project: ZooKeeper
  Issue Type: Bug
Affects Versions: 3.4.0
Reporter: Diogo
Assignee: Diogo
Priority: Minor
 Fix For: 3.5.0

 Attachments: second_case.patch, second_case.txt, 
 TEST-org.apache.zookeeper.test.FollowerResyncConcurrencyTest.txt, 
 ZOOKEEPER-876.patch


 When starting a new leadership, unnecessary snapshot transfers happen between 
 new leader and followers. This is so because of multiple small bugs. 
 1) the comparison of zxids is done based on a new proposal, instead of the 
 last logged zxid. (LearnerHandler.java ~ 297)
 2) if follower is one zxid behind, the check of the interval of committed 
 logs excludes the follower. (LearnerHandler.java ~ 277)
 3) the bug reported in ZOOKEEPER-874 (commitLogs are empty after recover).

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (ZOOKEEPER-1413) Use on-disk transaction log for learner sync up

2013-06-27 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1413:
---

Attachment: ZOOKEEPER-1413.patch

Add missing license header

 Use on-disk transaction log for learner sync up
 ---

 Key: ZOOKEEPER-1413
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1413
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Minor
  Labels: performance
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch, 
 ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch, 
 ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch


 Motivation:
 The learner syncs up with leader by retrieving committed log from the leader. 
 Currently, the leader only keeps 500 entries of recently committed log in 
 memory. If the learner falls behind more than 500 updates, the leader will 
 send the entire snapshot to the learner. 
 With the size of the snapshot for some of our Zookeeper deployments (~10G), 
 it is prohibitively expensive to send the entire snapshot over network. 
 Additionally, our Zookeeper may serve more than 4K updates per seconds. As a 
 result, a network hiccups for less than a second will cause the learner to 
 use snapshot transfer.
 Design:
 Instead of looking only at committed log in memory, the leader will also look 
 at transaction log on disk. The amount of transaction log kept on disk is 
 configurable and the current default is 100k. This will allow Zookeeper to 
 tolerate longer temporal network failure before initiating the snapshot 
 transfer.  
 Implementation:
 We plan to add interface to the persistence layer will can be use to retrieve 
 proposals from on-disk transaction log. These proposals can then be used to 
 send to the learner using existing protocol. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (ZOOKEEPER-1413) Use on-disk transaction log for learner sync up

2013-06-24 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1413:
---

Attachment: ZOOKEEPER-1413.patch

Fix bug

 Use on-disk transaction log for learner sync up
 ---

 Key: ZOOKEEPER-1413
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1413
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Minor
  Labels: performance
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch, 
 ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch, 
 ZOOKEEPER-1413.patch


 Motivation:
 The learner syncs up with leader by retrieving committed log from the leader. 
 Currently, the leader only keeps 500 entries of recently committed log in 
 memory. If the learner falls behind more than 500 updates, the leader will 
 send the entire snapshot to the learner. 
 With the size of the snapshot for some of our Zookeeper deployments (~10G), 
 it is prohibitively expensive to send the entire snapshot over network. 
 Additionally, our Zookeeper may serve more than 4K updates per seconds. As a 
 result, a network hiccups for less than a second will cause the learner to 
 use snapshot transfer.
 Design:
 Instead of looking only at committed log in memory, the leader will also look 
 at transaction log on disk. The amount of transaction log kept on disk is 
 configurable and the current default is 100k. This will allow Zookeeper to 
 tolerate longer temporal network failure before initiating the snapshot 
 transfer.  
 Implementation:
 We plan to add interface to the persistence layer will can be use to retrieve 
 proposals from on-disk transaction log. These proposals can then be used to 
 send to the learner using existing protocol. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1413) Use on-disk transaction log for learner sync up

2013-06-21 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1413:


Hmm, why didn't Hadoop QA ran. I thought I only need to attach a new patch file 
to trigger it

 Use on-disk transaction log for learner sync up
 ---

 Key: ZOOKEEPER-1413
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1413
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Minor
  Labels: performance
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch, 
 ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch


 Motivation:
 The learner syncs up with leader by retrieving committed log from the leader. 
 Currently, the leader only keeps 500 entries of recently committed log in 
 memory. If the learner falls behind more than 500 updates, the leader will 
 send the entire snapshot to the learner. 
 With the size of the snapshot for some of our Zookeeper deployments (~10G), 
 it is prohibitively expensive to send the entire snapshot over network. 
 Additionally, our Zookeeper may serve more than 4K updates per seconds. As a 
 result, a network hiccups for less than a second will cause the learner to 
 use snapshot transfer.
 Design:
 Instead of looking only at committed log in memory, the leader will also look 
 at transaction log on disk. The amount of transaction log kept on disk is 
 configurable and the current default is 100k. This will allow Zookeeper to 
 tolerate longer temporal network failure before initiating the snapshot 
 transfer.  
 Implementation:
 We plan to add interface to the persistence layer will can be use to retrieve 
 proposals from on-disk transaction log. These proposals can then be used to 
 send to the learner using existing protocol. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (ZOOKEEPER-1413) Use on-disk transaction log for learner sync up

2013-06-21 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1413:
---

Attachment: ZOOKEEPER-1413.patch

upload the patch again

 Use on-disk transaction log for learner sync up
 ---

 Key: ZOOKEEPER-1413
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1413
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Minor
  Labels: performance
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch, 
 ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch


 Motivation:
 The learner syncs up with leader by retrieving committed log from the leader. 
 Currently, the leader only keeps 500 entries of recently committed log in 
 memory. If the learner falls behind more than 500 updates, the leader will 
 send the entire snapshot to the learner. 
 With the size of the snapshot for some of our Zookeeper deployments (~10G), 
 it is prohibitively expensive to send the entire snapshot over network. 
 Additionally, our Zookeeper may serve more than 4K updates per seconds. As a 
 result, a network hiccups for less than a second will cause the learner to 
 use snapshot transfer.
 Design:
 Instead of looking only at committed log in memory, the leader will also look 
 at transaction log on disk. The amount of transaction log kept on disk is 
 configurable and the current default is 100k. This will allow Zookeeper to 
 tolerate longer temporal network failure before initiating the snapshot 
 transfer.  
 Implementation:
 We plan to add interface to the persistence layer will can be use to retrieve 
 proposals from on-disk transaction log. These proposals can then be used to 
 send to the learner using existing protocol. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (ZOOKEEPER-1413) Use on-disk transaction log for learner sync up

2013-06-20 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat updated ZOOKEEPER-1413:
---

Attachment: ZOOKEEPER-1413.patch

- Address comments
- Move to SLF4J 1.7.5 

 Use on-disk transaction log for learner sync up
 ---

 Key: ZOOKEEPER-1413
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1413
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Minor
  Labels: performance
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch, 
 ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch


 Motivation:
 The learner syncs up with leader by retrieving committed log from the leader. 
 Currently, the leader only keeps 500 entries of recently committed log in 
 memory. If the learner falls behind more than 500 updates, the leader will 
 send the entire snapshot to the learner. 
 With the size of the snapshot for some of our Zookeeper deployments (~10G), 
 it is prohibitively expensive to send the entire snapshot over network. 
 Additionally, our Zookeeper may serve more than 4K updates per seconds. As a 
 result, a network hiccups for less than a second will cause the learner to 
 use snapshot transfer.
 Design:
 Instead of looking only at committed log in memory, the leader will also look 
 at transaction log on disk. The amount of transaction log kept on disk is 
 configurable and the current default is 100k. This will allow Zookeeper to 
 tolerate longer temporal network failure before initiating the snapshot 
 transfer.  
 Implementation:
 We plan to add interface to the persistence layer will can be use to retrieve 
 proposals from on-disk transaction log. These proposals can then be used to 
 send to the learner using existing protocol. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1147) Add support for local sessions

2013-06-20 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1147:


Ah...  Thanks for catching this.  We never have to perform upgrade on ensemble 
that use local session or the upgrade was short enough that we didn't noticed 
the problem. As you proposed, the upgrade path will require 2 rolling restarts 
to enable this feature.

The patch that you proposed should work. I think it should be possible to just 
modify PrepRequestProcessor so that we can use local session validation path 
without actually turning on local session. This should make the patch smaller.  

For additional safety, it might be better to use Java property to control this 
parameter. So the value survive server restarts in case there is a leader 
election during the upgrade.   
 

 Add support for local sessions
 --

 Key: ZOOKEEPER-1147
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1147
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.3.3
Reporter: Vishal Kathuria
Assignee: Thawan Kooburat
  Labels: api-change, scaling
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1147.patch, ZOOKEEPER-1147.patch, 
 ZOOKEEPER-1147.patch, ZOOKEEPER-1147.patch, ZOOKEEPER-1147.patch, 
 ZOOKEEPER-1147.patch, ZOOKEEPER-1147.patch, ZOOKEEPER-1147.patch

   Original Estimate: 840h
  Remaining Estimate: 840h

 This improvement is in the bucket of making ZooKeeper work at a large scale. 
 We are planning on having about a 1 million clients connect to a ZooKeeper 
 ensemble through a set of 50-100 observers. Majority of these clients are 
 read only - ie they do not do any updates or create ephemeral nodes.
 In ZooKeeper today, the client creates a session and the session creation is 
 handled like any other update. In the above use case, the session create/drop 
 workload can easily overwhelm an ensemble. The following is a proposal for a 
 local session, to support a larger number of connections.
 1.   The idea is to introduce a new type of session - local session. A 
 local session doesn't have a full functionality of a normal session.
 2.   Local sessions cannot create ephemeral nodes.
 3.   Once a local session is lost, you cannot re-establish it using the 
 session-id/password. The session and its watches are gone for good.
 4.   When a local session connects, the session info is only maintained 
 on the zookeeper server (in this case, an observer) that it is connected to. 
 The leader is not aware of the creation of such a session and there is no 
 state written to disk.
 5.   The pings and expiration is handled by the server that the session 
 is connected to.
 With the above changes, we can make ZooKeeper scale to a much larger number 
 of clients without making the core ensemble a bottleneck.
 In terms of API, there are two options that are being considered
 1. Let the client specify at the connect time which kind of session do they 
 want.
 2. All sessions connect as local sessions and automatically get promoted to 
 global sessions when they do an operation that requires a global session 
 (e.g. creating an ephemeral node)
 Chubby took the approach of lazily promoting all sessions to global, but I 
 don't think that would work in our case, where we want to keep sessions which 
 never create ephemeral nodes as always local. Option 2 would make it more 
 broadly usable but option 1 would be easier to implement.
 We are thinking of implementing option 1 as the first cut. There would be a 
 client flag, IsLocalSession (much like the current readOnly flag) that would 
 be used to determine whether to create a local session or a global session.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1413) Use on-disk transaction log for learner sync up

2013-06-10 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1413:


I was thinking about this.  I think my design where is the leader is required 
to have a older history than the learner might be too strict in some cases, 
given that Zab guaranteed continuous history. 

Anyway, I don't think this case is common enough that we need to handle it. If 
there is additional change request, I will create another revision so we can 
review it.

 Use on-disk transaction log for learner sync up
 ---

 Key: ZOOKEEPER-1413
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1413
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Minor
  Labels: performance
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch, 
 ZOOKEEPER-1413.patch


 Motivation:
 The learner syncs up with leader by retrieving committed log from the leader. 
 Currently, the leader only keeps 500 entries of recently committed log in 
 memory. If the learner falls behind more than 500 updates, the leader will 
 send the entire snapshot to the learner. 
 With the size of the snapshot for some of our Zookeeper deployments (~10G), 
 it is prohibitively expensive to send the entire snapshot over network. 
 Additionally, our Zookeeper may serve more than 4K updates per seconds. As a 
 result, a network hiccups for less than a second will cause the learner to 
 use snapshot transfer.
 Design:
 Instead of looking only at committed log in memory, the leader will also look 
 at transaction log on disk. The amount of transaction log kept on disk is 
 configurable and the current default is 100k. This will allow Zookeeper to 
 tolerate longer temporal network failure before initiating the snapshot 
 transfer.  
 Implementation:
 We plan to add interface to the persistence layer will can be use to retrieve 
 proposals from on-disk transaction log. These proposals can then be used to 
 send to the learner using existing protocol. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1413) Use on-disk transaction log for learner sync up

2013-06-04 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1413:


It doesn't need to get a snapshot but I don't add a logic to handle this 
special case, since I thought it wasn't a common case. 

Currently, queueCommittedProposal() will try to use DIFF or TRUNC only if it 
saw at least 1 txn older than what the learner ask for, so we need to modify it 
a bit to handle this case. 

   



 Use on-disk transaction log for learner sync up
 ---

 Key: ZOOKEEPER-1413
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1413
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Minor
  Labels: performance
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch, 
 ZOOKEEPER-1413.patch


 Motivation:
 The learner syncs up with leader by retrieving committed log from the leader. 
 Currently, the leader only keeps 500 entries of recently committed log in 
 memory. If the learner falls behind more than 500 updates, the leader will 
 send the entire snapshot to the learner. 
 With the size of the snapshot for some of our Zookeeper deployments (~10G), 
 it is prohibitively expensive to send the entire snapshot over network. 
 Additionally, our Zookeeper may serve more than 4K updates per seconds. As a 
 result, a network hiccups for less than a second will cause the learner to 
 use snapshot transfer.
 Design:
 Instead of looking only at committed log in memory, the leader will also look 
 at transaction log on disk. The amount of transaction log kept on disk is 
 configurable and the current default is 100k. This will allow Zookeeper to 
 tolerate longer temporal network failure before initiating the snapshot 
 transfer.  
 Implementation:
 We plan to add interface to the persistence layer will can be use to retrieve 
 proposals from on-disk transaction log. These proposals can then be used to 
 send to the learner using existing protocol. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (ZOOKEEPER-1413) Use on-disk transaction log for learner sync up

2013-06-04 Thread Thawan Kooburat (JIRA)

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

Thawan Kooburat commented on ZOOKEEPER-1413:


Actually, this fails at db.getProposalsFromTxnLog() as shown in the log. The 
method return a list of proposal starting for zxid X only if the list of 
proposal start from zxid X or lower. 

Again this is to help simplify queueCommittedProposal() since it needs to 
enforce the constraint that I mentioned above.  

 Use on-disk transaction log for learner sync up
 ---

 Key: ZOOKEEPER-1413
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1413
 Project: ZooKeeper
  Issue Type: Improvement
  Components: server
Affects Versions: 3.4.3
Reporter: Thawan Kooburat
Assignee: Thawan Kooburat
Priority: Minor
  Labels: performance
 Fix For: 3.5.0

 Attachments: ZOOKEEPER-1413.patch, ZOOKEEPER-1413.patch, 
 ZOOKEEPER-1413.patch


 Motivation:
 The learner syncs up with leader by retrieving committed log from the leader. 
 Currently, the leader only keeps 500 entries of recently committed log in 
 memory. If the learner falls behind more than 500 updates, the leader will 
 send the entire snapshot to the learner. 
 With the size of the snapshot for some of our Zookeeper deployments (~10G), 
 it is prohibitively expensive to send the entire snapshot over network. 
 Additionally, our Zookeeper may serve more than 4K updates per seconds. As a 
 result, a network hiccups for less than a second will cause the learner to 
 use snapshot transfer.
 Design:
 Instead of looking only at committed log in memory, the leader will also look 
 at transaction log on disk. The amount of transaction log kept on disk is 
 configurable and the current default is 100k. This will allow Zookeeper to 
 tolerate longer temporal network failure before initiating the snapshot 
 transfer.  
 Implementation:
 We plan to add interface to the persistence layer will can be use to retrieve 
 proposals from on-disk transaction log. These proposals can then be used to 
 send to the learner using existing protocol. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


  1   2   3   >