Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions python/pyspark/conf.py
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,12 @@ def set(self, key, value):
self._jconf.set(key, unicode(value))
return self

def setIfMissing(self, key, value):
"""Set a configuration property, if not already set."""
if self.get(key) == None:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if key not in self:

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We may have to make SparkConf iterable for this to work. It does not work as is.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah, yes. but "self.get(key) is None" sounds more Pythonic :-)

self.set(key, value)
return self

def setMaster(self, value):
"""Set master URL to connect to."""
self._jconf.setMaster(value)
Expand Down
2 changes: 1 addition & 1 deletion python/pyspark/context.py
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,7 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None,
else:
self.serializer = BatchedSerializer(self._unbatched_serializer,
batchSize)

self._conf.setIfMissing("spark.rdd.compress", "true")
# Set any parameters passed directly to us on the conf
if master:
self._conf.setMaster(master)
Expand Down
4 changes: 2 additions & 2 deletions python/pyspark/rdd.py
Original file line number Diff line number Diff line change
Expand Up @@ -194,10 +194,10 @@ def context(self):

def cache(self):
"""
Persist this RDD with the default storage level (C{MEMORY_ONLY}).
Persist this RDD with the default storage level (C{MEMORY_ONLY_SER}).
"""
self.is_cached = True
self._jrdd.cache()
self.persist(StorageLevel.MEMORY_ONLY_SER)
return self

def persist(self, storageLevel):
Expand Down