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/06/30 13:23:45 UTC

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

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

Flavio Paiva Junqueira updated ZOOKEEPER-59:
--------------------------------------------

    Description: 
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:

{quote}
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();
                }
            } 
{quote}

{quote}
NIOServerCnxn.sendResponse@740
...
         synchronized (this.factory) {
                outstandingRequests--;
                // check throttling
                if (zk.getInProcess() < factory.outstandingLimit
                        || outstandingRequests < 1) {
                    sk.selector().wakeup();
                    enableRecv();
                }
            }
{quote}

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.

  was:
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:

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();
                }
            } 


NIOServerCnxn.sendResponse@740
...
         synchronized (this.factory) {
                outstandingRequests--;
                // check throttling
                if (zk.getInProcess() < factory.outstandingLimit
                        || outstandingRequests < 1) {
                    sk.selector().wakeup();
                    enableRecv();
                }
            }


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.

        Summary: Synchronized block in NIOServerCnxn  (was: Synchornized block in NIOServerCnxn)

> 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
>
> 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:
> {quote}
> 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();
>                 }
>             } 
> {quote}
> {quote}
> NIOServerCnxn.sendResponse@740
> ...
>          synchronized (this.factory) {
>                 outstandingRequests--;
>                 // check throttling
>                 if (zk.getInProcess() < factory.outstandingLimit
>                         || outstandingRequests < 1) {
>                     sk.selector().wakeup();
>                     enableRecv();
>                 }
>             }
> {quote}
> 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.