Skip to content

[SPARK-6886] [PySpark] fix big closure with shuffle #5496

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
15 changes: 5 additions & 10 deletions python/pyspark/rdd.py
Original file line number Diff line number Diff line change
Expand Up @@ -1197,7 +1197,7 @@ def take(self, num):
[91, 92, 93]
"""
items = []
totalParts = self._jrdd.partitions().size()
totalParts = self.getNumPartitions()
partsScanned = 0

while len(items) < num and partsScanned < totalParts:
Expand Down Expand Up @@ -1260,7 +1260,7 @@ def isEmpty(self):
>>> 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):
"""
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -2294,12 +2292,9 @@ def pipeline_func(split, iterator):
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):
Expand Down
6 changes: 2 additions & 4 deletions python/pyspark/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -550,10 +550,8 @@ def test_large_closure(self):
data = [float(i) for i in xrange(N)]
rdd = self.sc.parallelize(range(1), 1).map(lambda x: len(data))
self.assertEquals(N, rdd.first())
self.assertTrue(rdd._broadcast is not None)
rdd = self.sc.parallelize(range(1), 1).map(lambda x: 1)
self.assertEqual(1, rdd.first())
self.assertTrue(rdd._broadcast is None)
# regression test for SPARK-6886
self.assertEqual(1, rdd.map(lambda x: (x, 1)).groupByKey().count())

def test_zip_with_different_serializers(self):
a = self.sc.parallelize(range(5))
Expand Down