diff options
author | Yanbo Liang <ybliang8@gmail.com> | 2016-09-21 01:37:03 -0700 |
---|---|---|
committer | Yanbo Liang <ybliang8@gmail.com> | 2016-09-21 01:37:03 -0700 |
commit | d3b88697638dcf32854fe21a6c53dfb3782773b9 (patch) | |
tree | 8d5e243a53fb22c3e092556cc87ccd1c549329d6 /core | |
parent | 61876a42793bde0da90f54b44255148ed54b7f61 (diff) | |
download | spark-d3b88697638dcf32854fe21a6c53dfb3782773b9.tar.gz spark-d3b88697638dcf32854fe21a6c53dfb3782773b9.tar.bz2 spark-d3b88697638dcf32854fe21a6c53dfb3782773b9.zip |
[SPARK-17585][PYSPARK][CORE] PySpark SparkContext.addFile supports adding files recursively
## What changes were proposed in this pull request?
Users would like to add a directory as dependency in some cases, they can use ```SparkContext.addFile``` with argument ```recursive=true``` to recursively add all files under the directory by using Scala. But Python users can only add file not directory, we should also make it supported.
## How was this patch tested?
Unit test.
Author: Yanbo Liang <ybliang8@gmail.com>
Closes #15140 from yanboliang/spark-17585.
Diffstat (limited to 'core')
-rw-r--r-- | core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala | 13 |
1 files changed, 13 insertions, 0 deletions
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 131f36f547..4e50c2686d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -670,6 +670,19 @@ class JavaSparkContext(val sc: SparkContext) } /** + * Add a file to be downloaded with this Spark job on every node. + * The `path` passed 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(fileName)` to find its download location. + * + * A directory can be given if the recursive option is set to true. Currently directories are only + * supported for Hadoop-supported filesystems. + */ + def addFile(path: String, recursive: Boolean): Unit = { + sc.addFile(path, recursive) + } + + /** * Adds a JAR dependency for all tasks to be executed on this SparkContext in the future. * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported * filesystems), or an HTTP, HTTPS or FTP URI. |