Skip to content

Commit 31a44fa

Browse files
committed
add unit test for attaching and detaching new tab
1 parent db25ed2 commit 31a44fa

File tree

5 files changed

+55
-45
lines changed

5 files changed

+55
-45
lines changed

core/src/main/scala/org/apache/spark/ui/WebUI.scala

Lines changed: 4 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -46,7 +46,7 @@ private[spark] abstract class WebUI(
4646

4747
protected val tabs = ArrayBuffer[WebUITab]()
4848
protected val handlers = ArrayBuffer[ServletContextHandler]()
49-
protected val page2Handlers = new HashMap[WebUIPage, ArrayBuffer[ServletContextHandler]]
49+
protected val pageToHandlers = new HashMap[WebUIPage, ArrayBuffer[ServletContextHandler]]
5050
protected var serverInfo: Option[ServerInfo] = None
5151
protected val localHostName = Utils.localHostName()
5252
protected val publicHostName = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName)
@@ -69,9 +69,8 @@ private[spark] abstract class WebUI(
6969
}
7070

7171
def detachPage(page: WebUIPage) {
72-
page2Handlers.remove(page).foreach(_.foreach(detachHandler))
72+
pageToHandlers.remove(page).foreach(_.foreach(detachHandler))
7373
}
74-
7574

7675
/** Attach a page to this UI. */
7776
def attachPage(page: WebUIPage) {
@@ -82,9 +81,9 @@ private[spark] abstract class WebUI(
8281
(request: HttpServletRequest) => page.renderJson(request), securityManager, basePath)
8382
attachHandler(renderHandler)
8483
attachHandler(renderJsonHandler)
85-
page2Handlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]())
84+
pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]())
8685
.append(renderHandler)
87-
page2Handlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]())
86+
pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]())
8887
.append(renderJsonHandler)
8988

9089
}

core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala

Lines changed: 48 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,20 +17,24 @@
1717

1818
package org.apache.spark.ui
1919

20+
import javax.servlet.http.HttpServletRequest
21+
2022
import scala.collection.JavaConversions._
23+
import scala.xml.Node
2124

22-
import org.openqa.selenium.{By, WebDriver}
2325
import org.openqa.selenium.htmlunit.HtmlUnitDriver
26+
import org.openqa.selenium.{By, WebDriver}
2427
import org.scalatest._
2528
import org.scalatest.concurrent.Eventually._
2629
import org.scalatest.selenium.WebBrowser
2730
import org.scalatest.time.SpanSugar._
2831

29-
import org.apache.spark._
3032
import org.apache.spark.LocalSparkContext._
33+
import org.apache.spark._
3134
import org.apache.spark.api.java.StorageLevels
3235
import org.apache.spark.shuffle.FetchFailedException
3336

37+
3438
/**
3539
* Selenium tests for the Spark Web UI.
3640
*/
@@ -310,4 +314,46 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
310314
}
311315
}
312316
}
317+
318+
test("attaching and detaching a new tab") {
319+
withSpark(newSparkContext()) { sc =>
320+
val sparkUI = sc.ui.get
321+
322+
val newTab = new WebUITab(sparkUI, "foo") {
323+
attachPage(new WebUIPage("") {
324+
def render(request: HttpServletRequest): Seq[Node] = {
325+
<b>"html magic"</b>
326+
}
327+
})
328+
}
329+
sparkUI.attachTab(newTab)
330+
eventually(timeout(10 seconds), interval(50 milliseconds)) {
331+
go to (sc.ui.get.appUIAddress.stripSuffix("/"))
332+
find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None)
333+
find(cssSelector("""ul li a[href*="stages"]""")) should not be(None)
334+
find(cssSelector("""ul li a[href*="storage"]""")) should not be(None)
335+
find(cssSelector("""ul li a[href*="environment"]""")) should not be(None)
336+
find(cssSelector("""ul li a[href*="foo"]""")) should not be(None)
337+
}
338+
eventually(timeout(10 seconds), interval(50 milliseconds)) {
339+
// check whether new page exists
340+
go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo")
341+
find(cssSelector("b")).get.text.contains("magic")
342+
}
343+
sparkUI.detachTab(newTab)
344+
eventually(timeout(10 seconds), interval(50 milliseconds)) {
345+
go to (sc.ui.get.appUIAddress.stripSuffix("/"))
346+
find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None)
347+
find(cssSelector("""ul li a[href*="stages"]""")) should not be(None)
348+
find(cssSelector("""ul li a[href*="storage"]""")) should not be(None)
349+
find(cssSelector("""ul li a[href*="environment"]""")) should not be(None)
350+
find(cssSelector("""ul li a[href*="foo"]""")) should be(None)
351+
}
352+
eventually(timeout(10 seconds), interval(50 milliseconds)) {
353+
// check new page not exist
354+
go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo")
355+
find(cssSelector("b")) should be(None)
356+
}
357+
}
358+
}
313359
}

