GitHub user Chaos-Ju opened a pull request:
https://github.com/apache/spark/pull/19206
Client and ApplicationMaster resolvePath is inappropriate when use viewfs
## What changes were proposed in this pull request?
When HDFS use viewfs and spark construct Executor's and ApplicationMaster's
localResource Map ( the list of localized files ) ,can't covert viewfs://
path to the real hdfs:// path . Therefore , when NodeManager download the local
Resource, will throw java.io.IOException: ViewFs: Cannot initialize: Empty
Mount table in config for viewfs://clusterName/
Exception stackï¼
java.io.IOException: ViewFs: Cannot initialize: Empty Mount table in config
for viewfs://ns-view/
at org.apache.hadoop.fs.viewfs.InodeTree.<init>(InodeTree.java:337)
at
org.apache.hadoop.fs.viewfs.ViewFileSystem$1.<init>(ViewFileSystem.java:167)
at
org.apache.hadoop.fs.viewfs.ViewFileSystem.initialize(ViewFileSystem.java:167)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2669)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:94)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2703)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2685)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:373)
at org.apache.hadoop.fs.Path.getFileSystem(Path.java:295)
at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:251)
at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:63)
at org.apache.hadoop.yarn.util.FSDownload$2.run(FSDownload.java:361)
at org.apache.hadoop.yarn.util.FSDownload$2.run(FSDownload.java:359)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1700)
at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:358)
at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:62)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:748)
Failing this attempt. Failing the application
## How was this patch tested?
manual tests
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/Chaos-Ju/spark master
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/spark/pull/19206.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 #19206
----
commit f1fff009d32b8f7d1d2b24734e4d677c6264ec90
Author: Chaos-Ju <[email protected]>
Date: 2017-09-12T12:45:36Z
fix spark support viewfs
----
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]