-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-10643] [Core] Make spark-submit download remote files to local in client mode #18078
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
4cdeeed
6e86290
e5171ca
62e57df
7eb5d1a
3f18b81
2d6f2cd
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,12 +18,15 @@ | |
| package org.apache.spark.deploy | ||
|
|
||
| import java.io._ | ||
| import java.net.URI | ||
| import java.nio.charset.StandardCharsets | ||
|
|
||
| import scala.collection.mutable.ArrayBuffer | ||
| import scala.io.Source | ||
|
|
||
| import com.google.common.io.ByteStreams | ||
| import org.apache.commons.io.{FilenameUtils, FileUtils} | ||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.Path | ||
| import org.scalatest.{BeforeAndAfterEach, Matchers} | ||
| import org.scalatest.concurrent.Timeouts | ||
|
|
@@ -535,7 +538,7 @@ class SparkSubmitSuite | |
|
|
||
| test("resolves command line argument paths correctly") { | ||
| val jars = "/jar1,/jar2" // --jars | ||
| val files = "hdfs:/file1,file2" // --files | ||
| val files = "local:/file1,file2" // --files | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could you expand on why we are changing this?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. To make it not try to download file from hdfs
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It is kinda difficult to test download file from hdfs now, but we should cover this scene in the future. |
||
| val archives = "file:/archive1,archive2" // --archives | ||
| val pyFiles = "py-file1,py-file2" // --py-files | ||
|
|
||
|
|
@@ -587,7 +590,7 @@ class SparkSubmitSuite | |
|
|
||
| test("resolves config paths correctly") { | ||
| val jars = "/jar1,/jar2" // spark.jars | ||
| val files = "hdfs:/file1,file2" // spark.files / spark.yarn.dist.files | ||
| val files = "local:/file1,file2" // spark.files / spark.yarn.dist.files | ||
| val archives = "file:/archive1,archive2" // spark.yarn.dist.archives | ||
| val pyFiles = "py-file1,py-file2" // spark.submit.pyFiles | ||
|
|
||
|
|
@@ -705,6 +708,87 @@ class SparkSubmitSuite | |
| } | ||
| // scalastyle:on println | ||
|
|
||
| private def checkDownloadedFile(sourcePath: String, outputPath: String): Unit = { | ||
| if (sourcePath == outputPath) { | ||
| return | ||
| } | ||
|
|
||
| val sourceUri = new URI(sourcePath) | ||
| val outputUri = new URI(outputPath) | ||
| assert(outputUri.getScheme === "file") | ||
|
|
||
| // The path and filename are preserved. | ||
| assert(outputUri.getPath.endsWith(sourceUri.getPath)) | ||
| assert(FileUtils.readFileToString(new File(outputUri.getPath)) === | ||
| FileUtils.readFileToString(new File(sourceUri.getPath))) | ||
| } | ||
|
|
||
| private def deleteTempOutputFile(outputPath: String): Unit = { | ||
| val outputFile = new File(new URI(outputPath).getPath) | ||
| if (outputFile.exists) { | ||
| outputFile.delete() | ||
| } | ||
| } | ||
|
|
||
| test("downloadFile - invalid url") { | ||
| intercept[IOException] { | ||
| SparkSubmit.downloadFile("abc:/my/file", new Configuration()) | ||
| } | ||
| } | ||
|
|
||
| test("downloadFile - file doesn't exist") { | ||
| val hadoopConf = new Configuration() | ||
| // Set s3a implementation to local file system for testing. | ||
| hadoopConf.set("fs.s3a.impl", "org.apache.spark.deploy.TestFileSystem") | ||
| // Disable file system impl cache to make sure the test file system is picked up. | ||
| hadoopConf.set("fs.s3a.impl.disable.cache", "true") | ||
| intercept[FileNotFoundException] { | ||
| SparkSubmit.downloadFile("s3a:/no/such/file", hadoopConf) | ||
| } | ||
| } | ||
|
|
||
| test("downloadFile does not download local file") { | ||
| // empty path is considered as local file. | ||
| assert(SparkSubmit.downloadFile("", new Configuration()) === "") | ||
| assert(SparkSubmit.downloadFile("/local/file", new Configuration()) === "/local/file") | ||
| } | ||
|
|
||
| test("download one file to local") { | ||
| val jarFile = File.createTempFile("test", ".jar") | ||
| jarFile.deleteOnExit() | ||
| val content = "hello, world" | ||
| FileUtils.write(jarFile, content) | ||
| val hadoopConf = new Configuration() | ||
| // Set s3a implementation to local file system for testing. | ||
| hadoopConf.set("fs.s3a.impl", "org.apache.spark.deploy.TestFileSystem") | ||
| // Disable file system impl cache to make sure the test file system is picked up. | ||
| hadoopConf.set("fs.s3a.impl.disable.cache", "true") | ||
| val sourcePath = s"s3a://${jarFile.getAbsolutePath}" | ||
| val outputPath = SparkSubmit.downloadFile(sourcePath, hadoopConf) | ||
| checkDownloadedFile(sourcePath, outputPath) | ||
| deleteTempOutputFile(outputPath) | ||
| } | ||
|
|
||
| test("download list of files to local") { | ||
| val jarFile = File.createTempFile("test", ".jar") | ||
| jarFile.deleteOnExit() | ||
| val content = "hello, world" | ||
| FileUtils.write(jarFile, content) | ||
| val hadoopConf = new Configuration() | ||
| // Set s3a implementation to local file system for testing. | ||
| hadoopConf.set("fs.s3a.impl", "org.apache.spark.deploy.TestFileSystem") | ||
| // Disable file system impl cache to make sure the test file system is picked up. | ||
| hadoopConf.set("fs.s3a.impl.disable.cache", "true") | ||
| val sourcePaths = Seq("/local/file", s"s3a://${jarFile.getAbsolutePath}") | ||
| val outputPaths = SparkSubmit.downloadFileList(sourcePaths.mkString(","), hadoopConf).split(",") | ||
|
|
||
| assert(outputPaths.length === sourcePaths.length) | ||
| sourcePaths.zip(outputPaths).foreach { case (sourcePath, outputPath) => | ||
| checkDownloadedFile(sourcePath, outputPath) | ||
| deleteTempOutputFile(outputPath) | ||
| } | ||
| } | ||
|
|
||
| // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. | ||
| private def runSparkSubmit(args: Seq[String]): Unit = { | ||
| val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) | ||
|
|
@@ -807,3 +891,10 @@ object UserClasspathFirstTest { | |
| } | ||
| } | ||
| } | ||
|
|
||
| class TestFileSystem extends org.apache.hadoop.fs.LocalFileSystem { | ||
| override def copyToLocalFile(src: Path, dst: Path): Unit = { | ||
| // Ignore the scheme for testing. | ||
| super.copyToLocalFile(new Path(src.toUri.getPath), dst) | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
sorry I may not have enough background knowledge, why we only do this for client mode?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It seems it can handle remote files in Yarn/Mesos cluster mode. I haven't tested it, because we are using client mode.