Skip to content

Commit 163ba19

Browse files
committed
[SPARK-5190] Allow SparkListeners to be registered before SparkContext starts.
1 parent 74de94e commit 163ba19

File tree

3 files changed

+79
-10
lines changed

3 files changed

+79
-10
lines changed

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

Lines changed: 43 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -63,8 +63,12 @@ import org.apache.spark.util._
6363
*
6464
* @param config a Spark Config object describing the application configuration. Any settings in
6565
* this config overrides the default configs as well as system properties.
66+
* @param sparkListeners an optional list of [[SparkListener]]s to register.
6667
*/
67-
class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationClient {
68+
class SparkContext(
69+
config: SparkConf,
70+
sparkListeners: Seq[SparkListener] = Nil
71+
) extends Logging with ExecutorAllocationClient {
6872

6973
// The call site where this SparkContext was constructed.
7074
private val creationSite: CallSite = Utils.getCallSite()
@@ -89,7 +93,15 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
8993
* Create a SparkContext that loads settings from system properties (for instance, when
9094
* launching with ./bin/spark-submit).
9195
*/
92-
def this() = this(new SparkConf())
96+
def this() = this(new SparkConf(), Nil)
97+
98+
/**
99+
* Alternative constructor for binary compatibility.
100+
*
101+
* @param config a Spark Config object describing the application configuration. Any settings in
102+
* this config overrides the default configs as well as system properties.
103+
*/
104+
def this(config: SparkConf) = this(config, Nil)
93105

94106
/**
95107
* :: DeveloperApi ::
@@ -124,19 +136,40 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
124136
* @param jars Collection of JARs to send to the cluster. These can be paths on the local file
125137
* system or HDFS, HTTP, HTTPS, or FTP URLs.
126138
* @param environment Environment variables to set on worker nodes.
139+
* @param sparkListeners an optional list of [[SparkListener]]s to register.
127140
*/
128141
def this(
129142
master: String,
130143
appName: String,
131144
sparkHome: String = null,
132145
jars: Seq[String] = Nil,
133146
environment: Map[String, String] = Map(),
134-
preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) =
135-
{
136-
this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment))
147+
preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map(),
148+
sparkListeners: Seq[SparkListener] = Nil) = {
149+
this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment),
150+
sparkListeners)
137151
this.preferredNodeLocationData = preferredNodeLocationData
138152
}
139153

154+
/**
155+
* Alternative constructor for binary compatibility.
156+
*
157+
* @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
158+
* @param appName A name for your application, to display on the cluster web UI.
159+
* @param sparkHome Location where Spark is installed on cluster nodes.
160+
* @param jars Collection of JARs to send to the cluster. These can be paths on the local file
161+
* system or HDFS, HTTP, HTTPS, or FTP URLs.
162+
* @param environment Environment variables to set on worker nodes.
163+
*/
164+
def this(
165+
master: String,
166+
appName: String,
167+
sparkHome: String,
168+
jars: Seq[String],
169+
environment: Map[String, String],
170+
preferredNodeLocationData: Map[String, Set[SplitInfo]]) =
171+
this(master, appName, sparkHome, jars, environment, preferredNodeLocationData, Nil)
172+
140173
// NOTE: The below constructors could be consolidated using default arguments. Due to
141174
// Scala bug SI-8479, however, this causes the compile step to fail when generating docs.
142175
// Until we have a good workaround for that bug the constructors remain broken out.
@@ -148,7 +181,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
148181
* @param appName A name for your application, to display on the cluster web UI.
149182
*/
150183
private[spark] def this(master: String, appName: String) =
151-
this(master, appName, null, Nil, Map(), Map())
184+
this(master, appName, null, Nil, Map(), Map(), Nil)
152185

153186
/**
154187
* Alternative constructor that allows setting common Spark properties directly
@@ -158,7 +191,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
158191
* @param sparkHome Location where Spark is installed on cluster nodes.
159192
*/
160193
private[spark] def this(master: String, appName: String, sparkHome: String) =
161-
this(master, appName, sparkHome, Nil, Map(), Map())
194+
this(master, appName, sparkHome, Nil, Map(), Map(), Nil)
162195

