[ https://issues.apache.org/jira/browse/SPARK-32010?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Apache Spark reassigned SPARK-32010: ------------------------------------ Assignee: Hyukjin Kwon (was: Apache Spark) > Thread leaks in pinned thread mode > ---------------------------------- > > Key: SPARK-32010 > URL: https://issues.apache.org/jira/browse/SPARK-32010 > Project: Spark > Issue Type: Bug > Components: PySpark, SQL > Affects Versions: 3.1.0 > Reporter: Hyukjin Kwon > Assignee: Hyukjin Kwon > Priority: Major > > SPARK-22340 introduced a pin thread mode which guarantees you to sync Python > thread and JVM thread. > However, looks like the JVM threads are not finished even when the Python > thread is finished. It can be debugged via YourKit, and run multiple jobs > with multiple threads at the same time. > Easiest reproducer is: > {code} > PYSPARK_PIN_THREAD=true ./bin/pyspark > {code} > {code} > >>> from threading import Thread > >>> Thread(target=lambda: spark.range(1000).collect()).start() > >>> Thread(target=lambda: spark.range(1000).collect()).start() > >>> Thread(target=lambda: spark.range(1000).collect()).start() > >>> spark._jvm._gateway_client.deque > deque([<py4j.clientserver.ClientServerConnection object at 0x119f7aba8>, > <py4j.clientserver.ClientServerConnection object at 0x119fc9b70>, > <py4j.clientserver.ClientServerConnection object at 0x119fc9e10>, > <py4j.clientserver.ClientServerConnection object at 0x11a015358>, > <py4j.clientserver.ClientServerConnection object at 0x119fc00f0>]) > >>> Thread(target=lambda: spark.range(1000).collect()).start() > >>> spark._jvm._gateway_client.deque > deque([<py4j.clientserver.ClientServerConnection object at 0x119f7aba8>, > <py4j.clientserver.ClientServerConnection object at 0x119fc9b70>, > <py4j.clientserver.ClientServerConnection object at 0x119fc9e10>, > <py4j.clientserver.ClientServerConnection object at 0x11a015358>, > <py4j.clientserver.ClientServerConnection object at 0x119fc08d0>, > <py4j.clientserver.ClientServerConnection object at 0x119fc00f0>]) > {code} > The connection doesn't get closed, and it holds JVM thread running. -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org