From eff9714e1c88e39e28317358ca9ec87677f121dc Mon Sep 17 00:00:00 2001
From: Prashant Sharma <prashant.s@imaginea.com>
Date: Thu, 24 Jul 2014 18:15:37 -0700
Subject: [PATCH] [SPARK-2014] Make PySpark store RDDs in MEMORY_ONLY_SER with
 compression by default

Author: Prashant Sharma <prashant.s@imaginea.com>

Closes #1051 from ScrapCodes/SPARK-2014/pyspark-cache and squashes the following commits:

f192df7 [Prashant Sharma] Code Review
2a2f43f [Prashant Sharma] [SPARK-2014] Make PySpark store RDDs in MEMORY_ONLY_SER with compression by default
---
 python/pyspark/conf.py    | 6 ++++++
 python/pyspark/context.py | 2 +-
 python/pyspark/rdd.py     | 4 ++--
 3 files changed, 9 insertions(+), 3 deletions(-)

diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py
index b50590ab3b..b4c82f519b 100644
--- a/python/pyspark/conf.py
+++ b/python/pyspark/conf.py
@@ -100,6 +100,12 @@ class SparkConf(object):
         self._jconf.set(key, unicode(value))
         return self
 
+    def setIfMissing(self, key, value):
+        """Set a configuration property, if not already set."""
+        if self.get(key) is None:
+            self.set(key, value)
+        return self
+
     def setMaster(self, value):
         """Set master URL to connect to."""
         self._jconf.setMaster(value)
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index e21be0e10a..024fb88187 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -101,7 +101,7 @@ class SparkContext(object):
         else:
             self.serializer = BatchedSerializer(self._unbatched_serializer,
                                                 batchSize)
-
+        self._conf.setIfMissing("spark.rdd.compress", "true")
         # Set any parameters passed directly to us on the conf
         if master:
             self._conf.setMaster(master)
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 94ba22306a..a38dd0b923 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -231,10 +231,10 @@ class RDD(object):
 
     def cache(self):
         """
-        Persist this RDD with the default storage level (C{MEMORY_ONLY}).
+        Persist this RDD with the default storage level (C{MEMORY_ONLY_SER}).
         """
         self.is_cached = True
-        self._jrdd.cache()
+        self.persist(StorageLevel.MEMORY_ONLY_SER)
         return self
 
     def persist(self, storageLevel):
-- 
GitLab