Skip to content

Commit 31da755

Browse files
jerryshaorxin
authored andcommitted
[SPARK-18975][CORE] Add an API to remove SparkListener
## What changes were proposed in this pull request? In current Spark we could add customized SparkListener through `SparkContext#addListener` API, but there's no equivalent API to remove the registered one. In our scenario SparkListener will be added repeatedly accordingly to the changed environment. If lacks the ability to remove listeners, there might be many registered listeners finally, this is unnecessary and potentially affects the performance. So here propose to add an API to remove registered listener. ## How was this patch tested? Add an unit test to verify it. Author: jerryshao <sshao@hortonworks.com> Closes #16382 from jerryshao/SPARK-18975.
1 parent 2615100 commit 31da755

File tree

2 files changed

+25
-0
lines changed

2 files changed

+25
-0
lines changed

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1571,6 +1571,15 @@ class SparkContext(config: SparkConf) extends Logging {
15711571
listenerBus.addListener(listener)
15721572
}
15731573

1574+
/**
1575+
* :: DeveloperApi ::
1576+
* Deregister the listener from Spark's listener bus.
1577+
*/
1578+
@DeveloperApi
1579+
def removeSparkListener(listener: SparkListenerInterface): Unit = {
1580+
listenerBus.removeListener(listener)
1581+
}
1582+
15741583
private[spark] def getExecutorIds(): Seq[String] = {
15751584
schedulerBackend match {
15761585
case b: CoarseGrainedSchedulerBackend =>

core/src/test/scala/org/apache/spark/SparkContextSuite.scala

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@ import org.apache.hadoop.mapred.TextInputFormat
3131
import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat}
3232
import org.scalatest.Matchers._
3333

34+
import org.apache.spark.scheduler.SparkListener
3435
import org.apache.spark.util.Utils
3536

3637
class SparkContextSuite extends SparkFunSuite with LocalSparkContext {
@@ -451,4 +452,19 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext {
451452
sc.stop()
452453
}
453454
}
455+
456+
test("register and deregister Spark listener from SparkContext") {
457+
sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local"))
458+
try {
459+
val sparkListener1 = new SparkListener { }
460+
val sparkListener2 = new SparkListener { }
461+
sc.addSparkListener(sparkListener1)
462+
sc.addSparkListener(sparkListener2)
463+
assert(sc.listenerBus.listeners.contains(sparkListener1))
464+
assert(sc.listenerBus.listeners.contains(sparkListener2))
465+
sc.removeSparkListener(sparkListener1)
466+
assert(!sc.listenerBus.listeners.contains(sparkListener1))
467+
assert(sc.listenerBus.listeners.contains(sparkListener2))
468+
}
469+
}
454470
}

0 commit comments

Comments
 (0)