aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/context.py
diff options
context:
space:
mode:
Diffstat (limited to 'python/pyspark/context.py')
-rw-r--r--python/pyspark/context.py11
1 files changed, 5 insertions, 6 deletions
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 064a24bff5..8e7b00469e 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -171,7 +171,7 @@ class SparkContext(object):
SparkFiles._sc = self
root_dir = SparkFiles.getRootDirectory()
- sys.path.append(root_dir)
+ sys.path.insert(1, root_dir)
# Deploy any code dependencies specified in the constructor
self._python_includes = list()
@@ -183,10 +183,9 @@ class SparkContext(object):
for path in self._conf.get("spark.submit.pyFiles", "").split(","):
if path != "":
(dirname, filename) = os.path.split(path)
- self._python_includes.append(filename)
- sys.path.append(path)
- if dirname not in sys.path:
- sys.path.append(dirname)
+ if filename.lower().endswith("zip") or filename.lower().endswith("egg"):
+ self._python_includes.append(filename)
+ sys.path.insert(1, os.path.join(SparkFiles.getRootDirectory(), filename))
# Create a temporary directory inside spark.local.dir:
local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir(self._jsc.sc().conf())
@@ -667,7 +666,7 @@ class SparkContext(object):
if filename.endswith('.zip') or filename.endswith('.ZIP') or filename.endswith('.egg'):
self._python_includes.append(filename)
# for tests in local mode
- sys.path.append(os.path.join(SparkFiles.getRootDirectory(), filename))
+ sys.path.insert(1, os.path.join(SparkFiles.getRootDirectory(), filename))
def setCheckpointDir(self, dirName):
"""