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 E2452200C39 for ; Thu, 16 Mar 2017 20:55:49 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id E0EBE160B8B; Thu, 16 Mar 2017 19:55: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 1246B160B72 for ; Thu, 16 Mar 2017 20:55:48 +0100 (CET) Received: (qmail 1875 invoked by uid 500); 16 Mar 2017 19:55:48 -0000 Mailing-List: contact dev-help@giraph.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@giraph.apache.org Delivered-To: mailing list dev@giraph.apache.org Received: (qmail 1863 invoked by uid 500); 16 Mar 2017 19:55:48 -0000 Delivered-To: apmail-incubator-giraph-dev@incubator.apache.org Received: (qmail 1859 invoked by uid 99); 16 Mar 2017 19:55:48 -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; Thu, 16 Mar 2017 19:55:48 +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 C68961A0491 for ; Thu, 16 Mar 2017 19:55:47 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-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 (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id s1x_-zGikiZn for ; Thu, 16 Mar 2017 19:55: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 899D560E09 for ; Thu, 16 Mar 2017 19:55:45 +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 86B5EE0B21 for ; Thu, 16 Mar 2017 19:55:42 +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 DF1D6254C5 for ; Thu, 16 Mar 2017 19:55:41 +0000 (UTC) Date: Thu, 16 Mar 2017 19:55:41 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: giraph-dev@incubator.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (GIRAPH-1137) Remove channel probing from Netty worker thread for credit-based flow-control MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Thu, 16 Mar 2017 19:55:50 -0000 [ https://issues.apache.org/jira/browse/GIRAPH-1137?page=3Dcom.atlassia= n.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=3D159= 28741#comment-15928741 ]=20 ASF GitHub Bot commented on GIRAPH-1137: ---------------------------------------- GitHub user heslami opened a pull request: https://github.com/apache/giraph/pull/26 [GIRAPH-1137] Remove channel probing from Netty worker thread for credi= t-based flow=E2=80=A6 In credit-based flow-control, sometimes, client threads (one type of Ne= tty worker threads used in Giraph) try to send requests to other workers. T= his is bad practice for Netty and can cause Netty to mark the execution as = deadlock-prone (an example exception shown below). Client threads should on= ly be responsible for sending ACK/NACK messages in response to requests, an= d they should do so by reuseing the channel from which they received the re= quest. In the current implementation, client threads may try to send unsent= /cached requests in credit-based flow control. Sending such requests should= be delegated to other threads. =20 WARN 2017-03-08 06:06:22,104 [netty-client-worker-3] .... io.netty.util.concurrent.BlockingOperationException: DefaultChannelProm= ise@2c455378(incomplete) at io.netty.util.concurrent.DefaultPromise.checkDeadLock(DefaultPromise= .java:383) at io.netty.channel.DefaultChannelPromise.checkDeadLock(DefaultChannelP= romise.java:157) at io.netty.util.concurrent.DefaultPromise.await0(DefaultPromise.java:3= 43) at io.netty.util.concurrent.DefaultPromise.await(DefaultPromise.java:25= 9) at org.apache.giraph.utils.ProgressableUtils$ChannelFutureWaitable.wait= For(ProgressableUtils.java:461) at org.apache.giraph.utils.ProgressableUtils.waitFor(ProgressableUtils.= java:214) at org.apache.giraph.utils.ProgressableUtils.waitForever(ProgressableUt= ils.java:180) at org.apache.giraph.utils.ProgressableUtils.waitForever(ProgressableUt= ils.java:165) at org.apache.giraph.utils.ProgressableUtils.awaitChannelFuture(Progres= sableUtils.java:132) at org.apache.giraph.comm.netty.NettyClient.getNextChannel(NettyClient.= java:715) at org.apache.giraph.comm.netty.NettyClient.writeRequestToChannel(Netty= Client.java:799) at org.apache.giraph.comm.netty.NettyClient.doSend(NettyClient.java:789= ) at org.apache.giraph.comm.flow_control.CreditBasedFlowControl.trySendCa= chedRequests(CreditBasedFlowControl.java:515) at org.apache.giraph.comm.flow_control.CreditBasedFlowControl.messageAc= kReceived(CreditBasedFlowControl.java:485) at org.apache.giraph.comm.netty.NettyClient.messageReceived(NettyClient= .java:840) at org.apache.giraph.comm.netty.handler.ResponseClientHandler.channelRe= ad(ResponseClientHandler.java:87) at io.netty.channel.DefaultChannelHandlerContext.invokeChannelRead(Defa= ultChannelHandlerContext.java:338) at io.netty.channel.DefaultChannelHandlerContext.fireChannelRead(Defaul= tChannelHandlerContext.java:324) at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessag= eDecoder.java:153) at io.netty.channel.DefaultChannelHandlerContext.invokeChannelRead(Defa= ultChannelHandlerContext.java:338) at io.netty.channel.DefaultChannelHandlerContext.fireChannelRead(Defaul= tChannelHandlerContext.java:324) at org.apache.giraph.comm.netty.InboundByteCounter.channelRead(InboundB= yteCounter.java:89) at io.netty.channel.DefaultChannelHandlerContext.invokeChannelRead(Defa= ultChannelHandlerContext.java:338) at io.netty.channel.DefaultChannelHandlerContext.fireChannelRead(Defaul= tChannelHandlerContext.java:324) at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChann= elPipeline.java:785) at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(Abstr= actNioByteChannel.java:126) at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.ja= va:485) at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEv= entLoop.java:452) at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:346) at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThrea= dEventExecutor.java:101) at java.lang.Thread.run(Thread.java:745) You can merge this pull request into a Git repository by running: $ git pull https://github.com/heslami/giraph fix-credit-based Alternatively you can review and apply these changes as the patch at: https://github.com/apache/giraph/pull/26.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 #26 =20 ---- commit 4c8186cc8097877d5af20ef054630a629caaa026 Author: Hassan Eslami Date: 2017-03-16T19:52:12Z Remove channel probing from Netty worker thread for credit-based flow-c= ontrol =20 Closes GIRAPH-1137 ---- > Remove channel probing from Netty worker thread for credit-based flow-con= trol > -------------------------------------------------------------------------= ---- > > Key: GIRAPH-1137 > URL: https://issues.apache.org/jira/browse/GIRAPH-1137 > Project: Giraph > Issue Type: Bug > Reporter: Hassan Eslami > Assignee: Hassan Eslami > > In credit-based flow-control, sometimes, client threads (one type of Nett= y worker threads used in Giraph) try to send requests to other workers. Thi= s is bad practice for Netty and can cause Netty to mark the execution as de= adlock-prone (an example exception shown below). Client threads should only= be responsible for sending ACK/NACK messages in response to requests, and = they should do so by reuseing the channel from which they received the requ= est. In the current implementation, client threads may try to send unsent/c= ached requests in credit-based flow control. Sending such requests should b= e delegated to other threads. > WARN 2017-03-08 06:06:22,104 [netty-client-worker-3] .... > io.netty.util.concurrent.BlockingOperationException: DefaultChannelPromis= e@2c455378(incomplete) > at io.netty.util.concurrent.DefaultPromise.checkDeadLock(DefaultPromise.j= ava:383) > at io.netty.channel.DefaultChannelPromise.checkDeadLock(DefaultChannelPro= mise.java:157) > at io.netty.util.concurrent.DefaultPromise.await0(DefaultPromise.java:343= ) > at io.netty.util.concurrent.DefaultPromise.await(DefaultPromise.java:259) > at org.apache.giraph.utils.ProgressableUtils$ChannelFutureWaitable.waitFo= r(ProgressableUtils.java:461) > at org.apache.giraph.utils.ProgressableUtils.waitFor(ProgressableUtils.ja= va:214) > at org.apache.giraph.utils.ProgressableUtils.waitForever(ProgressableUtil= s.java:180) > at org.apache.giraph.utils.ProgressableUtils.waitForever(ProgressableUtil= s.java:165) > at org.apache.giraph.utils.ProgressableUtils.awaitChannelFuture(Progressa= bleUtils.java:132) > at org.apache.giraph.comm.netty.NettyClient.getNextChannel(NettyClient.ja= va:715) > at org.apache.giraph.comm.netty.NettyClient.writeRequestToChannel(NettyCl= ient.java:799) > at org.apache.giraph.comm.netty.NettyClient.doSend(NettyClient.java:789) > at org.apache.giraph.comm.flow_control.CreditBasedFlowControl.trySendCach= edRequests(CreditBasedFlowControl.java:515) > at org.apache.giraph.comm.flow_control.CreditBasedFlowControl.messageAckR= eceived(CreditBasedFlowControl.java:485) > at org.apache.giraph.comm.netty.NettyClient.messageReceived(NettyClient.j= ava:840) > at org.apache.giraph.comm.netty.handler.ResponseClientHandler.channelRead= (ResponseClientHandler.java:87) > at io.netty.channel.DefaultChannelHandlerContext.invokeChannelRead(Defaul= tChannelHandlerContext.java:338) > at io.netty.channel.DefaultChannelHandlerContext.fireChannelRead(DefaultC= hannelHandlerContext.java:324) > at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageD= ecoder.java:153) > at io.netty.channel.DefaultChannelHandlerContext.invokeChannelRead(Defaul= tChannelHandlerContext.java:338) > at io.netty.channel.DefaultChannelHandlerContext.fireChannelRead(DefaultC= hannelHandlerContext.java:324) > at org.apache.giraph.comm.netty.InboundByteCounter.channelRead(InboundByt= eCounter.java:89) > at io.netty.channel.DefaultChannelHandlerContext.invokeChannelRead(Defaul= tChannelHandlerContext.java:338) > at io.netty.channel.DefaultChannelHandlerContext.fireChannelRead(DefaultC= hannelHandlerContext.java:324) > at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannel= Pipeline.java:785) > at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(Abstrac= tNioByteChannel.java:126) > at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java= :485) > at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEven= tLoop.java:452) > at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:346) > at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadE= ventExecutor.java:101) > at java.lang.Thread.run(Thread.java:745) -- This message was sent by Atlassian JIRA (v6.3.15#6346)