@@ -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
0 commit comments