Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
28 changes: 23 additions & 5 deletions core/src/main/scala/org/apache/spark/ui/WebUI.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,14 +20,15 @@ package org.apache.spark.ui
import javax.servlet.http.HttpServletRequest

import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
import scala.xml.Node

import org.eclipse.jetty.servlet.ServletContextHandler
import org.json4s.JsonAST.{JNothing, JValue}

import org.apache.spark.{Logging, SecurityManager, SparkConf}
import org.apache.spark.ui.JettyUtils._
import org.apache.spark.util.Utils
import org.apache.spark.{Logging, SecurityManager, SparkConf}

/**
* The top level component of the UI hierarchy that contains the server.
Expand All @@ -45,6 +46,7 @@ private[spark] abstract class WebUI(

protected val tabs = ArrayBuffer[WebUITab]()
protected val handlers = ArrayBuffer[ServletContextHandler]()
protected val pageToHandlers = new HashMap[WebUIPage, ArrayBuffer[ServletContextHandler]]
protected var serverInfo: Option[ServerInfo] = None
protected val localHostName = Utils.localHostName()
protected val publicHostName = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName)
Expand All @@ -60,14 +62,30 @@ private[spark] abstract class WebUI(
tab.pages.foreach(attachPage)
tabs += tab
}

def detachTab(tab: WebUITab) {
tab.pages.foreach(detachPage)
tabs -= tab
}

def detachPage(page: WebUIPage) {
pageToHandlers.remove(page).foreach(_.foreach(detachHandler))
}

/** Attach a page to this UI. */
def attachPage(page: WebUIPage) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@JoshRosen Can you take a look at this code. This is your domain.

val pagePath = "/" + page.prefix
attachHandler(createServletHandler(pagePath,
(request: HttpServletRequest) => page.render(request), securityManager, basePath))
attachHandler(createServletHandler(pagePath.stripSuffix("/") + "/json",
(request: HttpServletRequest) => page.renderJson(request), securityManager, basePath))
val renderHandler = createServletHandler(pagePath,
(request: HttpServletRequest) => page.render(request), securityManager, basePath)
val renderJsonHandler = createServletHandler(pagePath.stripSuffix("/") + "/json",
(request: HttpServletRequest) => page.renderJson(request), securityManager, basePath)
attachHandler(renderHandler)
attachHandler(renderJsonHandler)
pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]())
.append(renderHandler)
pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]())
.append(renderJsonHandler)

}

/** Attach a handler to this UI. */
Expand Down
50 changes: 48 additions & 2 deletions core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,20 +17,24 @@

package org.apache.spark.ui

import javax.servlet.http.HttpServletRequest

import scala.collection.JavaConversions._
import scala.xml.Node

import org.openqa.selenium.{By, WebDriver}
import org.openqa.selenium.htmlunit.HtmlUnitDriver
import org.openqa.selenium.{By, WebDriver}
import org.scalatest._
import org.scalatest.concurrent.Eventually._
import org.scalatest.selenium.WebBrowser
import org.scalatest.time.SpanSugar._

import org.apache.spark._
import org.apache.spark.LocalSparkContext._
import org.apache.spark._
import org.apache.spark.api.java.StorageLevels
import org.apache.spark.shuffle.FetchFailedException


/**
* Selenium tests for the Spark Web UI.
*/
Expand Down Expand Up @@ -310,4 +314,46 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
}
}
}

test("attaching and detaching a new tab") {
withSpark(newSparkContext()) { sc =>
val sparkUI = sc.ui.get

val newTab = new WebUITab(sparkUI, "foo") {
attachPage(new WebUIPage("") {
def render(request: HttpServletRequest): Seq[Node] = {
<b>"html magic"</b>
}
})
}
sparkUI.attachTab(newTab)
eventually(timeout(10 seconds), interval(50 milliseconds)) {
go to (sc.ui.get.appUIAddress.stripSuffix("/"))
find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None)
find(cssSelector("""ul li a[href*="stages"]""")) should not be(None)
find(cssSelector("""ul li a[href*="storage"]""")) should not be(None)
find(cssSelector("""ul li a[href*="environment"]""")) should not be(None)
find(cssSelector("""ul li a[href*="foo"]""")) should not be(None)
}
eventually(timeout(10 seconds), interval(50 milliseconds)) {
// check whether new page exists
go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo")
find(cssSelector("b")).get.text should include ("html magic")
}
sparkUI.detachTab(newTab)
eventually(timeout(10 seconds), interval(50 milliseconds)) {
go to (sc.ui.get.appUIAddress.stripSuffix("/"))
find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None)
find(cssSelector("""ul li a[href*="stages"]""")) should not be(None)
find(cssSelector("""ul li a[href*="storage"]""")) should not be(None)
find(cssSelector("""ul li a[href*="environment"]""")) should not be(None)
find(cssSelector("""ul li a[href*="foo"]""")) should be(None)
}
eventually(timeout(10 seconds), interval(50 milliseconds)) {
// check new page not exist
go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo")
find(cssSelector("b")) should be(None)
}
}
}
}
38 changes: 1 addition & 37 deletions core/src/test/scala/org/apache/spark/ui/UISuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
package org.apache.spark.ui

import java.net.ServerSocket
import javax.servlet.http.HttpServletRequest

import scala.io.Source
import scala.util.{Failure, Success, Try}
Expand All @@ -28,9 +27,8 @@ import org.scalatest.FunSuite
import org.scalatest.concurrent.Eventually._
import org.scalatest.time.SpanSugar._

