diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index ead35662fc75a..05c4df891ed75 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -45,7 +45,7 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { def start() { try { - val (srv, bPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + val (srv, bPort) = JettyUtils.startJettyServer(host, port, handlers) server = Some(srv) boundPort = Some(bPort) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort.get)) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 8daa47b2b2435..c23b75d757456 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -56,7 +56,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I def start() { try { - val (srv, bPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + val (srv, bPort) = JettyUtils.startJettyServer(host, port, handlers) server = Some(srv) boundPort = Some(bPort) logInfo("Started Worker web UI at http://%s:%d".format(host, bPort)) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 7211dbc7c6681..b3deb41e761c8 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -30,12 +30,14 @@ import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHa import org.eclipse.jetty.util.thread.QueuedThreadPool import org.apache.spark.Logging +import java.net.InetSocketAddress /** Utilities for launching a web server using Jetty's HTTP Server class */ private[spark] object JettyUtils extends Logging { // Base type for a function that returns something based on an HTTP request. Allows for // implicit conversion from many types of functions to jetty Handlers. + type Responder[T] = HttpServletRequest => T // Conversions from various types of Responder's to jetty Handlers @@ -92,12 +94,13 @@ private[spark] object JettyUtils extends Logging { } /** - * Attempts to start a Jetty server at the supplied ip:port which uses the supplied handlers. + * Attempts to start a Jetty server at the supplied hostName:port which uses the supplied handlers. * * If the desired port number is contented, continues incrementing ports until a free port is * found. Returns the chosen port and the jetty Server object. */ - def startJettyServer(ip: String, port: Int, handlers: Seq[(String, Handler)]): (Server, Int) = { + def startJettyServer(hostName: String, port: Int, handlers: Seq[(String, Handler)]): (Server, Int) = { + val handlersToRegister = handlers.map { case(path, handler) => val contextHandler = new ContextHandler(path) contextHandler.setHandler(handler) @@ -109,7 +112,7 @@ private[spark] object JettyUtils extends Logging { @tailrec def connect(currentPort: Int): (Server, Int) = { - val server = new Server(currentPort) + val server = new Server(new InetSocketAddress(hostName, currentPort)) val pool = new QueuedThreadPool pool.setDaemon(true) server.setThreadPool(pool) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 50dfdbdf5ae9b..0196f43d7431b 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -54,7 +54,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { /** Bind the HTTP server which backs this web interface */ def bind() { try { - val (srv, usedPort) = JettyUtils.startJettyServer("0.0.0.0", port, allHandlers) + val (srv, usedPort) = JettyUtils.startJettyServer(host, port, allHandlers) logInfo("Started Spark Web UI at http://%s:%d".format(host, usedPort)) server = Some(srv) boundPort = Some(usedPort) diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 3764f4d1a0c9b..c17bbfe7d35ba 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -32,15 +32,15 @@ class UISuite extends FunSuite { case Failure(e) => // Either case server port is busy hence setup for test complete } - val (jettyServer1, boundPort1) = JettyUtils.startJettyServer("localhost", startPort, Seq()) - val (jettyServer2, boundPort2) = JettyUtils.startJettyServer("localhost", startPort, Seq()) + val (jettyServer1, boundPort1) = JettyUtils.startJettyServer("0.0.0.0", startPort, Seq()) + val (jettyServer2, boundPort2) = JettyUtils.startJettyServer("0.0.0.0", startPort, Seq()) // Allow some wiggle room in case ports on the machine are under contention assert(boundPort1 > startPort && boundPort1 < startPort + 10) assert(boundPort2 > boundPort1 && boundPort2 < boundPort1 + 10) } test("jetty binds to port 0 correctly") { - val (jettyServer, boundPort) = JettyUtils.startJettyServer("localhost", 0, Seq()) + val (jettyServer, boundPort) = JettyUtils.startJettyServer("0.0.0.0", 0, Seq()) assert(jettyServer.getState === "STARTED") assert(boundPort != 0) Try {new ServerSocket(boundPort)} match {