-
Notifications
You must be signed in to change notification settings - Fork 28.3k
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
[SPARK-48370][CONNECT][FOLLOW-UP] Use JDK's Cleaner instead #46726
Closed
Closed
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,130 +17,33 @@ | |
|
||
package org.apache.spark.sql.internal | ||
|
||
import java.lang.ref.{ReferenceQueue, WeakReference} | ||
import java.util.Collections | ||
import java.util.concurrent.ConcurrentHashMap | ||
import java.lang.ref.Cleaner | ||
|
||
import org.apache.spark.connect.proto | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.sql.SparkSession | ||
|
||
/** | ||
* Classes that represent cleaning tasks. | ||
*/ | ||
private sealed trait CleanupTask | ||
private case class CleanupCachedRemoteRelation(dfID: String) extends CleanupTask | ||
|
||
/** | ||
* A WeakReference associated with a CleanupTask. | ||
* | ||
* When the referent object becomes only weakly reachable, the corresponding | ||
* CleanupTaskWeakReference is automatically added to the given reference queue. | ||
*/ | ||
private class CleanupTaskWeakReference( | ||
val task: CleanupTask, | ||
referent: AnyRef, | ||
referenceQueue: ReferenceQueue[AnyRef]) | ||
extends WeakReference(referent, referenceQueue) | ||
|
||
/** | ||
* An asynchronous cleaner for objects. | ||
* | ||
* This maintains a weak reference for each CashRemoteRelation, etc. of interest, to be processed | ||
* when the associated object goes out of scope of the application. Actual cleanup is performed in | ||
* a separate daemon thread. | ||
*/ | ||
private[sql] class SessionCleaner(session: SparkSession) extends Logging { | ||
|
||
/** | ||
* How often (seconds) to trigger a garbage collection in this JVM. This context cleaner | ||
* triggers cleanups only when weak references are garbage collected. In long-running | ||
* applications with large driver JVMs, where there is little memory pressure on the driver, | ||
* this may happen very occasionally or not at all. Not cleaning at all may lead to executors | ||
* running out of disk space after a while. | ||
*/ | ||
private val refQueuePollTimeout: Long = 100 | ||
|
||
/** | ||
* A buffer to ensure that `CleanupTaskWeakReference`s are not garbage collected as long as they | ||
* have not been handled by the reference queue. | ||
*/ | ||
private val referenceBuffer = | ||
Collections.newSetFromMap[CleanupTaskWeakReference](new ConcurrentHashMap) | ||
|
||
private val referenceQueue = new ReferenceQueue[AnyRef] | ||
|
||
private val cleaningThread = new Thread() { override def run(): Unit = keepCleaning() } | ||
|
||
@volatile private var started = false | ||
@volatile private var stopped = false | ||
|
||
/** Start the cleaner. */ | ||
def start(): Unit = { | ||
cleaningThread.setDaemon(true) | ||
cleaningThread.setName("Spark Connect Context Cleaner") | ||
cleaningThread.start() | ||
} | ||
|
||
/** | ||
* Stop the cleaning thread and wait until the thread has finished running its current task. | ||
*/ | ||
def stop(): Unit = { | ||
stopped = true | ||
// Interrupt the cleaning thread, but wait until the current task has finished before | ||
// doing so. This guards against the race condition where a cleaning thread may | ||
// potentially clean similarly named variables created by a different SparkSession. | ||
synchronized { | ||
cleaningThread.interrupt() | ||
} | ||
cleaningThread.join() | ||
} | ||
private val cleaner = Cleaner.create() | ||
|
||
/** Register a CachedRemoteRelation for cleanup when it is garbage collected. */ | ||
def registerCachedRemoteRelationForCleanup(relation: proto.CachedRemoteRelation): Unit = { | ||
registerForCleanup(relation, CleanupCachedRemoteRelation(relation.getRelationId)) | ||
} | ||
|
||
/** Register an object for cleanup. */ | ||
private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask): Unit = { | ||
if (!started) { | ||
// Lazily starts when the first cleanup is registered. | ||
start() | ||
started = true | ||
} | ||
referenceBuffer.add(new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue)) | ||
def register(relation: proto.CachedRemoteRelation): Unit = { | ||
val dfID = relation.getRelationId | ||
cleaner.register(relation, () => doCleanupCachedRemoteRelation(dfID)) | ||
} | ||
|
||
/** Keep cleaning objects. */ | ||
private def keepCleaning(): Unit = { | ||
while (!stopped && !session.client.channel.isShutdown) { | ||
try { | ||
val reference = Option(referenceQueue.remove(refQueuePollTimeout)) | ||
.map(_.asInstanceOf[CleanupTaskWeakReference]) | ||
// Synchronize here to avoid being interrupted on stop() | ||
synchronized { | ||
reference.foreach { ref => | ||
logDebug("Got cleaning task " + ref.task) | ||
referenceBuffer.remove(ref) | ||
ref.task match { | ||
case CleanupCachedRemoteRelation(dfID) => | ||
doCleanupCachedRemoteRelation(dfID) | ||
} | ||
private[sql] def doCleanupCachedRemoteRelation(dfID: String): Unit = { | ||
try { | ||
if (!session.client.channel.isShutdown) { | ||
session.execute { | ||
session.newCommand { builder => | ||
builder.getRemoveCachedRemoteRelationCommandBuilder | ||
.setRelation(proto.CachedRemoteRelation.newBuilder().setRelationId(dfID).build()) | ||
} | ||
} | ||
} catch { | ||
case e: Throwable => logError("Error in cleaning thread", e) | ||
} | ||
} | ||
} | ||
|
||
/** Perform CleanupCachedRemoteRelation cleanup. */ | ||
private[spark] def doCleanupCachedRemoteRelation(dfID: String): Unit = { | ||
session.execute { | ||
session.newCommand { builder => | ||
builder.getRemoveCachedRemoteRelationCommandBuilder | ||
.setRelation(proto.CachedRemoteRelation.newBuilder().setRelationId(dfID).build()) | ||
} | ||
} catch { | ||
case e: Throwable => logError("Error in cleaning thread", e) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The error is swollen but I think it's better to explicitly log |
||
} | ||
} | ||
} |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
It seems that the lifecycle of
SessionCleaner
is the same asSparkSession
, so when the client holds multipleSparkSession
s, multiple instances ofjava.lang.ref.Cleaner
will be created. Ifcleaner
is defined in the companion object ofSessionCleaner
, it can allow multipleSessionCleaner
to share onejava.lang.ref.Cleaner
instance . Can this meet the requirements?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.
We can have one and share between other sessions but wanted to scope the cleaning specific to a session so it doesn't affect other sessions. I am fine either way 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.
We can do this in a follow-up? @LuciferYang is there any concrete concern here? Or are you just being tidy?
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.
It's fine, we can make follow-up when this really becomes an issue :)