core/src/test/scala/org/apache/spark/ui/UISuite.scala

Lines changed: 1 addition & 37 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@
1818
package org.apache.spark.ui
1919

2020
import java.net.ServerSocket
21-
import javax.servlet.http.HttpServletRequest
2221

2322
import scala.io.Source
2423
import scala.util.{Failure, Success, Try}
@@ -28,9 +27,8 @@ import org.scalatest.FunSuite
2827
import org.scalatest.concurrent.Eventually._
2928
import org.scalatest.time.SpanSugar._
3029

31-
import org.apache.spark.{SparkContext, SparkConf}
3230
import org.apache.spark.LocalSparkContext._
33-
import scala.xml.Node
31+
import org.apache.spark.{SparkConf, SparkContext}
3432

3533
class UISuite extends FunSuite {
3634

@@ -72,40 +70,6 @@ class UISuite extends FunSuite {
7270
}
7371
}
7472

75-
ignore("attaching a new tab") {
76-
withSpark(newSparkContext()) { sc =>
77-
val sparkUI = sc.ui.get
78-
79-
val newTab = new WebUITab(sparkUI, "foo") {
80-
attachPage(new WebUIPage("") {
81-
def render(request: HttpServletRequest): Seq[Node] = {
82-
<b>"html magic"</b>
83-
}
84-
})
85-
}
86-
sparkUI.attachTab(newTab)
87-
eventually(timeout(10 seconds), interval(50 milliseconds)) {
88-
val html = Source.fromURL(sparkUI.appUIAddress).mkString
89-
assert(!html.contains("random data that should not be present"))
90-
91-
// check whether new page exists
92-
assert(html.toLowerCase.contains("foo"))
93-
94-
// check whether other pages still exist
95-
assert(html.toLowerCase.contains("stages"))
96-
assert(html.toLowerCase.contains("storage"))
97-
assert(html.toLowerCase.contains("environment"))
98-
assert(html.toLowerCase.contains("executors"))
99-
}
100-
101-
eventually(timeout(10 seconds), interval(50 milliseconds)) {
102-
val html = Source.fromURL(sparkUI.appUIAddress.stripSuffix("/") + "/foo").mkString
103-
// check whether new page exists
104-
assert(html.contains("magic"))
105-
}
106-
}
107-
}
108-
10973
test("jetty selects different port under contention") {
11074
val server = new ServerSocket(0)
11175
val startPort = server.getLocalPort

streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -553,7 +553,6 @@ class StreamingContext private[streaming] (
553553
*/
554554
def stop(stopSparkContext: Boolean = true): Unit = synchronized {
555555
stop(stopSparkContext, false)
556-
this.uiTab.foreach(StreamingTab.detachStreamingTab(this, _))
557556
}
558557

559558
/**
@@ -579,6 +578,7 @@ class StreamingContext private[streaming] (
579578
// Even if we have already stopped, we still need to attempt to stop the SparkContext because
580579
// a user might stop(stopSparkContext = false) and then call stop(stopSparkContext = true).
581580
if (stopSparkContext) sc.stop()
581+
uiTab.foreach(StreamingTab.detachStreamingTab(this, _))
582582
// The state should always be Stopped after calling `stop()`, even if we haven't started yet:
583583
state = Stopped
584584
}

streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,7 @@ private[spark] object StreamingTab {
4444
throw new SparkException("Parent SparkUI to attach this tab to not found!")
4545
}
4646
}
47+
4748
def detachStreamingTab(ssc: StreamingContext, tab: SparkUITab) {
4849
getSparkUI(ssc).detachTab(tab)
4950
}

0 commit comments

Comments
 (0)