[ https://issues.apache.org/jira/browse/SPARK-32165?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17259777#comment-17259777 ]
Xianjin YE commented on SPARK-32165: ------------------------------------ > could we add a bit more detail here as to what problem is or what this >causes? I assume by leak you mean this causes memory leak but no other >function issues? [~tgraves] Please see this comment([https://github.com/apache/spark/pull/28128#issuecomment-653102770]) for the context. The main issues is memory leak, which could be a problem for applications which create SparkSession frequently. > SessionState leaks SparkListener with multiple SparkSession > ----------------------------------------------------------- > > Key: SPARK-32165 > URL: https://issues.apache.org/jira/browse/SPARK-32165 > Project: Spark > Issue Type: Improvement > Components: SQL > Affects Versions: 3.0.0 > Reporter: Xianjin YE > Priority: Major > > Copied from > [https://github.com/apache/spark/pull/28128#issuecomment-653102770] > > {code:java} > test("SPARK-31354: SparkContext only register one SparkSession > ApplicationEnd listener") { > val conf = new SparkConf() > .setMaster("local") > .setAppName("test-app-SPARK-31354-1") > val context = new SparkContext(conf) > SparkSession > .builder() > .sparkContext(context) > .master("local") > .getOrCreate() > .sessionState // this touches the sessionState > val postFirstCreation = context.listenerBus.listeners.size() > SparkSession.clearActiveSession() > SparkSession.clearDefaultSession() > SparkSession > .builder() > .sparkContext(context) > .master("local") > .getOrCreate() > .sessionState // this touches the sessionState > val postSecondCreation = context.listenerBus.listeners.size() > SparkSession.clearActiveSession() > SparkSession.clearDefaultSession() > assert(postFirstCreation == postSecondCreation) > } > {code} > The problem can be reproduced by the above code. -- 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