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

Path Inconsistency when Operating statCache within Yarn Client

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • 3.4.0, 3.5.0
    • 3.5.0, 4.0.0
    • Spark Submit
    • None

    Description

      The addResource from ClientDistributedCacheManager can add FileStatus to 

      statCache when it is not yet cached. Also, there is a subtle bug from isPublic from 

      getVisibility method. uri.getPath() will not retain URI information like 

      scheme, host, etc. So, the uri passed to checkPermissionOfOther will differ from the original uri.

      For example, if uri is "file:/foo.invalid.com:8080/tmp/testing", then 

      uri.getPath -> /foo.invalid.com:8080/tmp/testing
      uri.toString -> file:/foo.invalid.com:8080/tmp/testing

      The consequence of this bug is that we will double RPC calls when the resources are remote, which is unnecessary. We see nontrivial overhead when checking those resources from our HDFS, especially when HDFS is overloaded. 

       

      Ref: related code within ClientDistributedCacheManager

      def addResource(...) {
          val destStatus = statCache.getOrElse(destPath.toUri(), fs.getFileStatus(destPath))
      val visibility = getVisibility(conf, destPath.toUri(), statCache)
      }
      private[yarn] def getVisibility() {
      isPublic(conf, uri, statCache)
      }
      private def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = {
      val current = new Path(uri.getPath()) // Should not use getPath
      checkPermissionOfOther(fs, uri, FsAction.READ, statCache)
      }
      

       

      Attachments

        Issue Links

          Activity

            People

              shuwang SHU WANG
              shuwang SHU WANG
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: