GitHub user jerryshao opened a pull request: https://github.com/apache/spark/pull/19130
[SPARK-21917][CORE][YARN] Supporting Download http(s) resources in yarn mode ## What changes were proposed in this pull request? In the current Spark, when submitting application on YARN with remote resources `./bin/spark-shell --jars http://central.maven.org/maven2/com/github/swagger-akka-http/swagger-akka-http_2.11/0.10.1/swagger-akka-http_2.11-0.10.1.jar --master yarn-client -v`, Spark will be failed with: ``` java.io.IOException: No FileSystem for scheme: http at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2586) at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2593) at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:91) at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2632) at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2614) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:370) at org.apache.hadoop.fs.Path.getFileSystem(Path.java:296) at org.apache.spark.deploy.yarn.Client.copyFileToRemote(Client.scala:354) at org.apache.spark.deploy.yarn.Client.org$apache$spark$deploy$yarn$Client$$distribute$1(Client.scala:478) at org.apache.spark.deploy.yarn.Client$$anonfun$prepareLocalResources$11$$anonfun$apply$6.apply(Client.scala:600) at org.apache.spark.deploy.yarn.Client$$anonfun$prepareLocalResources$11$$anonfun$apply$6.apply(Client.scala:599) at scala.collection.mutable.ArraySeq.foreach(ArraySeq.scala:74) at org.apache.spark.deploy.yarn.Client$$anonfun$prepareLocalResources$11.apply(Client.scala:599) at org.apache.spark.deploy.yarn.Client$$anonfun$prepareLocalResources$11.apply(Client.scala:598) at scala.collection.immutable.List.foreach(List.scala:381) at org.apache.spark.deploy.yarn.Client.prepareLocalResources(Client.scala:598) at org.apache.spark.deploy.yarn.Client.createContainerLaunchContext(Client.scala:848) at org.apache.spark.deploy.yarn.Client.submitApplication(Client.scala:173) ``` This is because `YARN#client` assumes resources are on the Hadoop compatible FS. To fix this problem, here propose to download remote http(s) resources to local and add this local downloaded resources to dist cache. This solution has one downside: remote resources are downloaded and uploaded again, but it only restricted to only remote http(s) resources, also the overhead is not so big. The advantages of this solution is that it is simple and the code changes restricts to only `SparkSubmit`. ## How was this patch tested? Unit test added, also verified in local cluster. You can merge this pull request into a Git repository by running: $ git pull https://github.com/jerryshao/apache-spark SPARK-21917 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/spark/pull/19130.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #19130 ---- commit 42a79ab5dab8f956eb923a2260b5aed7ae2660cf Author: jerryshao <ss...@hortonworks.com> Date: 2017-09-05T08:18:36Z Download remote http(s) resources to local in yarn mode Change-Id: I7897817ceaaafecd779a6e085c96d2a28363d7d6 ---- --- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org