aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/rdd.py
diff options
context:
space:
mode:
authorAaron Davidson <aaron@databricks.com>2013-09-05 23:36:27 -0700
committerAaron Davidson <aaron@databricks.com>2013-09-05 23:36:27 -0700
commita63d4c7dc2970900b116f7287e3d6b302d9d5698 (patch)
treedc30e2dd8435e8a3cb95717c39e491be630fcae3 /python/pyspark/rdd.py
parent714e7f9e32590c302ad315b7cbee72b2e8b32b9b (diff)
downloadspark-a63d4c7dc2970900b116f7287e3d6b302d9d5698.tar.gz
spark-a63d4c7dc2970900b116f7287e3d6b302d9d5698.tar.bz2
spark-a63d4c7dc2970900b116f7287e3d6b302d9d5698.zip
SPARK-660: Add StorageLevel support in Python
It uses reflection... I am not proud of that fact, but it at least ensures compatibility (sans refactoring of the StorageLevel stuff).
Diffstat (limited to 'python/pyspark/rdd.py')
-rw-r--r--python/pyspark/rdd.py18
1 files changed, 18 insertions, 0 deletions
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 914118ccdd..332258f5d1 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -70,6 +70,24 @@ class RDD(object):
self._jrdd.cache()
return self
+ def persist(self, storageLevel):
+ """
+ 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.
+ """
+ self.is_cached = True
+ self._jrdd.persist(storageLevel)
+ return self
+
+ def unpersist(self):
+ """
+ Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.
+ """
+ self.is_cached = False
+ self._jrdd.unpersist()
+ return self
+
def checkpoint(self):
"""
Mark this RDD for checkpointing. It will be saved to a file inside the