Github user jerryshao commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18962#discussion_r133907328
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala ---
    @@ -330,18 +330,45 @@ object SparkSubmit extends CommandLineUtils {
         args.archives = Option(args.archives).map(resolveGlobPaths(_, 
hadoopConf)).orNull
     
         // In client mode, download remote files.
    +    var localPrimaryResource: String = null
    +    var localJars: String = null
    +    var localPyFiles: String = null
         if (deployMode == CLIENT) {
    -      args.primaryResource = Option(args.primaryResource).map {
    +      localPrimaryResource = Option(args.primaryResource).map {
             downloadFile(_, targetDir, args.sparkProperties, hadoopConf)
           }.orNull
    -      args.jars = Option(args.jars).map {
    +      localJars = Option(args.jars).map {
             downloadFileList(_, targetDir, args.sparkProperties, hadoopConf)
           }.orNull
    -      args.pyFiles = Option(args.pyFiles).map {
    +      localPyFiles = Option(args.pyFiles).map {
             downloadFileList(_, targetDir, args.sparkProperties, hadoopConf)
           }.orNull
         }
     
    +    if (clusterManager == YARN) {
    +      def isNoneFsFileExist(paths: String): Boolean = {
    +        Option(paths).exists { p =>
    +          p.split(",").map(_.trim).filter(_.nonEmpty).exists { path =>
    +            val url = Utils.resolveURI(path)
    +            url.getScheme match {
    +              case "http" | "https" | "ftp" => true
    +              case _ => false
    +            }
    +          }
    +        }
    +      }
    +
    +      // Spark on YARN doesn't support upload remote resources from http, 
https or ftp server
    +      // directly to distributed cache, so print a warning and exit the 
process.
    +      if (isNoneFsFileExist(args.jars) ||
    --- End diff --
    
    From my understanding, to solve this issue, there's some solutions:
    
    1. Filtering resource from http(s) and add to spark.jars/spark.files, using 
Spark's internal fileServer to distribute. This should be worked in yarn client 
mode, but in yarn cluster mode it cannot be worked, since AM doesn't have such 
resources.
    2. Filtering resources from http(s) and download them to local both in yarn 
client and cluster mode. This will re-upload resources to Hadoop FS, so the 
problem here is resources are downloaded and uploaded unnecessarily. But this 
could solve the whole problem here, also if there's no internet access other 
than gateway nodes, this should be a possible solution. I assume not so many 
users will use resources on http(s), so as a compromise solution it looks 
acceptable.
    
    What do you think? @vanzin @tgravescs .
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to