[ 
https://issues.apache.org/jira/browse/HADOOP-13436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15398070#comment-15398070
 ] 

Daryn Sharp commented on HADOOP-13436:
--------------------------------------

We encountered this exact problem, fixed it, but apparently haven't pushed back 
to the community.  I think #1 is the correct approach.  Below is the quick & 
dirty patch we used.  I'd suggest scrubbing all the policies for correctness.  
Plus re-building strings for hashCode/equals is a horrible thing that should be 
changed.

{code}
index 0000000..5aab5c2
--- /dev/null
+++ b/Y-CHANGES/YHADOOP-977
@@ -0,0 +1 @@
+[YHADOOP-977] Webhdfs causes datanodes to create excessive connections.
index fab406d..032e38e 100644
--- 
a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryUtils.java
+++ 
b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryUtils.java
@@ -115,6 +115,16 @@ public RetryAction shouldRetry(Exception e, int retries, 
int failovers,
         }
 
         @Override
+        public int hashCode() {
+          return multipleLinearRandomRetry.hashCode();
+        }
+
+        @Override
+        public boolean equals(final Object that) {
+          return this.toString().equals(that.toString());
+        }
+
+        @Override
         public String toString() {
           return "RetryPolicy[" + multipleLinearRandomRetry + ", "
               + RetryPolicies.TRY_ONCE_THEN_FAIL.getClass().getSimpleName();
{code}


> 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 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:
> {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: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to