diff options
author | Holden Karau <holden@pigscanfly.ca> | 2014-09-06 14:49:25 -0700 |
---|---|---|
committer | Josh Rosen <joshrosen@apache.org> | 2014-09-06 14:49:25 -0700 |
commit | da35330e830a85008c0bf9f0725418e4dfe7ac66 (patch) | |
tree | 4b9e0a37919e9e0daed35a2ade99a5bc3159437b /python/pyspark/rdd.py | |
parent | baff7e936101635d9bd4245e45335878bafb75e0 (diff) | |
download | spark-da35330e830a85008c0bf9f0725418e4dfe7ac66.tar.gz spark-da35330e830a85008c0bf9f0725418e4dfe7ac66.tar.bz2 spark-da35330e830a85008c0bf9f0725418e4dfe7ac66.zip |
Spark-3406 add a default storage level to python RDD persist API
Author: Holden Karau <holden@pigscanfly.ca>
Closes #2280 from holdenk/SPARK-3406-Python-RDD-persist-api-does-not-have-default-storage-level and squashes the following commits:
33eaade [Holden Karau] As Josh pointed out, sql also override persist. Make persist behave the same as in the underlying RDD as well
e658227 [Holden Karau] Fix the test I added
e95a6c5 [Holden Karau] The Python persist function did not have a default storageLevel unlike the Scala API. Noticed this issue because we got a bug report back from the book where we had documented it as if it was the same as the Scala API
Diffstat (limited to 'python/pyspark/rdd.py')
-rw-r--r-- | python/pyspark/rdd.py | 7 |
1 files changed, 6 insertions, 1 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) |