Skip to content

Commit

Permalink
Add batch links and unit tests
Browse files Browse the repository at this point in the history
  • Loading branch information
zsxwing committed Apr 15, 2015
1 parent 4bf66b6 commit 15bdf9b
Show file tree
Hide file tree
Showing 2 changed files with 33 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,11 @@ private[ui] abstract class BatchTableBase(tableId: String) {
val processingTime = batch.processingDelay
val formattedProcessingTime = processingTime.map(UIUtils.formatDuration).getOrElse("-")

<td sorttable_customkey={batchTime.toString}>{formattedBatchTime}</td>
<td sorttable_customkey={batchTime.toString}>
<a href={s"batch?id=$batchTime"}>
{formattedBatchTime}
</a>
</td>
<td sorttable_customkey={eventCount.toString}>{eventCount.toString} events</td>
<td sorttable_customkey={schedulingDelay.getOrElse(Long.MaxValue).toString}>
{formattedSchedulingDelay}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,8 +88,8 @@ class UISeleniumSuite
statisticText should contain("Batch interval:")

val h4Text = findAll(cssSelector("h4")).map(_.text).toSeq
h4Text should contain("Active Batches (0)")
h4Text should contain("Completed Batches (last 0 out of 0)")
h4Text.exists(_.matches("Active Batches \\(\\d+\\)")) should be (true)
h4Text.exists(_.matches("Completed Batches \\(last \\d+ out of \\d+\\)")) should be (true)

findAll(cssSelector("""#active-batches-table th""")).map(_.text).toSeq should be {
List("Batch Time", "Input Size", "Scheduling Delay", "Processing Time", "Status")
Expand All @@ -98,7 +98,32 @@ class UISeleniumSuite
List("Batch Time", "Input Size", "Scheduling Delay", "Processing Time", "Total Delay")
}

// TODO add tests once SPARK-6796 is merged
val batchLinks =
findAll(cssSelector("""#completed-batches-table a""")).flatMap(_.attribute("href")).toSeq
batchLinks.size should be >= 1

// Check a normal batch page
go to (batchLinks.last) // Last should be the first batch, so it will have some jobs
val summaryText = findAll(cssSelector("li strong")).map(_.text).toSeq
summaryText should contain ("Batch Duration:")
summaryText should contain ("Input data size:")
summaryText should contain ("Scheduling delay:")
summaryText should contain ("Processing time:")
summaryText should contain ("Total delay:")

findAll(cssSelector("""#batch-job-table th""")).map(_.text).toSeq should be {
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 the job link in the batch page is right
go to (jobLinks(0))
val jobDetails = findAll(cssSelector("li strong")).map(_.text).toSeq
jobDetails should contain("Status:")
jobDetails should contain("Completed Stages:")

// Check a batch page without id
go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming/batch/")
Expand Down

0 comments on commit 15bdf9b

Please sign in to comment.