zookeeper-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (ZOOKEEPER-2711) Deadlock between concurrent 4LW commands that iterate over connections with Netty server
Date Tue, 07 Mar 2017 21:10:38 GMT

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

ASF GitHub Bot commented on ZOOKEEPER-2711:
-------------------------------------------

GitHub user joshelser opened a pull request:

    https://github.com/apache/zookeeper/pull/186

    ZOOKEEPER-2711 Avoid synchronization on NettyServerCnxn in Factory

    NettyServerCnxnFactory previously synchronized on the (Netty)ServerCnxn
    object to provide mutual exclusion at the RPC layer. However, this was
    at odds with the synchronized methods in ServerCnxn (which shared the
    same monitor). As such, it was possible to deadlock between concurrent
    4LW commands that were invoking one of these synchronized methods
    on ServerCnxn.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/joshelser/zookeeper 2711-4lw

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/186.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #186
    
----
commit 9953e32963e6167f88b97519233b0872cce69a71
Author: Josh Elser <elserj@apache.org>
Date:   2017-03-07T21:03:57Z

    ZOOKEEPER-2711 Avoid synchronization on NettyServerCnxn in Factory
    
    NettyServerCnxnFactory previously synchronized on the (Netty)ServerCnxn
    object to provide mutual exclusion at the RPC layer. However, this was
    at odds with the synchronized methods in ServerCnxn (which shared the
    same monitor). As such, it was possible to deadlock between concurrent
    4LW commands that were invoking one of these synchronized methods
    on ServerCnxn.

----


> Deadlock between concurrent 4LW commands that iterate over connections with Netty server
> ----------------------------------------------------------------------------------------
>
>                 Key: ZOOKEEPER-2711
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2711
>             Project: ZooKeeper
>          Issue Type: Bug
>            Reporter: Josh Elser
>            Priority: Critical
>
> Observed the following issue in some $dayjob testing environments. Line numbers are a
little off compared to master/branch-3.5, but I did confirm the same issue exists there.
> With the NettyServerCnxnFactory, before a request is dispatched, the code synchronizes
on the {{NettyServerCnxn}} object. However, with some 4LW commands (like {{stat}}), each {{ServerCnxn}}
object is also synchronized to (safely) iterate over the internal contents of the object to
generate the necessary debug message. As such, multiple concurrent {{stat}} commands can both
lock their own {{NettyServerCnxn}} objects, and then be blocked waiting to lock each others'
{{ServerCnxn}} in the {{StatCommand}}, deadlocked.
> {noformat}
> "New I/O worker #55":
> 	at org.apache.zookeeper.server.ServerCnxn.dumpConnectionInfo(ServerCnxn.java:407)
> 	- waiting to lock <0x00000000fabc01b8> (a org.apache.zookeeper.server.NettyServerCnxn)
> 	at org.apache.zookeeper.server.NettyServerCnxn$StatCommand.commandRun(NettyServerCnxn.java:478)
> 	at org.apache.zookeeper.server.NettyServerCnxn$CommandThread.run(NettyServerCnxn.java:311)
> 	at org.apache.zookeeper.server.NettyServerCnxn$CommandThread.start(NettyServerCnxn.java:306)
> 	at org.apache.zookeeper.server.NettyServerCnxn.checkFourLetterWord(NettyServerCnxn.java:677)
> 	at org.apache.zookeeper.server.NettyServerCnxn.receiveMessage(NettyServerCnxn.java:790)
> 	at org.apache.zookeeper.server.NettyServerCnxnFactory$CnxnChannelHandler.processMessage(NettyServerCnxnFactory.java:211)
> 	at org.apache.zookeeper.server.NettyServerCnxnFactory$CnxnChannelHandler.messageReceived(NettyServerCnxnFactory.java:135)
> 	- locked <0x00000000fab68178> (a org.apache.zookeeper.server.NettyServerCnxn)
> 	at org.jboss.netty.channel.SimpleChannelHandler.handleUpstream(SimpleChannelHandler.java:88)
> 	at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
> 	at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:559)
> 	at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:268)
> 	at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:255)
> 	at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88)
> 	at org.jboss.netty.channel.socket.nio.AbstractNioWorker.process(AbstractNioWorker.java:109)
> 	at org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:312)
> 	at org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:90)
> 	at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178)
> 	at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
> 	at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> "New I/O worker #51":
> 	at org.apache.zookeeper.server.ServerCnxn.dumpConnectionInfo(ServerCnxn.java:407)
> 	- waiting to lock <0x00000000fab68178> (a org.apache.zookeeper.server.NettyServerCnxn)
> 	at org.apache.zookeeper.server.NettyServerCnxn$StatCommand.commandRun(NettyServerCnxn.java:478)
> 	at org.apache.zookeeper.server.NettyServerCnxn$CommandThread.run(NettyServerCnxn.java:311)
> 	at org.apache.zookeeper.server.NettyServerCnxn$CommandThread.start(NettyServerCnxn.java:306)
> 	at org.apache.zookeeper.server.NettyServerCnxn.checkFourLetterWord(NettyServerCnxn.java:677)
> 	at org.apache.zookeeper.server.NettyServerCnxn.receiveMessage(NettyServerCnxn.java:790)
> 	at org.apache.zookeeper.server.NettyServerCnxnFactory$CnxnChannelHandler.processMessage(NettyServerCnxnFactory.java:211)
> 	at org.apache.zookeeper.server.NettyServerCnxnFactory$CnxnChannelHandler.messageReceived(NettyServerCnxnFactory.java:135)
> 	- locked <0x00000000fabc01b8> (a org.apache.zookeeper.server.NettyServerCnxn)
> 	at org.jboss.netty.channel.SimpleChannelHandler.handleUpstream(SimpleChannelHandler.java:88)
> 	at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
> 	at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:559)
> 	at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:268)
> 	at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:255)
> 	at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88)
> 	at org.jboss.netty.channel.socket.nio.AbstractNioWorker.process(AbstractNioWorker.java:109)
> 	at org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:312)
> 	at org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:90)
> 	at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178)
> 	at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
> 	at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> {noformat}
> It would appear that the synchronization on the {{NettyServerCnxn}} in {{NettyServerCnxnFactory}}
is to blame (and I can see why it was done originally). I think we can just use a different
Object (and monitor) to provide mutual exclusion at Netty layer (and avoid synchronization
issues at the "application" layer).



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Mime
View raw message