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

Max Schmidt commented on HDFS-6973:
-----------------------------------

I can relate to that. We're using a org.apache.hadoop.fs.FSDataInputStream for 
reading multiple files continuously 2 times an hour from a 2.7.1 cluster. 

I've added "-Djava.net.preferIPv4Stack=true" and 
"-Djava.net.preferIPv6Addresses=false" but it only changed that the sockets are 
now ipv4 instead of ipv6.

After 12 hours usage, 1.4K open sockets:

java    10486 root 2233u  IPv4           28226850      0t0      TCP 
10.134.160.9:55927->10.134.160.28:50010 (CLOSE_WAIT)
java    10486 root 2237u  IPv4           28223758      0t0      TCP 
10.134.160.9:37363->10.134.160.17:50010 (CLOSE_WAIT)
java    10486 root 2240u  IPv4           28223759      0t0      TCP 
10.134.160.9:48976->10.134.160.41:50010 (CLOSE_WAIT)
java    10486 root 2248u  IPv4           28222398      0t0      TCP 
10.134.160.9:55976->10.134.160.28:50010 (CLOSE_WAIT)
java    10486 root 2274u  IPv4           28222403      0t0      TCP 
10.134.160.9:53185->10.134.160.35:50010 (CLOSE_WAIT)
java    10486 root 2283u  IPv4           28211085      0t0      TCP 
10.134.160.9:56009->10.134.160.28:50010 (CLOSE_WAIT)

10.134.160.9 ip of the host with the driver programm, dst-ips are the 
hadoop-nodes.

> DFSClient does not closing a closed socket resulting in thousand of 
> CLOSE_WAIT sockets
> --------------------------------------------------------------------------------------
>
>                 Key: HDFS-6973
>                 URL: https://issues.apache.org/jira/browse/HDFS-6973
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: hdfs-client
>    Affects Versions: 2.4.0
>         Environment: RHEL 6.3 -HDP 2.1 -6 RegionServers/Datanode -18T per 
> node -3108Regions
>            Reporter: steven xu
>
> HBase as HDFS Client dose not close a dead connection with the datanode.
> This resulting in over 30K+ CLOSE_WAIT and at some point HBase can not 
> connect to the datanode because too many mapped sockets from one host to 
> another on the same port:50010. 
> After I restart all RSs, the count of CLOSE_WAIT will increase always.
> $ netstat -an|grep CLOSE_WAIT|wc -l
> 2545
> netstat -nap|grep CLOSE_WAIT|grep 6569|wc -l
> 2545
> ps -ef|grep 6569
> hbase 6569 6556 21 Aug25 ? 09:52:33 /opt/jdk1.6.0_25/bin/java 
> -Dproc_regionserver -XX:OnOutOfMemoryError=kill -9 %p -Xmx1000m 
> -XX:+UseConcMarkSweepGC
> I aslo have reviewed these issues:
> [HDFS-5697]
> [HDFS-5671]
> [HDFS-1836]
> [HBASE-9393]
> I found in HBase 0.98/Hadoop 2.4.0 source codes of these patchs have been 
> added.
> But I donot understand why HBase 0.98/Hadoop 2.4.0 also have this isssue. 
> Please check. Thanks a lot.
> These codes have been added into 
> BlockReaderFactory.getRemoteBlockReaderFromTcp(). Another bug maybe lead my 
> problem,
> {code:title=BlockReaderFactory.java|borderStyle=solid}
> // Some comments here
>   private BlockReader getRemoteBlockReaderFromTcp() throws IOException {
>     if (LOG.isTraceEnabled()) {
>       LOG.trace(this + ": trying to create a remote block reader from a " +
>           "TCP socket");
>     }
>     BlockReader blockReader = null;
>     while (true) {
>       BlockReaderPeer curPeer = null;
>       Peer peer = null;
>       try {
>         curPeer = nextTcpPeer();
>         if (curPeer == null) break;
>         if (curPeer.fromCache) remainingCacheTries--;
>         peer = curPeer.peer;
>         blockReader = getRemoteBlockReader(peer);
>         return blockReader;
>       } catch (IOException ioe) {
>         if (isSecurityException(ioe)) {
>           if (LOG.isTraceEnabled()) {
>             LOG.trace(this + ": got security exception while constructing " +
>                 "a remote block reader from " + peer, ioe);
>           }
>           throw ioe;
>         }
>         if ((curPeer != null) && curPeer.fromCache) {
>           // Handle an I/O error we got when using a cached peer.  These are
>           // considered less serious, because the underlying socket may be
>           // stale.
>           if (LOG.isDebugEnabled()) {
>             LOG.debug("Closed potentially stale remote peer " + peer, ioe);
>           }
>         } else {
>           // Handle an I/O error we got when using a newly created peer.
>           LOG.warn("I/O error constructing remote block reader.", ioe);
>           throw ioe;
>         }
>       } finally {
>         if (blockReader == null) {
>           IOUtils.cleanup(LOG, peer);
>         }
>       }
>     }
>     return null;
>   }
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to