From dev-return-74689-archive-asf-public=cust-asf.ponee.io@zookeeper.apache.org Tue Oct 16 12:05:11 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 8B768180649 for ; Tue, 16 Oct 2018 12:05:10 +0200 (CEST) Received: (qmail 24446 invoked by uid 500); 16 Oct 2018 10:05:09 -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 24435 invoked by uid 99); 16 Oct 2018 10:05:09 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 16 Oct 2018 10:05:09 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id B11C21A0584 for ; Tue, 16 Oct 2018 10:05:08 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -109.501 X-Spam-Level: X-Spam-Status: No, score=-109.501 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, KAM_ASCII_DIVIDERS=0.8, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, USER_IN_DEF_SPF_WL=-7.5, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id 27RV7AXzE7v5 for ; Tue, 16 Oct 2018 10:05:02 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id 706065F478 for ; Tue, 16 Oct 2018 10:05:01 +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 B3F40E0E87 for ; Tue, 16 Oct 2018 10:05:00 +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 706A524DEA for ; Tue, 16 Oct 2018 10:05:00 +0000 (UTC) Date: Tue, 16 Oct 2018 10:05:00 +0000 (UTC) From: "maoling (JIRA)" To: dev@zookeeper.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (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: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/ZOOKEEPER-2711?page=3Dcom.atla= ssian.jira.plugin.system.issuetabpanels:all-tabpanel ] maoling updated ZOOKEEPER-2711: ------------------------------- Description:=20 Observed the following issue in some $dayjob testing environments. Line num= bers 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 s= ynchronizes on the {{NettyServerCnxn}} object. However, with some 4LW comma= nds (like {{stat}}), each {{ServerCnxn}} object is also synchronized to (sa= fely) iterate over the internal contents of the object to generate the nece= ssary debug message. As such, multiple concurrent {{stat}} commands can bot= h lock their own {{NettyServerCnxn}} objects, and then be blocked waiting t= o lock each others' {{ServerCnxn}} in the {{StatCommand}}, deadlocked. {noformat} "New I/O worker #55": =09at org.apache.zookeeper.server.ServerCnxn.dumpConnectionInfo(ServerCnxn.= java:407) =09- waiting to lock <0x00000000fabc01b8> (a org.apache.zookeeper.server.Ne= ttyServerCnxn) =09at org.apache.zookeeper.server.NettyServerCnxn$StatCommand.commandRun(Ne= ttyServerCnxn.java:478) =09at org.apache.zookeeper.server.NettyServerCnxn$CommandThread.run(NettySe= rverCnxn.java:311) =09at org.apache.zookeeper.server.NettyServerCnxn$CommandThread.start(Netty= ServerCnxn.java:306) =09at org.apache.zookeeper.server.NettyServerCnxn.checkFourLetterWord(Netty= ServerCnxn.java:677) =09at org.apache.zookeeper.server.NettyServerCnxn.receiveMessage(NettyServe= rCnxn.java:790) =09at org.apache.zookeeper.server.NettyServerCnxnFactory$CnxnChannelHandler= .processMessage(NettyServerCnxnFactory.java:211) =09at org.apache.zookeeper.server.NettyServerCnxnFactory$CnxnChannelHandler= .messageReceived(NettyServerCnxnFactory.java:135) =09- locked <0x00000000fab68178> (a org.apache.zookeeper.server.NettyServer= Cnxn) =09at org.jboss.netty.channel.SimpleChannelHandler.handleUpstream(SimpleCha= nnelHandler.java:88) =09at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultCh= annelPipeline.java:564) =09at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultCh= annelPipeline.java:559) =09at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:26= 8) =09at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:25= 5) =09at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88) =09at org.jboss.netty.channel.socket.nio.AbstractNioWorker.process(Abstract= NioWorker.java:109) =09at org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNi= oSelector.java:312) =09at org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioW= orker.java:90) =09at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178) =09at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnabl= e.java:108) =09at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProof= Worker.java:42) =09at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.= java:1142) =09at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor= .java:617) =09at java.lang.Thread.run(Thread.java:745) "New I/O worker #51": =09at org.apache.zookeeper.server.ServerCnxn.dumpConnectionInfo(ServerCnxn.= java:407) =09- waiting to lock <0x00000000fab68178> (a org.apache.zookeeper.server.Ne= ttyServerCnxn) =09at org.apache.zookeeper.server.NettyServerCnxn$StatCommand.commandRun(Ne= ttyServerCnxn.java:478) =09at org.apache.zookeeper.server.NettyServerCnxn$CommandThread.run(NettySe= rverCnxn.java:311) =09at org.apache.zookeeper.server.NettyServerCnxn$CommandThread.start(Netty= ServerCnxn.java:306) =09at org.apache.zookeeper.server.NettyServerCnxn.checkFourLetterWord(Netty= ServerCnxn.java:677) =09at org.apache.zookeeper.server.NettyServerCnxn.receiveMessage(NettyServe= rCnxn.java:790) =09at org.apache.zookeeper.server.NettyServerCnxnFactory$CnxnChannelHandler= .processMessage(NettyServerCnxnFactory.java:211) =09at org.apache.zookeeper.server.NettyServerCnxnFactory$CnxnChannelHandler= .messageReceived(NettyServerCnxnFactory.java:135) =09- locked <0x00000000fabc01b8> (a org.apache.zookeeper.server.NettyServer= Cnxn) =09at org.jboss.netty.channel.SimpleChannelHandler.handleUpstream(SimpleCha= nnelHandler.java:88) =09at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultCh= annelPipeline.java:564) =09at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultCh= annelPipeline.java:559) =09at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:26= 8) =09at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:25= 5) =09at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88) =09at org.jboss.netty.channel.socket.nio.AbstractNioWorker.process(Abstract= NioWorker.java:109) =09at org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNi= oSelector.java:312) =09at org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioW= orker.java:90) =09at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178) =09at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnabl= e.java:108) =09at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProof= Worker.java:42) =09at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.= java:1142) =09at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor= .java:617) =09at java.lang.Thread.run(Thread.java:745) {noformat} It would appear that the synchronization on the {{NettyServerCnxn}} in {{Ne= ttyServerCnxnFactory}} is to blame (and I can see why it was done originall= y). I think we can just use a different Object (and monitor) to provide mut= ual exclusion at Netty layer (and avoid synchronization issues at the "appl= ication" layer). =C2=A0 was: Observed the following issue in some $dayjob testing environments. Line num= bers 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 s= ynchronizes on the {{NettyServerCnxn}} object. However, with some 4LW comma= nds (like {{stat}}), each {{ServerCnxn}} object is also synchronized to (sa= fely) iterate over the internal contents of the object to generate the nece= ssary debug message. As such, multiple concurrent {{stat}} commands can bot= h lock their own {{NettyServerCnxn}} objects, and then be blocked waiting t= o lock each others' {{ServerCnxn}} in the {{StatCommand}}, deadlocked. {noformat} "New I/O worker #55": =09at org.apache.zookeeper.server.ServerCnxn.dumpConnectionInfo(ServerCnxn.= java:407) =09- waiting to lock <0x00000000fabc01b8> (a org.apache.zookeeper.server.Ne= ttyServerCnxn) =09at org.apache.zookeeper.server.NettyServerCnxn$StatCommand.commandRun(Ne= ttyServerCnxn.java:478) =09at org.apache.zookeeper.server.NettyServerCnxn$CommandThread.run(NettySe= rverCnxn.java:311) =09at org.apache.zookeeper.server.NettyServerCnxn$CommandThread.start(Netty= ServerCnxn.java:306) =09at org.apache.zookeeper.server.NettyServerCnxn.checkFourLetterWord(Netty= ServerCnxn.java:677) =09at org.apache.zookeeper.server.NettyServerCnxn.receiveMessage(NettyServe= rCnxn.java:790) =09at org.apache.zookeeper.server.NettyServerCnxnFactory$CnxnChannelHandler= .processMessage(NettyServerCnxnFactory.java:211) =09at org.apache.zookeeper.server.NettyServerCnxnFactory$CnxnChannelHandler= .messageReceived(NettyServerCnxnFactory.java:135) =09- locked <0x00000000fab68178> (a org.apache.zookeeper.server.NettyServer= Cnxn) =09at org.jboss.netty.channel.SimpleChannelHandler.handleUpstream(SimpleCha= nnelHandler.java:88) =09at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultCh= annelPipeline.java:564) =09at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultCh= annelPipeline.java:559) =09at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:26= 8) =09at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:25= 5) =09at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88) =09at org.jboss.netty.channel.socket.nio.AbstractNioWorker.process(Abstract= NioWorker.java:109) =09at org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNi= oSelector.java:312) =09at org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioW= orker.java:90) =09at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178) =09at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnabl= e.java:108) =09at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProof= Worker.java:42) =09at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.= java:1142) =09at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor= .java:617) =09at java.lang.Thread.run(Thread.java:745) "New I/O worker #51": =09at org.apache.zookeeper.server.ServerCnxn.dumpConnectionInfo(ServerCnxn.= java:407) =09- waiting to lock <0x00000000fab68178> (a org.apache.zookeeper.server.Ne= ttyServerCnxn) =09at org.apache.zookeeper.server.NettyServerCnxn$StatCommand.commandRun(Ne= ttyServerCnxn.java:478) =09at org.apache.zookeeper.server.NettyServerCnxn$CommandThread.run(NettySe= rverCnxn.java:311) =09at org.apache.zookeeper.server.NettyServerCnxn$CommandThread.start(Netty= ServerCnxn.java:306) =09at org.apache.zookeeper.server.NettyServerCnxn.checkFourLetterWord(Netty= ServerCnxn.java:677) =09at org.apache.zookeeper.server.NettyServerCnxn.receiveMessage(NettyServe= rCnxn.java:790) =09at org.apache.zookeeper.server.NettyServerCnxnFactory$CnxnChannelHandler= .processMessage(NettyServerCnxnFactory.java:211) =09at org.apache.zookeeper.server.NettyServerCnxnFactory$CnxnChannelHandler= .messageReceived(NettyServerCnxnFactory.java:135) =09- locked <0x00000000fabc01b8> (a org.apache.zookeeper.server.NettyServer= Cnxn) =09at org.jboss.netty.channel.SimpleChannelHandler.handleUpstream(SimpleCha= nnelHandler.java:88) =09at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultCh= annelPipeline.java:564) =09at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultCh= annelPipeline.java:559) =09at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:26= 8) =09at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:25= 5) =09at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88) =09at org.jboss.netty.channel.socket.nio.AbstractNioWorker.process(Abstract= NioWorker.java:109) =09at org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNi= oSelector.java:312) =09at org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioW= orker.java:90) =09at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178) =09at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnabl= e.java:108) =09at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProof= Worker.java:42) =09at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.= java:1142) =09at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor= .java:617) =09at java.lang.Thread.run(Thread.java:745) {noformat} It would appear that the synchronization on the {{NettyServerCnxn}} in {{Ne= ttyServerCnxnFactory}} is to blame (and I can see why it was done originall= y). I think we can just use a different Object (and monitor) to provide mut= ual exclusion at Netty layer (and avoid synchronization issues at the "appl= ication" layer). > Deadlock between concurrent 4LW commands that iterate over connections wi= th 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 n= umbers are a little off compared to master/branch-3.5, but I did confirm th= e same issue exists there. > With the NettyServerCnxnFactory, before a request is dispatched, the code= synchronizes on the {{NettyServerCnxn}} object. However, with some 4LW com= mands (like {{stat}}), each {{ServerCnxn}} object is also synchronized to (= safely) iterate over the internal contents of the object to generate the ne= cessary debug message. As such, multiple concurrent {{stat}} commands can b= oth 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": > =09at org.apache.zookeeper.server.ServerCnxn.dumpConnectionInfo(ServerCnx= n.java:407) > =09- waiting to lock <0x00000000fabc01b8> (a org.apache.zookeeper.server.= NettyServerCnxn) > =09at org.apache.zookeeper.server.NettyServerCnxn$StatCommand.commandRun(= NettyServerCnxn.java:478) > =09at org.apache.zookeeper.server.NettyServerCnxn$CommandThread.run(Netty= ServerCnxn.java:311) > =09at org.apache.zookeeper.server.NettyServerCnxn$CommandThread.start(Net= tyServerCnxn.java:306) > =09at org.apache.zookeeper.server.NettyServerCnxn.checkFourLetterWord(Net= tyServerCnxn.java:677) > =09at org.apache.zookeeper.server.NettyServerCnxn.receiveMessage(NettySer= verCnxn.java:790) > =09at org.apache.zookeeper.server.NettyServerCnxnFactory$CnxnChannelHandl= er.processMessage(NettyServerCnxnFactory.java:211) > =09at org.apache.zookeeper.server.NettyServerCnxnFactory$CnxnChannelHandl= er.messageReceived(NettyServerCnxnFactory.java:135) > =09- locked <0x00000000fab68178> (a org.apache.zookeeper.server.NettyServ= erCnxn) > =09at org.jboss.netty.channel.SimpleChannelHandler.handleUpstream(SimpleC= hannelHandler.java:88) > =09at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(Default= ChannelPipeline.java:564) > =09at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(Default= ChannelPipeline.java:559) > =09at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:= 268) > =09at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:= 255) > =09at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88= ) > =09at org.jboss.netty.channel.socket.nio.AbstractNioWorker.process(Abstra= ctNioWorker.java:109) > =09at org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(Abstract= NioSelector.java:312) > =09at org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNi= oWorker.java:90) > =09at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178= ) > =09at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunna= ble.java:108) > =09at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockPro= ofWorker.java:42) > =09at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecuto= r.java:1142) > =09at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecut= or.java:617) > =09at java.lang.Thread.run(Thread.java:745) > "New I/O worker #51": > =09at org.apache.zookeeper.server.ServerCnxn.dumpConnectionInfo(ServerCnx= n.java:407) > =09- waiting to lock <0x00000000fab68178> (a org.apache.zookeeper.server.= NettyServerCnxn) > =09at org.apache.zookeeper.server.NettyServerCnxn$StatCommand.commandRun(= NettyServerCnxn.java:478) > =09at org.apache.zookeeper.server.NettyServerCnxn$CommandThread.run(Netty= ServerCnxn.java:311) > =09at org.apache.zookeeper.server.NettyServerCnxn$CommandThread.start(Net= tyServerCnxn.java:306) > =09at org.apache.zookeeper.server.NettyServerCnxn.checkFourLetterWord(Net= tyServerCnxn.java:677) > =09at org.apache.zookeeper.server.NettyServerCnxn.receiveMessage(NettySer= verCnxn.java:790) > =09at org.apache.zookeeper.server.NettyServerCnxnFactory$CnxnChannelHandl= er.processMessage(NettyServerCnxnFactory.java:211) > =09at org.apache.zookeeper.server.NettyServerCnxnFactory$CnxnChannelHandl= er.messageReceived(NettyServerCnxnFactory.java:135) > =09- locked <0x00000000fabc01b8> (a org.apache.zookeeper.server.NettyServ= erCnxn) > =09at org.jboss.netty.channel.SimpleChannelHandler.handleUpstream(SimpleC= hannelHandler.java:88) > =09at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(Default= ChannelPipeline.java:564) > =09at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(Default= ChannelPipeline.java:559) > =09at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:= 268) > =09at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:= 255) > =09at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88= ) > =09at org.jboss.netty.channel.socket.nio.AbstractNioWorker.process(Abstra= ctNioWorker.java:109) > =09at org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(Abstract= NioSelector.java:312) > =09at org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNi= oWorker.java:90) > =09at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178= ) > =09at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunna= ble.java:108) > =09at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockPro= ofWorker.java:42) > =09at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecuto= r.java:1142) > =09at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecut= or.java:617) > =09at 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 origina= lly). I think we can just use a different Object (and monitor) to provide m= utual exclusion at Netty layer (and avoid synchronization issues at the "ap= plication" layer). > =C2=A0 -- This message was sent by Atlassian JIRA (v7.6.3#76005)