163196
/**
164197
* Alternative constructor that allows setting common Spark properties directly
@@ -170,7 +203,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
170203
* system or HDFS, HTTP, HTTPS, or FTP URLs.
171204
*/
172205
private[spark] def this(master: String, appName: String, sparkHome: String, jars: Seq[String]) =
173-
this(master, appName, sparkHome, jars, Map(), Map())
206+
this(master, appName, sparkHome, jars, Map(), Map(), Nil)
174207

175208
// log out Spark Version in Spark driver log
176209
logInfo(s"Running Spark version $SPARK_VERSION")
@@ -379,6 +412,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
379412
}
380413
executorAllocationManager.foreach(_.start())
381414

415+
sparkListeners.foreach(listenerBus.addListener)
416+
382417
// At this point, all relevant SparkListeners have been registered, so begin releasing events
383418
listenerBus.start()
384419

core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala

Lines changed: 16 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,7 @@ import org.apache.spark.annotation.Experimental
3838
import org.apache.spark.api.java.JavaSparkContext.fakeClassTag
3939
import org.apache.spark.broadcast.Broadcast
4040
import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD}
41+
import org.apache.spark.scheduler.SparkListener
4142

4243
/**
4344
* A Java-friendly version of [[org.apache.spark.SparkContext]] that returns
@@ -104,7 +105,21 @@ class JavaSparkContext(val sc: SparkContext)
104105
*/
105106
def this(master: String, appName: String, sparkHome: String, jars: Array[String],
106107
environment: JMap[String, String]) =
107-
this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment, Map()))
108+
this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment, Map(), Nil))
109+
110+
/**
111+
* @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
112+
* @param appName A name for your application, to display on the cluster web UI
113+
* @param sparkHome The SPARK_HOME directory on the slave nodes
114+
* @param jars Collection of JARs to send to the cluster. These can be paths on the local file
115+
* system or HDFS, HTTP, HTTPS, or FTP URLs.
116+
* @param environment Environment variables to set on worker nodes
117+
* @param sparkListeners an optional list of [[SparkListener]]s to register.
118+
*/
119+
def this(master: String, appName: String, sparkHome: String, jars: Array[String],
120+
environment: JMap[String, String], sparkListeners: Array[SparkListener]) =
121+
this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment, Map(),
122+
sparkListeners))
108123

109124
private[spark] val env = sc.env
110125

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

Lines changed: 20 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,9 +17,14 @@
1717

1818
package org.apache.spark
1919

20+
import org.apache.hadoop.io.BytesWritable
2021
import org.scalatest.FunSuite
22+
import org.scalatest.concurrent.Eventually._
2123

22-
import org.apache.hadoop.io.BytesWritable
24+
import scala.concurrent.duration._
25+
import scala.language.{implicitConversions, postfixOps}
26+
27+
import org.apache.spark.scheduler.{SparkListener, SparkListenerEnvironmentUpdate}
2328

2429
class SparkContextSuite extends FunSuite with LocalSparkContext {
2530

@@ -72,4 +77,18 @@ class SparkContextSuite extends FunSuite with LocalSparkContext {
7277
val byteArray2 = converter.convert(bytesWritable)
7378
assert(byteArray2.length === 0)
7479
}
80+
81+
test("SparkListeners can be registered via the SparkContext constructor (SPARK-5190)") {
82+
@volatile var gotEnvironmentUpdate: Boolean = false
83+
val listener = new SparkListener {
84+
override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate): Unit = {
85+
gotEnvironmentUpdate = true
86+
}
87+
}
88+
val conf = new SparkConf().setAppName("test").setMaster("local")
89+
sc = new SparkContext(conf, Seq(listener))
90+
eventually(timeout(10 seconds)) {
91+
assert(gotEnvironmentUpdate === true)
92+
}
93+
}
7594
}

0 commit comments

Comments
 (0)