aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/storagelevel.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/storagelevel.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/storagelevel.py')
-rw-r--r--python/pyspark/storagelevel.py31
1 files changed, 21 insertions, 10 deletions
diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py
index 676aa0f714..d4f184a85d 100644
--- a/python/pyspark/storagelevel.py
+++ b/python/pyspark/storagelevel.py
@@ -23,8 +23,10 @@ class StorageLevel(object):
"""
Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory,
whether to drop the RDD to disk if it falls out of memory, whether to keep the data in memory
- in a serialized format, and whether to replicate the RDD partitions on multiple nodes.
- Also contains static constants for some commonly used storage levels, such as MEMORY_ONLY.
+ in a JAVA-specific serialized format, and whether to replicate the RDD partitions on multiple
+ nodes. Also contains static constants for some commonly used storage levels, MEMORY_ONLY.
+ Since the data is always serialized on the Python side, all the constants use the serialized
+ formats.
"""
def __init__(self, useDisk, useMemory, useOffHeap, deserialized, replication=1):
@@ -49,12 +51,21 @@ class StorageLevel(object):
StorageLevel.DISK_ONLY = StorageLevel(True, False, False, False)
StorageLevel.DISK_ONLY_2 = StorageLevel(True, False, False, False, 2)
-StorageLevel.MEMORY_ONLY = StorageLevel(False, True, False, True)
-StorageLevel.MEMORY_ONLY_2 = StorageLevel(False, True, False, True, 2)
-StorageLevel.MEMORY_ONLY_SER = StorageLevel(False, True, False, False)
-StorageLevel.MEMORY_ONLY_SER_2 = StorageLevel(False, True, False, False, 2)
-StorageLevel.MEMORY_AND_DISK = StorageLevel(True, True, False, True)
-StorageLevel.MEMORY_AND_DISK_2 = StorageLevel(True, True, False, True, 2)
-StorageLevel.MEMORY_AND_DISK_SER = StorageLevel(True, True, False, False)
-StorageLevel.MEMORY_AND_DISK_SER_2 = StorageLevel(True, True, False, False, 2)
+StorageLevel.MEMORY_ONLY = StorageLevel(False, True, False, False)
+StorageLevel.MEMORY_ONLY_2 = StorageLevel(False, True, False, False, 2)
+StorageLevel.MEMORY_AND_DISK = StorageLevel(True, True, False, False)
+StorageLevel.MEMORY_AND_DISK_2 = StorageLevel(True, True, False, False, 2)
StorageLevel.OFF_HEAP = StorageLevel(False, False, True, False, 1)
+
+"""
+.. note:: The following four storage level constants are deprecated in 2.0, since the records \
+will always be serialized in Python.
+"""
+StorageLevel.MEMORY_ONLY_SER = StorageLevel.MEMORY_ONLY
+""".. note:: Deprecated in 2.0, use ``StorageLevel.MEMORY_ONLY`` instead."""
+StorageLevel.MEMORY_ONLY_SER_2 = StorageLevel.MEMORY_ONLY_2
+""".. note:: Deprecated in 2.0, use ``StorageLevel.MEMORY_ONLY_2`` instead."""
+StorageLevel.MEMORY_AND_DISK_SER = StorageLevel.MEMORY_AND_DISK
+""".. note:: Deprecated in 2.0, use ``StorageLevel.MEMORY_AND_DISK`` instead."""
+StorageLevel.MEMORY_AND_DISK_SER_2 = StorageLevel.MEMORY_AND_DISK_2
+""".. note:: Deprecated in 2.0, use ``StorageLevel.MEMORY_AND_DISK_2`` instead."""