aboutsummaryrefslogtreecommitdiff
path: root/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
blob: aa22f3ba2b4d14c51f508e55c3b164383561e93a (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
/*
 * 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.ui

import java.net.{HttpURLConnection, URL}
import javax.servlet.http.{HttpServletRequest, HttpServletResponse}

import scala.io.Source
import scala.xml.Node

import com.gargoylesoftware.htmlunit.DefaultCssErrorHandler
import org.json4s._
import org.json4s.jackson.JsonMethods
import org.openqa.selenium.{By, 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.w3c.css.sac.CSSParseException

import org.apache.spark._
import org.apache.spark.LocalSparkContext._
import org.apache.spark.api.java.StorageLevels
import org.apache.spark.deploy.history.HistoryServerSuite
import org.apache.spark.shuffle.FetchFailedException
import org.apache.spark.status.api.v1.{JacksonMessageWriter, StageStatus}

private[spark] class SparkUICssErrorHandler extends DefaultCssErrorHandler {

  private val cssWhiteList = List("bootstrap.min.css", "vis.min.css")

  private def isInWhileList(uri: String): Boolean = cssWhiteList.exists(uri.endsWith)

  override def warning(e: CSSParseException): Unit = {
    if (!isInWhileList(e.getURI)) {
      super.warning(e)
    }
  }

  override def fatalError(e: CSSParseException): Unit = {
    if (!isInWhileList(e.getURI)) {
      super.fatalError(e)
    }
  }

  override def error(e: CSSParseException): Unit = {
    if (!isInWhileList(e.getURI)) {
      super.error(e)
    }
  }
}

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

  implicit var webDriver: WebDriver = _
  implicit val formats = DefaultFormats


  override def beforeAll(): Unit = {
    super.beforeAll()
    webDriver = new HtmlUnitDriver {
      getWebClient.setCssErrorHandler(new SparkUICssErrorHandler)
    }
  }

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

  /**
   * Create a test SparkContext with the SparkUI enabled.
   * It is safe to `get` the SparkUI directly from the SparkContext returned here.
   */
  private def newSparkContext(killEnabled: Boolean = true): SparkContext = {
    val conf = new SparkConf()
      .setMaster("local")
      .setAppName("test")
      .set("spark.ui.enabled", "true")
      .set("spark.ui.port", "0")
      .set("spark.ui.killEnabled", killEnabled.toString)
    val sc = new SparkContext(conf)
    assert(sc.ui.isDefined)
    sc
  }

  test("effects of unpersist() / persist() should be reflected") {
    // Regression test for SPARK-2527
    withSpark(newSparkContext()) { sc =>
      val ui = sc.ui.get
      val rdd = sc.parallelize(Seq(1, 2, 3))
      rdd.persist(StorageLevels.DISK_ONLY).count()
      eventually(timeout(5 seconds), interval(50 milliseconds)) {
        goToUi(ui, "/storage")
        val tableRowText = findAll(cssSelector("#storage-by-rdd-table td")).map(_.text).toSeq
        tableRowText should contain (StorageLevels.DISK_ONLY.description)
      }
      eventually(timeout(5 seconds), interval(50 milliseconds)) {
        goToUi(ui, "/storage/rdd/?id=0")
        val tableRowText = findAll(cssSelector("#rdd-storage-by-block-table td")).map(_.text).toSeq
        tableRowText should contain (StorageLevels.DISK_ONLY.description)
      }

      val storageJson = getJson(ui, "storage/rdd")
      storageJson.children.length should be (1)
      (storageJson \ "storageLevel").extract[String] should be (StorageLevels.DISK_ONLY.description)
      val rddJson = getJson(ui, "storage/rdd/0")
      (rddJson  \ "storageLevel").extract[String] should be (StorageLevels.DISK_ONLY.description)

      rdd.unpersist()
      rdd.persist(StorageLevels.MEMORY_ONLY).count()
      eventually(timeout(5 seconds), interval(50 milliseconds)) {
        goToUi(ui, "/storage")
        val tableRowText = findAll(cssSelector("#storage-by-rdd-table td")).map(_.text).toSeq
        tableRowText should contain (StorageLevels.MEMORY_ONLY.description)
      }
      eventually(timeout(5 seconds), interval(50 milliseconds)) {
        goToUi(ui, "/storage/rdd/?id=0")
        val tableRowText = findAll(cssSelector("#rdd-storage-by-block-table td")).map(_.text).toSeq
        tableRowText should contain (StorageLevels.MEMORY_ONLY.description)
      }

      val updatedStorageJson = getJson(ui, "storage/rdd")
      updatedStorageJson.children.length should be (1)
      (updatedStorageJson \ "storageLevel").extract[String] should be (
        StorageLevels.MEMORY_ONLY.description)
      val updatedRddJson = getJson(ui, "storage/rdd/0")
      (updatedRddJson  \ "storageLevel").extract[String] should be (
        StorageLevels.MEMORY_ONLY.description)
    }
  }

  test("failed stages should not appear to be active") {
    withSpark(newSparkContext()) { sc =>
      // Regression test for SPARK-3021
      intercept[SparkException] {
        sc.parallelize(1 to 10).map { x => throw new Exception()}.collect()
      }
      eventually(timeout(5 seconds), interval(50 milliseconds)) {
        goToUi(sc, "/stages")
        find(id("active")) should be(None)  // Since we hide empty tables
        find(id("failed")).get.text should be("Failed Stages (1)")
      }
      val stageJson = getJson(sc.ui.get, "stages")
      stageJson.children.length should be (1)
      (stageJson \ "status").extract[String] should be (StageStatus.FAILED.name())

      // Regression test for SPARK-2105
      class NotSerializable
      val unserializableObject = new NotSerializable
      intercept[SparkException] {
        sc.parallelize(1 to 10).map { x => unserializableObject}.collect()
      }
      eventually(timeout(5 seconds), interval(50 milliseconds)) {
        goToUi(sc, "/stages")
        find(id("active")) should be(None)  // Since we hide empty tables
        // The failure occurs before the stage becomes active, hence we should still show only one
        // failed stage, not two:
        find(id("failed")).get.text should be("Failed Stages (1)")
      }

      val updatedStageJson = getJson(sc.ui.get, "stages")
      updatedStageJson should be (stageJson)
    }
  }

  test("spark.ui.killEnabled should properly control kill button display") {
    def hasKillLink: Boolean = find(className("kill-link")).isDefined
    def runSlowJob(sc: SparkContext) {
      sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync()
    }

    withSpark(newSparkContext(killEnabled = true)) { sc =>
      runSlowJob(sc)
      eventually(timeout(5 seconds), interval(50 milliseconds)) {
        goToUi(sc, "/stages")
        assert(hasKillLink)
      }
    }

    withSpark(newSparkContext(killEnabled = false)) { sc =>
      runSlowJob(sc)
      eventually(timeout(5 seconds), interval(50 milliseconds)) {
        goToUi(sc, "/stages")
        assert(!hasKillLink)
      }
    }
  }

  test("jobs page should not display job group name unless some job was submitted in a job group") {
    withSpark(newSparkContext()) { sc =>
      // If no job has been run in a job group, then "(Job Group)" should not appear in the header
      sc.parallelize(Seq(1, 2, 3)).count()
      eventually(timeout(5 seconds), interval(50 milliseconds)) {
        goToUi(sc, "/jobs")
        val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq
        tableHeaders should not contain "Job Id (Job Group)"
      }
      // Once at least one job has been run in a job group, then we should display the group name:
      sc.setJobGroup("my-job-group", "my-job-group-description")
      sc.parallelize(Seq(1, 2, 3)).count()
      eventually(timeout(5 seconds), interval(50 milliseconds)) {
        goToUi(sc, "/jobs")
        val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq
        tableHeaders should contain ("Job Id (Job Group)")
      }

      val jobJson = getJson(sc.ui.get, "jobs")
      for {
        job @ JObject(_) <- jobJson
        JInt(jobId) <- job \ "jobId"
        jobGroup = job \ "jobGroup"
      } {
        jobId.toInt match {
          case 0 => jobGroup should be (JNothing)
          case 1 => jobGroup should be (JString("my-job-group"))
        }
      }
    }
  }

  test("job progress bars should handle stage / task failures") {
    withSpark(newSparkContext()) { sc =>
      val data = sc.parallelize(Seq(1, 2, 3), 1).map(identity).groupBy(identity)
      val shuffleHandle =
        data.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle
      // Simulate fetch failures:
      val mappedData = data.map { x =>
        val taskContext = TaskContext.get
        if (taskContext.taskAttemptId() == 1) {
          // Cause the post-shuffle stage to fail on its first attempt with a single task failure
          val env = SparkEnv.get
          val bmAddress = env.blockManager.blockManagerId
          val shuffleId = shuffleHandle.shuffleId
          val mapId = 0
          val reduceId = taskContext.partitionId()
          val message = "Simulated fetch failure"
          throw new FetchFailedException(bmAddress, shuffleId, mapId, reduceId, message)
        } else {
          x
        }
      }
      mappedData.count()
      eventually(timeout(5 seconds), interval(50 milliseconds)) {
        goToUi(sc, "/jobs")
        find(cssSelector(".stage-progress-cell")).get.text should be ("2/2 (1 failed)")
        // Ideally, the following test would pass, but currently we overcount completed tasks
        // if task recomputations occur:
        // find(cssSelector(".progress-cell .progress")).get.text should be ("2/2 (1 failed)")
        // Instead, we guarantee that the total number of tasks is always correct, while the number
        // of completed tasks may be higher:
        find(cssSelector(".progress-cell .progress")).get.text should be ("3/2 (1 failed)")
      }
      val jobJson = getJson(sc.ui.get, "jobs")
      (jobJson \ "numTasks").extract[Int]should be (2)
      (jobJson \ "numCompletedTasks").extract[Int] should be (3)
      (jobJson \ "numFailedTasks").extract[Int] should be (1)
      (jobJson \ "numCompletedStages").extract[Int] should be (2)
      (jobJson \ "numFailedStages").extract[Int] should be (1)
      val stageJson = getJson(sc.ui.get, "stages")

      for {
        stage @ JObject(_) <- stageJson
        JString(status) <- stage \ "status"
        JInt(stageId) <- stage \ "stageId"
        JInt(attemptId) <- stage \ "attemptId"
      } {
        val exp = if (attemptId == 0 && stageId == 1) StageStatus.FAILED else StageStatus.COMPLETE
        status should be (exp.name())
      }

      for {
        stageId <- 0 to 1
        attemptId <- 0 to 1
      } {
        val exp = if (attemptId == 0 && stageId == 1) StageStatus.FAILED else StageStatus.COMPLETE
        val stageJson = getJson(sc.ui.get, s"stages/$stageId/$attemptId")
        (stageJson \ "status").extract[String] should be (exp.name())
      }
    }
  }

  test("job details page should display useful information for stages that haven't started") {
    withSpark(newSparkContext()) { sc =>
      // Create a multi-stage job with a long delay in the first stage:
      val rdd = sc.parallelize(Seq(1, 2, 3)).map { x =>
        // This long sleep call won't slow down the tests because we don't actually need to wait
        // for the job to finish.
        Thread.sleep(20000)
      }.groupBy(identity).map(identity).groupBy(identity).map(identity)
      // Start the job:
      rdd.countAsync()
      eventually(timeout(10 seconds), interval(50 milliseconds)) {
        goToUi(sc, "/jobs/job/?id=0")
        find(id("active")).get.text should be ("Active Stages (1)")
        find(id("pending")).get.text should be ("Pending Stages (2)")
        // Essentially, we want to check that none of the stage rows show
        // "No data available for this stage". Checking for the absence of that string is brittle
        // because someone could change the error message and cause this test to pass by accident.
        // Instead, it's safer to check that each row contains a link to a stage details page.
        findAll(cssSelector("tbody tr")).foreach { row =>
          val link = row.underlying.findElement(By.xpath("./td/div/a"))
          link.getAttribute("href") should include ("stage")
        }
      }
    }
  }

  test("job progress bars / cells reflect skipped stages / tasks") {
    withSpark(newSparkContext()) { sc =>
      // Create an RDD that involves multiple stages:
      val rdd = sc.parallelize(1 to 8, 8)
        .map(x => x).groupBy((x: Int) => x, numPartitions = 8)
        .flatMap(x => x._2).groupBy((x: Int) => x, numPartitions = 8)
      // Run it twice; this will cause the second job to have two "phantom" stages that were
      // mentioned in its job start event but which were never actually executed:
      rdd.count()
      rdd.count()
      eventually(timeout(10 seconds), interval(50 milliseconds)) {
        goToUi(sc, "/jobs")
        // The completed jobs table should have two rows. The first row will be the most recent job:
        val firstRow = find(cssSelector("tbody tr")).get.underlying
        val firstRowColumns = firstRow.findElements(By.tagName("td"))
        firstRowColumns.get(0).getText should be ("1")
        firstRowColumns.get(4).getText should be ("1/1 (2 skipped)")
        firstRowColumns.get(5).getText should be ("8/8 (16 skipped)")
        // The second row is the first run of the job, where nothing was skipped:
        val secondRow = findAll(cssSelector("tbody tr")).toSeq(1).underlying
        val secondRowColumns = secondRow.findElements(By.tagName("td"))
        secondRowColumns.get(0).getText should be ("0")
        secondRowColumns.get(4).getText should be ("3/3")
        secondRowColumns.get(5).getText should be ("24/24")
      }
    }
  }

  test("stages that aren't run appear as 'skipped stages' after a job finishes") {
    withSpark(newSparkContext()) { sc =>
      // Create an RDD that involves multiple stages:
      val rdd =
        sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity).map(identity).groupBy(identity)
      // Run it twice; this will cause the second job to have two "phantom" stages that were
      // mentioned in its job start event but which were never actually executed:
      rdd.count()
      rdd.count()
      eventually(timeout(10 seconds), interval(50 milliseconds)) {
        goToUi(sc, "/jobs/job/?id=1")
        find(id("pending")) should be (None)
        find(id("active")) should be (None)
        find(id("failed")) should be (None)
        find(id("completed")).get.text should be ("Completed Stages (1)")
        find(id("skipped")).get.text should be ("Skipped Stages (2)")
        // Essentially, we want to check that none of the stage rows show
        // "No data available for this stage". Checking for the absence of that string is brittle
        // because someone could change the error message and cause this test to pass by accident.
        // Instead, it's safer to check that each row contains a link to a stage details page.
        findAll(cssSelector("tbody tr")).foreach { row =>
          val link = row.underlying.findElement(By.xpath(".//a"))
          link.getAttribute("href") should include ("stage")
        }
      }
    }
  }

  test("jobs with stages that are skipped should show correct link descriptions on all jobs page") {
    withSpark(newSparkContext()) { sc =>
      // Create an RDD that involves multiple stages:
      val rdd =
        sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity).map(identity).groupBy(identity)
      // Run it twice; this will cause the second job to have two "phantom" stages that were
      // mentioned in its job start event but which were never actually executed:
      rdd.count()
      rdd.count()
      eventually(timeout(10 seconds), interval(50 milliseconds)) {
        goToUi(sc, "/jobs")
        findAll(cssSelector("tbody tr a")).foreach { link =>
          link.text.toLowerCase should include ("count")
          link.text.toLowerCase should not include "unknown"
        }
      }
    }
  }

  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)) {
        goToUi(sc, "")
        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
        goToUi(sc, "/foo")
        find(cssSelector("b")).get.text should include ("html magic")
      }
      sparkUI.detachTab(newTab)
      eventually(timeout(10 seconds), interval(50 milliseconds)) {
        goToUi(sc, "")
        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
        goToUi(sc, "/foo")
        find(cssSelector("b")) should be(None)
      }
    }
  }

  test("kill stage POST/GET response is correct") {
    def getResponseCode(url: URL, method: String): Int = {
      val connection = url.openConnection().asInstanceOf[HttpURLConnection]
      connection.setRequestMethod(method)
      connection.connect()
      val code = connection.getResponseCode()
      connection.disconnect()
      code
    }

    withSpark(newSparkContext(killEnabled = true)) { sc =>
      sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync()
      eventually(timeout(5 seconds), interval(50 milliseconds)) {
        val url = new URL(
          sc.ui.get.appUIAddress.stripSuffix("/") + "/stages/stage/kill/?id=0&terminate=true")
        // SPARK-6846: should be POST only but YARN AM doesn't proxy POST
        getResponseCode(url, "GET") should be (200)
        getResponseCode(url, "POST") should be (200)
      }
    }
  }

  test("stage & job retention") {
    val conf = new SparkConf()
      .setMaster("local")
      .setAppName("test")
      .set("spark.ui.enabled", "true")
      .set("spark.ui.port", "0")
      .set("spark.ui.retainedStages", "3")
      .set("spark.ui.retainedJobs", "2")
    val sc = new SparkContext(conf)
    assert(sc.ui.isDefined)

    withSpark(sc) { sc =>
      // run a few jobs & stages ...
      (0 until 5).foreach { idx =>
        // NOTE: if we reverse the order, things don't really behave nicely
        // we lose the stage for a job we keep, and then the job doesn't know
        // about its last stage
        sc.parallelize(idx to (idx + 3)).map(identity).groupBy(identity).map(identity)
          .groupBy(identity).count()
        sc.parallelize(idx to (idx + 3)).collect()
      }

      val expJobInfo = Seq(
        ("9", "collect"),
        ("8", "count")
      )

      eventually(timeout(1 second), interval(50 milliseconds)) {
        goToUi(sc, "/jobs")
        // The completed jobs table should have two rows. The first row will be the most recent job:
        find("completed-summary").get.text should be ("Completed Jobs: 10, only showing 2")
        find("completed").get.text should be ("Completed Jobs (10, only showing 2)")
        val rows = findAll(cssSelector("tbody tr")).toIndexedSeq.map{_.underlying}
        rows.size should be (expJobInfo.size)
        for {
          (row, idx) <- rows.zipWithIndex
          columns = row.findElements(By.tagName("td"))
          id = columns.get(0).getText()
          description = columns.get(1).getText()
        } {
          id should be (expJobInfo(idx)._1)
          description should include (expJobInfo(idx)._2)
        }
      }

      val jobsJson = getJson(sc.ui.get, "jobs")
      jobsJson.children.size should be (expJobInfo.size)
      for {
        (job @ JObject(_), idx) <- jobsJson.children.zipWithIndex
        id = (job \ "jobId").extract[String]
        name = (job \ "name").extract[String]
      } {
        withClue(s"idx = $idx; id = $id; name = ${name.substring(0, 20)}") {
          id should be (expJobInfo(idx)._1)
          name should include (expJobInfo(idx)._2)
        }
      }

      // what about when we query for a job that did exist, but has been cleared?
      goToUi(sc, "/jobs/job/?id=7")
      find("no-info").get.text should be ("No information to display for job 7")

      val badJob = HistoryServerSuite.getContentAndCode(apiUrl(sc.ui.get, "jobs/7"))
      badJob._1 should be (HttpServletResponse.SC_NOT_FOUND)
      badJob._2 should be (None)
      badJob._3 should be (Some("unknown job: 7"))

      val expStageInfo = Seq(
        ("19", "collect"),
        ("18", "count"),
        ("17", "groupBy")
      )

      eventually(timeout(1 second), interval(50 milliseconds)) {
        goToUi(sc, "/stages")
        find("completed-summary").get.text should be ("Completed Stages: 20, only showing 3")
        find("completed").get.text should be ("Completed Stages (20, only showing 3)")
        val rows = findAll(cssSelector("tbody tr")).toIndexedSeq.map{_.underlying}
        rows.size should be (3)
        for {
          (row, idx) <- rows.zipWithIndex
          columns = row.findElements(By.tagName("td"))
          id = columns.get(0).getText()
          description = columns.get(1).getText()
        } {
          id should be (expStageInfo(idx)._1)
          description should include (expStageInfo(idx)._2)
        }
      }

      val stagesJson = getJson(sc.ui.get, "stages")
      stagesJson.children.size should be (3)
      for {
        (stage @ JObject(_), idx) <- stagesJson.children.zipWithIndex
        id = (stage \ "stageId").extract[String]
        name = (stage \ "name").extract[String]
      } {
        id should be (expStageInfo(idx)._1)
        name should include (expStageInfo(idx)._2)
      }

      // nonexistent stage

      goToUi(sc, "/stages/stage/?id=12&attempt=0")
      find("no-info").get.text should be ("No information to display for Stage 12 (Attempt 0)")
      val badStage = HistoryServerSuite.getContentAndCode(apiUrl(sc.ui.get, "stages/12/0"))
      badStage._1 should be (HttpServletResponse.SC_NOT_FOUND)
      badStage._2 should be (None)
      badStage._3 should be (Some("unknown stage: 12"))

      val badAttempt = HistoryServerSuite.getContentAndCode(apiUrl(sc.ui.get, "stages/19/15"))
      badAttempt._1 should be (HttpServletResponse.SC_NOT_FOUND)
      badAttempt._2 should be (None)
      badAttempt._3 should be (Some("unknown attempt for stage 19.  Found attempts: [0]"))

      val badStageAttemptList = HistoryServerSuite.getContentAndCode(
        apiUrl(sc.ui.get, "stages/12"))
      badStageAttemptList._1 should be (HttpServletResponse.SC_NOT_FOUND)
      badStageAttemptList._2 should be (None)
      badStageAttemptList._3 should be (Some("unknown stage: 12"))
    }
  }

  test("live UI json application list") {
    withSpark(newSparkContext()) { sc =>
      val appListRawJson = HistoryServerSuite.getUrl(new URL(
        sc.ui.get.appUIAddress + "/api/v1/applications"))
      val appListJsonAst = JsonMethods.parse(appListRawJson)
      appListJsonAst.children.length should be (1)
      val attempts = (appListJsonAst \ "attempts").children
      attempts.size should be (1)
      (attempts(0) \ "completed").extract[Boolean] should be (false)
      parseDate(attempts(0) \ "startTime") should be (sc.startTime)
      parseDate(attempts(0) \ "endTime") should be (-1)
      val oneAppJsonAst = getJson(sc.ui.get, "")
      oneAppJsonAst should be (appListJsonAst.children(0))
    }
  }

  test("job stages should have expected dotfile under DAG visualization") {
    withSpark(newSparkContext()) { sc =>
      // Create a multi-stage job
      val rdd =
        sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity).map(identity).groupBy(identity)
      rdd.count()

      val stage0 = Source.fromURL(sc.ui.get.appUIAddress +
        "/stages/stage/?id=0&attempt=0&expandDagViz=true").mkString
      assert(stage0.contains("digraph G {\n  subgraph clusterstage_0 {\n    " +
        "label=&quot;Stage 0&quot;;\n    subgraph "))
      assert(stage0.contains("{\n      label=&quot;parallelize&quot;;\n      " +
        "0 [label=&quot;ParallelCollectionRDD [0]"))
      assert(stage0.contains("{\n      label=&quot;map&quot;;\n      " +
        "1 [label=&quot;MapPartitionsRDD [1]"))
      assert(stage0.contains("{\n      label=&quot;groupBy&quot;;\n      " +
        "2 [label=&quot;MapPartitionsRDD [2]"))

      val stage1 = Source.fromURL(sc.ui.get.appUIAddress +
        "/stages/stage/?id=1&attempt=0&expandDagViz=true").mkString
      assert(stage1.contains("digraph G {\n  subgraph clusterstage_1 {\n    " +
        "label=&quot;Stage 1&quot;;\n    subgraph "))
      assert(stage1.contains("{\n      label=&quot;groupBy&quot;;\n      " +
        "3 [label=&quot;ShuffledRDD [3]"))
      assert(stage1.contains("{\n      label=&quot;map&quot;;\n      " +
        "4 [label=&quot;MapPartitionsRDD [4]"))
      assert(stage1.contains("{\n      label=&quot;groupBy&quot;;\n      " +
        "5 [label=&quot;MapPartitionsRDD [5]"))

      val stage2 = Source.fromURL(sc.ui.get.appUIAddress +
        "/stages/stage/?id=2&attempt=0&expandDagViz=true").mkString
      assert(stage2.contains("digraph G {\n  subgraph clusterstage_2 {\n    " +
        "label=&quot;Stage 2&quot;;\n    subgraph "))
      assert(stage2.contains("{\n      label=&quot;groupBy&quot;;\n      " +
        "6 [label=&quot;ShuffledRDD [6]"))
    }
  }

  def goToUi(sc: SparkContext, path: String): Unit = {
    goToUi(sc.ui.get, path)
  }

  def goToUi(ui: SparkUI, path: String): Unit = {
    go to (ui.appUIAddress.stripSuffix("/") + path)
  }

  def parseDate(json: JValue): Long = {
    JacksonMessageWriter.makeISODateFormat.parse(json.extract[String]).getTime
  }

  def getJson(ui: SparkUI, path: String): JValue = {
    JsonMethods.parse(HistoryServerSuite.getUrl(apiUrl(ui, path)))
  }

  def apiUrl(ui: SparkUI, path: String): URL = {
    new URL(ui.appUIAddress + "/api/v1/applications/" + ui.sc.get.applicationId + "/" + path)
  }
}