-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-21012][Submit] Add glob support for resources adding to Spark #18235
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
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 |
|---|---|---|
|
|
@@ -17,17 +17,21 @@ | |
|
|
||
| package org.apache.spark.deploy | ||
|
|
||
| import java.io.{File, IOException} | ||
| import java.io._ | ||
| import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowableException} | ||
| import java.net.URL | ||
| import java.nio.file.Files | ||
| import java.security.PrivilegedExceptionAction | ||
| import java.security.{KeyStore, PrivilegedExceptionAction} | ||
| import java.security.cert.X509Certificate | ||
| import java.text.ParseException | ||
| import javax.net.ssl._ | ||
|
|
||
| import scala.annotation.tailrec | ||
| import scala.collection.mutable.{ArrayBuffer, HashMap, Map} | ||
| import scala.util.Properties | ||
|
|
||
| import com.google.common.io.ByteStreams | ||
| import org.apache.commons.io.FileUtils | ||
| import org.apache.commons.lang3.StringUtils | ||
| import org.apache.hadoop.conf.{Configuration => HadoopConfiguration} | ||
| import org.apache.hadoop.fs.{FileSystem, Path} | ||
|
|
@@ -310,33 +314,33 @@ object SparkSubmit extends CommandLineUtils { | |
| RPackageUtils.checkAndBuildRPackage(args.jars, printStream, args.verbose) | ||
| } | ||
|
|
||
| // In client mode, download remote files. | ||
| if (deployMode == CLIENT) { | ||
| val hadoopConf = new HadoopConfiguration() | ||
| args.primaryResource = Option(args.primaryResource).map(downloadFile(_, hadoopConf)).orNull | ||
| args.jars = Option(args.jars).map(downloadFileList(_, hadoopConf)).orNull | ||
| args.pyFiles = Option(args.pyFiles).map(downloadFileList(_, hadoopConf)).orNull | ||
| args.files = Option(args.files).map(downloadFileList(_, hadoopConf)).orNull | ||
| } | ||
|
|
||
| // Require all python files to be local, so we can add them to the PYTHONPATH | ||
| // In YARN cluster mode, python files are distributed as regular files, which can be non-local. | ||
| // In Mesos cluster mode, non-local python files are automatically downloaded by Mesos. | ||
| if (args.isPython && !isYarnCluster && !isMesosCluster) { | ||
| if (Utils.nonLocalPaths(args.primaryResource).nonEmpty) { | ||
| printErrorAndExit(s"Only local python files are supported: ${args.primaryResource}") | ||
| val hadoopConf = new HadoopConfiguration() | ||
| val targetDir = Files.createTempDirectory("tmp").toFile | ||
| // scalastyle:off runtimeaddshutdownhook | ||
| Runtime.getRuntime.addShutdownHook(new Thread() { | ||
| override def run(): Unit = { | ||
| FileUtils.deleteQuietly(targetDir) | ||
| } | ||
| val nonLocalPyFiles = Utils.nonLocalPaths(args.pyFiles).mkString(",") | ||
| if (nonLocalPyFiles.nonEmpty) { | ||
| printErrorAndExit(s"Only local additional python files are supported: $nonLocalPyFiles") | ||
| } | ||
| } | ||
| }) | ||
| // scalastyle:on runtimeaddshutdownhook | ||
|
|
||
| // Require all R files to be local | ||
| if (args.isR && !isYarnCluster && !isMesosCluster) { | ||
| if (Utils.nonLocalPaths(args.primaryResource).nonEmpty) { | ||
| printErrorAndExit(s"Only local R files are supported: ${args.primaryResource}") | ||
| } | ||
| // Resolve glob path for different resources. | ||
| args.jars = Option(args.jars).map(resolveGlobPaths(_, hadoopConf)).orNull | ||
| args.files = Option(args.files).map(resolveGlobPaths(_, hadoopConf)).orNull | ||
| args.pyFiles = Option(args.pyFiles).map(resolveGlobPaths(_, hadoopConf)).orNull | ||
| args.archives = Option(args.archives).map(resolveGlobPaths(_, hadoopConf)).orNull | ||
|
|
||
| // In client mode, download remote files. | ||
| if (deployMode == CLIENT) { | ||
| args.primaryResource = Option(args.primaryResource).map { | ||
| downloadFile(_, targetDir, args.sparkProperties, hadoopConf) | ||
| }.orNull | ||
| args.jars = Option(args.jars).map { | ||
| downloadFileList(_, targetDir, args.sparkProperties, hadoopConf) | ||
| }.orNull | ||
| args.pyFiles = Option(args.pyFiles).map { | ||
| downloadFileList(_, targetDir, args.sparkProperties, hadoopConf) | ||
| }.orNull | ||
| } | ||
|
|
||
| // The following modes are not supported or applicable | ||
|
|
@@ -841,36 +845,132 @@ object SparkSubmit extends CommandLineUtils { | |
| * Download a list of remote files to temp local files. If the file is local, the original file | ||
| * will be returned. | ||
| * @param fileList A comma separated file list. | ||
| * @param targetDir A temporary directory for which downloaded files | ||
| * @param sparkProperties Spark properties | ||
| * @return A comma separated local files list. | ||
| */ | ||
| private[deploy] def downloadFileList( | ||
| fileList: String, | ||
| targetDir: File, | ||
| sparkProperties: Map[String, String], | ||
| hadoopConf: HadoopConfiguration): String = { | ||
| require(fileList != null, "fileList cannot be null.") | ||
| fileList.split(",").map(downloadFile(_, hadoopConf)).mkString(",") | ||
| fileList.split(",") | ||
| .map(downloadFile(_, targetDir, sparkProperties, hadoopConf)) | ||
| .mkString(",") | ||
| } | ||
|
|
||
| /** | ||
| * Download a file from the remote to a local temporary directory. If the input path points to | ||
| * a local path, returns it with no operation. | ||
| * @param path A file path from where the files will be downloaded. | ||
| * @param targetDir A temporary directory for which downloaded files | ||
| * @param sparkProperties Spark properties | ||
| * @return A comma separated local files list. | ||
| */ | ||
| private[deploy] def downloadFile(path: String, hadoopConf: HadoopConfiguration): String = { | ||
| private[deploy] def downloadFile( | ||
| path: String, | ||
| targetDir: File, | ||
|
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. ditto |
||
| sparkProperties: Map[String, String], | ||
| hadoopConf: HadoopConfiguration): String = { | ||
| require(path != null, "path cannot be null.") | ||
| val uri = Utils.resolveURI(path) | ||
| uri.getScheme match { | ||
| case "file" | "local" => | ||
| path | ||
| case "file" | "local" => path | ||
| case "http" | "https" | "ftp" => | ||
| val uc = uri.toURL.openConnection() | ||
| uc match { | ||
| case https: HttpsURLConnection => | ||
| val trustStore = sparkProperties.get("spark.ssl.fs.trustStore") | ||
|
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. Should we make this a common util?
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. Also, should we test against 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. There is a common util method in Do you have any thought? |
||
| .orElse(sparkProperties.get("spark.ssl.trustStore")) | ||
|
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. Should we move these properties to internal/config?
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. These configurations are dynamic configurations based on the component
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. make sense, let's keep this way. |
||
| val trustStorePwd = sparkProperties.get("spark.ssl.fs.trustStorePassword") | ||
| .orElse(sparkProperties.get("spark.ssl.trustStorePassword")) | ||
| .map(_.toCharArray) | ||
| .orNull | ||
| val protocol = sparkProperties.get("spark.ssl.fs.protocol") | ||
| .orElse(sparkProperties.get("spark.ssl.protocol")) | ||
| if (protocol.isEmpty) { | ||
| printErrorAndExit("spark ssl protocol is required when enabling SSL connection.") | ||
| } | ||
|
|
||
| val trustStoreManagers = trustStore.map { t => | ||
| var input: InputStream = null | ||
| try { | ||
| input = new FileInputStream(new File(t)) | ||
| val ks = KeyStore.getInstance(KeyStore.getDefaultType) | ||
| ks.load(input, trustStorePwd) | ||
| val tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm) | ||
| tmf.init(ks) | ||
| tmf.getTrustManagers | ||
| } finally { | ||
| if (input != null) { | ||
| input.close() | ||
| input = null | ||
| } | ||
| } | ||
| }.getOrElse { | ||
| Array({ | ||
| new X509TrustManager { | ||
| override def getAcceptedIssuers: Array[X509Certificate] = null | ||
| override def checkClientTrusted( | ||
| x509Certificates: Array[X509Certificate], s: String) {} | ||
| override def checkServerTrusted( | ||
| x509Certificates: Array[X509Certificate], s: String) {} | ||
| }: TrustManager | ||
| }) | ||
| } | ||
| val sslContext = SSLContext.getInstance(protocol.get) | ||
| sslContext.init(null, trustStoreManagers, null) | ||
| https.setSSLSocketFactory(sslContext.getSocketFactory) | ||
| https.setHostnameVerifier(new HostnameVerifier { | ||
| override def verify(s: String, sslSession: SSLSession): Boolean = false | ||
| }) | ||
|
|
||
| case _ => | ||
| } | ||
|
|
||
| uc.setConnectTimeout(60 * 1000) | ||
| uc.setReadTimeout(60 * 1000) | ||
| uc.connect() | ||
| val in = uc.getInputStream | ||
| val fileName = new Path(uri).getName | ||
| val tempFile = new File(targetDir, fileName) | ||
| val out = new FileOutputStream(tempFile) | ||
| // scalastyle:off println | ||
| printStream.println(s"Downloading ${uri.toString} to ${tempFile.getAbsolutePath}.") | ||
| // scalastyle:on println | ||
| try { | ||
| ByteStreams.copy(in, out) | ||
| } finally { | ||
| in.close() | ||
| out.close() | ||
| } | ||
| tempFile.toURI.toString | ||
| case _ => | ||
| val fs = FileSystem.get(uri, hadoopConf) | ||
| val tmpFile = new File(Files.createTempDirectory("tmp").toFile, uri.getPath) | ||
| val tmpFile = new File(targetDir, new Path(uri).getName) | ||
| // scalastyle:off println | ||
| printStream.println(s"Downloading ${uri.toString} to ${tmpFile.getAbsolutePath}.") | ||
| // scalastyle:on println | ||
| fs.copyToLocalFile(new Path(uri), new Path(tmpFile.getAbsolutePath)) | ||
| Utils.resolveURI(tmpFile.getAbsolutePath).toString | ||
| tmpFile.toURI.toString | ||
| } | ||
| } | ||
|
|
||
| private def resolveGlobPaths(paths: String, hadoopConf: HadoopConfiguration): String = { | ||
| require(paths != null, "paths cannot be null.") | ||
| paths.split(",").map(_.trim).filter(_.nonEmpty).flatMap { path => | ||
| val uri = Utils.resolveURI(path) | ||
| uri.getScheme match { | ||
| case "local" | "http" | "https" | "ftp" => Array(path) | ||
| case _ => | ||
| val fs = FileSystem.get(uri, hadoopConf) | ||
| Option(fs.globStatus(new Path(uri))).map { status => | ||
| status.filter(_.isFile).map(_.getPath.toUri.toString) | ||
| }.getOrElse(Array(path)) | ||
| } | ||
| }.mkString(",") | ||
| } | ||
| } | ||
|
|
||
| /** Provides utility functions to be used inside SparkSubmit. */ | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -520,7 +520,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | |
| | (Default: client). | ||
| | --class CLASS_NAME Your application's main class (for Java / Scala apps). | ||
| | --name NAME A name of your application. | ||
| | --jars JARS Comma-separated list of local jars to include on the driver | ||
|
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. This is a new feature(support non-local jars), shall we create a separated jira ticket?
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. and we should also add a test case for it.
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. Thanks @cloud-fan for your review. Supporting remote jar has been addressed in #18078 , but it still left some codes and docs should be updated, also it doesn't support downloading jars from HTTP(S), FTP, here also adding this support. So basically this PR address the left problem of #18078 and add glob support. |
||
| | --jars JARS Comma-separated list of jars to include on the driver | ||
| | and executor classpaths. | ||
| | --packages Comma-separated list of maven coordinates of jars to include | ||
| | on the driver and executor classpaths. Will search the local | ||
|
|
||
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.
Let's explain the meaning of each param.