Skip to content

Commit f8e1053

Browse files
committed
Added Spark and Streaming UI unit tests.
1 parent caa5e05 commit f8e1053

File tree

8 files changed

+94
-110
lines changed

8 files changed

+94
-110
lines changed

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

Lines changed: 0 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -132,9 +132,6 @@ private[spark] abstract class WebUITab(parent: WebUI, val prefix: String) {
132132
pages += page
133133
}
134134

135-
/** Initialize this tab and attach all relevant pages. */
136-
def initialize()
137-
138135
/** Get a list of header tabs from the parent UI. */
139136
def headerTabs: Seq[WebUITab] = parent.getTabs
140137
}

core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala

Lines changed: 2 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -25,12 +25,8 @@ private[ui] class EnvironmentTab(parent: SparkUI) extends WebUITab(parent, "envi
2525
val basePath = parent.basePath
2626
val listener = new EnvironmentListener
2727

28-
initialize()
29-
30-
def initialize() {
31-
attachPage(new IndexPage(this))
32-
parent.registerListener(listener)
33-
}
28+
attachPage(new IndexPage(this))
29+
parent.registerListener(listener)
3430
}
3531

3632
/**

core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala

Lines changed: 2 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -29,12 +29,8 @@ private[ui] class ExecutorsTab(parent: SparkUI) extends WebUITab(parent, "execut
2929
val basePath = parent.basePath
3030
val listener = new ExecutorsListener(parent.storageStatusListener)
3131

32-
initialize()
33-
34-
def initialize() {
35-
attachPage(new IndexPage(this))
36-
parent.registerListener(listener)
37-
}
32+
attachPage(new IndexPage(this))
33+
parent.registerListener(listener)
3834
}
3935

4036
/**

core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala

Lines changed: 4 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -30,14 +30,10 @@ private[ui] class JobProgressTab(parent: SparkUI) extends WebUITab(parent, "stag
3030
val conf = if (live) sc.conf else new SparkConf
3131
val listener = new JobProgressListener(conf)
3232

33-
initialize()
34-
35-
def initialize() {
36-
attachPage(new IndexPage(this))
37-
attachPage(new StagePage(this))
38-
attachPage(new PoolPage(this))
39-
parent.registerListener(listener)
40-
}
33+
attachPage(new IndexPage(this))
34+
attachPage(new StagePage(this))
35+
attachPage(new PoolPage(this))
36+
parent.registerListener(listener)
4137

4238
def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR)
4339
}

core/src/main/scala/org/apache/spark/ui/storage/BlockManagerTab.scala

Lines changed: 3 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -29,13 +29,9 @@ private[ui] class BlockManagerTab(parent: SparkUI) extends WebUITab(parent, "sto
2929
val basePath = parent.basePath
3030
val listener = new BlockManagerListener(parent.storageStatusListener)
3131

32-
initialize()
33-
34-
def initialize() {
35-
attachPage(new IndexPage(this))
36-
attachPage(new RddPage(this))
37-
parent.registerListener(listener)
38-
}
32+
attachPage(new IndexPage(this))
33+
attachPage(new RddPage(this))
34+
parent.registerListener(listener)
3935
}
4036

4137
/**

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

Lines changed: 66 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,16 +18,81 @@
1818
package org.apache.spark.ui
1919

2020
import java.net.ServerSocket
21+
import javax.servlet.http.HttpServletRequest
2122

23+
import scala.io.Source
2224
import scala.util.{Failure, Success, Try}
2325

2426
import org.eclipse.jetty.server.Server
2527
import org.eclipse.jetty.servlet.ServletContextHandler
2628
import org.scalatest.FunSuite
29+
import org.scalatest.concurrent.Eventually._
30+
import org.scalatest.time.SpanSugar._
2731

28-
import org.apache.spark.SparkConf
32+
import org.apache.spark.{SparkContext, SparkConf}
33+
import org.apache.spark.LocalSparkContext._
34+
import scala.xml.Node
2935

3036
class UISuite extends FunSuite {
37+
38+
test("basic ui visibility") {
39+
withSpark(new SparkContext("local", "test")) { sc =>
40+
// test if the ui is visible, and all the expected tabs are visible
41+
eventually(timeout(10 seconds), interval(50 milliseconds)) {
42+
val html = Source.fromURL(sc.ui.appUIAddress).mkString
43+
assert(!html.contains("random data that should not be present"))
44+
assert(html.toLowerCase.contains("stages"))
45+
assert(html.toLowerCase.contains("storage"))
46+
assert(html.toLowerCase.contains("environment"))
47+
assert(html.toLowerCase.contains("executors"))
48+
}
49+
}
50+
}
51+
52+
test("visibility at localhost:4040") {
53+
withSpark(new SparkContext("local", "test")) { sc =>
54+
// test if visible from http://localhost:4040
55+
eventually(timeout(10 seconds), interval(50 milliseconds)) {
56+
val html = Source.fromURL("http://localhost:4040").mkString
57+
assert(html.toLowerCase.contains("stages"))
58+
}
59+
}
60+
}
61+
62+
test("attaching a new tab") {
63+
withSpark(new SparkContext("local", "test")) { sc =>
64+
val sparkUI = sc.ui
65+
66+
val newTab = new WebUITab(sparkUI, "foo") {
67+
attachPage(new WebUIPage("") {
68+
override def render(request: HttpServletRequest): Seq[Node] = {
69+
<b>"html magic"</b>
70+
}
71+
})
72+
}
73+
sparkUI.attachTab(newTab)
74+
eventually(timeout(10 seconds), interval(50 milliseconds)) {
75+
val html = Source.fromURL(sc.ui.appUIAddress).mkString
76+
assert(!html.contains("random data that should not be present"))
77+
78+
// check whether new page exists
79+
assert(html.toLowerCase.contains("foo"))
80+
81+
// check whether other pages still exist
82+
assert(html.toLowerCase.contains("stages"))
83+
assert(html.toLowerCase.contains("storage"))
84+
assert(html.toLowerCase.contains("environment"))
85+
assert(html.toLowerCase.contains("executors"))
86+
}
87+
88+
eventually(timeout(10 seconds), interval(50 milliseconds)) {
89+
val html = Source.fromURL(sc.ui.appUIAddress.stripSuffix("/") + "/foo").mkString
90+
// check whether new page exists
91+
assert(html.contains("magic"))
92+
}
93+
}
94+
}
95+
3196
test("jetty port increases under contention") {
3297
val startPort = 4040
3398
val server = new Server(startPort)

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

Lines changed: 3 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,6 @@
1717

1818
package org.apache.spark.streaming.ui
1919

20-
import java.util.concurrent.atomic.AtomicInteger
21-
2220
import org.apache.spark.Logging
2321
import org.apache.spark.streaming.StreamingContext
2422
import org.apache.spark.ui.WebUITab
@@ -32,11 +30,7 @@ private[spark] class StreamingTab(ssc: StreamingContext)
3230
val basePath = parent.basePath
3331
val listener = new StreamingJobProgressListener(ssc)
3432

35-
initialize()
36-
37-
def initialize() {
38-
ssc.addStreamingListener(listener)
39-
attachPage(new StreamingPage(this))
40-
parent.attachTab(this)
41-
}
33+
ssc.addStreamingListener(listener)
34+
attachPage(new StreamingPage(this))
35+
parent.attachTab(this)
4236
}

streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala

Lines changed: 14 additions & 70 deletions
Original file line numberDiff line numberDiff line change
@@ -17,86 +17,30 @@
1717

1818
package org.apache.spark.streaming
1919

20-
import scala.reflect.ClassTag
21-
import scala.util.Random
2220
import scala.io.Source
2321

24-
import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite}
22+
import org.scalatest.FunSuite
2523
import org.scalatest.concurrent.Eventually._
2624
import org.scalatest.time.SpanSugar._
27-
import org.scalatest.matchers.ShouldMatchers
2825

29-
import org.apache.spark.{SparkConf, SparkContext}
30-
import org.apache.spark.rdd.RDD
31-
import org.apache.spark.streaming.dstream.InputDStream
32-
33-
class UISuite extends FunSuite with ShouldMatchers with BeforeAndAfterAll with BeforeAndAfter {
34-
var sc: SparkContext = null
35-
var ssc: StreamingContext = null
36-
37-
override def beforeAll() {
38-
val conf = new SparkConf().setMaster("local").setAppName(this.getClass.getSimpleName)
39-
conf.set("spark.cleaner.ttl", "1800")
40-
sc = new SparkContext(conf)
41-
}
42-
43-
override def afterAll() {
44-
if (sc != null) sc.stop()
45-
}
46-
47-
before {
48-
ssc = new StreamingContext(sc, Seconds(1))
49-
}
50-
51-
after {
52-
if (ssc != null) {
53-
ssc.stop()
54-
ssc = null
55-
}
56-
}
26+
class UISuite extends FunSuite {
5727

5828
test("streaming tab in spark UI") {
59-
val ssc = new StreamingContext(sc, Seconds(1))
29+
val ssc = new StreamingContext("local", "test", Seconds(1))
6030
eventually(timeout(10 seconds), interval(50 milliseconds)) {
61-
val uiData = Source.fromURL(
62-
ssc.sparkContext.ui.appUIAddress).mkString
63-
assert(!uiData.contains("random data that should not be present"))
64-
assert(uiData.contains("streaming"))
31+
val html = Source.fromURL(ssc.sparkContext.ui.appUIAddress).mkString
32+
assert(!html.contains("random data that should not be present"))
33+
// test if streaming tab exist
34+
assert(html.toLowerCase.contains("streaming"))
35+
// test if other Spark tabs still exist
36+
assert(html.toLowerCase.contains("stages"))
6537
}
66-
}
6738

68-
ignore("Testing") {
69-
runStreaming(1000000)
70-
}
71-
72-
def runStreaming(duration: Long) {
73-
val ssc1 = new StreamingContext(sc, Seconds(1))
74-
val servers1 = (1 to 3).map { i => new TestServer(10000 + i) }
75-
76-
val inputStream1 = ssc1.union(servers1.map(server => ssc1.socketTextStream("localhost", server.port)))
77-
inputStream1.count.print
78-
79-
ssc1.start()
80-
servers1.foreach(_.start())
81-
82-
val startTime = System.currentTimeMillis()
83-
while (System.currentTimeMillis() - startTime < duration) {
84-
servers1.map(_.send(Random.nextString(10) + "\n"))
85-
//Thread.sleep(1)
39+
eventually(timeout(10 seconds), interval(50 milliseconds)) {
40+
val html = Source.fromURL(
41+
ssc.sparkContext.ui.appUIAddress.stripSuffix("/") + "/streaming").mkString
42+
assert(html.toLowerCase.contains("batch"))
43+
assert(html.toLowerCase.contains("network"))
8644
}
87-
ssc1.stop()
88-
servers1.foreach(_.stop())
8945
}
9046
}
91-
92-
class FunctionBasedInputDStream[T: ClassTag](
93-
ssc_ : StreamingContext,
94-
function: (StreamingContext, Time) => Option[RDD[T]]
95-
) extends InputDStream[T](ssc_) {
96-
97-
def start(): Unit = {}
98-
99-
def stop(): Unit = {}
100-
101-
def compute(validTime: Time): Option[RDD[T]] = function(ssc, validTime)
102-
}

0 commit comments

Comments
 (0)