Skip to content

Commit

Permalink
[SPARK-34087][FOLLOW-UP][SQL] Manage ExecutionListenerBus register in…
Browse files Browse the repository at this point in the history
…side itself

### What changes were proposed in this pull request?

Move `ExecutionListenerBus` register (both `ListenerBus` and `ContextCleaner` register) into  itself.

Also with a minor change that put `registerSparkListenerForCleanup` to a better place.

### Why are the changes needed?

improve code

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass existing tests.

Closes apache#31919 from Ngone51/SPARK-34087-followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
  • Loading branch information
Ngone51 authored and cloud-fan committed Mar 23, 2021
1 parent d32bb4e commit e00afd3
Show file tree
Hide file tree
Showing 2 changed files with 15 additions and 11 deletions.
10 changes: 5 additions & 5 deletions core/src/main/scala/org/apache/spark/ContextCleaner.scala
Original file line number Diff line number Diff line change
Expand Up @@ -172,18 +172,18 @@ private[spark] class ContextCleaner(
registerForCleanup(rdd, CleanCheckpoint(parentId))
}

/** Register an object for cleanup. */
private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask): Unit = {
referenceBuffer.add(new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue))
}

/** Register a SparkListener to be cleaned up when its owner is garbage collected. */
def registerSparkListenerForCleanup(
listenerOwner: AnyRef,
listener: SparkListener): Unit = {
registerForCleanup(listenerOwner, CleanSparkListener(listener))
}

/** Register an object for cleanup. */
private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask): Unit = {
referenceBuffer.add(new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue))
}

/** Keep cleaning RDD, shuffle, and broadcast state. */
private def keepCleaning(): Unit = Utils.tryOrStopSparkContext(sc) {
while (!stopped) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,11 +76,7 @@ trait QueryExecutionListener {
class ExecutionListenerManager private[sql](session: SparkSession, loadExtensions: Boolean)
extends Logging {

private val listenerBus = new ExecutionListenerBus(session.sessionUUID)
session.sparkContext.listenerBus.addToSharedQueue(listenerBus)
session.sparkContext.cleaner.foreach { cleaner =>
cleaner.registerSparkListenerForCleanup(this, listenerBus)
}
private val listenerBus = new ExecutionListenerBus(this, session)

if (loadExtensions) {
val conf = session.sparkContext.conf
Expand Down Expand Up @@ -128,9 +124,17 @@ class ExecutionListenerManager private[sql](session: SparkSession, loadExtension
}
}

private[sql] class ExecutionListenerBus(sessionUUID: String)
private[sql] class ExecutionListenerBus private(sessionUUID: String)
extends SparkListener with ListenerBus[QueryExecutionListener, SparkListenerSQLExecutionEnd] {

def this(manager: ExecutionListenerManager, session: SparkSession) = {
this(session.sessionUUID)
session.sparkContext.listenerBus.addToSharedQueue(this)
session.sparkContext.cleaner.foreach { cleaner =>
cleaner.registerSparkListenerForCleanup(manager, this)
}
}

override def onOtherEvent(event: SparkListenerEvent): Unit = event match {
case e: SparkListenerSQLExecutionEnd => postToAll(e)
case _ =>
Expand Down

0 comments on commit e00afd3

Please sign in to comment.