Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 1B8FB200C28 for ; Mon, 13 Mar 2017 16:37:49 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 1A3BC160B60; Mon, 13 Mar 2017 15:37:49 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 457E6160B85 for ; Mon, 13 Mar 2017 16:37:48 +0100 (CET) Received: (qmail 57523 invoked by uid 500); 13 Mar 2017 15:37:46 -0000 Mailing-List: contact dev-help@zookeeper.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@zookeeper.apache.org Delivered-To: mailing list dev@zookeeper.apache.org Received: (qmail 57511 invoked by uid 99); 13 Mar 2017 15:37:46 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 13 Mar 2017 15:37:46 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 95131CA87F for ; Mon, 13 Mar 2017 15:37:46 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 1.451 X-Spam-Level: * X-Spam-Status: No, score=1.451 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RP_MATCHES_RCVD=-0.001, SPF_NEUTRAL=0.652] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id vlRu2-MGB_1E for ; Mon, 13 Mar 2017 15:37:45 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id 333FE60FE2 for ; Mon, 13 Mar 2017 15:37:44 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 29B52E0BB3 for ; Mon, 13 Mar 2017 15:37:43 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id D6D0E243BE for ; Mon, 13 Mar 2017 15:37:41 +0000 (UTC) Date: Mon, 13 Mar 2017 15:37:41 +0000 (UTC) From: "Josh Elser (JIRA)" To: dev@zookeeper.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Assigned] (ZOOKEEPER-2711) Deadlock between concurrent 4LW commands that iterate over connections with Netty server MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Mon, 13 Mar 2017 15:37:49 -0000 [ https://issues.apache.org/jira/browse/ZOOKEEPER-2711?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Josh Elser reassigned ZOOKEEPER-2711: ------------------------------------- Assignee: Josh Elser > 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 > Assignee: 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)