Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-6862][Streaming][WebUI] Add BatchPage to display details of a batch #5473

Closed
wants to merge 15 commits into from
Closed

Conversation

zsxwing
Copy link
Member

@zsxwing zsxwing commented Apr 11, 2015

This is an initial commit for SPARK-6862. Once SPARK-6796 is merged, I will add the links to StreamingPage so that the user can jump to BatchPage.

Screenshots:
success
failure

@zsxwing
Copy link
Member Author

zsxwing commented Apr 11, 2015

My test codes:

import org.apache.spark._
import org.apache.spark.streaming._

object StreamingApp {

  def main(args: Array[String]): Unit = {
    val conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount").set("spark.streaming.concurrentJobs", "3")
    val ssc = new StreamingContext(conf, Seconds(10))
    val lines = ssc.socketTextStream("localhost", 9999)
    val words = lines.flatMap(_.split(" "))
    val pairs = words.map(word => {
      Thread.sleep(1000)
      (word, 1)
    })
    val wordCounts = pairs.reduceByKey((x: Int, y: Int) => x + y, 2)
    wordCounts.foreachRDD {
      rdd => rdd.foreach { v =>
        println(v)
      }
    }
    wordCounts.foreachRDD { rdd =>
      rdd.foreach { v =>
        println(v)
      }
      rdd.foreach { v =>
        println(v)
        //throw new RuntimeException("Oops!")
      }
    }
    ssc.start()
    ssc.awaitTermination()
  }
}

@zsxwing
Copy link
Member Author

zsxwing commented Apr 11, 2015

cc @tdas

@SparkQA
Copy link

SparkQA commented Apr 11, 2015

Test build #30084 has finished for PR 5473 at commit 0c7b2eb.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class BatchPage(parent: StreamingTab) extends WebUIPage("batch")
  • This patch does not change any dependencies.

