diff options
author | Josh Rosen <joshrosen@eecs.berkeley.edu> | 2013-01-23 10:36:18 -0800 |
---|---|---|
committer | Josh Rosen <joshrosen@eecs.berkeley.edu> | 2013-01-23 10:58:50 -0800 |
commit | ae2ed2947d43860c74a8d40767e289ca78073977 (patch) | |
tree | c72a585db233832ae53e52c4e015b8d52813643c /python/pyspark/files.py | |
parent | 35168d9c89904f0dc0bb470c1799f5ca3b04221f (diff) | |
download | spark-ae2ed2947d43860c74a8d40767e289ca78073977.tar.gz spark-ae2ed2947d43860c74a8d40767e289ca78073977.tar.bz2 spark-ae2ed2947d43860c74a8d40767e289ca78073977.zip |
Allow PySpark's SparkFiles to be used from driver
Fix minor documentation formatting issues.
Diffstat (limited to 'python/pyspark/files.py')
-rw-r--r-- | python/pyspark/files.py | 20 |
1 files changed, 17 insertions, 3 deletions
diff --git a/python/pyspark/files.py b/python/pyspark/files.py index de1334f046..98f6a399cc 100644 --- a/python/pyspark/files.py +++ b/python/pyspark/files.py @@ -4,13 +4,15 @@ import os class SparkFiles(object): """ Resolves paths to files added through - L{addFile()<pyspark.context.SparkContext.addFile>}. + L{SparkContext.addFile()<pyspark.context.SparkContext.addFile>}. SparkFiles contains only classmethods; users should not create SparkFiles instances. """ _root_directory = None + _is_running_on_worker = False + _sc = None def __init__(self): raise NotImplementedError("Do not construct SparkFiles objects") @@ -18,7 +20,19 @@ class SparkFiles(object): @classmethod def get(cls, filename): """ - Get the absolute path of a file added through C{addFile()}. + Get the absolute path of a file added through C{SparkContext.addFile()}. """ - path = os.path.join(SparkFiles._root_directory, filename) + path = os.path.join(SparkFiles.getRootDirectory(), filename) return os.path.abspath(path) + + @classmethod + def getRootDirectory(cls): + """ + Get the root directory that contains files added through + C{SparkContext.addFile()}. + """ + if cls._is_running_on_worker: + return cls._root_directory + else: + # This will have to change if we support multiple SparkContexts: + return cls._sc.jvm.spark.SparkFiles.getRootDirectory() |