From bc87cc410fae59660c13b6ae1c14204df77237b8 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 7 Oct 2014 12:20:12 -0700 Subject: [SPARK-3731] [PySpark] fix memory leak in PythonRDD The parent.getOrCompute() of PythonRDD is executed in a separated thread, it should release the memory reserved for shuffle and unrolling finally. Author: Davies Liu Closes #2668 from davies/leak and squashes the following commits: ae98be2 [Davies Liu] fix memory leak in PythonRDD --- core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala | 5 +++++ 1 file changed, 5 insertions(+) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index ad6eb9ef50..c74f86548e 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -247,6 +247,11 @@ private[spark] class PythonRDD( // will kill the whole executor (see org.apache.spark.executor.Executor). _exception = e worker.shutdownOutput() + } finally { + // Release memory used by this thread for shuffles + env.shuffleMemoryManager.releaseMemoryForThisThread() + // Release memory used by this thread for unrolling blocks + env.blockManager.memoryStore.releaseUnrollMemoryForThisThread() } } } -- cgit v1.2.3