-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-31354] SparkContext only register one SparkSession ApplicationEnd listener #28128
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
|
Hey @srowen - could I get an "ok to test" on here? also thoughts on this approach? |
|
Jenkins test this please |
|
Test build #120948 has finished for PR 28128 at commit
|
|
hey @srowen - apologies for the repeated ping but I think you had some thoughts about this one. is this approach preferred compared to my previous one? tagging some interested parties from the other ticket as well @cloud-fan @jiangxb1987 @rdblue @HyukjinKwon @britishbadger |
|
I'm not against adding a clear method to free a session, but I'd like to understand more about the memory leak. According to #24807 (comment) , if the memory leak is caused by the listener, can we simply register that listener only once? That listener just null out the |
|
Hi @cloud-fan - Good question! A few thoughts here:
Does that make sense? |
|
per-session listeners is a common use case. Does the new method remove these user-registered per-session listeners? (registered by AFAIK the reported issue is memory leak. If it's caused by listeners, I think there are 2 ways to fix it:
Can you explain more about how we leak the spark session instance? Then we can discuss how to fix it. |
sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
Outdated
Show resolved
Hide resolved
|
Hey @cloud-fan - Sure, right now the listener issue is coupled with the operating model for Currently, every time a Once the listener is created, because of the way it is created, the reference is lost after it is attached to the SparkContext. This means that there is currently no way to remove the listener from the SparkContext (even after the lifetime of the session is "done"). Even if I call I think my PR description may have been unclear- there isn't a leak of the The reason this PR does more than just removing the listeners, is because there isn't a lifecycle method that actually allows to mark the end of a So, in order to ever clean up the listener leak, we need a way to mark the session as over, and that currently doesn't exist. Does that make sense? |
|
If the only problem is https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala#L947 Can we just register it only once? The listener watches the |
|
@cloud-fan - Okay, I've pushed a fix to immediately resolve this leak. The context keep track of whether the Does this approach make sense to you? |
| val startTime = System.currentTimeMillis() | ||
|
|
||
| private[spark] val stopped: AtomicBoolean = new AtomicBoolean(false) | ||
| private[spark] val sessionListenerRegistered: AtomicBoolean = new AtomicBoolean(false) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
can we move this to object SparkSession?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not easily - that was my original plan, however a SparkSession object can be constructed with an instance of SparkContext. Meaning that it's really on the SparkContext to know whether a session has already attached a listener. The SparkSession instance can then decide whether or not to attach the listener, but the context will need to report whether the listener is attached first.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry if I missed anything. object SparkSession is a singleton and it can make sure we do something only once. Are you saying that we can register a session cleanup listener to SparkContext outside of SparkSession?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah so the challenge is that even though SparkSession is a singleton, I can create various spark session instances that are all "backed" by a different spark context instance.
val conf = new SparkConf()
.setMaster("local")
.setAppName("test-app-SPARK-31354-1")
val context = new SparkContext(conf)
val session1 = SparkSession.builder()
.master("local")
.sparkContext(context) // <- here
.getOrCreate()
So, the state actually needs to moreso be on the context, since the context is the only one that knows whether it has already had a session listener attached. I agree that this feels really weird, and that's kind of why I was initially pursuing adding lifecycle methods to encapsulate the state in one place. Definitely open to other ways to resolve this though.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think we can assume there is only one SparkContext instance. See #23311
One problem is we create a SparkContext, stop it and create it again. Then it's not safe to rely on the flag in singleton object SparkSession
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If we really want to guide against the corner case, we can track the SparkContext instance id in object SparkSession. If SparkContext is re-created, we can reset the boolean flag.
It's a bit hacky to leak the session concept to the core module.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@cloud-fan - updated, how does this look?
|
ok to test |
sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
Outdated
Show resolved
Hide resolved
sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
Outdated
Show resolved
Hide resolved
|
Test build #122886 has finished for PR 28128 at commit
|
|
Thanks, @cloud-fan and @stczwd. Did some cleanup and added a test. If anyone was curious, before this PR, running the following would result in a After this PR, the execution of the same code above results in |
|
Test build #122905 has finished for PR 28128 at commit
|
|
|
||
| /** Register the AppEnd listener onto the Context */ | ||
| private def registerContextListener(sparkContext: SparkContext): Unit = { | ||
| if (!SparkSession.listenerRegistered.get()) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: we are in the same class so we can remove SparkSession.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
fixed!
| defaultSession.set(null) | ||
| } | ||
| }) | ||
| SparkSession.listenerRegistered.set(true) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ditto
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
fixed!
|
Test build #122929 has finished for PR 28128 at commit
|
|
thanks, merging to master/3.0! |
…End listener ## What changes were proposed in this pull request? This change was made as a result of the conversation on https://issues.apache.org/jira/browse/SPARK-31354 and is intended to continue work from that ticket here. This change fixes a memory leak where SparkSession listeners are never cleared off of the SparkContext listener bus. Before running this PR, the following code: ``` SparkSession.builder().master("local").getOrCreate() SparkSession.clearActiveSession() SparkSession.clearDefaultSession() SparkSession.builder().master("local").getOrCreate() SparkSession.clearActiveSession() SparkSession.clearDefaultSession() ``` would result in a SparkContext with the following listeners on the listener bus: ``` [org.apache.spark.status.AppStatusListener5f610071, org.apache.spark.HeartbeatReceiverd400c17, org.apache.spark.sql.SparkSession$$anon$125849aeb, <-First instance org.apache.spark.sql.SparkSession$$anon$1fadb9a0] <- Second instance ``` After this PR, the execution of the same code above results in SparkContext with the following listeners on the listener bus: ``` [org.apache.spark.status.AppStatusListener5f610071, org.apache.spark.HeartbeatReceiverd400c17, org.apache.spark.sql.SparkSession$$anon$125849aeb] <-One instance ``` ## How was this patch tested? * Unit test included as a part of the PR Closes #28128 from vinooganesh/vinooganesh/SPARK-27958. Lead-authored-by: Vinoo Ganesh <[email protected]> Co-authored-by: Vinoo Ganesh <[email protected]> Co-authored-by: Vinoo Ganesh <[email protected]> Signed-off-by: Wenchen Fan <[email protected]> (cherry picked from commit dae7988) Signed-off-by: Wenchen Fan <[email protected]>
|
Hi, @vinooganesh and @cloud-fan It can be reproduced easily as |
|
@advancedxy - can you file a new jira ticket? I can look into fixing this. |
@vinooganesh filed https://issues.apache.org/jira/browse/SPARK-32165 |
…End listener This change was made as a result of the conversation on https://issues.apache.org/jira/browse/SPARK-31354 and is intended to continue work from that ticket here. This change fixes a memory leak where SparkSession listeners are never cleared off of the SparkContext listener bus. Before running this PR, the following code: ``` SparkSession.builder().master("local").getOrCreate() SparkSession.clearActiveSession() SparkSession.clearDefaultSession() SparkSession.builder().master("local").getOrCreate() SparkSession.clearActiveSession() SparkSession.clearDefaultSession() ``` would result in a SparkContext with the following listeners on the listener bus: ``` [org.apache.spark.status.AppStatusListener5f610071, org.apache.spark.HeartbeatReceiverd400c17, org.apache.spark.sql.SparkSession$$anon$125849aeb, <-First instance org.apache.spark.sql.SparkSession$$anon$1fadb9a0] <- Second instance ``` After this PR, the execution of the same code above results in SparkContext with the following listeners on the listener bus: ``` [org.apache.spark.status.AppStatusListener5f610071, org.apache.spark.HeartbeatReceiverd400c17, org.apache.spark.sql.SparkSession$$anon$125849aeb] <-One instance ``` * Unit test included as a part of the PR Closes apache#28128 from vinooganesh/vinooganesh/SPARK-27958. Lead-authored-by: Vinoo Ganesh <[email protected]> Co-authored-by: Vinoo Ganesh <[email protected]> Co-authored-by: Vinoo Ganesh <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>

What changes were proposed in this pull request?
This change was made as a result of the conversation on https://issues.apache.org/jira/browse/SPARK-31354 and is intended to continue work from that ticket here.
This change fixes a memory leak where SparkSession listeners are never cleared off of the SparkContext listener bus.
Before running this PR, the following code:
would result in a SparkContext with the following listeners on the listener bus:
After this PR, the execution of the same code above results in SparkContext with the following listeners on the listener bus:
How was this patch tested?