GitHub user davies opened a pull request:
https://github.com/apache/spark/pull/15089
[SPARK-15621] [SQL] Support spilling for Python UDF
## What changes were proposed in this pull request?
When execute a Python UDF, we buffer the input row into as queue, then pull
them out to join with the result from Python UDF. In the case that Python UDF
is slow or the input row is too wide, we could ran out of memory because of the
queue. Since we can't flush all the buffers (sockets) between JVM and Python
process from JVM side, we can't limit the rows in the queue, otherwise it could
deadlock.
This PR will manage the memory used by the queue, spill that into disk when
there is no enough memory (also release the memory and disk space as soon as
possible).
## How was this patch tested?
Added unit tests. Also manually ran a workload with large input row and
slow python UDF (with large broadcast) like this:
```
b = range(1<<24)
add = udf(lambda x: x + len(b), IntegerType())
df = sqlContext.range(1, 1<<26, 1, 4)
print df.select(df.id, lit("adf"*10000).alias("s"),
add(df.id).alias("add")).groupBy(length("s")).sum().collect()
```
It ran out of memory (hang because of full GC) before the patch, ran
smoothly after the patch.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/davies/spark spill_udf
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/spark/pull/15089.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #15089
----
commit 4964b9a611ed01aaa5252ac642df94db07a38868
Author: Davies Liu <[email protected]>
Date: 2016-09-13T23:47:31Z
spill the buffer for Python UDF into disk
----
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]