[ 
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, 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)

Reply via email to