aboutsummaryrefslogtreecommitdiff
path: root/python
diff options
context:
space:
mode:
Diffstat (limited to 'python')
-rw-r--r--python/pyspark/rdd.py7
-rw-r--r--python/pyspark/sql.py3
2 files changed, 8 insertions, 2 deletions
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 04f13523b4..aa90297855 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -212,11 +212,16 @@ class RDD(object):
self.persist(StorageLevel.MEMORY_ONLY_SER)
return self
- def persist(self, storageLevel):
+ def persist(self, storageLevel=StorageLevel.MEMORY_ONLY_SER):
"""
Set this RDD's storage level to persist its values across operations
after the first time it is computed. This can only be used to assign
a new storage level if the RDD does not have a storage level set yet.
+ If no storage level is specified defaults to (C{MEMORY_ONLY_SER}).
+
+ >>> rdd = sc.parallelize(["b", "a", "c"])
+ >>> rdd.persist().is_cached
+ True
"""
self.is_cached = True
javaStorageLevel = self.ctx._getJavaStorageLevel(storageLevel)
diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py
index e7f573cf6d..97a51b9f8a 100644
--- a/python/pyspark/sql.py
+++ b/python/pyspark/sql.py
@@ -29,6 +29,7 @@ from operator import itemgetter
from pyspark.rdd import RDD, PipelinedRDD
from pyspark.serializers import BatchedSerializer, PickleSerializer, CloudPickleSerializer
+from pyspark.storagelevel import StorageLevel
from itertools import chain, ifilter, imap
@@ -1665,7 +1666,7 @@ class SchemaRDD(RDD):
self._jschema_rdd.cache()
return self
- def persist(self, storageLevel):
+ def persist(self, storageLevel=StorageLevel.MEMORY_ONLY_SER):
self.is_cached = True
javaStorageLevel = self.ctx._getJavaStorageLevel(storageLevel)
self._jschema_rdd.persist(javaStorageLevel)