[
https://issues.apache.org/jira/browse/HADOOP-13436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418013#comment-15418013
]
Xiaobing Zhou commented on HADOOP-13436:
----------------------------------------
Thank you [~daryn] and [~liuml07] for your comments. How about defining a new
interface ConnectionRetryPolicy that extends RetryPolicy with extension of
ConnectionRetryPolicy#reuseConnection? ConnectionId#equals should instead
compare ConnectionRetryPolicy#reuseConnection to conclude if a new connection
needs be created. Without the specific interface, there are several issues:
1. Exception checks in RetryInvocationHandler level could be tightly coupled
with those in connection level. It's error prone to mix them together since the
same RetryPolicy can be used in both levels. A good case in point is
RetryUtils#getDefaultRetryPolicy. It composed checks of ServiceException and
IOException, whereas ServiceException is never thrown in connection level.
2. For the same reason, it's hard to correctly implement retry logic in those
exceptions dependent retry polices, e.g. ExceptionDependentRetry,
OtherThanRemoteExceptionDependentRetry and RemoteExceptionDependentRetry.
We need to avoid the jumbo retry policy anyway.
I will start with a couple of sub tasks addressing the issue aforementioned.
Thank you [~jingzhao] very much for your suggestions.
> RPC connections are leaking due to missing equals override in
> RetryUtils#getDefaultRetryPolicy
> ----------------------------------------------------------------------------------------------
>
> Key: HADOOP-13436
> URL: https://issues.apache.org/jira/browse/HADOOP-13436
> Project: Hadoop Common
> Issue Type: Bug
> Components: ipc
> Affects Versions: 2.7.1
> Reporter: Xiaobing Zhou
> Assignee: Xiaobing Zhou
> Attachments: repro.sh
>
>
> We've noticed RPC connections are increasing dramatically in a Kerberized
> HDFS cluster with {noformat}dfs.client.retry.policy.enabled{noformat}
> enabled. Internally, Client#getConnection is doing lookup relying on
> ConnectionId#equals, which composes checking Subclass-of-RetryPolicy#equals.
> If subclasses of RetryPolicy neglect overriding RetryPolicy#equals, every
> instances of RetryPolicy with equivalent fields' values (e.g.
> MultipleLinearRandomRetry[6x10000ms, 10x60000ms]) will lead to a brand new
> connection because the check will fall back to Object#equals.
> This is stack trace where RetryUtils#getDefaultRetryPolicy is called:
> {noformat}
> at
> org.apache.hadoop.io.retry.RetryUtils.getDefaultRetryPolicy(RetryUtils.java:82)
> at
> org.apache.hadoop.hdfs.NameNodeProxies.createNNProxyWithClientProtocol(NameNodeProxies.java:409)
> at
> org.apache.hadoop.hdfs.NameNodeProxies.createNonHAProxy(NameNodeProxies.java:315)
> at
> org.apache.hadoop.hdfs.NameNodeProxies.createProxy(NameNodeProxies.java:176)
> at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:678)
> at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:619)
> at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:609)
> at
> org.apache.hadoop.hdfs.server.datanode.web.webhdfs.WebHdfsHandler.newDfsClient(WebHdfsHandler.java:272)
> at
> org.apache.hadoop.hdfs.server.datanode.web.webhdfs.WebHdfsHandler.onOpen(WebHdfsHandler.java:215)
> at
> org.apache.hadoop.hdfs.server.datanode.web.webhdfs.WebHdfsHandler.handle(WebHdfsHandler.java:135)
> at
> org.apache.hadoop.hdfs.server.datanode.web.webhdfs.WebHdfsHandler$1.run(WebHdfsHandler.java:117)
> at
> org.apache.hadoop.hdfs.server.datanode.web.webhdfs.WebHdfsHandler$1.run(WebHdfsHandler.java:114)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1657)
> at
> org.apache.hadoop.hdfs.server.datanode.web.webhdfs.WebHdfsHandler.channelRead0(WebHdfsHandler.java:114)
> at
> org.apache.hadoop.hdfs.server.datanode.web.URLDispatcher.channelRead0(URLDispatcher.java:52)
> at
> org.apache.hadoop.hdfs.server.datanode.web.URLDispatcher.channelRead0(URLDispatcher.java:32)
> at
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
> at
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333)
> at
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319)
> at
> io.netty.channel.ChannelInboundHandlerAdapter.channelRead(ChannelInboundHandlerAdapter.java:86)
> at
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333)
> at
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319)
> at
> io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:163)
> at
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333)
> at
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319)
> at
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:787)
> at
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:130)
> at
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
> at
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
> at
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
> at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
> at
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:116)
> at
> io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> Three options to fix the problem:
> 1. All subclasses of RetryPolicy must override equals and hashCode to deliver
> less discriminating equivalence relation, i.e. they are equal if they have
> meaningful equivalent fields' values (e.g.
> MultipleLinearRandomRetry[6x10000ms, 10x60000ms])
> 2. Change ConnectionId#equals by removing RetryPolicy#equals compoment.
> 3. Let WebHDFS reuse the DFSClient.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]