Uploaded image for project: 'Hadoop Common'
  1. Hadoop Common
  2. HADOOP-13436

RPC connections are leaking due to not overriding hashCode and equals

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 2.7.1
    • None
    • ipc
    • None

    Description

      We've noticed RPC connections are increasing dramatically in a Kerberized HDFS cluster with

      dfs.client.retry.policy.enabled

      enabled. Internally, Client#getConnection is doing lookup relying on ConnectionId #hashCode and then #equals, which compose checking Subclass-of-RetryPolicy #hashCode and #equals. If subclasses of RetryPolicy neglect overriding #hashCode or #equals, every instance 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#hashCode and Object#equals which is distinct and false for distinct instances.

      This is stack trace where the anonymous RetryPolicy implementation (neglecting overriding hashCode and equals) in RetryUtils#getDefaultRetryPolicy is called:

      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)
      

      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.

      Attachments

        1. repro.sh
          1 kB
          Xiaobing Zhou
        2. Proposal-of-Fixing-Connection-Leakage.pdf
          73 kB
          Xiaobing Zhou

        Issue Links

          Activity

            People

              xiaobingo Xiaobing Zhou
              xiaobingo Xiaobing Zhou
              Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

                Created:
                Updated: