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

Kihwal Lee commented on HDFS-12737:
-----------------------------------

If the same user (i.e. the same UGI instance) talks to the same namenode via 
RPC, a connection will be shared.  If the files system cache is disabled, the 
sharing cannot happen. Are those requests being made by separate processes?

> Thousands of sockets lingering in TIME_WAIT state due to frequent file open 
> operations
> --------------------------------------------------------------------------------------
>
>                 Key: HDFS-12737
>                 URL: https://issues.apache.org/jira/browse/HDFS-12737
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: ipc
>         Environment: CDH5.10.2, HBase Multi-WAL=2, 250 replication peers
>            Reporter: Wei-Chiu Chuang
>            Assignee: Wei-Chiu Chuang
>
> On a HBase cluster we found HBase RegionServers have thousands of sockets in 
> TIME_WAIT state. It depleted system resources and caused other services to 
> fail.
> After months of troubleshooting, we found the issue is the cluster has 
> hundreds of replication peers, and has multi-WAL = 2. That creates hundreds 
> of replication threads in HBase RS, and each thread opens WAL file *every 
> second*.
> We found that the IPC client closes socket right away, and does not reuse 
> socket connection. Since each closed socket stays in TIME_WAIT state for 60 
> seconds in Linux by default, that generates thousands of TIME_WAIT sockets.
> {code:title=ClientDatanodeProtocolTranslatorPB:createClientDatanodeProtocolProxy}
>     // Since we're creating a new UserGroupInformation here, we know that no
>     // future RPC proxies will be able to re-use the same connection. And
>     // usages of this proxy tend to be one-off calls.
>     //
>     // This is a temporary fix: callers should really achieve this by using
>     // RPC.stopProxy() on the resulting object, but this is currently not
>     // working in trunk. See the discussion on HDFS-1965.
>     Configuration confWithNoIpcIdle = new Configuration(conf);
>     confWithNoIpcIdle.setInt(CommonConfigurationKeysPublic
>         .IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, 0);
> {code}
> This piece of code is used in DistributedFileSystem#open()
> {noformat}
> 2017-10-27 14:01:44,152 DEBUG org.apache.hadoop.ipc.Client: New connection 
> Thread[IPC Client (1838187805) connection to /172.131.21.48:20001 from 
> blk_1013754707_14032,5,main] for remoteId /172.131.21.48:20001
> java.lang.Throwable: For logging stack trace, not a real exception
>         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1556)
>         at org.apache.hadoop.ipc.Client.call(Client.java:1482)
>         at org.apache.hadoop.ipc.Client.call(Client.java:1443)
>         at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:230)
>         at com.sun.proxy.$Proxy28.getReplicaVisibleLength(Unknown Source)
>         at 
> org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB.getReplicaVisibleLength(ClientDatanodeProtocolTranslatorPB.java:198)
>         at 
> org.apache.hadoop.hdfs.DFSInputStream.readBlockLength(DFSInputStream.java:365)
>         at 
> org.apache.hadoop.hdfs.DFSInputStream.fetchLocatedBlocksAndGetLastBlockLength(DFSInputStream.java:335)
>         at 
> org.apache.hadoop.hdfs.DFSInputStream.openInfo(DFSInputStream.java:271)
>         at 
> org.apache.hadoop.hdfs.DFSInputStream.<init>(DFSInputStream.java:263)
>         at org.apache.hadoop.hdfs.DFSClient.open(DFSClient.java:1585)
>         at 
> org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(DistributedFileSystem.java:326)
>         at 
> org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(DistributedFileSystem.java:322)
>         at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>         at 
> org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:322)
>         at 
> org.apache.hadoop.fs.FilterFileSystem.open(FilterFileSystem.java:162)
>         at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:783)
>         at 
> org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:293)
>         at 
> org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:267)
>         at 
> org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:255)
>         at 
> org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:414)
>         at 
> org.apache.hadoop.hbase.replication.regionserver.ReplicationWALReaderManager.openReader(ReplicationWALReaderManager.java:70)
>         at 
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSource$ReplicationSourceWorkerThread.openReader(ReplicationSource.java:747)
>         at 
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSource$ReplicationSourceWorkerThread.run(ReplicationSource.java:543)
> {noformat}
> Unfortunately, given the HBase's usage pattern, this hack creates the problem.
> Ignoring the fact that having hundreds of HBase replication peers is a bad 
> practice (I'll probably file a HBASE jira to help remedy that), the fact that 
> Hadoop IPC client does not reuse socket seems not right. The relevant code is 
> historical and deep in the stack, so I'd like to invite comments. I have a 
> patch but it's pretty hacky.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to