zookeeper-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Flavio Paiva Junqueira (JIRA)" <j...@apache.org>
Subject [jira] Commented: (ZOOKEEPER-59) Synchronized block in NIOServerCnxn
Date Tue, 08 Jul 2008 21:03:31 GMT

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

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

Mahadev, Thanks for fixing the patch. My eclipse still hasn't leaned how to generate diffs
appropriately. Your assessment of the patch is right, and the observation on "outstandingRequests"
seems correct to me.

Andrew, I agree with you on getInProcess(). It is probably best to make it synchronized or
simply to use an atomic integer for requestsInProcess. Now, on your description of the connection
problem, I think this is an important observation because when I observed the problem, the
log messages did not show the server receiving anything. Also, if the server receives and
we don't disable sending, then either ping requests are not going through the pipeline of
request processors (once it gets to FinalRequestProcessor, the server sends a response immediately)
or sendResponse is not doing its job.



> 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, ZOOKEEPER-59_2.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.


Mime
View raw message