import org.apache.spark.{SparkContext, SparkConf}
import org.apache.spark.LocalSparkContext._
import scala.xml.Node
import org.apache.spark.{SparkConf, SparkContext}

class UISuite extends FunSuite {

Expand Down Expand Up @@ -72,40 +70,6 @@ class UISuite extends FunSuite {
}
}

ignore("attaching a new tab") {
withSpark(newSparkContext()) { sc =>
val sparkUI = sc.ui.get

val newTab = new WebUITab(sparkUI, "foo") {
attachPage(new WebUIPage("") {
def render(request: HttpServletRequest): Seq[Node] = {
<b>"html magic"</b>
}
})
}
sparkUI.attachTab(newTab)
eventually(timeout(10 seconds), interval(50 milliseconds)) {
val html = Source.fromURL(sparkUI.appUIAddress).mkString
assert(!html.contains("random data that should not be present"))

// check whether new page exists
assert(html.toLowerCase.contains("foo"))

// check whether other pages still exist
assert(html.toLowerCase.contains("stages"))
assert(html.toLowerCase.contains("storage"))
assert(html.toLowerCase.contains("environment"))
assert(html.toLowerCase.contains("executors"))
}

eventually(timeout(10 seconds), interval(50 milliseconds)) {
val html = Source.fromURL(sparkUI.appUIAddress.stripSuffix("/") + "/foo").mkString
// check whether new page exists
assert(html.contains("magic"))
}
}
}

test("jetty selects different port under contention") {
val server = new ServerSocket(0)
val startPort = server.getLocalPort
Expand Down
5 changes: 5 additions & 0 deletions streaming/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,11 @@
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.seleniumhq.selenium</groupId>
<artifactId>selenium-java</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.novocode</groupId>
<artifactId>junit-interface</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -578,6 +578,7 @@ class StreamingContext private[streaming] (
// Even if we have already stopped, we still need to attempt to stop the SparkContext because
// a user might stop(stopSparkContext = false) and then call stop(stopSparkContext = true).
if (stopSparkContext) sc.stop()
uiTab.foreach(_.detach())
// The state should always be Stopped after calling `stop()`, even if we haven't started yet:
state = Stopped
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ private[ui] class StreamingPage(parent: StreamingTab)
extends WebUIPage("") with Logging {

private val listener = parent.listener
private val startTime = Calendar.getInstance().getTime()
private val startTime = System.currentTimeMillis()
private val emptyCell = "-"

/** Render the page */
Expand All @@ -47,7 +47,7 @@ private[ui] class StreamingPage(parent: StreamingTab)

/** Generate basic stats of the streaming program */
private def generateBasicStats(): Seq[Node] = {
val timeSinceStart = System.currentTimeMillis() - startTime.getTime
val timeSinceStart = System.currentTimeMillis() - startTime
<ul class ="unstyled">
<li>
<strong>Started at: </strong> {startTime.toString}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,10 @@ private[spark] class StreamingTab(ssc: StreamingContext)
ssc.addStreamingListener(listener)
attachPage(new StreamingPage(this))
parent.attachTab(this)

def detach() {
getSparkUI(ssc).detachTab(this)
}
}

private object StreamingTab {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,95 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.streaming

import org.openqa.selenium.WebDriver
import org.openqa.selenium.htmlunit.HtmlUnitDriver
import org.scalatest._
import org.scalatest.concurrent.Eventually._
import org.scalatest.selenium.WebBrowser
import org.scalatest.time.SpanSugar._

import org.apache.spark._




/**
* Selenium tests for the Spark Web UI.
*/
class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with BeforeAndAfterAll with TestSuiteBase {

implicit var webDriver: WebDriver = _

override def beforeAll(): Unit = {
webDriver = new HtmlUnitDriver
}

override def afterAll(): Unit = {
if (webDriver != null) {
webDriver.quit()
}
}

/**
* Create a test SparkStreamingContext with the SparkUI enabled.
*/
private def newSparkStreamingContext(): StreamingContext = {
val conf = new SparkConf()
.setMaster("local")
.setAppName("test")
.set("spark.ui.enabled", "true")
val ssc = new StreamingContext(conf, Seconds(1))
assert(ssc.sc.ui.isDefined, "Spark UI is not started!")
ssc
}

test("attaching and detaching a Streaming tab") {
withStreamingContext(newSparkStreamingContext()) { ssc =>
val sparkUI = ssc.sparkContext.ui.get

eventually(timeout(10 seconds), interval(50 milliseconds)) {
go to (sparkUI.appUIAddress.stripSuffix("/"))
find(cssSelector( """ul li a[href*="streaming"]""")) should not be (None)
}

eventually(timeout(10 seconds), interval(50 milliseconds)) {
// check whether streaming page exists
go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming")
val statisticText = findAll(cssSelector("li strong")).map(_.text).toSeq
statisticText should contain("Network receivers:")
statisticText should contain("Batch interval:")
}

ssc.stop(false)

eventually(timeout(10 seconds), interval(50 milliseconds)) {
go to (sparkUI.appUIAddress.stripSuffix("/"))
find(cssSelector( """ul li a[href*="streaming"]""")) should be(None)
}

eventually(timeout(10 seconds), interval(50 milliseconds)) {
go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming")
val statisticText = findAll(cssSelector("li strong")).map(_.text).toSeq
statisticText should not contain ("Network receivers:")
statisticText should not contain ("Batch interval:")
}
}
}
}

55 changes: 0 additions & 55 deletions streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala

This file was deleted.