aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/context.py
diff options
context:
space:
mode:
authorYanbo Liang <ybliang8@gmail.com>2016-09-21 01:37:03 -0700
committerYanbo Liang <ybliang8@gmail.com>2016-09-21 01:37:03 -0700
commitd3b88697638dcf32854fe21a6c53dfb3782773b9 (patch)
tree8d5e243a53fb22c3e092556cc87ccd1c549329d6 /python/pyspark/context.py
parent61876a42793bde0da90f54b44255148ed54b7f61 (diff)
downloadspark-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 'python/pyspark/context.py')
-rw-r--r--python/pyspark/context.py7
1 files changed, 5 insertions, 2 deletions
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 5c32f8ea1d..7a7f59cb50 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -767,7 +767,7 @@ class SparkContext(object):
SparkContext._next_accum_id += 1
return Accumulator(SparkContext._next_accum_id - 1, value, accum_param)
- def addFile(self, path):
+ def addFile(self, path, recursive=False):
"""
Add a file to be downloaded with this Spark job on every node.
The C{path} passed can be either a local file, a file in HDFS
@@ -778,6 +778,9 @@ class SparkContext(object):
L{SparkFiles.get(fileName)<pyspark.files.SparkFiles.get>} with the
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.
+
>>> from pyspark import SparkFiles
>>> path = os.path.join(tempdir, "test.txt")
>>> with open(path, "w") as testFile:
@@ -790,7 +793,7 @@ class SparkContext(object):
>>> sc.parallelize([1, 2, 3, 4]).mapPartitions(func).collect()
[100, 200, 300, 400]
"""
- self._jsc.sc().addFile(path)
+ self._jsc.sc().addFile(path, recursive)
def addPyFile(self, path):
"""