Skip to content

Conversation

jerryshao
Copy link
Contributor

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.

@jerryshao jerryshao changed the title [SPARK-21917][CORE][YARN] Supporting Download http(s) resources in yarn mode [SPARK-21917][CORE][YARN] Supporting adding http(s) resources in yarn mode Sep 5, 2017
@SparkQA
Copy link

SparkQA commented Sep 5, 2017

Test build #81402 has finished for PR 19130 at commit 42a79ab.

  • This patch fails SparkR unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@jerryshao
Copy link
Contributor Author

Jenkins, retest this please.

@SparkQA
Copy link

SparkQA commented Sep 5, 2017

Test build #81410 has finished for PR 19130 at commit 42a79ab.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Sep 6, 2017

Test build #81447 has finished for PR 19130 at commit 047578e.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@jerryshao
Copy link
Contributor Author

@vanzin @tgravescs , would you please help to review, thanks!

@tgravescs
Copy link
Contributor

sorry probably wont' get to this today, will look tomorrow.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is a Filesystem.getFileSystemClass function we could use here instead of calling dummy uri

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we somehow want to make this configurable per scheme? Right now its basically http/https, in the future would we want to possibly handle other filesystems that hadoop doesn't support. Making this a settable config would make that easier

@jerryshao
Copy link
Contributor Author

@tgravescs , thanks for your comments, can you review again, if it is what you expected.

@SparkQA
Copy link

SparkQA commented Sep 12, 2017

Test build #81658 has finished for PR 19130 at commit 4bbc09d.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@jerryshao
Copy link
Contributor Author

Jenkins, retest this please.

@SparkQA
Copy link

SparkQA commented Sep 12, 2017

Test build #81665 has finished for PR 19130 at commit 4bbc09d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

"When running in YARN cluster manager, ?"

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry for the broken comment, my bad, I will fix it.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Try { ... }.isSuccess? You could also avoid this call if the scheme is in the blacklist.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Better wording:

Comma-separated list of schemes for which files will be downloaded to the local disk prior to being added to YARN's distributed cache. For use in cases where the YARN service does not support schemes that are supported by Spark.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I was going to say this is missing spark.yarn.dist.files and .jars, but later those properties seem to be set based on args.files and args.jars.

Which kinda raises the question of what happens when the user sets both. From the documentation it sounds like that should work (both sets of files get added), but from the code it seems --files and --jars would overwrite the spark.yarn.* configs...

In any case, that's not the fault of your change.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

From the code --files and --jars overwrite spark.yarn.* long ago AFAIK. What I think is that we should make spark.yarn.* as an internal configurations to reduce the discrepancy.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It would be better if tests could avoid this... you could start a local http server, but that feels like a lot of work. Is there some way to mock the behavior instead?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, that's my concern, let me think out another way to handle this.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

...still are...

Also I'm not sure I understand the comment.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems you have 3 different tests in this block (at least), could you break them into separate tests?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@vanzin , it is a little difficult to mock the download behavior, so here I check if "spark.testing" is configured, return a dummy local path if it is configured. What do you think about this approach?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sounds good.

@SparkQA
Copy link

SparkQA commented Sep 14, 2017

Test build #81775 has finished for PR 19130 at commit d479ff0.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Sep 14, 2017

Test build #81776 has finished for PR 19130 at commit cc69bc8.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

@vanzin vanzin left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A few minor things to address.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Code like this (break a comma-separate string into a list) is copy & pasted in so many places that it probably deserves a method in Utils.

There's one in ConfigHelpers.stringToSeq but that class is private to its package.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I added a help method in Utils and changed in SparkSubmit related codes. There still have some other places which requires to change, but I will not touch them in this PR.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

break multi-line args one per line.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Better: "force download from blacklisted schemes"

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should already be set by the build scripts, was it not working for you?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I haven't tried, I saw some UT also set this configuration, let me check if it is explicitly required or not.

@SparkQA
Copy link

SparkQA commented Sep 15, 2017

Test build #81804 has finished for PR 19130 at commit 1c5487c.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Sep 15, 2017

Test build #81805 has finished for PR 19130 at commit fc2eb2b.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

@jiangxb1987 jiangxb1987 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is it a problem only for YARN? Do standalone and Mesos have this problem?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a problem for YARN currently, because YARN uses dist cache to distribute resources to yarn cluster, dist cache requires supported Hadoop FS to copy resources, if our resource scheme is http, it will try to find http FS to handle such resource, which will be failed since no http FS supported in current Hadoop.

In standalone and Mesos cluster, we use Spark's internal logic to handle http resources, this logic handles well for the http(s) resources, so there should be no issue for standalone and mesos mode.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why make it a function? Can't we just inline it?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It can be, let me change the code.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shall we explicitly list "http" | "https" | "ftp"?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, it is not required, because shouldDownload logic will handle this. If 1) this resource scheme is blacklisted, or 2) it is not support by Hadoop, then Spark will handle this through downloadFile method. Since "http" | "https" | "ftp" is not supported by Hadoop before 2.9, so it implies that resources with such scheme will be handled by Spark itself.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can you give an example of these schemes?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK, sure.

@SparkQA
Copy link

SparkQA commented Sep 19, 2017

Test build #81905 has finished for PR 19130 at commit 580d587.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

update here too

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shall we make these 3 the default value of this config?

Copy link
Contributor Author

@jerryshao jerryshao Sep 19, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is not necessary, we still want to leverage Hadoop's http(s) FS to distribute resources by default if it is running on Hadoop 2.9+ (https://issues.apache.org/jira/browse/HADOOP-14383)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah got it

@SparkQA
Copy link

SparkQA commented Sep 19, 2017

Test build #81912 has finished for PR 19130 at commit 9a2c8c7.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

Change-Id: I7897817ceaaafecd779a6e085c96d2a28363d7d6
@SparkQA
Copy link

SparkQA commented Sep 19, 2017

Test build #81914 has finished for PR 19130 at commit 0fb7943.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cloud-fan
Copy link
Contributor

LGTM, just one question: why we need spark.yarn.dist.forceDownloadSchemes?

@jerryshao
Copy link
Contributor Author

Hi @cloud-fan , the main purpose of spark.yarn.dist.forceDownloadSchemes is to explicitly using Spark's own logic to handle remote resources instead of relying on Hadoop. For example if spark.yarn.dist.forceDownloadSchemes is configured to http,https, then this 2 kinds of resources will be downloaded by Spark prior to add to dist cache, even if they're supported by http FS in Hadoop 2.9+. For now if we use Hadoop 2.9-, since Hadoop doesn't support http FS, so we will always leverage Spark's own logic to download resources, it is not necessary to configure this parameter.

@asfgit asfgit closed this in 8319432 Sep 19, 2017
@cloud-fan
Copy link
Contributor

thanks, merging to master!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants