[ 
https://issues.apache.org/jira/browse/SPARK-21917?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16154097#comment-16154097
 ] 

Marcelo Vanzin commented on SPARK-21917:
----------------------------------------

I think the optimal way would be to try to add things to the cache and, if that 
fails, fallback to download + re-upload.

The sucky part is that there's no reliable way to do it. The libraries 
available on the client side may support different file systems than the 
libraries available on the NM; so if you have the http fs in your classpath, 
but the NM does not, the container localizer would probably fail.

#1 works but it also penalizes those who are running YARN 2.9 or any other 
future version where that support exists.

So perhaps a compromise could be:

- by default, assume that client and NM libraries are "in sync"; if 
{{FileSystem.get()}} does not complain, assume the NM can also download files 
from that scheme. If it does, then download the file and re-upload it to HDFS.
- add a config option where users can blacklist schemes and force those to 
download and re-upload.


> Remote http(s) resources is not supported in YARN mode
> ------------------------------------------------------
>
>                 Key: SPARK-21917
>                 URL: https://issues.apache.org/jira/browse/SPARK-21917
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Submit, YARN
>    Affects Versions: 2.2.0
>            Reporter: Saisai Shao
>            Priority: Minor
>
> 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:
> {noformat}
> 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)
> {noformat}
> This is because {{YARN#client}} assumes resources must be on the Hadoop 
> compatible FS, also in the NM 
> (https://github.com/apache/hadoop/blob/99e558b13ba4d5832aea97374e1d07b4e78e5e39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java#L245)
>  it will only use Hadoop compatible FS to download resources. So this makes 
> Spark on YARN fail to support remote http(s) resources.
> To solve this problem, there might be several options:
> * Download remote http(s) resources to local and add this local downloaded 
> resources to dist cache. The downside of this option is that remote resources 
> will be uploaded again unnecessarily.
> * Filter remote http(s) resources and add them with spark.jars or 
> spark.files, to leverage Spark's internal fileserver to distribute remote 
> http(s) resources. The problem of this solution is: for some resources which 
> require to be available before application start may not work.
> * Leverage Hadoop's support http(s) file system 
> (https://issues.apache.org/jira/browse/HADOOP-14383). This is only worked in 
> Hadoop 2.9+, and I think even we implement a similar one in Spark will not be 
> worked.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to