diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html b/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html
index e5ed5b30724dd..5a7a252231053 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html
+++ b/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html
@@ -67,7 +67,7 @@
{{id}} |
{{name}} |
{{#attempts}}
- {{attemptId}} |
+ {{attemptId}} |
{{startTime}} |
{{endTime}} |
{{duration}} |
diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage.js b/core/src/main/resources/org/apache/spark/ui/static/historypage.js
index 4ff083146e9a9..2571cc3971932 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/historypage.js
+++ b/core/src/main/resources/org/apache/spark/ui/static/historypage.js
@@ -110,7 +110,7 @@ $(document).ready(function() {
requestedIncomplete = getParameterByName("showIncomplete", searchString);
requestedIncomplete = (requestedIncomplete == "true" ? true : false);
- $.getJSON("/api/v1/applications", function(response,status,jqXHR) {
+ $.getJSON("api/v1/applications", function(response,status,jqXHR) {
var array = [];
var hasMultipleAttempts = false;
for (i in response) {
@@ -139,9 +139,9 @@ $(document).ready(function() {
var url = null
if (maxAttemptId == null) {
- url = "/history/" + id + "/"
+ url = "history/" + id + "/"
} else {
- url = "/history/" + id + "/" + maxAttemptId + "/"
+ url = "history/" + id + "/" + maxAttemptId + "/"
}
var app_clone = {"id" : id, "name" : name, "url" : url, "attempts" : [attempt]};
@@ -150,7 +150,7 @@ $(document).ready(function() {
}
var data = {"applications": array}
- $.get("/static/historypage-template.html", function(template) {
+ $.get("static/historypage-template.html", function(template) {
historySummary.append(Mustache.render($(template).filter("#history-summary-template").html(),data));
var selector = "#history-summary-table";
var conf = {
diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala
index 9b31497adfb12..7091513df07e3 100644
--- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala
@@ -73,7 +73,7 @@ private[mesos] class MesosClusterDispatcher(
def start(): Unit = {
webUi.bind()
- scheduler.frameworkUrl = webUi.activeWebUiUrl
+ scheduler.frameworkUrl = conf.get("spark.mesos.dispatcher.webui.url", webUi.activeWebUiUrl)
scheduler.start()
server.start()
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
index 38935e3209869..b97805a28bdcf 100644
--- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
@@ -47,7 +47,7 @@ private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf:
port = value
parse(tail)
- case ("--webui-port" | "-p") :: IntParam(value) :: tail =>
+ case ("--webui-port") :: IntParam(value) :: tail =>
webUiPort = value
parse(tail)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index 622f361ec2a3c..e1180980eed68 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -149,7 +149,8 @@ private[spark] class CoarseMesosSchedulerBackend(
sc.sparkUser,
sc.appName,
sc.conf,
- sc.ui.map(_.appUIAddress))
+ sc.conf.getOption("spark.mesos.driver.webui.url").orElse(sc.ui.map(_.appUIAddress))
+ )
startScheduler(driver)
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
index 8929d8a427789..1a94aee2ca30c 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
@@ -78,7 +78,8 @@ private[spark] class MesosSchedulerBackend(
sc.sparkUser,
sc.appName,
sc.conf,
- sc.ui.map(_.appUIAddress))
+ sc.conf.getOption("spark.mesos.driver.webui.url").orElse(sc.ui.map(_.appUIAddress))
+ )
startScheduler(driver)
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala
index 2df05401f3a3c..dd76644288b4c 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala
@@ -208,6 +208,35 @@ class CoarseMesosSchedulerBackendSuite extends SparkFunSuite
verify(driver, times(1)).killTask(createTaskId("0"))
}
+ test("weburi is set in created scheduler driver") {
+ setBackend()
+ val taskScheduler = mock[TaskSchedulerImpl]
+ when(taskScheduler.sc).thenReturn(sc)
+ val driver = mock[SchedulerDriver]
+ when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING)
+ val securityManager = mock[SecurityManager]
+
+ val backend = new CoarseMesosSchedulerBackend(taskScheduler, sc, "master", securityManager) {
+ override protected def createSchedulerDriver(
+ masterUrl: String,
+ scheduler: Scheduler,
+ sparkUser: String,
+ appName: String,
+ conf: SparkConf,
+ webuiUrl: Option[String] = None,
+ checkpoint: Option[Boolean] = None,
+ failoverTimeout: Option[Double] = None,
+ frameworkId: Option[String] = None): SchedulerDriver = {
+ markRegistered()
+ assert(webuiUrl.isDefined)
+ assert(webuiUrl.get.equals("http://webui"))
+ driver
+ }
+ }
+
+ backend.start()
+ }
+
private def verifyDeclinedOffer(driver: SchedulerDriver,
offerId: OfferID,
filter: Boolean = false): Unit = {
@@ -316,6 +345,7 @@ class CoarseMesosSchedulerBackendSuite extends SparkFunSuite
.setMaster("local[*]")
.setAppName("test-mesos-dynamic-alloc")
.setSparkHome("/path")
+ .set("spark.mesos.driver.webui.url", "http://webui")
if (sparkConfVars != null) {
for (attr <- sparkConfVars) {
diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala
index 3fb3279073f24..7d6b7bde68253 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala
@@ -26,9 +26,9 @@ import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
+import org.apache.mesos.{Protos, Scheduler, SchedulerDriver}
import org.apache.mesos.Protos._
import org.apache.mesos.Protos.Value.Scalar
-import org.apache.mesos.SchedulerDriver
import org.mockito.{ArgumentCaptor, Matchers}
import org.mockito.Matchers._
import org.mockito.Mockito._
@@ -42,6 +42,41 @@ import org.apache.spark.scheduler.cluster.ExecutorInfo
class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar {
+ test("weburi is set in created scheduler driver") {
+ val conf = new SparkConf
+ conf.set("spark.mesos.driver.webui.url", "http://webui")
+ conf.set("spark.app.name", "name1")
+
+ val sc = mock[SparkContext]
+ when(sc.conf).thenReturn(conf)
+ when(sc.sparkUser).thenReturn("sparkUser1")
+ when(sc.appName).thenReturn("appName1")
+
+ val taskScheduler = mock[TaskSchedulerImpl]
+ val driver = mock[SchedulerDriver]
+ when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING)
+
+ val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") {
+ override protected def createSchedulerDriver(
+ masterUrl: String,
+ scheduler: Scheduler,
+ sparkUser: String,
+ appName: String,
+ conf: SparkConf,
+ webuiUrl: Option[String] = None,
+ checkpoint: Option[Boolean] = None,
+ failoverTimeout: Option[Double] = None,
+ frameworkId: Option[String] = None): SchedulerDriver = {
+ markRegistered()
+ assert(webuiUrl.isDefined)
+ assert(webuiUrl.get.equals("http://webui"))
+ driver
+ }
+ }
+
+ backend.start()
+ }
+
test("Use configured mesosExecutor.cores for ExecutorInfo") {
val mesosExecutorCores = 3
val conf = new SparkConf
diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md
index 912a0108129c2..3a832de95f10d 100644
--- a/docs/running-on-mesos.md
+++ b/docs/running-on-mesos.md
@@ -390,6 +390,22 @@ See the [configuration page](configuration.html) for information on Spark config
+
+ spark.mesos.driver.webui.url |
+ (none) |
+
+ Set the Spark Mesos driver webui_url for interacting with the framework.
+ If unset it will point to Spark's internal web UI.
+ |
+
+
+ spark.mesos.dispatcher.webui.url |
+ (none) |
+
+ Set the Spark Mesos dispatcher webui_url for interacting with the framework.
+ If unset it will point to Spark's internal web UI.
+ |
+
# Troubleshooting and Debugging