You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by pw...@apache.org on 2014/03/27 02:22:21 UTC

git commit: SPARK-1324: SparkUI Should Not Bind to SPARK_PUBLIC_DNS

Repository: spark
Updated Branches:
  refs/heads/master e15e57413 -> be6d96c15


SPARK-1324: SparkUI Should Not Bind to SPARK_PUBLIC_DNS

/cc @aarondav and @andrewor14

Author: Patrick Wendell <pw...@gmail.com>

Closes #231 from pwendell/ui-binding and squashes the following commits:

e8025f8 [Patrick Wendell] SPARK-1324: SparkUI Should Not Bind to SPARK_PUBLIC_DNS


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/be6d96c1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/be6d96c1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/be6d96c1

Branch: refs/heads/master
Commit: be6d96c15b3c31cd27bdd79fb259072479151ae6
Parents: e15e574
Author: Patrick Wendell <pw...@gmail.com>
Authored: Wed Mar 26 18:22:15 2014 -0700
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Mar 26 18:22:15 2014 -0700

----------------------------------------------------------------------
 core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/be6d96c1/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
----------------------------------------------------------------------
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 fd638c8..ef1ad87 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -47,7 +47,8 @@ private[spark] class SparkUI(
 
   val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf)
 
-  private val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName())
+  private val bindHost = Utils.localHostName()
+  private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost)
   private val port = conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt
   private var serverInfo: Option[ServerInfo] = None
 
@@ -79,8 +80,8 @@ private[spark] class SparkUI(
   /** Bind the HTTP server which backs this web interface */
   def bind() {
     try {
-      serverInfo = Some(startJettyServer(host, port, handlers, sc.conf))
-      logInfo("Started Spark Web UI at http://%s:%d".format(host, boundPort))
+      serverInfo = Some(startJettyServer(bindHost, port, handlers, sc.conf))
+      logInfo("Started Spark Web UI at http://%s:%d".format(publicHost, boundPort))
     } catch {
       case e: Exception =>
         logError("Failed to create Spark JettyUtils", e)
@@ -111,7 +112,7 @@ private[spark] class SparkUI(
     logInfo("Stopped Spark Web UI at %s".format(appUIAddress))
   }
 
-  private[spark] def appUIAddress = "http://" + host + ":" + boundPort
+  private[spark] def appUIAddress = "http://" + publicHost + ":" + boundPort
 
 }