You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by "Flavio Paiva Junqueira (JIRA)" <ji...@apache.org> on 2008/07/08 13:20:34 UTC

[jira] Commented: (ZOOKEEPER-59) Synchronized block in NIOServerCnxn

    [ https://issues.apache.org/jira/browse/ZOOKEEPER-59?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12611527#action_12611527 ] 

Flavio Paiva Junqueira commented on ZOOKEEPER-59:
-------------------------------------------------

It turns out that outstandingRequests is not the variable controlling the throttling, but ZooKeeperServer.requestsInProcess. It seems to me that we should do the increment before calling the first request processor in the following code block:

{noformat}
          if (validpacket) {
                firstProcessor.processRequest(si);
                if (cnxn != null) {
                    incInProcess();
                }
            }
{noformat}

This is in ZooKeeperServer@870. 

> Synchronized block in NIOServerCnxn
> -----------------------------------
>
>                 Key: ZOOKEEPER-59
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-59
>             Project: Zookeeper
>          Issue Type: Bug
>          Components: server
>            Reporter: Flavio Paiva Junqueira
>            Assignee: Benjamin Reed
>         Attachments: ZOOKEEPER-59_1.patch
>
>
> There are two synchronized blocks locking on different objects, and to me they should be guarded by the same object. Here are the parts of the code I'm talking about:
> {noformat}
> NIOServerCnxn.readRequest@444
> ...
>           synchronized (this) {
>                 outstandingRequests++;
>                 // check throttling
>                 if (zk.getInProcess() > factory.outstandingLimit) {
>                     disableRecv();
>                     // following lines should not be needed since we are already
>                     // reading
>                     // } else {
>                     // enableRecv();
>                 }
>             } 
> {noformat}
> {noformat}
> NIOServerCnxn.sendResponse@740
> ...
>          synchronized (this.factory) {
>                 outstandingRequests--;
>                 // check throttling
>                 if (zk.getInProcess() < factory.outstandingLimit
>                         || outstandingRequests < 1) {
>                     sk.selector().wakeup();
>                     enableRecv();
>                 }
>             }
> {noformat}
> I think the second one is correct, and the first synchronized block should be guarded by "this.factory". 
> This could be related to issue ZOOKEEPER-57, but I have no concrete indication that this is the case so far.

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