[jira] [Commented] (ZOOKEEPER-1549) Data inconsistency when follower is receiving a DIFF with a dirty snapshot
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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
[ 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
[ 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
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1887?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1887?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1879?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1875) NullPointerException in ClientCnxn$EventThread.processEvent
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1875?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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.
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1863) Race condition in commit processor leading to out of order request completion, xid mismatch on client.
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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, ge
[jira] [Commented] (ZOOKEEPER-1863) Race condition in commit processor leading to out of order request completion, xid mismatch on client.
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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.
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1847) Normalize line endings in repository
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Assigned] (ZOOKEEPER-1504) Multi-thread NIOServerCnxn
[ 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] [Assigned] (ZOOKEEPER-1851) Follower and Observer Request Processors Do Not Forward create2 Requests
[ 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] [Commented] (ZOOKEEPER-1851) Follower and Observer Request Processors Do Not Forward create2 Requests
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1851?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Commented] (ZOOKEEPER-1504) Multi-thread NIOServerCnxn
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1504?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1863) Race condition in commit processor leading to out of order request completion, xid mismatch on client.
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Created] (ZOOKEEPER-1865) Fix retry logic in Learner.connectToLeader()
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
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1573?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1813) Zookeeper restart fails due to missing node from snapshot
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1813?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13823364#comment-13823364 ] Thawan Kooburat commented on ZOOKEEPER-1813: I believe this is similar issue as ZOOKEEPER-1573. In that case, I found that it is a false positive. If the check is disable, the result after snapshot restore is still a valid state of the system. I don't quite follow your example but it seem to be similar that we can ignore the failure right? Anyway the work around is to actually remove snapshot and txnlog for a given server. > Zookeeper restart fails due to missing node from snapshot > - > > Key: ZOOKEEPER-1813 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1813 > Project: ZooKeeper > Issue Type: Bug >Affects Versions: 3.4.5, 3.5.0 >Reporter: Vinay >Priority: Blocker > Attachments: ZOOKEEPER-1813-test.patch > > > Due to following exception Zookeeper restart is failing > {noformat}java.io.IOException: Failed to process transaction type: 1 error: > KeeperErrorCode = NoNode for /test/subdir2/subdir2/subdir > at > org.apache.zookeeper.server.persistence.FileTxnSnapLog.restore(FileTxnSnapLog.java:183) > at > org.apache.zookeeper.server.ZKDatabase.loadDataBase(ZKDatabase.java:222) > at > org.apache.zookeeper.server.ZooKeeperServer.loadData(ZooKeeperServer.java:255) > at > org.apache.zookeeper.server.ZooKeeperServer.startdata(ZooKeeperServer.java:380) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.startup(NIOServerCnxnFactory.java:748) > at > org.apache.zookeeper.server.ZooKeeperServerMain.runFromConfig(ZooKeeperServerMain.java:111) > at > org.apache.zookeeper.server.ZooKeeperServerMain.initializeAndRun(ZooKeeperServerMain.java:90) > at > org.apache.zookeeper.server.ZooKeeperServerMainTest$2.run(ZooKeeperServerMainTest.java:218) > Caused by: org.apache.zookeeper.KeeperException$NoNodeException: > KeeperErrorCode = NoNode for /test/subdir2/subdir2/subdir > at > org.apache.zookeeper.server.persistence.FileTxnSnapLog.processTransaction(FileTxnSnapLog.java:268) > at > org.apache.zookeeper.server.persistence.FileTxnSnapLog.restore(FileTxnSnapLog.java:181) > ... 7 more{noformat} -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (ZOOKEEPER-1798) Fix race condition in testNormalObserverRun
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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: > > Testcase: testNormalObserverRun took 4.221 sec > FAILED > expected: but was: > junit.framework.AssertionFailedError: expected: but was: > 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) > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (ZOOKEEPER-1798) Fix race condition in testNormalObserverRun
[ 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: > > Testcase: testNormalObserverRun took 4.221 sec > FAILED > expected: but was: > junit.framework.AssertionFailedError: expected: but was: > 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) > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (ZOOKEEPER-1798) Fix race condition in testNormalObserverRun
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13814482#comment-13814482 ] Thawan Kooburat commented on ZOOKEEPER-1798: Yeah, you are right. I will create another patch to fix this > 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: > > Testcase: testNormalObserverRun took 4.221 sec > FAILED > expected: but was: > junit.framework.AssertionFailedError: expected: but was: > 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) > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (ZOOKEEPER-1807) Observers spam each other creating connections to the election addr
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1807?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1807) Observers spam each other creating connections to the election addr
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1807?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1790) Deal with special ObserverId in QuorumCnxManager.receiveConnection
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1790?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1798) Fix race condition in testNormalObserverRun
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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: > > Testcase: testNormalObserverRun took 4.221 sec > FAILED > expected: but was: > junit.framework.AssertionFailedError: expected: but was: > 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) > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (ZOOKEEPER-1802) flakey test testResyncByTxnlogThenDiffAfterFollowerCrashes
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1802?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1800) jenkins failure in testGetProposalFromTxn
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1800?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1732) ZooKeeper server unable to join established ensemble
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1732?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Resolved] (ZOOKEEPER-1793) Zab1_0Test.testNormalObserverRun() is flaky
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1793?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Thawan Kooburat resolved ZOOKEEPER-1793. Resolution: Duplicate Fixed by ZOOKEEPER-1798 > 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-1798) Fix race condition in testNormalObserverRun
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13804538#comment-13804538 ] Thawan Kooburat commented on ZOOKEEPER-1798: Committed trunk 1535491 b3.4 1535497 > 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: > > Testcase: testNormalObserverRun took 4.221 sec > FAILED > expected: but was: > junit.framework.AssertionFailedError: expected: but was: > 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) > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (ZOOKEEPER-1798) Fix race condition in testNormalObserverRun
[ 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: > > Testcase: testNormalObserverRun took 4.221 sec > FAILED > expected: but was: > junit.framework.AssertionFailedError: expected: but was: > 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) > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (ZOOKEEPER-1798) Fix race condition in testNormalObserverRun
[ 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: > > Testcase: testNormalObserverRun took 4.221 sec > FAILED > expected: but was: > junit.framework.AssertionFailedError: expected: but was: > 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) > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (ZOOKEEPER-1798) Fix race condition in testNormalObserverRun
[ 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: > > Testcase: testNormalObserverRun took 4.221 sec > FAILED > expected: but was: > junit.framework.AssertionFailedError: expected: but was: > 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) > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (ZOOKEEPER-1798) testNormalObserverRun failed on branch 3.4
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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: > > Testcase: testNormalObserverRun took 4.221 sec > FAILED > expected: but was: > junit.framework.AssertionFailedError: expected: but was: > 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) > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (ZOOKEEPER-1557) jenkins jdk7 test failure in testBadSaslAuthNotifiesWatch
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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: > > Testcase: testNormalObserverRun took 4.221 sec > FAILED > expected: but was: > junit.framework.AssertionFailedError: expected: but was: > 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) > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (ZOOKEEPER-1798) testNormalObserverRun failed on branch 3.4
[ 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: > > Testcase: testNormalObserverRun took 4.221 sec > FAILED > expected: but was: > junit.framework.AssertionFailedError: expected: but was: > 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) > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (ZOOKEEPER-1798) testNormalObserverRun failed on branch 3.4
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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: > > Testcase: testNormalObserverRun took 4.221 sec > FAILED > expected: but was: > junit.framework.AssertionFailedError: expected: but was: > 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) > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (ZOOKEEPER-1798) testNormalObserverRun failed on branch 3.4
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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: > > Testcase: testNormalObserverRun took 4.221 sec > FAILED > expected: but was: > junit.framework.AssertionFailedError: expected: but was: > 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) > -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (ZOOKEEPER-1558) Leader should not snapshot uncommitted state
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Updated] (ZOOKEEPER-1793) Zab1_0Test.testNormalObserverRun() is flaky
[ 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
[ 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
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1793?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Commented] (ZOOKEEPER-1558) Leader should not snapshot uncommitted state
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1793?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Assigned] (ZOOKEEPER-1793) Zab1_0Test.testNormalObserverRun() is flaky
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1793?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Thawan Kooburat reassigned ZOOKEEPER-1793: -- Assignee: 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 >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
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Updated] (ZOOKEEPER-1558) Leader should not snapshot uncommitted state
[ 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-1558) Leader should not snapshot uncommitted state
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Commented] (ZOOKEEPER-1147) Add support for local sessions
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1147?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1787) Add support enabling local session in rolling upgrade
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1787?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Created] (ZOOKEEPER-1787) Add support enabling local session in rolling upgrade
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-1624) PrepRequestProcessor abort multi-operation incorrectly
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1624?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Commented] (ZOOKEEPER-1551) Observers ignore txns that come after snapshot and UPTODATE
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1551?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13788871#comment-13788871 ] Thawan Kooburat commented on ZOOKEEPER-1551: committed to 3.5.0 and 3.4.6 > 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] [Updated] (ZOOKEEPER-1551) Observers ignore txns that come after snapshot and UPTODATE
[ 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
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1551?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1777?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1777) Missing ephemeral nodes in one of the members of the ensemble
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1777?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13784814#comment-13784814 ] Thawan Kooburat commented on ZOOKEEPER-1777: Similar to what Flavio already said, here is what I see. Between step 4 and 5, you actually lose majority of the machine at once, so the quorum move forward without committed txns from (1,7c) to (1,a9) At step 6, A should get a TRUNC to (1,7b) and start getting DIFF with txn from (2,1) to (2,4). If A ever produced a snapshot after (1, 7b) , A won't be able to process TRUNC correctly and crash and never join a quorum. If this is not the behavior that you observe, it is a bug in an implementation not the protocol. > 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 > > > 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] [Updated] (ZOOKEEPER-1551) Observer ignore txns that comes after snapshot and UPTODATE
[ 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-1551) Observer ignore txns that comes after snapshot and UPTODATE
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1551?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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
[ 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-1558) Leader should not snapshot uncommitted state
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1758) Add documentation for zookeeper.observer.syncEnabled flag
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1758?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Thawan Kooburat updated ZOOKEEPER-1758: --- Summary: Add documentation for zookeeper.observer.syncEnabled flag (was: Add documentation for zookeeper.observer.syncRequestProcessorEnabled flag) > Add documentation for zookeeper.observer.syncEnabled 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
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1552?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1758) Add documentation for zookeeper.observer.syncRequestProcessorEnabled flag
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1758?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1674) There is no need to clear & load the database across leader election
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1674?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1552) Enable sync request processor in Observer
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1552?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1638) Redundant zk.getZKDatabase().clear();
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1638?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1552?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1552) Enable sync request processor in Observer
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1552?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13770433#comment-13770433 ] Thawan Kooburat commented on ZOOKEEPER-1552: No problem, please go ahead > 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-1462) Read-only server does not initialize database properly
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1462?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1624) PrepRequestProcessor abort multi-operation incorrectly
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1624?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Commented] (ZOOKEEPER-1552) Enable sync request processor in Observer
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1552?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1552) Enable sync request processor in Observer
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1552?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1757) Adler32 may not be sufficient to protect against data corruption
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1757?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Updated] (ZOOKEEPER-1757) Adler32 may not be sufficient to protect against data corruption
[ 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] [Updated] (ZOOKEEPER-1551) Observer ignore txns that comes after snapshot and UPTODATE
[ 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] [Commented] (ZOOKEEPER-1552) Enable sync request processor in Observer
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1552?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1758) Add documentation for zookeeper.observer.syncRequestProcessorEnabled flag
[ 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] [Created] (ZOOKEEPER-1758) Add documentation for zookeeper.observer.syncRequestProcessorEnabled flag
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-1552) Enable sync request processor in Observer
[ 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] [Updated] (ZOOKEEPER-1552) Enable sync request processor in Observer
[ 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-1757) Adler32 may not be sufficient to protect against data corruption
[ 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-1757) Adler32 may not be sufficient to protect aginst data corruption
[ 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
[ 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] [Updated] (ZOOKEEPER-1757) Adler32 may not be sufficient to protect aginst data corruption
[ 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] [Created] (ZOOKEEPER-1757) Adler32 may not be sufficient to protect aginst data corruption
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] [Commented] (ZOOKEEPER-1392) Should not allow to read ACL when not authorized to read node
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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-1314) improve zkpython synchronous api implementation
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1314?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13721024#comment-13721024 ] Thawan Kooburat commented on ZOOKEEPER-1314: [~henryr] Do you think this patch is good to go? I can help commit the patch > improve zkpython synchronous api implementation > --- > > Key: ZOOKEEPER-1314 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1314 > Project: ZooKeeper > Issue Type: Improvement > Components: contrib-bindings >Affects Versions: 3.3.3 >Reporter: Daniel Lescohier >Assignee: Daniel Lescohier >Priority: Minor > Attachments: ZOOKEEPER-1314.patch, ZOOKEEPER-1314.patch > > Original Estimate: 0.5h > Remaining Estimate: 0.5h > > Improves the following items in zkpython which are related to the Zookeeper > synchronous API: > # For pyzoo_create, no longer limit the returned znode name to 256 bytes; > dynamically allocate memory on the heap. > # For all the synchronous api calls, release the Python Global Interpreter > Lock just before doing the synchronous call. > I will attach the patch shortly. -- 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
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1730?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Commented] (ZOOKEEPER-1730) Make ZooKeeper easier to test - support simulating a session expiration
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1730?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Updated] (ZOOKEEPER-1730) Make ZooKeeper easier to test - support simulating a session expiration
[ 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
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1730?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1730?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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
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] [Commented] (ZOOKEEPER-1722) Make ZooKeeper clients more test friendly
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1722?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Commented] (ZOOKEEPER-1722) Make ZooKeeper clients more test friendly
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1722?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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] [Created] (ZOOKEEPER-1722) Make ZooKeeper clients more test friendly
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