-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-33530][CORE] Support --archives and spark.archives option natively #30486
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 |
|---|---|---|
|
|
@@ -22,6 +22,7 @@ import java.net.URI | |
| import java.util.{Arrays, Locale, Properties, ServiceLoader, UUID} | ||
| import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} | ||
| import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicReference} | ||
| import javax.ws.rs.core.UriBuilder | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
| import scala.collection.Map | ||
|
|
@@ -39,7 +40,7 @@ import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, Sequence | |
| import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob} | ||
| import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} | ||
|
|
||
| import org.apache.spark.annotation.DeveloperApi | ||
| import org.apache.spark.annotation.{DeveloperApi, Experimental} | ||
| import org.apache.spark.broadcast.Broadcast | ||
| import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} | ||
| import org.apache.spark.executor.{Executor, ExecutorMetrics, ExecutorMetricsSource} | ||
|
|
@@ -221,6 +222,7 @@ class SparkContext(config: SparkConf) extends Logging { | |
| private var _listenerBusStarted: Boolean = false | ||
| private var _jars: Seq[String] = _ | ||
| private var _files: Seq[String] = _ | ||
| private var _archives: Seq[String] = _ | ||
| private var _shutdownHookRef: AnyRef = _ | ||
| private var _statusStore: AppStatusStore = _ | ||
| private var _heartbeater: Heartbeater = _ | ||
|
|
@@ -246,6 +248,7 @@ class SparkContext(config: SparkConf) extends Logging { | |
|
|
||
| def jars: Seq[String] = _jars | ||
| def files: Seq[String] = _files | ||
| def archives: Seq[String] = _archives | ||
| def master: String = _conf.get("spark.master") | ||
| def deployMode: String = _conf.get(SUBMIT_DEPLOY_MODE) | ||
| def appName: String = _conf.get("spark.app.name") | ||
|
|
@@ -278,6 +281,7 @@ class SparkContext(config: SparkConf) extends Logging { | |
|
|
||
| // Used to store a URL for each static file/jar together with the file's local timestamp | ||
| private[spark] val addedFiles = new ConcurrentHashMap[String, Long]().asScala | ||
| private[spark] val addedArchives = new ConcurrentHashMap[String, Long]().asScala | ||
| private[spark] val addedJars = new ConcurrentHashMap[String, Long]().asScala | ||
|
|
||
| // Keeps track of all persisted RDDs | ||
|
|
@@ -422,6 +426,7 @@ class SparkContext(config: SparkConf) extends Logging { | |
| _jars = Utils.getUserJars(_conf) | ||
| _files = _conf.getOption(FILES.key).map(_.split(",")).map(_.filter(_.nonEmpty)) | ||
| .toSeq.flatten | ||
| _archives = _conf.getOption(ARCHIVES.key).map(Utils.stringToSeq).toSeq.flatten | ||
|
|
||
| _eventLogDir = | ||
| if (isEventLogEnabled) { | ||
|
|
@@ -506,6 +511,13 @@ class SparkContext(config: SparkConf) extends Logging { | |
| } | ||
| } | ||
|
|
||
| if (archives != null) { | ||
| archives.foreach(file => addFile(file, false, true, isArchive = true)) | ||
| if (addedArchives.nonEmpty) { | ||
| _conf.set("spark.app.initial.archive.urls", addedArchives.keys.toSeq.mkString(",")) | ||
| } | ||
| } | ||
|
|
||
| _executorMemory = _conf.getOption(EXECUTOR_MEMORY.key) | ||
| .orElse(Option(System.getenv("SPARK_EXECUTOR_MEMORY"))) | ||
| .orElse(Option(System.getenv("SPARK_MEM")) | ||
|
|
@@ -1520,6 +1532,36 @@ class SparkContext(config: SparkConf) extends Logging { | |
| */ | ||
| def listFiles(): Seq[String] = addedFiles.keySet.toSeq | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Add an archive to be downloaded and unpacked with this Spark job on every node. | ||
| * | ||
| * If an archive is added during execution, it will not be available until the next TaskSet | ||
| * starts. | ||
| * | ||
| * @param path can be either a local file, a file in HDFS (or other Hadoop-supported | ||
| * filesystems), or an HTTP, HTTPS or FTP URI. To access the file in Spark jobs, | ||
| * use `SparkFiles.get(paths-to-files)` to find its download/unpacked location. | ||
| * The given path should be one of .zip, .tar, .tar.gz, .tgz and .jar. | ||
| * | ||
| * @note A path can be added only once. Subsequent additions of the same path are ignored. | ||
| * | ||
| * @since 3.1.0 | ||
| */ | ||
| @Experimental | ||
| def addArchive(path: String): Unit = { | ||
| addFile(path, false, false, isArchive = true) | ||
| } | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Returns a list of archive paths that are added to resources. | ||
| * | ||
| * @since 3.1.0 | ||
| */ | ||
| @Experimental | ||
| def listArchives(): Seq[String] = addedArchives.keySet.toSeq | ||
|
|
||
| /** | ||
| * Add a file to be downloaded with this Spark job on every node. | ||
| * | ||
|
|
@@ -1537,8 +1579,14 @@ class SparkContext(config: SparkConf) extends Logging { | |
| addFile(path, recursive, false) | ||
| } | ||
|
|
||
| private def addFile(path: String, recursive: Boolean, addedOnSubmit: Boolean): Unit = { | ||
| val uri = new Path(path).toUri | ||
| private def addFile( | ||
| path: String, recursive: Boolean, addedOnSubmit: Boolean, isArchive: Boolean = false | ||
| ): Unit = { | ||
| val uri = if (!isArchive) { | ||
| new Path(path).toUri | ||
| } else { | ||
| Utils.resolveURI(path) | ||
|
||
| } | ||
| val schemeCorrectedURI = uri.getScheme match { | ||
| case null => new File(path).getCanonicalFile.toURI | ||
| case "local" => | ||
|
|
@@ -1550,7 +1598,7 @@ class SparkContext(config: SparkConf) extends Logging { | |
|
|
||
| val hadoopPath = new Path(schemeCorrectedURI) | ||
| val scheme = schemeCorrectedURI.getScheme | ||
| if (!Array("http", "https", "ftp").contains(scheme)) { | ||
| if (!Array("http", "https", "ftp").contains(scheme) && !isArchive) { | ||
|
||
| val fs = hadoopPath.getFileSystem(hadoopConfiguration) | ||
| val isDir = fs.getFileStatus(hadoopPath).isDirectory | ||
| if (!isLocal && scheme == "file" && isDir) { | ||
|
|
@@ -1568,21 +1616,39 @@ class SparkContext(config: SparkConf) extends Logging { | |
|
|
||
| val key = if (!isLocal && scheme == "file") { | ||
| env.rpcEnv.fileServer.addFile(new File(uri.getPath)) | ||
| } else if (uri.getScheme == null) { | ||
| schemeCorrectedURI.toString | ||
| } else if (isArchive) { | ||
| uri.toString | ||
|
||
| } else { | ||
| if (uri.getScheme == null) { | ||
| schemeCorrectedURI.toString | ||
| } else { | ||
| path | ||
| } | ||
| path | ||
| } | ||
|
|
||
| val timestamp = if (addedOnSubmit) startTime else System.currentTimeMillis | ||
| if (addedFiles.putIfAbsent(key, timestamp).isEmpty) { | ||
| if (!isArchive && addedFiles.putIfAbsent(key, timestamp).isEmpty) { | ||
| logInfo(s"Added file $path at $key with timestamp $timestamp") | ||
| // Fetch the file locally so that closures which are run on the driver can still use the | ||
| // SparkFiles API to access files. | ||
| Utils.fetchFile(uri.toString, new File(SparkFiles.getRootDirectory()), conf, | ||
| env.securityManager, hadoopConfiguration, timestamp, useCache = false) | ||
| postEnvironmentUpdate() | ||
| } else if ( | ||
| isArchive && | ||
| addedArchives.putIfAbsent( | ||
| UriBuilder.fromUri(new URI(key)).fragment(uri.getFragment).build().toString, | ||
| timestamp).isEmpty) { | ||
| logInfo(s"Added archive $path at $key with timestamp $timestamp") | ||
| val uriToDownload = UriBuilder.fromUri(new URI(key)).fragment(null).build() | ||
| val source = Utils.fetchFile(uriToDownload.toString, Utils.createTempDir(), conf, | ||
HyukjinKwon marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| env.securityManager, hadoopConfiguration, timestamp, useCache = false, shouldUntar = false) | ||
| val dest = new File( | ||
| SparkFiles.getRootDirectory(), | ||
| if (uri.getFragment != null) uri.getFragment else source.getName) | ||
| logInfo( | ||
| s"Unpacking an archive $path from ${source.getAbsolutePath} to ${dest.getAbsolutePath}") | ||
| Utils.deleteRecursively(dest) | ||
| Utils.unpack(source, dest) | ||
| postEnvironmentUpdate() | ||
| } else { | ||
| logWarning(s"The path $path has been added already. Overwriting of added paths " + | ||
| "is not supported in the current version.") | ||
|
|
@@ -2494,8 +2560,9 @@ class SparkContext(config: SparkConf) extends Logging { | |
| val schedulingMode = getSchedulingMode.toString | ||
| val addedJarPaths = addedJars.keys.toSeq | ||
| val addedFilePaths = addedFiles.keys.toSeq | ||
| val addedArchivePaths = addedArchives.keys.toSeq | ||
| val environmentDetails = SparkEnv.environmentDetails(conf, hadoopConfiguration, | ||
| schedulingMode, addedJarPaths, addedFilePaths) | ||
| schedulingMode, addedJarPaths, addedFilePaths, addedArchivePaths) | ||
| val environmentUpdate = SparkListenerEnvironmentUpdate(environmentDetails) | ||
| listenerBus.post(environmentUpdate) | ||
| } | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.