aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/rdd.py
diff options
context:
space:
mode:
authorDavies Liu <davies@databricks.com>2015-04-15 12:58:02 -0700
committerJosh Rosen <joshrosen@databricks.com>2015-04-15 12:58:02 -0700
commitf11288d5272bc18585b8cad4ee3bd59eade7c296 (patch)
tree0016cd5ccb7c7025d00ce79c3aefc3b9799420c5 /python/pyspark/rdd.py
parent6c5ed8a6d552abd967d27cdb94b68d46ccb57221 (diff)
downloadspark-f11288d5272bc18585b8cad4ee3bd59eade7c296.tar.gz
spark-f11288d5272bc18585b8cad4ee3bd59eade7c296.tar.bz2
spark-f11288d5272bc18585b8cad4ee3bd59eade7c296.zip
[SPARK-6886] [PySpark] fix big closure with shuffle
Currently, the created broadcast object will have same life cycle as RDD in Python. For multistage jobs, an PythonRDD will be created in JVM and the RDD in Python may be GCed, then the broadcast will be destroyed in JVM before the PythonRDD. This PR change to use PythonRDD to track the lifecycle of the broadcast object. It also have a refactor about getNumPartitions() to avoid unnecessary creation of PythonRDD, which could be heavy. cc JoshRosen Author: Davies Liu <davies@databricks.com> Closes #5496 from davies/big_closure and squashes the following commits: 9a0ea4c [Davies Liu] fix big closure with shuffle
Diffstat (limited to 'python/pyspark/rdd.py')
-rw-r--r--python/pyspark/rdd.py15
1 files changed, 5 insertions, 10 deletions
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index c9ac95d117..93e658eded 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -1197,7 +1197,7 @@ class RDD(object):
[91, 92, 93]
"""
items = []
- totalParts = self._jrdd.partitions().size()
+ totalParts = self.getNumPartitions()
partsScanned = 0
while len(items) < num and partsScanned < totalParts:
@@ -1260,7 +1260,7 @@ class RDD(object):
>>> sc.parallelize([1]).isEmpty()
False
"""
- return self._jrdd.partitions().size() == 0 or len(self.take(1)) == 0
+ return self.getNumPartitions() == 0 or len(self.take(1)) == 0
def saveAsNewAPIHadoopDataset(self, conf, keyConverter=None, valueConverter=None):
"""
@@ -2235,11 +2235,9 @@ def _prepare_for_python_RDD(sc, command, obj=None):
ser = CloudPickleSerializer()
pickled_command = ser.dumps((command, sys.version_info[:2]))
if len(pickled_command) > (1 << 20): # 1M
+ # The broadcast will have same life cycle as created PythonRDD
broadcast = sc.broadcast(pickled_command)
pickled_command = ser.dumps(broadcast)
- # tracking the life cycle by obj
- if obj is not None:
- obj._broadcast = broadcast
broadcast_vars = ListConverter().convert(
[x._jbroadcast for x in sc._pickled_broadcast_vars],
sc._gateway._gateway_client)
@@ -2294,12 +2292,9 @@ class PipelinedRDD(RDD):
self._jrdd_deserializer = self.ctx.serializer
self._bypass_serializer = False
self.partitioner = prev.partitioner if self.preservesPartitioning else None
- self._broadcast = None
- def __del__(self):
- if self._broadcast:
- self._broadcast.unpersist()
- self._broadcast = None
+ def getNumPartitions(self):
+ return self._prev_jrdd.partitions().size()
@property
def _jrdd(self):