aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/rdd.py
diff options
context:
space:
mode:
authorgatorsmile <gatorsmile@gmail.com>2015-12-18 20:06:05 -0800
committerDavies Liu <davies.liu@gmail.com>2015-12-18 20:06:05 -0800
commit499ac3e69a102f9b10a1d7e14382fa191516f7b5 (patch)
treedcb84cda0ddb75094a39946dfd09f0ed29dd058c /python/pyspark/rdd.py
parenta78a91f4d7239c14bd5d0b18cdc87d55594a8d8a (diff)
downloadspark-499ac3e69a102f9b10a1d7e14382fa191516f7b5.tar.gz
spark-499ac3e69a102f9b10a1d7e14382fa191516f7b5.tar.bz2
spark-499ac3e69a102f9b10a1d7e14382fa191516f7b5.zip
[SPARK-12091] [PYSPARK] Deprecate the JAVA-specific deserialized storage levels
The current default storage level of Python persist API is MEMORY_ONLY_SER. This is different from the default level MEMORY_ONLY in the official document and RDD APIs. davies Is this inconsistency intentional? Thanks! Updates: Since the data is always serialized on the Python side, the storage levels of JAVA-specific deserialization are not removed, such as MEMORY_ONLY. Updates: Based on the reviewers' feedback. In Python, stored objects will always be serialized with the [Pickle](https://docs.python.org/2/library/pickle.html) library, so it does not matter whether you choose a serialized level. The available storage levels in Python include `MEMORY_ONLY`, `MEMORY_ONLY_2`, `MEMORY_AND_DISK`, `MEMORY_AND_DISK_2`, `DISK_ONLY`, `DISK_ONLY_2` and `OFF_HEAP`. Author: gatorsmile <gatorsmile@gmail.com> Closes #10092 from gatorsmile/persistStorageLevel.
Diffstat (limited to 'python/pyspark/rdd.py')
-rw-r--r--python/pyspark/rdd.py8
1 files changed, 4 insertions, 4 deletions
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 00bb9a62e9..a019c05862 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -220,18 +220,18 @@ class RDD(object):
def cache(self):
"""
- Persist this RDD with the default storage level (C{MEMORY_ONLY_SER}).
+ Persist this RDD with the default storage level (C{MEMORY_ONLY}).
"""
self.is_cached = True
- self.persist(StorageLevel.MEMORY_ONLY_SER)
+ self.persist(StorageLevel.MEMORY_ONLY)
return self
- def persist(self, storageLevel=StorageLevel.MEMORY_ONLY_SER):
+ def persist(self, storageLevel=StorageLevel.MEMORY_ONLY):
"""
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}).
+ If no storage level is specified defaults to (C{MEMORY_ONLY}).
>>> rdd = sc.parallelize(["b", "a", "c"])
>>> rdd.persist().is_cached