You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Tao Li (JIRA)" <ji...@apache.org> on 2015/06/26 13:18:04 UTC

[jira] [Updated] (SPARK-8657) Fail to upload conf archive to viewfs

     [ https://issues.apache.org/jira/browse/SPARK-8657?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Tao Li updated SPARK-8657:
--------------------------
    Labels: distributed_cache viewfs  (was: )

> Fail to upload conf archive to viewfs
> -------------------------------------
>
>                 Key: SPARK-8657
>                 URL: https://issues.apache.org/jira/browse/SPARK-8657
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core
>    Affects Versions: 1.4.0, 1.4.1, 1.4.2
>         Environment: spark-1.4.2 & hadoop-2.5.0-cdh5.3.2
>            Reporter: Tao Li
>              Labels: distributed_cache, viewfs
>             Fix For: 1.4.2
>
>
> When I run in spark-1.4 yarn-client mode, I throws the following Exception when trying to upload conf archive to viewfs:
> 15/06/26 17:56:37 INFO yarn.Client: Uploading resource file:/tmp/spark-095ec3d2-5dad-468c-8d46-2c813457404d/__hadoop_conf__8436284925771788661
> .zip -> viewfs://nsX/user/ultraman/.sparkStaging/application_1434370929997_191242/__hadoop_conf__8436284925771788661.zip
> 15/06/26 17:56:38 INFO yarn.Client: Deleting staging directory .sparkStaging/application_1434370929997_191242
> 15/06/26 17:56:38 ERROR spark.SparkContext: Error initializing SparkContext.
> java.lang.IllegalArgumentException: Wrong FS: hdfs://SunshineNameNode2:8020/user/ultraman/.sparkStaging/application_1434370929997_191242/__had
> oop_conf__8436284925771788661.zip, expected: viewfs://nsX/
>         at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:645)
>         at org.apache.hadoop.fs.viewfs.ViewFileSystem.getUriPath(ViewFileSystem.java:117)
>         at org.apache.hadoop.fs.viewfs.ViewFileSystem.getFileStatus(ViewFileSystem.java:346)
>         at org.apache.spark.deploy.yarn.ClientDistributedCacheManager.addResource(ClientDistributedCacheManager.scala:67)
>         at org.apache.spark.deploy.yarn.Client$$anonfun$prepareLocalResources$5.apply(Client.scala:341)
>         at org.apache.spark.deploy.yarn.Client$$anonfun$prepareLocalResources$5.apply(Client.scala:338)
>         at scala.Option.foreach(Option.scala:236)
>         at org.apache.spark.deploy.yarn.Client.prepareLocalResources(Client.scala:338)
>         at org.apache.spark.deploy.yarn.Client.createContainerLaunchContext(Client.scala:559)
>         at org.apache.spark.deploy.yarn.Client.submitApplication(Client.scala:115)
>         at org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend.start(YarnClientSchedulerBackend.scala:58)
>         at org.apache.spark.scheduler.TaskSchedulerImpl.start(TaskSchedulerImpl.scala:141)
>         at org.apache.spark.SparkContext.<init>(SparkContext.scala:497)
>         at org.apache.spark.repl.SparkILoop.createSparkContext(SparkILoop.scala:1017)
>         at $line3.$read$$iwC$$iwC.<init>(<console>:9)
>         at $line3.$read$$iwC.<init>(<console>:18)
>         at $line3.$read.<init>(<console>:20)
>         at $line3.$read$.<init>(<console>:24)
>         at $line3.$read$.<clinit>(<console>)
>         at $line3.$eval$.<init>(<console>:7)
>         at $line3.$eval$.<clinit>(<console>)
>         at $line3.$eval.$print(<console>)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:606)
>         at org.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)
>         at org.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1338)
>         at org.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840)
> The bug is easy to fix, we should pass the correct file system object to addResource. The similar issure is: https://github.com/apache/spark/pull/1483. I will attach my bug fix PR very soon.
> The code in Client.scala is need to fix:
> createConfArchive().foreach { file =>
>       require(addDistributedUri(file.toURI()))
>       val destPath = copyFileToRemote(dst, new Path(file.toURI()), replication)
>       val destFs = FileSystem.get(destPath.toUri(), hadoopConf)  // here! use correct dest FileSystem
>       distCacheMgr.addResource(destFs, hadoopConf, destPath, localResources, LocalResourceType.ARCHIVE,
>         LOCALIZED_HADOOP_CONF_DIR, statCache, appMasterOnly = true)
>     }



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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