diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index cfa2c028a807b6a78a11b39f28159698eea1e49e..5017273e87c07e5d0b049b855fdb2f41b7745921 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -697,7 +697,7 @@ private[spark] class Master(
       appIdToUI(app.id) = ui
       webUi.attachSparkUI(ui)
       // Application UI is successfully rebuilt, so link the Master UI to it
-      app.desc.appUiUrl = ui.basePath
+      app.desc.appUiUrl = ui.getBasePath
       true
     } catch {
       case e: Exception =>
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 6c788a37dc70b9688f232429081ad5555bde084f..cccd59d122a92fad02d02270e2324b169ddb5eb7 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -76,6 +76,8 @@ private[spark] class SparkUI(
     }
   }
 
+  def getAppName = appName
+
   /** Set the app name for this UI. */
   def setAppName(name: String) {
     appName = name
@@ -100,6 +102,13 @@ private[spark] class SparkUI(
   private[spark] def appUIAddress = s"http://$appUIHostPort"
 }
 
+private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String)
+  extends WebUITab(parent, prefix) {
+
+  def appName: String = parent.getAppName
+
+}
+
 private[spark] object SparkUI {
   val DEFAULT_PORT = 4040
   val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static"
diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
index 715cc2f4df8dd1a6422974feeaae18b09ef70dbc..bee6dad3387e5a26c68ef06864b08f8ffb0ad98b 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
@@ -163,17 +163,15 @@ private[spark] object UIUtils extends Logging {
 
   /** Returns a spark page with correctly formatted headers */
   def headerSparkPage(
-      content: => Seq[Node],
-      basePath: String,
-      appName: String,
       title: String,
-      tabs: Seq[WebUITab],
-      activeTab: WebUITab,
+      content: => Seq[Node],
+      activeTab: SparkUITab,
       refreshInterval: Option[Int] = None): Seq[Node] = {
 
-    val header = tabs.map { tab =>
+    val appName = activeTab.appName
+    val header = activeTab.headerTabs.map { tab =>
       <li class={if (tab == activeTab) "active" else ""}>
-        <a href={prependBaseUri(basePath, "/" + tab.prefix)}>{tab.name}</a>
+        <a href={prependBaseUri(activeTab.basePath, "/" + tab.prefix)}>{tab.name}</a>
       </li>
     }
 
diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
index 5f52f9508800734e1bd8b1cc852a347c11b71944..5d88ca403a6740857de816e67f7fdedfaeee783b 100644
--- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
@@ -50,6 +50,7 @@ private[spark] abstract class WebUI(
   protected val publicHostName = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName)
   private val className = Utils.getFormattedClassName(this)
 
+  def getBasePath: String = basePath
   def getTabs: Seq[WebUITab] = tabs.toSeq
   def getHandlers: Seq[ServletContextHandler] = handlers.toSeq
   def getSecurityManager: SecurityManager = securityManager
@@ -135,6 +136,8 @@ private[spark] abstract class WebUITab(parent: WebUI, val prefix: String) {
 
   /** Get a list of header tabs from the parent UI. */
   def headerTabs: Seq[WebUITab] = parent.getTabs
+
+  def basePath: String = parent.getBasePath
 }
 
 
diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala
index b347eb1b83c1fe77856990273c87f6054be51582..f0a1174a71d34cd44b031d20b8dd73bac8215d92 100644
--- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala
@@ -24,8 +24,6 @@ import scala.xml.Node
 import org.apache.spark.ui.{UIUtils, WebUIPage}
 
 private[ui] class EnvironmentPage(parent: EnvironmentTab) extends WebUIPage("") {
-  private val appName = parent.appName
-  private val basePath = parent.basePath
   private val listener = parent.listener
 
   def render(request: HttpServletRequest): Seq[Node] = {
@@ -45,7 +43,7 @@ private[ui] class EnvironmentPage(parent: EnvironmentTab) extends WebUIPage("")
         <h4>Classpath Entries</h4> {classpathEntriesTable}
       </span>
 
-    UIUtils.headerSparkPage(content, basePath, appName, "Environment", parent.headerTabs, parent)
+    UIUtils.headerSparkPage("Environment", content, parent)
   }
 
   private def propertyHeader = Seq("Name", "Value")
diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala
index bbbe55ecf44a1b603657d06bb30106713610dcc1..0d158fbe638d3a07bda5ccb68cdc014ac0cd1dc7 100644
--- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala
@@ -21,9 +21,7 @@ import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.scheduler._
 import org.apache.spark.ui._
 
-private[ui] class EnvironmentTab(parent: SparkUI) extends WebUITab(parent, "environment") {
-  val appName = parent.appName
-  val basePath = parent.basePath
+private[ui] class EnvironmentTab(parent: SparkUI) extends SparkUITab(parent, "environment") {
   val listener = new EnvironmentListener
 
   attachPage(new EnvironmentPage(this))
diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
index b814b0e6b85090ad8a20f3a6b73d908200cfafb0..02df4e8fe61afcf0ac198e691b17299c6218282a 100644
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
@@ -43,8 +43,6 @@ private case class ExecutorSummaryInfo(
     maxMemory: Long)
 
 private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") {
-  private val appName = parent.appName
-  private val basePath = parent.basePath
   private val listener = parent.listener
 
   def render(request: HttpServletRequest): Seq[Node] = {
@@ -101,8 +99,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") {
         </div>
       </div>;
 
-    UIUtils.headerSparkPage(content, basePath, appName, "Executors (" + execInfo.size + ")",
-      parent.headerTabs, parent)
+    UIUtils.headerSparkPage("Executors (" + execInfo.size + ")", content, parent)
   }
 
   /** Render an HTML row representing an executor */
diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala
index 5c2d1d1fe75d3effc559a98e22fcb7f092c04731..61eb111cd9100a3895e507db7016769c7435d106 100644
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala
@@ -23,11 +23,9 @@ import org.apache.spark.ExceptionFailure
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.scheduler._
 import org.apache.spark.storage.StorageStatusListener
-import org.apache.spark.ui.{SparkUI, WebUITab}
+import org.apache.spark.ui.{SparkUI, SparkUITab}
 
-private[ui] class ExecutorsTab(parent: SparkUI) extends WebUITab(parent, "executors") {
-  val appName = parent.appName
-  val basePath = parent.basePath
+private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "executors") {
   val listener = new ExecutorsListener(parent.storageStatusListener)
 
   attachPage(new ExecutorsPage(this))
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala
index 0da62892118d41a56e4f6b2dee56a57206a8b195..a82f71ed0847500f0f68f55247d072a058fc5072 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala
@@ -26,8 +26,6 @@ import org.apache.spark.ui.{WebUIPage, UIUtils}
 
 /** Page showing list of all ongoing and recently finished stages and pools */
 private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") {
-  private val appName = parent.appName
-  private val basePath = parent.basePath
   private val live = parent.live
   private val sc = parent.sc
   private val listener = parent.listener
@@ -94,7 +92,7 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("")
         <h4 id ="failed">Failed Stages ({failedStages.size})</h4> ++
         failedStagesTable.toNodeSeq
 
-      UIUtils.headerSparkPage(content, basePath, appName, "Spark Stages", parent.headerTabs, parent)
+      UIUtils.headerSparkPage("Spark Stages", content, parent)
     }
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala
index 8a01ec80c9dd623aa693e2e264c843f24be836ec..c16542c9db30f27309ffd181c98fb5586c5f905b 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala
@@ -21,12 +21,10 @@ import javax.servlet.http.HttpServletRequest
 
 import org.apache.spark.SparkConf
 import org.apache.spark.scheduler.SchedulingMode
-import org.apache.spark.ui.{SparkUI, WebUITab}
+import org.apache.spark.ui.{SparkUI, SparkUITab}
 
 /** Web UI showing progress status of all jobs in the given SparkContext. */
-private[ui] class JobProgressTab(parent: SparkUI) extends WebUITab(parent, "stages") {
-  val appName = parent.appName
-  val basePath = parent.basePath
+private[ui] class JobProgressTab(parent: SparkUI) extends SparkUITab(parent, "stages") {
   val live = parent.live
   val sc = parent.sc
   val conf = if (live) sc.conf else new SparkConf
@@ -53,4 +51,5 @@ private[ui] class JobProgressTab(parent: SparkUI) extends WebUITab(parent, "stag
       Thread.sleep(100)
     }
   }
+
 }
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
index 0a2bf31833d2b75fd2aea4fbe3e6bdd8eb5ec8ea..7a6c7d1a497ed5fe8199ebc102f40c3c1ab028b5 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
@@ -26,8 +26,6 @@ import org.apache.spark.ui.{WebUIPage, UIUtils}
 
 /** Page showing specific pool details */
 private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") {
-  private val appName = parent.appName
-  private val basePath = parent.basePath
   private val live = parent.live
   private val sc = parent.sc
   private val listener = parent.listener
@@ -51,8 +49,7 @@ private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") {
         <h4>Summary </h4> ++ poolTable.toNodeSeq ++
         <h4>{activeStages.size} Active Stages</h4> ++ activeStagesTable.toNodeSeq
 
-      UIUtils.headerSparkPage(content, basePath, appName, "Fair Scheduler Pool: " + poolName,
-        parent.headerTabs, parent)
+      UIUtils.headerSparkPage("Fair Scheduler Pool: " + poolName, content, parent)
     }
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
index f4b68f241966d96df32604dc3fd213465bef12fb..64178e1e33d414578494247eebd14d9f987ce544 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
@@ -25,7 +25,6 @@ import org.apache.spark.ui.UIUtils
 
 /** Table showing list of pools */
 private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressTab) {
-  private val basePath = parent.basePath
   private val listener = parent.listener
 
   def toNodeSeq: Seq[Node] = {
@@ -59,11 +58,11 @@ private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressTab) {
       case Some(stages) => stages.size
       case None => 0
     }
+    val href = "%s/stages/pool?poolname=%s"
+      .format(UIUtils.prependBaseUri(parent.basePath), p.name)
     <tr>
       <td>
-        <a href={"%s/stages/pool?poolname=%s".format(UIUtils.prependBaseUri(basePath), p.name)}>
-          {p.name}
-        </a>
+        <a href={href}>{p.name}</a>
       </td>
       <td>{p.minShare}</td>
       <td>{p.weight}</td>
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index 8bc1ba758cf77325810d05ba6ebb1cf279520c4c..d4eb02722ad123f83c4470f7f2570ede6692d63e 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -29,8 +29,6 @@ import org.apache.spark.scheduler.AccumulableInfo
 
 /** Page showing statistics and task list for a given stage */
 private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
-  private val appName = parent.appName
-  private val basePath = parent.basePath
   private val listener = parent.listener
 
   def render(request: HttpServletRequest): Seq[Node] = {
@@ -44,8 +42,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
             <h4>Summary Metrics</h4> No tasks have started yet
             <h4>Tasks</h4> No tasks have started yet
           </div>
-        return UIUtils.headerSparkPage(content, basePath, appName,
-          "Details for Stage %s".format(stageId), parent.headerTabs, parent)
+        return UIUtils.headerSparkPage("Details for Stage %s".format(stageId), content, parent)
       }
 
       val stageData = stageDataOption.get
@@ -227,8 +224,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
         maybeAccumulableTable ++
         <h4>Tasks</h4> ++ taskTable
 
-      UIUtils.headerSparkPage(content, basePath, appName, "Details for Stage %d".format(stageId),
-        parent.headerTabs, parent)
+      UIUtils.headerSparkPage("Details for Stage %d".format(stageId), content, parent)
     }
   }
 
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
index 15998404ed612dbde584ce7413399ddf82060524..16ad0df45aa0da914089b34c59195aed833cfe3d 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
@@ -32,7 +32,6 @@ private[ui] class StageTableBase(
     parent: JobProgressTab,
     killEnabled: Boolean = false) {
 
-  private val basePath = parent.basePath
   private val listener = parent.listener
   protected def isFairScheduler = parent.isFairScheduler
 
@@ -88,17 +87,19 @@ private[ui] class StageTableBase(
   private def makeDescription(s: StageInfo): Seq[Node] = {
     // scalastyle:off
     val killLink = if (killEnabled) {
+      val killLinkUri = "%s/stages/stage/kill?id=%s&terminate=true"
+        .format(UIUtils.prependBaseUri(parent.basePath), s.stageId)
+      val confirm = "return window.confirm('Are you sure you want to kill stage %s ?');"
+        .format(s.stageId)
       <span class="kill-link">
-        (<a href={"%s/stages/stage/kill?id=%s&terminate=true".format(UIUtils.prependBaseUri(basePath), s.stageId)}
-            onclick={"return window.confirm('Are you sure you want to kill stage %s ?');".format(s.stageId)}>kill</a>)
+        (<a href={killLinkUri} onclick={confirm}>kill</a>)
       </span>
     }
     // scalastyle:on
 
-    val nameLink =
-      <a href={"%s/stages/stage?id=%s".format(UIUtils.prependBaseUri(basePath), s.stageId)}>
-        {s.name}
-      </a>
+    val nameLinkUri ="%s/stages/stage?id=%s"
+      .format(UIUtils.prependBaseUri(parent.basePath), s.stageId)
+    val nameLink = <a href={nameLinkUri}>{s.name}</a>
 
     val cachedRddInfos = s.rddInfos.filter(_.numCachedPartitions > 0)
     val details = if (s.details.nonEmpty) {
@@ -111,7 +112,7 @@ private[ui] class StageTableBase(
           Text("RDD: ") ++
           // scalastyle:off
           cachedRddInfos.map { i =>
-            <a href={"%s/storage/rdd?id=%d".format(UIUtils.prependBaseUri(basePath), i.id)}>{i.name}</a>
+            <a href={"%s/storage/rdd?id=%d".format(UIUtils.prependBaseUri(parent.basePath), i.id)}>{i.name}</a>
           }
           // scalastyle:on
         }}
@@ -157,7 +158,7 @@ private[ui] class StageTableBase(
     {if (isFairScheduler) {
       <td>
         <a href={"%s/stages/pool?poolname=%s"
-          .format(UIUtils.prependBaseUri(basePath), stageData.schedulingPool)}>
+          .format(UIUtils.prependBaseUri(parent.basePath), stageData.schedulingPool)}>
           {stageData.schedulingPool}
         </a>
       </td>
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
index 84ac53da4755287a73408b050e0aa7ed6eb444ab..8a0075ae8daf7e3d34905e3fe53bd9b5fbd79e9c 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
@@ -27,8 +27,6 @@ import org.apache.spark.util.Utils
 
 /** Page showing storage details for a given RDD */
 private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
-  private val appName = parent.appName
-  private val basePath = parent.basePath
   private val listener = parent.listener
 
   def render(request: HttpServletRequest): Seq[Node] = {
@@ -36,8 +34,7 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
     val storageStatusList = listener.storageStatusList
     val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse {
       // Rather than crashing, render an "RDD Not Found" page
-      return UIUtils.headerSparkPage(Seq[Node](), basePath, appName, "RDD Not Found",
-        parent.headerTabs, parent)
+      return UIUtils.headerSparkPage("RDD Not Found", Seq[Node](), parent)
     }
 
     // Worker table
@@ -96,8 +93,7 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
         </div>
       </div>;
 
-    UIUtils.headerSparkPage(content, basePath, appName, "RDD Storage Info for " + rddInfo.name,
-      parent.headerTabs, parent)
+    UIUtils.headerSparkPage("RDD Storage Info for " + rddInfo.name, content, parent)
   }
 
   /** Header fields for the worker table */
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
index 9813d9330ac7fdf231c8b5c19228e44f8f9dc5f6..716591c9ed4498c9e5fd40c802b54d6a60de6b70 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
@@ -27,14 +27,12 @@ import org.apache.spark.util.Utils
 
 /** Page showing list of RDD's currently stored in the cluster */
 private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") {
-  private val appName = parent.appName
-  private val basePath = parent.basePath
   private val listener = parent.listener
 
   def render(request: HttpServletRequest): Seq[Node] = {
     val rdds = listener.rddInfoList
     val content = UIUtils.listingTable(rddHeader, rddRow, rdds)
-    UIUtils.headerSparkPage(content, basePath, appName, "Storage ", parent.headerTabs, parent)
+    UIUtils.headerSparkPage("Storage", content, parent)
   }
 
   /** Header fields for the RDD table */
@@ -52,7 +50,7 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") {
     // scalastyle:off
     <tr>
       <td>
-        <a href={"%s/storage/rdd?id=%s".format(UIUtils.prependBaseUri(basePath), rdd.id)}>
+        <a href={"%s/storage/rdd?id=%s".format(UIUtils.prependBaseUri(parent.basePath), rdd.id)}>
           {rdd.name}
         </a>
       </td>
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
index 5f6740d49552190511e175b56596ee6ef509566f..67f72a94f02690946302346685a6687e5768dc21 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
@@ -25,9 +25,7 @@ import org.apache.spark.scheduler._
 import org.apache.spark.storage._
 
 /** Web UI showing storage status of all RDD's in the given SparkContext. */
-private[ui] class StorageTab(parent: SparkUI) extends WebUITab(parent, "storage") {
-  val appName = parent.appName
-  val basePath = parent.basePath
+private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storage") {
   val listener = new StorageListener(parent.storageStatusListener)
 
   attachPage(new StoragePage(this))
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala
index 451b23e01c99545bc031188c261ca2df5dfe30d3..1353e487c72cfaed557d8735f3f40cea076d9cd1 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala
@@ -42,8 +42,7 @@ private[ui] class StreamingPage(parent: StreamingTab)
       <h4>Statistics over last {listener.retainedCompletedBatches.size} processed batches</h4> ++
       generateReceiverStats() ++
       generateBatchStatsTable()
-    UIUtils.headerSparkPage(
-      content, parent.basePath, parent.appName, "Streaming", parent.headerTabs, parent, Some(5000))
+    UIUtils.headerSparkPage("Streaming", content, parent, Some(5000))
   }
 
   /** Generate basic stats of the streaming program */
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala
index 51448d15c65161b96dc9f47eb7ab3cf43044db8d..34ac254f337eb6ac332a5b21727e39aae0a471b4 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala
@@ -19,15 +19,13 @@ package org.apache.spark.streaming.ui
 
 import org.apache.spark.Logging
 import org.apache.spark.streaming.StreamingContext
-import org.apache.spark.ui.WebUITab
+import org.apache.spark.ui.SparkUITab
 
 /** Spark Web UI tab that shows statistics of a streaming job */
 private[spark] class StreamingTab(ssc: StreamingContext)
-  extends WebUITab(ssc.sc.ui, "streaming") with Logging {
+  extends SparkUITab(ssc.sc.ui, "streaming") with Logging {
 
   val parent = ssc.sc.ui
-  val appName = parent.appName
-  val basePath = parent.basePath
   val listener = new StreamingJobProgressListener(ssc)
 
   ssc.addStreamingListener(listener)