Uploaded image for project: 'Hadoop HDFS'
  1. Hadoop HDFS
  2. HDFS-12737

Thousands of sockets lingering in TIME_WAIT state due to frequent file open operations

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • None
    • None
    • ipc
    • None
    • CDH5.10.2, HBase Multi-WAL=2, 250 replication peers

    Description

      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.

      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);
      

      This piece of code is used in DistributedFileSystem#open()

      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)
      

      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.

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              weichiu Wei-Chiu Chuang
              Votes:
              1 Vote for this issue
              Watchers:
              17 Start watching this issue

              Dates

                Created:
                Updated: