Github user andrewor14 commented on a diff in the pull request:
https://github.com/apache/spark/pull/10779#discussion_r50048973
--- Diff:
core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala ---
@@ -17,24 +17,169 @@
package org.apache.spark.scheduler
+import java.util.concurrent._
import java.util.concurrent.atomic.AtomicBoolean
-import org.apache.spark.util.AsynchronousListenerBus
+import scala.util.DynamicVariable
+
+import org.apache.spark.SparkContext
+import org.apache.spark.util.Utils
/**
* Asynchronously passes SparkListenerEvents to registered SparkListeners.
*
- * Until start() is called, all posted events are only buffered. Only
after this listener bus
+ * Until `start()` is called, all posted events are only buffered. Only
after this listener bus
* has started will events be actually propagated to all attached
listeners. This listener bus
- * is stopped when it receives a SparkListenerShutdown event, which is
posted using stop().
+ * is stopped when `stop()` is called, and it will drop further events
after stopping.
*/
-private[spark] class LiveListenerBus
- extends AsynchronousListenerBus[SparkListener,
SparkListenerEvent]("SparkListenerBus")
- with SparkListenerBus {
+private[spark] class LiveListenerBus extends SparkListenerBus {
+
+ self =>
+
+ import LiveListenerBus._
+
+ private var sparkContext: SparkContext = null
+
+ /* Cap the capacity of the event queue so we get an explicit error
(rather than
+ * an OOM exception) if it's perpetually being added to more quickly
than it's being drained. */
+ private val EVENT_QUEUE_CAPACITY = 10000
+ private val eventQueue = new
LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY)
+
+ // Indicate if `start()` is called
+ private val started = new AtomicBoolean(false)
+ // Indicate if `stop()` is called
+ private val stopped = new AtomicBoolean(false)
+
+ // Indicate if we are processing some event
+ // Guarded by `self`
+ private var processingEvent = false
private val logDroppedEvent = new AtomicBoolean(false)
- override def onDropEvent(event: SparkListenerEvent): Unit = {
+ // A counter that represents the number of events produced and consumed
in the queue
+ private val eventLock = new Semaphore(0)
+
+ private val listenerThread = new Thread(name) {
+ setDaemon(true)
+ override def run(): Unit = Utils.tryOrStopSparkContext(sparkContext) {
+ LiveListenerBus.withinListenerThread.withValue(true) {
+ while (true) {
+ eventLock.acquire()
+ self.synchronized {
+ processingEvent = true
+ }
+ try {
+ val event = eventQueue.poll
+ if (event == null) {
+ // Get out of the while loop and shutdown the daemon thread
+ if (!stopped.get) {
+ throw new IllegalStateException("Polling `null` from
eventQueue means" +
+ " the listener bus has been stopped. So `stopped` must
be true")
+ }
+ return
+ }
+ postToAll(event)
+ } finally {
+ self.synchronized {
+ processingEvent = false
+ }
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Start sending events to attached listeners.
+ *
+ * This first sends out all buffered events posted before this listener
bus has started, then
+ * listens for any additional events asynchronously while the listener
bus is still running.
+ * This should only be called once.
+ *
+ * @param sc Used to stop the SparkContext in case the listener thread
dies.
+ */
+ def start(sc: SparkContext) {
--- End diff --
`Unit` return type
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]