/**
* The number of recorders received by the receivers in this batch.
*/
def numRecords: Long = receivedBlockInfo.map { case (_, infos) =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 on exposing this.

@tdas
Copy link
Contributor

tdas commented Apr 13, 2015

Its looking pretty good! Here are some preliminary comments. I will do a more detailed pass tomorrow during the day.

  1. There should be tests in the StreamingListenerSuite that tests the local property setting.
  2. In the screenshots, is there any way to limit the max col size of the error? When there is error, it squishes all the other columns.

<th>Duration</th>
<th class="sorttable_nosort">Stages: Succeeded/Total</th>
<th class="sorttable_nosort">Tasks (for all stages): Succeeded/Total</th>
<th>Last Error</th>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is this Last Error and not just Error

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I was thinking Last Error of tasks. But here is Stage. I agree it should be Error. Fixed.

@tdas
Copy link
Contributor

tdas commented Apr 14, 2015

@zsxwing Any thoughts on this?

@zsxwing
Copy link
Member Author

zsxwing commented Apr 14, 2015

There should be tests in the StreamingListenerSuite that tests the local property setting.

As I explained in https://github.com/apache/spark/pull/5288#issuecomment-92579899, we may get wrong properties in SparkListenerJobStart.properties. Because the tests will be non-deterministic, I think it's better to add them after @JoshRosen fixes the issue.

In the screenshots, is there any way to limit the max col size of the error? When there is error, it squishes all the other columns.

Let me try some css style to fix it.

@tdas
Copy link
Contributor

tdas commented Apr 14, 2015

Why would we get wrong properties in this case? We are explicitly setting it in the thread that is launching the Spark job, so there is no question of inheriting local properties by child threads, etc. So that issue should not affect this case, isnt it? Or am I missing something?

@zsxwing
Copy link
Member Author

zsxwing commented Apr 14, 2015

Why would we get wrong properties in this case? We are explicitly setting it in the thread that is launching the Spark job, so there is no question of inheriting local properties by child threads, etc. So that issue should not affect this case, isnt it? Or am I missing something?

Without #5288, SparkListenerJobStart is created like SparkListenerJobStart(..., localProperties.get). So the properties in SparkListenerJobStart is the same object of localProperties.get, which is supposed to be limited in its thread.

And after job.run, we will clear some properties in localProperties.get, because SparkListenerJobStart.properties is the same object of localProperties.get, if job.run is very fast, we may not find the properties we need, because they have been cleared.

@zsxwing
Copy link
Member Author

zsxwing commented Apr 14, 2015

I set the max-width of the error column to 300px. Here is the new screenshot:

batch_page

@zsxwing
Copy link
Member Author

zsxwing commented Apr 14, 2015

Please ignore that Processing time and Total delay are -. It's an issue about JobScheduler. I will open a JIRA for it.

@zsxwing
Copy link
Member Author

zsxwing commented Apr 14, 2015

Please ignore that Processing time and Total delay are -. It's an issue about JobScheduler. I will open a JIRA for it.

After thinking it carefully, this may not be an issue, since StreamingContext should be stopped in such case. @tdas, do you think if we still need to show Processing time and Total delay in this case?

@SparkQA
Copy link

SparkQA commented Apr 14, 2015

Test build #30220 has finished for PR 5473 at commit 0b226f9.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class BatchPage(parent: StreamingTab) extends WebUIPage("batch")
  • This patch does not change any dependencies.

@SparkQA
Copy link

SparkQA commented Apr 14, 2015

Test build #30223 has finished for PR 5473 at commit 7168807.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class BatchPage(parent: StreamingTab) extends WebUIPage("batch")
  • This patch does not change any dependencies.

Conflicts:
	streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala
@zsxwing zsxwing changed the title [SPARK-6862][Streaming][WebUI][WIP] Add BatchPage to display details of a batch [SPARK-6862][Streaming][WebUI] Add BatchPage to display details of a batch Apr 15, 2015
@SparkQA
Copy link

SparkQA commented Apr 15, 2015

Test build #30330 has finished for PR 5473 at commit 15bdf9b.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class BatchPage(parent: StreamingTab) extends WebUIPage("batch")
  • This patch does not change any dependencies.

@zsxwing
Copy link
Member Author

zsxwing commented Apr 15, 2015

@tdas this is ready for code review.

Conflicts:
	streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
@SparkQA
Copy link

SparkQA commented Apr 20, 2015

Test build #30585 has finished for PR 5473 at commit 35ffd80.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class BatchPage(parent: StreamingTab) extends WebUIPage("batch")
  • This patch does not change any dependencies.

@zsxwing
Copy link
Member Author

zsxwing commented Apr 22, 2015

In addition, because of rowspan, the table cannot be sortable.

@tdas
Copy link
Contributor

tdas commented Apr 28, 2015

High level points as discussed offline.

We are planning to increase the number of retainedBatches, hence we need to be careful about all the data that we retain in memory. And its not a good idea to retain BatchInfo objects in memory because it contains a lot of arbitrary ReceivedBlockInfo objects, which can have arbitrary metadata ( #5732 ). So a better design would be to train only BatchUIData objects which contains only the data necessary for rendering the UI (timings, numrecords, etc.). So waitingBatches, runningBatches, completedBatches should have only BatchUIData objects.

@SparkQA
Copy link

SparkQA commented Apr 29, 2015

Test build #31237 has finished for PR 5473 at commit 087ba98.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
  • This patch does not change any dependencies.

private val runningBatchInfos = new HashMap[Time, BatchInfo]
private val completedBatchInfos = new Queue[BatchInfo]
private val batchInfoLimit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100)
private val waitingBatchUIDatas = new HashMap[Time, BatchUIData]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Datas sounds really weird. Just leave it as BatchUIData. Or if you want to follow other hashmaps naming styles timeToWaitingBatchUIData. Either is fine by me.

@zsxwing
Copy link
Member Author

zsxwing commented Apr 29, 2015

@tdas UISeleniumSuite is unstable because we may get wrong batchTime and outputOpId properties due to #5288.

@SparkQA
Copy link

SparkQA commented Apr 29, 2015

Test build #31284 has finished for PR 5473 at commit b380cfb.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
  • This patch does not change any dependencies.

@SparkQA
Copy link

SparkQA commented Apr 29, 2015

Test build #31287 has finished for PR 5473 at commit 0727d35.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
  • This patch does not change any dependencies.

@tdas
Copy link
Contributor

tdas commented Apr 29, 2015

I have merged #5288. Let me take another look at this PR.

@tdas
Copy link
Contributor

tdas commented Apr 29, 2015

retest this please.

@tdas
Copy link
Contributor

tdas commented Apr 29, 2015

rest this please.

@SparkQA
Copy link

SparkQA commented Apr 29, 2015

Test build #31323 has finished for PR 5473 at commit 0727d35.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
  • This patch does not change any dependencies.

@tdas
Copy link
Contributor

tdas commented Apr 29, 2015

retest this please.

@SparkQA
Copy link

SparkQA commented Apr 29, 2015

Test build #733 has started for PR 5473 at commit 0727d35.

@zsxwing
Copy link
Member Author

zsxwing commented Apr 29, 2015

retest this please.

@SparkQA
Copy link

SparkQA commented Apr 30, 2015

Test build #31346 has finished for PR 5473 at commit 0727d35.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
  • This patch does not change any dependencies.

@tdas
Copy link
Contributor

tdas commented Apr 30, 2015

The test has passed. Merging this! Thanks @zsxwing !

@asfgit asfgit closed this in 1b7106b Apr 30, 2015
@zsxwing zsxwing deleted the SPARK-6862 branch April 30, 2015 03:34
jeanlyn pushed a commit to jeanlyn/spark that referenced this pull request May 28, 2015
…a batch

This is an initial commit for SPARK-6862. Once SPARK-6796 is merged, I will add the links to StreamingPage so that the user can jump to BatchPage.

Screenshots:
![success](https://cloud.githubusercontent.com/assets/1000778/7102439/bbe75406-e0b3-11e4-84fe-3e6de629a49a.png)
![failure](https://cloud.githubusercontent.com/assets/1000778/7102440/bc124454-e0b3-11e4-921a-c8b39d6b61bc.png)

Author: zsxwing <zsxwing@gmail.com>

Closes apache#5473 from zsxwing/SPARK-6862 and squashes the following commits:

0727d35 [zsxwing] Change BatchUIData to a case class
b380cfb [zsxwing] Add createJobStart to eliminate duplicate codes
9a3083d [zsxwing] Rename XxxDatas -> XxxData
087ba98 [zsxwing] Refactor BatchInfo to store only necessary fields
cb62e4f [zsxwing] Use Seq[(OutputOpId, SparkJobId)] to store the id relations
72f8e7e [zsxwing] Add unit tests for BatchPage
1282b10 [zsxwing] Handle some corner cases and add tests for StreamingJobProgressListener
77a69ae [zsxwing] Refactor codes as per TD's comments
35ffd80 [zsxwing] Merge branch 'master' into SPARK-6862
15bdf9b [zsxwing] Add batch links and unit tests
4bf66b6 [zsxwing] Merge branch 'master' into SPARK-6862
7168807 [zsxwing] Limit the max width of the error message and fix nits in the UI
0b226f9 [zsxwing] Change 'Last Error' to 'Error'
fc98a43 [zsxwing] Put clearing local properties to finally and remove redundant private[streaming]
0c7b2eb [zsxwing] Add BatchPage to display details of a batch
jeanlyn pushed a commit to jeanlyn/spark that referenced this pull request Jun 12, 2015
…a batch

This is an initial commit for SPARK-6862. Once SPARK-6796 is merged, I will add the links to StreamingPage so that the user can jump to BatchPage.

Screenshots:
![success](https://cloud.githubusercontent.com/assets/1000778/7102439/bbe75406-e0b3-11e4-84fe-3e6de629a49a.png)
![failure](https://cloud.githubusercontent.com/assets/1000778/7102440/bc124454-e0b3-11e4-921a-c8b39d6b61bc.png)

Author: zsxwing <zsxwing@gmail.com>

Closes apache#5473 from zsxwing/SPARK-6862 and squashes the following commits:

0727d35 [zsxwing] Change BatchUIData to a case class
b380cfb [zsxwing] Add createJobStart to eliminate duplicate codes
9a3083d [zsxwing] Rename XxxDatas -> XxxData
087ba98 [zsxwing] Refactor BatchInfo to store only necessary fields
cb62e4f [zsxwing] Use Seq[(OutputOpId, SparkJobId)] to store the id relations
72f8e7e [zsxwing] Add unit tests for BatchPage
1282b10 [zsxwing] Handle some corner cases and add tests for StreamingJobProgressListener
77a69ae [zsxwing] Refactor codes as per TD's comments
35ffd80 [zsxwing] Merge branch 'master' into SPARK-6862
15bdf9b [zsxwing] Add batch links and unit tests
4bf66b6 [zsxwing] Merge branch 'master' into SPARK-6862
7168807 [zsxwing] Limit the max width of the error message and fix nits in the UI
0b226f9 [zsxwing] Change 'Last Error' to 'Error'
fc98a43 [zsxwing] Put clearing local properties to finally and remove redundant private[streaming]
0c7b2eb [zsxwing] Add BatchPage to display details of a batch
nemccarthy pushed a commit to nemccarthy/spark that referenced this pull request Jun 19, 2015
…a batch

This is an initial commit for SPARK-6862. Once SPARK-6796 is merged, I will add the links to StreamingPage so that the user can jump to BatchPage.

Screenshots:
![success](https://cloud.githubusercontent.com/assets/1000778/7102439/bbe75406-e0b3-11e4-84fe-3e6de629a49a.png)
![failure](https://cloud.githubusercontent.com/assets/1000778/7102440/bc124454-e0b3-11e4-921a-c8b39d6b61bc.png)

Author: zsxwing <zsxwing@gmail.com>

Closes apache#5473 from zsxwing/SPARK-6862 and squashes the following commits:

0727d35 [zsxwing] Change BatchUIData to a case class
b380cfb [zsxwing] Add createJobStart to eliminate duplicate codes
9a3083d [zsxwing] Rename XxxDatas -> XxxData
087ba98 [zsxwing] Refactor BatchInfo to store only necessary fields
cb62e4f [zsxwing] Use Seq[(OutputOpId, SparkJobId)] to store the id relations
72f8e7e [zsxwing] Add unit tests for BatchPage
1282b10 [zsxwing] Handle some corner cases and add tests for StreamingJobProgressListener
77a69ae [zsxwing] Refactor codes as per TD's comments
35ffd80 [zsxwing] Merge branch 'master' into SPARK-6862
15bdf9b [zsxwing] Add batch links and unit tests
4bf66b6 [zsxwing] Merge branch 'master' into SPARK-6862
7168807 [zsxwing] Limit the max width of the error message and fix nits in the UI
0b226f9 [zsxwing] Change 'Last Error' to 'Error'
fc98a43 [zsxwing] Put clearing local properties to finally and remove redundant private[streaming]
0c7b2eb [zsxwing] Add BatchPage to display details of a batch
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants