Skip to content

Commit

Permalink
Add unit tests for BatchPage
Browse files Browse the repository at this point in the history
  • Loading branch information
zsxwing committed Apr 27, 2015
1 parent 1282b10 commit 72f8e7e
Show file tree
Hide file tree
Showing 2 changed files with 40 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -80,9 +80,10 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") {

// In the first row, output op id and its information needs to be shown. In other rows, these
// cells will be taken up due to "rowspan".
// scalastyle:off
val prefixCells =
if (isFirstRow) {
<td rowspan={numSparkJobRowsInOutputOp.toString}>{outputOpId.toString}</td>
<td class="output-op-id-cell" rowspan={numSparkJobRowsInOutputOp.toString}>{outputOpId.toString}</td>
<td rowspan={numSparkJobRowsInOutputOp.toString}>
<span class="description-input" title={lastStageDescription}>
{lastStageDescription}
Expand All @@ -92,6 +93,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") {
} else {
Nil
}
// scalastyle:on

<tr>
{prefixCells}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,18 @@ class UISeleniumSuite
private def setupStreams(ssc: StreamingContext): Unit = {
val rdds = Queue(ssc.sc.parallelize(1 to 4, 4))
val inputStream = ssc.queueStream(rdds)
inputStream.foreachRDD(rdd => rdd.foreach(_ => {}))
inputStream.foreachRDD { rdd =>
rdd.foreach(_ => {})
rdd.foreach(_ => {})
}
inputStream.foreachRDD { rdd =>
rdd.foreach(_ => {})
try {
rdd.foreach(_ => throw new RuntimeException("Oops"))
} catch {
case e: SparkException if e.getMessage.contains("Oops") =>
}
}
}

test("attaching and detaching a Streaming tab") {
Expand Down Expand Up @@ -115,9 +126,31 @@ class UISeleniumSuite
List("Output Op Id", "Description", "Duration", "Job Id", "Duration",
"Stages: Succeeded/Total", "Tasks (for all stages): Succeeded/Total", "Error")
}
val jobLinks =
findAll(cssSelector("""#batch-job-table a""")).flatMap(_.attribute("href")).toSeq
jobLinks.size should be >= (1)

// Check we have 2 output op ids
val outputOpIds = findAll(cssSelector(".output-op-id-cell")).toSeq
outputOpIds.map(_.attribute("rowspan")) should be (List(Some("2"), Some("2")))
outputOpIds.map(_.text) should be (List("0", "1"))

// Check job ids
val jobIdCells = findAll(cssSelector( """#batch-job-table a""")).toSeq
jobIdCells.map(_.text) should be (List("0", "1", "2", "3"))

val jobLinks = jobIdCells.flatMap(_.attribute("href"))
jobLinks.size should be (4)

// Check stage progress
findAll(cssSelector(""".stage-progress-cell""")).map(_.text).toSeq should be
(List("1/1", "1/1", "1/1", "0/1 (1 failed)"))

// Check job progress
findAll(cssSelector(""".progress-cell""")).map(_.text).toSeq should be
(List("1/1", "1/1", "1/1", "0/1 (1 failed)"))

// Check stacktrack
val errorCells = findAll(cssSelector(""".stacktrace-details""")).map(_.text).toSeq
errorCells should have size 1
errorCells(0) should include("java.lang.RuntimeException: Oops")

// Check the job link in the batch page is right
go to (jobLinks(0))
Expand Down

0 comments on commit 72f8e7e

Please sign in to comment.