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-29543][SS][UI]Structured Streaming Web UI #26201

Closed
wants to merge 45 commits into from

Conversation

uncleGen
Copy link
Contributor

@uncleGen uncleGen commented Oct 22, 2019

What changes were proposed in this pull request?

This PR adds two pages to Web UI for Structured Streaming:

  • "/streamingquery": Streaming Query Page, providing some aggregate information for running/completed streaming queries.
  • "/streamingquery/statistics": Streaming Query Statistics Page, providing detailed information for streaming query, including Input Rate, Process Rate, Input Rows, Batch Duration and Operation Duration

Screen Shot 2020-01-29 at 1 38 00 PM
Screen Shot 2020-01-29 at 1 39 16 PM

Why are the changes needed?

It helps users to better monitor Structured Streaming query.

Does this PR introduce any user-facing change?

No

How was this patch tested?

  • new added and existing UTs
  • manual test

@SparkQA
Copy link

SparkQA commented Oct 22, 2019

Test build #112437 has finished for PR 26201 at commit 9b33018.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class SQLTab(
  • class StreamingQueryPage(parent: SQLTab, store: Option[HashSet[(StreamingQuery, Long)]])
  • class StreamingQueryStatisticsPage(
  • class JsCollector
  • class GraphUIData(

@SparkQA
Copy link

SparkQA commented Oct 22, 2019

Test build #112440 has finished for PR 26201 at commit 1e349ac.

  • This patch fails MiMa tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@uncleGen
Copy link
Contributor Author

Currently, the streaming query page entry is in SQL tab. I am not sure whether we need to add a new StreamingTab as there is already a StreamingTab which is for old version DStream. But it don't matter, we can think about it in future.

@SparkQA
Copy link

SparkQA commented Oct 22, 2019

Test build #112444 has finished for PR 26201 at commit 5633431.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@uncleGen
Copy link
Contributor Author

retest this please.

@HeartSaVioR
Copy link
Contributor

Amazing work! I've seen multiple requests on having tab for structured streaming be equivalent to streaming tab, and happy to see this effort.

I've just skimmed through the code (don't know about front-end tech.) and have general comment:

  • It may be better to define a new event for SQL and pass through listener bus, so that it could be available to SHS as well. It seems to be the right approach even we just want to expose it in running application.
  • Is there any chance to reuse resources for streaming tab? It's basically close to streaming tab, but looks like it doesn't reuse anything.
  • At least we could add UTs for API layer.
  • It would be great if statistic page would have query information as well on top.

@uncleGen
Copy link
Contributor Author

uncleGen commented Oct 22, 2019

@HeartSaVioR Thanks for your feedback

It may be better to define a new event for SQL and pass through listener bus, so that it could be available to SHS as well. It seems to be the right approach even we just want to expose it in running application.

This is a great suggestion. I agree with you. But IMHO, we can do it in a separate PR after/before this.

Is there any chance to reuse resources for streaming tab? It's basically close to streaming tab, but looks like it doesn't reuse anything.

Basically, streaming query is also a sql query, so it is reasonable to place the entry in SQL Tab. But it does not mean I object to use a streaming tab. Besides, I am not sure it is a good choice to reuse existing streaming tab.

At least we could add UTs for API layer.

I will add some UTs

It would be great if statistic page would have query information as well on top.

Sure, I will add them.

@dongjoon-hyun
Copy link
Member

Thank you, @uncleGen . Nice UI. I'm very sorry that 3.0-preview misses this.
cc @gatorsmile .

@SparkQA
Copy link

SparkQA commented Oct 22, 2019

Test build #112446 has finished for PR 26201 at commit 5633431.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HeartSaVioR
Copy link
Contributor

It may be better to define a new event for SQL and pass through listener bus, so that it could be available to SHS as well. It seems to be the right approach even we just want to expose it in running application.

This is a great suggestion. I agree with you. But IMHO, we can do it in a separate PR after/before this.

Actually my point was, it seems to be a bit weird to make SQL event listener to be couple with other stuff. It's an event listener, and the best way to ingest the data into that listener is sending event. I'm not saying we should enable this in SHS right now, but it will be available smoothly if we follow the general way to ingest the data.

Is there any chance to reuse resources for streaming tab? It's basically close to streaming tab, but looks like it doesn't reuse anything.

Basically, streaming query is also a sql query, so it is reasonable to place the entry in SQL Tab. But it does not mean I object to use a streaming tab. Besides, I am not sure it is a good choice to reuse existing streaming tab.

I meant the graph between streaming tab and here would be quite similar, so curious it could be reused, maybe with refactoring if necessary.

Copy link
Contributor

@HeartSaVioR HeartSaVioR left a comment

Choose a reason for hiding this comment

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

Still in skimmed review.

@uncleGen
Copy link
Contributor Author

@HeartSaVioR

I meant the graph between streaming tab and here would be quite similar, so curious it could be reused, maybe with refactoring if necessary.

Get it. If so, we should do some refactoring work, since these two code path are located in two different packages. I do not mind to do this work in a follow-up PR. What is your opinion?

Actually my point was, it seems to be a bit weird to make SQL event listener to be couple with other stuff.

Sorry, I do not get what you mean. Could you please point out the codebase where you are referring

@uncleGen
Copy link
Contributor Author

Add streaming query information on top.
image

@HeartSaVioR
Copy link
Contributor

Uh, I meant the query information like name, query ID, etc, as statistic page doesn't show any information on query.

@HeartSaVioR
Copy link
Contributor

I meant the graph between streaming tab and here would be quite similar, so curious it could be reused, maybe with refactoring if necessary.

Get it. If so, we should do some refactoring work, since these two code path are located in two different packages. I do not mind to do this work in a follow-up PR. What is your opinion?

I'm OK with it. Let's reconsider if someone claims again.

Actually my point was, it seems to be a bit weird to make SQL event listener to be couple with other stuff.

Sorry, I do not get what you mean. Could you please point out the codebase where you are referring

Sorry my bad. I was confused SQLTab with SQL listener. I'd still suggest looping through SQL listener and access through SQLAppStore, but that's OK as it is once we are OK with supporting it only for Spark UI (not SHS).

@gaborgsomogyi
Copy link
Contributor

I think it's a good addition. Started to catch up and testing...
My first suggestion is similar what @HeartSaVioR mentioned, I'm pretty sure we can add more tests since the coverage is not really high.

@SparkQA
Copy link

SparkQA commented Oct 22, 2019

Test build #112457 has finished for PR 26201 at commit cb2b44d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

@HeartSaVioR HeartSaVioR left a comment

Choose a reason for hiding this comment

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

Had a look a bit but stopped it as I wouldn't like to make myself continuously ask where the code comes from. I hope there're comments everywhere whenever it copied code from somewhere. It would save reviewers' time significantly.

Bunch of codes seem to be copied from streaming - while we review this, we might be better to find the way how to reuse them. It wouldn't be a blocker for this PR, but doing it in prior would even save times for reviewing, helps everyone.

@uncleGen
Copy link
Contributor Author

uncleGen commented Oct 23, 2019

I've just updated the code, including:

  • refactor to better reuse code and reduce duplicate code references
  • fix time zone bug
  • add query related information on top
  • update and add some UTs

Some updated UI:
image

@SparkQA
Copy link

SparkQA commented Jan 14, 2020

Test build #116682 has finished for PR 26201 at commit 1e35c0c.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 14, 2020

Test build #116681 has finished for PR 26201 at commit a0022a7.

  • This patch fails Spark unit tests.
  • This patch does not merge cleanly.
  • This patch adds no public classes.

@uncleGen
Copy link
Contributor Author

Today, I will give a commit to fix ut failure.

@SparkQA
Copy link

SparkQA commented Jan 15, 2020

Test build #116744 has finished for PR 26201 at commit dd5ca20.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 17, 2020

Test build #116899 has finished for PR 26201 at commit dd5ca20.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@uncleGen
Copy link
Contributor Author

uncleGen commented Jan 17, 2020

Kindly reminder to @sarutak @dongjoon-hyun and @zsxwing

Copy link
Member

@zsxwing zsxwing left a comment

Choose a reason for hiding this comment

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

The current approach looks good! Left some minor comments. Regarding the UI, it's better to make the following changes to make this page consistent with others:

  • Query Name -> Name.
  • Submit Time -> Submitted.
  • Last Batch Id -> Latest Batch.

import org.apache.spark.sql.streaming.ui.UIUtils._
import org.apache.spark.ui.{GraphUIData, JsCollector, UIUtils => SparkUIUtils, WebUIPage}

class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
Copy link
Member

Choose a reason for hiding this comment

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

nit: let's make this a package private class.

import org.apache.spark.sql.streaming.ui.UIUtils._
import org.apache.spark.ui.{UIUtils => SparkUIUtils, WebUIPage}

class StreamingQueryPage(parent: StreamingQueryTab)
Copy link
Member

Choose a reason for hiding this comment

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

nit: let's make this a package private class

* data to show.
*/
lazy val streamingQueryStatusListener: Option[StreamingQueryStatusListener] =
if (conf.get(UI_ENABLED)) {
Copy link
Member

Choose a reason for hiding this comment

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

Could you add one more flag spark.sql.streaming.ui.enabled for SS UI separately? Since we are adding one more streaming listener, it would be better if there is a flag to turn if off separately rather than turning off the whole Spark UI.

Copy link
Member

Choose a reason for hiding this comment

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

Copy that.

* UI data for both active and inactive query.
* TODO: Add support for history server.
*/
class StreamingQueryStatusListener(sqlConf: SQLConf) extends StreamingQueryListener {
Copy link
Member

Choose a reason for hiding this comment

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

nit: please make this package private.

import org.apache.spark.internal.Logging
import org.apache.spark.ui.{SparkUI, SparkUITab}

class StreamingQueryTab(val statusListener: StreamingQueryStatusListener, sparkUI: SparkUI)
Copy link
Member

Choose a reason for hiding this comment

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

ditto

Copy link
Member

Choose a reason for hiding this comment

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

We might need to keep this public cause it was called in SharedStatus, also StreamingQueryStatusListener.

Copy link
Member

Choose a reason for hiding this comment

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

I meant adding private[streaming] or private[sql].

Copy link
Member

Choose a reason for hiding this comment

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

Ah, thanks for clarifying, done in uncleGen#22

parent.addStaticHandler(StreamingQueryTab.STATIC_RESOURCE_DIR, "/static/sql")
}

object StreamingQueryTab {
Copy link
Member

Choose a reason for hiding this comment

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

ditto

queryStatus.updateProcess(event.progress, streamingProgressRetention)
}

override def onQueryTerminated(event: StreamingQueryListener.QueryTerminatedEvent): Unit = {
Copy link
Member

Choose a reason for hiding this comment

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

It's better to lock the whole method. There is a race condition that allQueryStatus may miss a query that's being removed from activeQueryStatus and added to inactiveQueryStatus. Since we are locking the whole method. It's better to use synchronized directly rather than inactiveQueryStatus.synchronized.

Copy link
Member

@xuanyuanking xuanyuanking Jan 22, 2020

Choose a reason for hiding this comment

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

Thanks, use synchronized directly in the next commit.

* @since 2.1.0
*/
@Evolving
class QueryStartedEvent private[sql](
val id: UUID,
val runId: UUID,
val name: String) extends Event
val name: String,
val submitTime: Long) extends Event
Copy link
Member

Choose a reason for hiding this comment

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

Use submissionTime to make it consistent with other places.

@uncleGen
Copy link
Contributor Author

uncleGen commented Jan 26, 2020

Thanks for reviewing @zsxwing and thanks for your help when i was busy with other things @xuanyuanking

@SparkQA
Copy link

SparkQA commented Jan 26, 2020

Test build #117413 has finished for PR 26201 at commit cb1e68c.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HeartSaVioR
Copy link
Contributor

Could you check the test failure on the build 117413? Looks to be related.

@SparkQA
Copy link

SparkQA commented Jan 27, 2020

Test build #117435 has finished for PR 26201 at commit 2d5f66c.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@zsxwing
Copy link
Member

zsxwing commented Jan 29, 2020

retest this please

@SparkQA
Copy link

SparkQA commented Jan 29, 2020

Test build #117516 has finished for PR 26201 at commit 2d5f66c.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@zsxwing
Copy link
Member

zsxwing commented Jan 29, 2020

LGTM. Thanks! Merging to master.

@zsxwing zsxwing changed the title [SPARK-29543][SS][UI] Init structured streaming ui [SPARK-29543][SS][UI]Structured Streaming Web UI Jan 29, 2020
@asfgit asfgit closed this in 7173786 Jan 29, 2020
@HeartSaVioR
Copy link
Contributor

Finally! Thanks again for the amazing effort to get this done. This has been missed one.

gengliangwang added a commit that referenced this pull request Mar 13, 2020
### What changes were proposed in this pull request?

There is a minor issue in #26201
In the streaming statistics page, there is such error
```
streaming-page.js:211 Uncaught TypeError: Cannot read property 'top' of undefined
at SVGCircleElement.<anonymous> (streaming-page.js:211)
at SVGCircleElement.__onclick (d3.min.js:1)
```
in the console after clicking the timeline graph.
![image](https://user-images.githubusercontent.com/1097932/76479745-14b26280-63ca-11ea-9079-0065321795f9.png)

This PR is to fix it.
### Why are the changes needed?

Fix the error of javascript execution.

### Does this PR introduce any user-facing change?

No, the error shows up in the console.

### How was this patch tested?

Manual test.

Closes #27883 from gengliangwang/fixSelector.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
gengliangwang added a commit that referenced this pull request Mar 13, 2020
### What changes were proposed in this pull request?

There is a minor issue in #26201
In the streaming statistics page, there is such error
```
streaming-page.js:211 Uncaught TypeError: Cannot read property 'top' of undefined
at SVGCircleElement.<anonymous> (streaming-page.js:211)
at SVGCircleElement.__onclick (d3.min.js:1)
```
in the console after clicking the timeline graph.
![image](https://user-images.githubusercontent.com/1097932/76479745-14b26280-63ca-11ea-9079-0065321795f9.png)

This PR is to fix it.
### Why are the changes needed?

Fix the error of javascript execution.

### Does this PR introduce any user-facing change?

No, the error shows up in the console.

### How was this patch tested?

Manual test.

Closes #27883 from gengliangwang/fixSelector.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
(cherry picked from commit 0f46325)
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
sjincho pushed a commit to sjincho/spark that referenced this pull request Apr 15, 2020
### What changes were proposed in this pull request?

There is a minor issue in apache#26201
In the streaming statistics page, there is such error
```
streaming-page.js:211 Uncaught TypeError: Cannot read property 'top' of undefined
at SVGCircleElement.<anonymous> (streaming-page.js:211)
at SVGCircleElement.__onclick (d3.min.js:1)
```
in the console after clicking the timeline graph.
![image](https://user-images.githubusercontent.com/1097932/76479745-14b26280-63ca-11ea-9079-0065321795f9.png)

This PR is to fix it.
### Why are the changes needed?

Fix the error of javascript execution.

### Does this PR introduce any user-facing change?

No, the error shows up in the console.

### How was this patch tested?

Manual test.

Closes apache#27883 from gengliangwang/fixSelector.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet