Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-31219

YarnShuffleService doesn't close idle netty channel

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.4.5, 3.0.0
    • 3.0.0, 3.1.0
    • Shuffle, Spark Core
    • None

    Description

      Recently, we find our YarnShuffleService has a lot of half-open connections where shuffle servers' connections are active while clients have already closed. 

      For example, from server's `ss -nt sport = :7337` output we have

      ESTAB 0 0 server:7337 client:port
      
      

      However, on client `ss -nt dport =: 7337 | grep server` would return nothing.

      Looking at the code,  `YarnShuffleService` creates a `TransportContext` with `closeIdleConnections` set to false.

      public class YarnShuffleService extends AuxiliaryService {
        ...
        @Override  protected void serviceInit(Configuration conf) throws Exception { 
          ...     
          transportContext = new TransportContext(transportConf, blockHandler); 
          ...
        }
        ...
      }
      
      public class TransportContext implements Closeable {
        ...
      
        public TransportContext(TransportConf conf, RpcHandler rpcHandler) {       
          this(conf, rpcHandler, false, false);  
        }
        public TransportContext(TransportConf conf, RpcHandler rpcHandler, boolean closeIdleConnections) {    
          this(conf, rpcHandler, closeIdleConnections, false);  
        }
        ...
      }

      Hence, it's possible the channel  may never get closed at server side if the server misses the event that the client has closed it.

      I find that parameter is true for `ExternalShuffleService`.

      Is there any reason for the difference here ?  Can we enable closeIdleConnections in YarnShuffleService or at least add a configuration to enable it ?

       

      Attachments

        Issue Links

          Activity

            People

              mauzhang Manu Zhang
              mauzhang Manu Zhang
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: