Skip to content
This repository was archived by the owner on Jan 9, 2020. It is now read-only.

Commit c6ff59a

Browse files
Marcelo Vanzincloud-fan
authored andcommitted
[SPARK-18838][CORE] Add separate listener queues to LiveListenerBus.
This change modifies the live listener bus so that all listeners are added to queues; each queue has its own thread to dispatch events, making it possible to separate slow listeners from other more performance-sensitive ones. The public API has not changed - all listeners added with the existing "addListener" method, which after this change mostly means all user-defined listeners, end up in a default queue. Internally, there's an API allowing listeners to be added to specific queues, and that API is used to separate the internal Spark listeners into 3 categories: application status listeners (e.g. UI), executor management (e.g. dynamic allocation), and the event log. The queueing logic, while abstracted away in a separate class, is kept as much as possible hidden away from consumers. Aside from choosing their queue, there's no code change needed to take advantage of queues. Test coverage relies on existing tests; a few tests had to be tweaked because they relied on `LiveListenerBus.postToAll` being synchronous, and the change makes that method asynchronous. Other tests were simplified not to use the asynchronous LiveListenerBus. Author: Marcelo Vanzin <[email protected]> Closes apache#19211 from vanzin/SPARK-18838.
1 parent 718bbc9 commit c6ff59a

File tree

16 files changed

+473
-290
lines changed

16 files changed

+473
-290
lines changed

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -217,7 +217,7 @@ private[spark] class ExecutorAllocationManager(
217217
* the scheduling task.
218218
*/
219219
def start(): Unit = {
220-
listenerBus.addListener(listener)
220+
listenerBus.addToManagementQueue(listener)
221221

222222
val scheduleTask = new Runnable() {
223223
override def run(): Unit = {

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -63,7 +63,7 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
6363
this(sc, new SystemClock)
6464
}
6565

66-
sc.addSparkListener(this)
66+
sc.listenerBus.addToManagementQueue(this)
6767

6868
override val rpcEnv: RpcEnv = sc.env.rpcEnv
6969

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

Lines changed: 6 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -419,7 +419,7 @@ class SparkContext(config: SparkConf) extends Logging {
419419
// "_jobProgressListener" should be set up before creating SparkEnv because when creating
420420
// "SparkEnv", some messages will be posted to "listenerBus" and we should not miss them.
421421
_jobProgressListener = new JobProgressListener(_conf)
422-
listenerBus.addListener(jobProgressListener)
422+
listenerBus.addToStatusQueue(jobProgressListener)
423423

424424
// Create the Spark execution environment (cache, map output tracker, etc)
425425
_env = createSparkEnv(_conf, isLocal, listenerBus)
@@ -442,7 +442,7 @@ class SparkContext(config: SparkConf) extends Logging {
442442

443443
_ui =
444444
if (conf.getBoolean("spark.ui.enabled", true)) {
445-
Some(SparkUI.createLiveUI(this, _conf, listenerBus, _jobProgressListener,
445+
Some(SparkUI.createLiveUI(this, _conf, _jobProgressListener,
446446
_env.securityManager, appName, startTime = startTime))
447447
} else {
448448
// For tests, do not enable the UI
@@ -522,7 +522,7 @@ class SparkContext(config: SparkConf) extends Logging {
522522
new EventLoggingListener(_applicationId, _applicationAttemptId, _eventLogDir.get,
523523
_conf, _hadoopConfiguration)
524524
logger.start()
525-
listenerBus.addListener(logger)
525+
listenerBus.addToEventLogQueue(logger)
526526
Some(logger)
527527
} else {
528528
None
@@ -1563,7 +1563,7 @@ class SparkContext(config: SparkConf) extends Logging {
15631563
*/
15641564
@DeveloperApi
15651565
def addSparkListener(listener: SparkListenerInterface) {
1566-
listenerBus.addListener(listener)
1566+
listenerBus.addToSharedQueue(listener)
15671567
}
15681568

15691569
/**
@@ -1879,8 +1879,7 @@ class SparkContext(config: SparkConf) extends Logging {
18791879
*/
18801880
def stop(): Unit = {
18811881
if (LiveListenerBus.withinListenerThread.value) {
1882-
throw new SparkException(
1883-
s"Cannot stop SparkContext within listener thread of ${LiveListenerBus.name}")
1882+
throw new SparkException(s"Cannot stop SparkContext within listener bus thread.")
18841883
}
18851884
// Use the stopping variable to ensure no contention for the stop scenario.
18861885
// Still track the stopped variable for use elsewhere in the code.
@@ -2378,7 +2377,7 @@ class SparkContext(config: SparkConf) extends Logging {
23782377
" parameter from breaking Spark's ability to find a valid constructor.")
23792378
}
23802379
}
2381-
listenerBus.addListener(listener)
2380+
listenerBus.addToSharedQueue(listener)
23822381
logInfo(s"Registered listener $className")
23832382
}
23842383
} catch {
Lines changed: 196 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,196 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.scheduler
19+
20+
import java.util.concurrent.LinkedBlockingQueue
21+
import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}
22+
23+
import com.codahale.metrics.{Gauge, Timer}
24+
25+
import org.apache.spark.{SparkConf, SparkContext}
26+
import org.apache.spark.internal.Logging
27+
import org.apache.spark.internal.config._
28+
import org.apache.spark.util.Utils
29+
30+
/**
31+
* An asynchronous queue for events. All events posted to this queue will be delivered to the child
32+
* listeners in a separate thread.
33+
*
34+
* Delivery will only begin when the `start()` method is called. The `stop()` method should be
35+
* called when no more events need to be delivered.
36+
*/
37+
private class AsyncEventQueue(val name: String, conf: SparkConf, metrics: LiveListenerBusMetrics)
38+
extends SparkListenerBus
39+
with Logging {
40+
41+
import AsyncEventQueue._
42+
43+
// Cap the capacity of the queue so we get an explicit error (rather than an OOM exception) if
44+
// it's perpetually being added to more quickly than it's being drained.
45+
private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](
46+
conf.get(LISTENER_BUS_EVENT_QUEUE_CAPACITY))
47+
48+
// Keep the event count separately, so that waitUntilEmpty() can be implemented properly;
49+
// this allows that method to return only when the events in the queue have been fully
50+
// processed (instead of just dequeued).
51+
private val eventCount = new AtomicLong()
52+
53+
/** A counter for dropped events. It will be reset every time we log it. */
54+
private val droppedEventsCounter = new AtomicLong(0L)
55+
56+
/** When `droppedEventsCounter` was logged last time in milliseconds. */
57+
@volatile private var lastReportTimestamp = 0L
58+
59+
private val logDroppedEvent = new AtomicBoolean(false)
60+
61+
private var sc: SparkContext = null
62+
63+
private val started = new AtomicBoolean(false)
64+
private val stopped = new AtomicBoolean(false)
65+
66+
private val droppedEvents = metrics.metricRegistry.counter(s"queue.$name.numDroppedEvents")
67+
private val processingTime = metrics.metricRegistry.timer(s"queue.$name.listenerProcessingTime")
68+
69+
// Remove the queue size gauge first, in case it was created by a previous incarnation of
70+
// this queue that was removed from the listener bus.
71+
metrics.metricRegistry.remove(s"queue.$name.size")
72+
metrics.metricRegistry.register(s"queue.$name.size", new Gauge[Int] {
73+
override def getValue: Int = eventQueue.size()
74+
})
75+
76+
private val dispatchThread = new Thread(s"spark-listener-group-$name") {
77+
setDaemon(true)
78+
override def run(): Unit = Utils.tryOrStopSparkContext(sc) {
79+
dispatch()
80+
}
81+
}
82+
83+
private def dispatch(): Unit = LiveListenerBus.withinListenerThread.withValue(true) {
84+
try {
85+
var next: SparkListenerEvent = eventQueue.take()
86+
while (next != POISON_PILL) {
87+
val ctx = processingTime.time()
88+
try {
89+
super.postToAll(next)
90+
} finally {
91+
ctx.stop()
92+
}
93+
eventCount.decrementAndGet()
94+
next = eventQueue.take()
95+
}
96+
eventCount.decrementAndGet()
97+
} catch {
98+
case ie: InterruptedException =>
99+
logInfo(s"Stopping listener queue $name.", ie)
100+
}
101+
}
102+
103+
override protected def getTimer(listener: SparkListenerInterface): Option[Timer] = {
104+
metrics.getTimerForListenerClass(listener.getClass.asSubclass(classOf[SparkListenerInterface]))
105+
}
106+
107+
/**
108+
* Start an asynchronous thread to dispatch events to the underlying listeners.
109+
*
110+
* @param sc Used to stop the SparkContext in case the async dispatcher fails.
111+
*/
112+
private[scheduler] def start(sc: SparkContext): Unit = {
113+
if (started.compareAndSet(false, true)) {
114+
this.sc = sc
115+
dispatchThread.start()
116+
} else {
117+
throw new IllegalStateException(s"$name already started!")
118+
}
119+
}
120+
121+
/**
122+
* Stop the listener bus. It will wait until the queued events have been processed, but new
123+
* events will be dropped.
124+
*/
125+
private[scheduler] def stop(): Unit = {
126+
if (!started.get()) {
127+
throw new IllegalStateException(s"Attempted to stop $name that has not yet started!")
128+
}
129+
if (stopped.compareAndSet(false, true)) {
130+
eventQueue.put(POISON_PILL)
131+
eventCount.incrementAndGet()
132+
}
133+
dispatchThread.join()
134+
}
135+
136+
def post(event: SparkListenerEvent): Unit = {
137+
if (stopped.get()) {
138+
return
139+
}
140+
141+
eventCount.incrementAndGet()
142+
if (eventQueue.offer(event)) {
143+
return
144+
}
145+
146+
eventCount.decrementAndGet()
147+
droppedEvents.inc()
148+
droppedEventsCounter.incrementAndGet()
149+
if (logDroppedEvent.compareAndSet(false, true)) {
150+
// Only log the following message once to avoid duplicated annoying logs.
151+
logError(s"Dropping event from queue $name. " +
152+
"This likely means one of the listeners is too slow and cannot keep up with " +
153+
"the rate at which tasks are being started by the scheduler.")
154+
}
155+
logTrace(s"Dropping event $event")
156+
157+
val droppedCount = droppedEventsCounter.get
158+
if (droppedCount > 0) {
159+
// Don't log too frequently
160+
if (System.currentTimeMillis() - lastReportTimestamp >= 60 * 1000) {
161+
// There may be multiple threads trying to decrease droppedEventsCounter.
162+
// Use "compareAndSet" to make sure only one thread can win.
163+
// And if another thread is increasing droppedEventsCounter, "compareAndSet" will fail and
164+
// then that thread will update it.
165+
if (droppedEventsCounter.compareAndSet(droppedCount, 0)) {
166+
val prevLastReportTimestamp = lastReportTimestamp
167+
lastReportTimestamp = System.currentTimeMillis()
168+
val previous = new java.util.Date(prevLastReportTimestamp)
169+
logWarning(s"Dropped $droppedEvents events from $name since $previous.")
170+
}
171+
}
172+
}
173+
}
174+
175+
/**
176+
* For testing only. Wait until there are no more events in the queue.
177+
*
178+
* @return true if the queue is empty.
179+
*/
180+
def waitUntilEmpty(deadline: Long): Boolean = {
181+
while (eventCount.get() != 0) {
182+
if (System.currentTimeMillis > deadline) {
183+
return false
184+
}
185+
Thread.sleep(10)
186+
}
187+
true
188+
}
189+
190+
}
191+
192+
private object AsyncEventQueue {
193+
194+
val POISON_PILL = new SparkListenerEvent() { }
195+
196+
}

0 commit comments

Comments
 (0)