[ https://issues.apache.org/jira/browse/HDFS-12737?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17721964#comment-17721964 ]
Dheeren Beborrtha commented on HDFS-12737: ------------------------------------------ We are observing this issue in Hbase cluster of around 75 RSs experiencing the issue where Region Server is littered with following logs: {noformat} 2023-05-09 18:47:46,092 WARN [RpcServer.default.FPBQ.Fifo.handler=27,queue=3,port=16020] hdfs.DFSClient: Connection failure: Failed to connect to hbase1wn41-0.subnetpoc1.vcn12231050.oraclevcn.com/10.1.64.234:1019 for file /apps/hbase/data/data/default/usertable2/fe172ff893d8afcf20c008e3765077da/cf/921cfad177b0434a957079cd4506c834 for block BP-1395570538-10.1.21.157-1682117242080:blk_1093623349_19885353:org.apache.hadoop.net.ConnectTimeoutException: 60000 millis timeout while waiting for channel to be ready for connect. ch : java.nio.channels.SocketChannel[connection-pending remote=hbase1wn41-0.subnetpoc1.vcn12231050.oraclevcn.com/10.1.64.234:1019] org.apache.hadoop.net.ConnectTimeoutException: 60000 millis timeout while waiting for channel to be ready for connect. ch : java.nio.channels.SocketChannel[connection-pending remote=hbase1wn41-0.subnetpoc1.vcn12231050.oraclevcn.com/10.1.64.234:1019] at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:589) at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:3033) at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:829) at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:754) at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.build(BlockReaderFactory.java:381) at org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:755) at org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1199) at org.apache.hadoop.hdfs.DFSInputStream.fetchBlockByteRange(DFSInputStream.java:1151) at org.apache.hadoop.hdfs.DFSInputStream.pread(DFSInputStream.java:1511) at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:1475) at org.apache.hadoop.fs.FSDataInputStream.read(FSDataInputStream.java:98) at org.apache.hadoop.hbase.io.util.BlockIOUtils.preadWithExtra(BlockIOUtils.java:233) at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readAtOffset(HFileBlock.java:1456) at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockDataInternal(HFileBlock.java:1679) at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockData(HFileBlock.java:1490) at org.apache.hadoop.hbase.io.hfile.HFileReaderImpl.readBlock(HFileReaderImpl.java:1308) at org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$CellBasedKeyBlockIndexReader.loadDataBlockWithScanInfo(HFileBlockIndex.java:318) at org.apache.hadoop.hbase.io.hfile.HFileReaderImpl$HFileScannerImpl.seekTo(HFileReaderImpl.java:659) at org.apache.hadoop.hbase.io.hfile.HFileReaderImpl$HFileScannerImpl.seekTo(HFileReaderImpl.java:612) at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seekAtOrAfter(StoreFileScanner.java:306) at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seek(StoreFileScanner.java:214) at org.apache.hadoop.hbase.regionserver.StoreScanner.seekScanners(StoreScanner.java:408) at org.apache.hadoop.hbase.regionserver.StoreScanner.<init>(StoreScanner.java:253) at org.apache.hadoop.hbase.regionserver.HStore.createScanner(HStore.java:2100) at org.apache.hadoop.hbase.regionserver.HStore.getScanner(HStore.java:2091) at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.initializeScanners(HRegion.java:7049) at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.<init>(HRegion.java:7029) at org.apache.hadoop.hbase.regionserver.HRegion.instantiateRegionScanner(HRegion.java:3043) at org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:3023) at org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:3005) at org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:2999) at org.apache.hadoop.hbase.regionserver.RSRpcServices.get(RSRpcServices.java:2614) at org.apache.hadoop.hbase.regionserver.RSRpcServices.get(RSRpcServices.java:2538) at org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:45945) at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:384) at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:131){noformat} || || |[root@hbase1wn61-0 ~]# netstat -nat \| awk '\{print $6}' \| sort \| uniq -c \| sort -n 1 established) 1 Foreign 1 SYN_RECV 2 FIN_WAIT1 2 LAST_ACK 27 LISTEN 138 CLOSE_WAIT 155 ESTABLISHED 120175 TIME_WAIT| The TIME_WAIT shooting up to 200K+. > 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 > Priority: Major > > 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 (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org