[ https://issues.apache.org/jira/browse/ZOOKEEPER-4537?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17533972#comment-17533972 ]
Kfir Lev-Ari commented on ZOOKEEPER-4537: ----------------------------------------- Nice catch Jithin! Based on the code that you shared here, your solution looks right to me and should be merged in order to deal with this corner case. Specifically, I don't see a reason from the correctness perspective not to call the sync before looking at the queues, given that we already doing it a few lines later. I.e., we're not introducing here a new order between sync operation and reading the queues, so I don't see why calling it earlier will danger the rest of the code, even the parts that you didn't share here. I don't believe performance is an issue - but If the 1 extra sync cost in terms of performance is noticeable, I guess we can consider avoiding syncs altogether, and move to a busy-wait approach (i.e., use bounded wait instead of unbounded). > Race between SyncThread and CommitProcessor thread > -------------------------------------------------- > > Key: ZOOKEEPER-4537 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-4537 > Project: ZooKeeper > Issue Type: Bug > Components: server > Reporter: Jithin Girish > Priority: Major > > Zookeeper server can get stuck when it has just one client and the only way > it recovers is due to a socket timeout or another client commit request. > Details: > We have a system where a system manager app (SM) launches all the other > applications based on some config criteria. When system boots up, SM > connects to zookeeper and is the only client talking to zookeeper until all > other apps are launched. SM does a bunch of sync create() calls to zookeeper, > before it starts up all the other apps. So at this point zookeeper server has > got just one connection, which is from SM. 1 out of 3 times during system > startup, SM gets stuck at a random create calls and there are only two ways > this gets unwedged. > 1. The socket has to time out and we get ZOPERATIONTIMEOUT > 2. If we start another connection to zookeeper (I used zkCli.sh to do this), > this unwedges the exiting connection to SM. > > From strace I can see that there is a race between SyncThread and > CommitProcessor thread. > > Sync thread > class CommitProcessor { > run() { > int requestsToProcess = 0; > boolean commitIsWaiting = false; > do { > commitIsWaiting = !committedRequests.isEmpty(); <<<<< we are > checking if there are more messages to process here > requestsToProcess = queuedRequests.size(); > // Avoid sync if we have something to do > if (requestsToProcess == 0 && !commitIsWaiting) { > // Waiting for requests to process > synchronized (this) { > while (!stopped && requestsToProcess == 0 && > !commitIsWaiting) { <<<<< and acting on the information read above inside the > sync block > wait(); <<<<<< wait here > commitIsWaiting = !committedRequests.isEmpty(); > requestsToProcess = queuedRequests.size(); > } > } > } > > > Commit thread > public void commit(Request request) { > if (stopped || request == null) { > return; > } > LOG.debug("Committing request:: {}", request); > request.commitRecvTime = Time.currentElapsedTime(); > ServerMetrics.getMetrics().COMMITS_QUEUED.add(1); > committedRequests.add(request); <<<<<< enqueue messages here > wakeup(); > } > > @SuppressFBWarnings("NN_NAKED_NOTIFY") > private synchronized void wakeup() { <<<<< wakeup call is synchronized > notifyAll(); > } > > > Now lets consider the following scenario > 1. Sync thread reads commitIsWaiting and there are no commits pending. > 2. This thread gets scheduled out > 3. We got a commit request – CommitProcessor thread adds the request to > committedRequests and calls wakeup > 4. CommitProcessor goes ahead and does a notifyAll(). > 5. Since the sync thread has not reached the wait() yet, there is no one to > wake up. > 6. Sync thread gets scheduled back in. > 7. It goes ahead and does a wait() but since there are no other connections > or new commit requests no one does a wakeup(). So this thread waits here > until the socket is timed out. > 7a. Or if another commit is made where we endup calling notifyAll() > which wakes up the waiting thread. > > I modified the CommitProcessor::run() like this > do { > /* > * Since requests are placed in the queue before being sent to > * the leader, if commitIsWaiting = true, the commit belongs > to > * the first update operation in the queuedRequests or to a > * request from a client on another server (i.e., the order of > * the following two lines is important!). > */ > synchronized (this) { <<<<< > commitIsWaiting = !committedRequests.isEmpty(); > <<<<< moved the queue checks inside the sync block to ensure we don’t have > the race condition > requestsToProcess = queuedRequests.size(); > // Avoid sync if we have something to do > if (requestsToProcess == 0 && !commitIsWaiting) { > // Waiting for requests to process > while (!stopped && requestsToProcess == 0 > && !commitIsWaiting) { > wait(); > commitIsWaiting = > !committedRequests.isEmpty(); > requestsToProcess = > queuedRequests.size(); > } > } > } > > This seems to have fixed the issue. -- This message was sent by Atlassian Jira (v8.20.7#820007)