diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 566472e597958..1cd37036f9171 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -155,8 +155,14 @@ class SparkContext(
new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf)
// Initialize the Spark UI
- private[spark] val ui = new SparkUI(this)
- ui.bind()
+ private[spark] val ui: Option[SparkUI] =
+ if (conf.getBoolean("spark.ui.enabled", true)) {
+ Some(new SparkUI(this))
+ } else {
+ // For tests, do not enable the UI
+ None
+ }
+ ui.foreach(_.bind())
val startTime = System.currentTimeMillis()
@@ -202,7 +208,7 @@ class SparkContext(
@volatile private[spark] var dagScheduler = new DAGScheduler(taskScheduler)
dagScheduler.start()
- ui.start()
+ ui.foreach(_.start())
/** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */
val hadoopConfiguration = {
@@ -777,7 +783,7 @@ class SparkContext(
/** Shut down the SparkContext. */
def stop() {
- ui.stop()
+ ui.foreach(_.stop())
// Do this only if not stopped already - best case effort.
// prevent NPE if stopped more than once.
val dagSchedulerCopy = dagScheduler
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
index d99c76117c168..149c9fd1efbaa 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
@@ -44,16 +44,17 @@ private[spark] class SimrSchedulerBackend(
val conf = new Configuration()
val fs = FileSystem.get(conf)
+ val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("")
logInfo("Writing to HDFS file: " + driverFilePath)
logInfo("Writing Akka address: " + driverUrl)
- logInfo("Writing Spark UI Address: " + sc.ui.appUIAddress)
+ logInfo("Writing Spark UI Address: " + appUIAddress)
// Create temporary file to prevent race condition where executors get empty driverUrl file
val temp = fs.create(tmpPath, true)
temp.writeUTF(driverUrl)
temp.writeInt(maxCores)
- temp.writeUTF(sc.ui.appUIAddress)
+ temp.writeUTF(appUIAddress)
temp.close()
// "Atomic" rename
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index faa6e1ebe886f..dfcb22ae131dc 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -51,8 +51,9 @@ private[spark] class SparkDeploySchedulerBackend(
val command = Command(
"org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs)
val sparkHome = sc.getSparkHome().getOrElse(null)
- val appDesc = new ApplicationDescription(appName, maxCores, sc.executorMemory, command, sparkHome,
- "http://" + sc.ui.appUIAddress)
+ val appUIAddress = sc.ui.map { x => "http://" + x.appUIAddress }.getOrElse("")
+ val appDesc = new ApplicationDescription(
+ appName, maxCores, sc.executorMemory, command, sparkHome, appUIAddress)
client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf)
client.start()
diff --git a/pom.xml b/pom.xml
index e3f4893f70723..6a12a4ecc0b15 100644
--- a/pom.xml
+++ b/pom.xml
@@ -636,6 +636,10 @@
${project.build.directory}/SparkTestSuite.txt
-Xms64m -Xmx3g
+
+ true
+ false
+
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 05b430afcec8f..2c8ee00143f0f 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -149,6 +149,7 @@ object SparkBuild extends Build {
fork := true,
javaOptions in Test += "-Dspark.home=" + sparkHome,
javaOptions in Test += "-Dspark.testing=1",
+ javaOptions in Test += "-Dspark.ui.enabled=false",
javaOptions += "-Xmx3g",
// Show full stack trace and duration in test cases.
testOptions in Test += Tests.Argument("-oDF"),
diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index 66bcaf1deed3a..b4c06fa0538a7 100644
--- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -227,7 +227,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
assert(sparkContext != null || count >= numTries)
if (null != sparkContext) {
- uiAddress = sparkContext.ui.appUIAddress
+ uiAddress = sparkContext.ui.map(_.appUIAddress).getOrElse("")
this.yarnAllocator = YarnAllocationHandler.newAllocator(
yarnConf,
resourceManager,
diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index 83f8234746290..0cb347460469b 100644
--- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -199,7 +199,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
assert(sparkContext != null || numTries >= maxNumTries)
if (sparkContext != null) {
- uiAddress = sparkContext.ui.appUIAddress
+ uiAddress = sparkContext.ui.map(_.appUIAddress).getOrElse("")
this.yarnAllocator = YarnAllocationHandler.newAllocator(
yarnConf,
amClient,