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
This commit is contained in:
parent
baff7e9361
commit
da35330e83
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
Loading…
Reference in a new issue