diff options
author | Ameet Talwalkar <atalwalkar@gmail.com> | 2013-09-08 18:41:38 -0700 |
---|---|---|
committer | Ameet Talwalkar <atalwalkar@gmail.com> | 2013-09-08 18:41:38 -0700 |
commit | bf280c8b0faa542061a42f9ea56e93380b6d37f8 (patch) | |
tree | 5640cfb29ccc3098e550e93233dda1e2e889a657 /python/pyspark/rdd.py | |
parent | 5ac62dbbd0d604d699017a5956f3c79172e09896 (diff) | |
parent | f68848d95d896b578235c063be51483b4fce518e (diff) | |
download | spark-bf280c8b0faa542061a42f9ea56e93380b6d37f8.tar.gz spark-bf280c8b0faa542061a42f9ea56e93380b6d37f8.tar.bz2 spark-bf280c8b0faa542061a42f9ea56e93380b6d37f8.zip |
Merge remote-tracking branch 'upstream/master'
Diffstat (limited to 'python/pyspark/rdd.py')
-rw-r--r-- | python/pyspark/rdd.py | 19 |
1 files changed, 19 insertions, 0 deletions
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 914118ccdd..58e1849cad 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -70,6 +70,25 @@ 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 + javaStorageLevel = self.ctx._getJavaStorageLevel(storageLevel) + self._jrdd.persist(javaStorageLevel) + 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 |