From 5531830246dd3ee9e1feed96ec565c7405ee8b95 Mon Sep 17 00:00:00 2001 From: Davies Liu <davies.liu@gmail.com> Date: Tue, 7 Oct 2014 12:20:12 -0700 Subject: [PATCH] [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 <davies.liu@gmail.com> Closes #2668 from davies/leak and squashes the following commits: ae98be2 [Davies Liu] fix memory leak in PythonRDD (cherry picked from commit bc87cc410fae59660c13b6ae1c14204df77237b8) Signed-off-by: Josh Rosen <joshrosen@apache.org> Conflicts: core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala --- .../main/scala/org/apache/spark/api/python/PythonRDD.scala | 4 ++++ 1 file changed, 4 insertions(+) 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 747023812f..2d41502a06 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 @@ -218,6 +218,10 @@ private[spark] class PythonRDD( _exception = e } finally { Try(worker.shutdownOutput()) // kill Python worker process + // Release memory used by this thread for shuffles + env.shuffleMemoryManager.releaseMemoryForThisThread() + // Release memory used by this thread for unrolling blocks + env.blockManager.memoryStore.releaseUnrollMemoryForThisThread() } } } -- GitLab