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-4598][WebUI]Task table pagination for the Stage page #7399

Closed
wants to merge 13 commits into from
Closed

[SPARK-4598][WebUI]Task table pagination for the Stage page #7399

wants to merge 13 commits into from

Conversation

zsxwing
Copy link
Member

@zsxwing zsxwing commented Jul 14, 2015

This PR adds pagination for the task table to solve the scalability issue of the stage page. Here is the initial screenshot:
pagination
The task table only shows 100 tasks. There is a page navigation above the table. Users can click the page navigation or type the page number to jump to another page. The table can be sorted by clicking the headers. However, unlike previous implementation, the sorting work is done in the server now. So clicking a table column to sort needs to refresh the web page.

@zsxwing
Copy link
Member Author

zsxwing commented Jul 14, 2015

I will clean this PR soon. @pwendell any thoughts about the screenshot?

@zsxwing zsxwing changed the title [SPARK-9037][WebUI][WIP]Task table pagination for the Stage page [SPARK-4598][WebUI][WIP]Task table pagination for the Stage page Jul 14, 2015
@SparkQA
Copy link

SparkQA commented Jul 14, 2015

Test build #37239 has finished for PR 7399 at commit db6c859.

  • This patch passes all tests.
  • This patch does not merge cleanly.
  • This patch adds no public classes.

@andrewor14
Copy link
Contributor

Yes!!

@XuTingjun
Copy link
Contributor

Can the table sort globally by any field?

@zsxwing
Copy link
Member Author

zsxwing commented Jul 15, 2015

Can the table sort globally by any field?

Yes. It will sort the data by the column and slice the corresponding data for a page in the server.

@zsxwing zsxwing changed the title [SPARK-4598][WebUI][WIP]Task table pagination for the Stage page [SPARK-4598][WebUI]Task table pagination for the Stage page Jul 15, 2015
@zsxwing
Copy link
Member Author

zsxwing commented Jul 15, 2015

Here is the rule for page navigation.

  • If the totalPages is 1, the page navigation will be empty
  • If the totalPages is more than 1, it will create a page navigation including a group of page numbers (10 pages) and a form to submit the page number.

Here are some examples of the page navigation:

<< < 11 12 13* 14 15 16 17 18 19 20 > >>

This is the first group, so "<<" is hidden.
< 1 2* 3 4 5 6 7 8 9 10 > >>

This is the first group and the first page, so "<<" and "<" are hidden.
1* 2 3 4 5 6 7 8 9 10 > >>

Assume totalPages is 19. This is the last group, so ">>" is hidden.
<< < 11 12 13* 14 15 16 17 18 19 >

Assume totalPages is 19. This is the last group and the last page, so ">>" and ">" are hidden.
<< < 11 12 13 14 15 16 17 18 19*

* means the current page number
<< means jumping to the first page of the previous group.
< means jumping to the previous page.
>> means jumping to the first page of the next group.
> means jumping to the next page.

And a screenshot for 100k tasks:

pagination-100k

It's about 1.6 seconds to return the content from the server.

@@ -231,52 +241,25 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
accumulableRow,
accumulables.values.toSeq)

val taskHeadersAndCssClasses: Seq[(String, String)] =
Copy link
Member Author

Choose a reason for hiding this comment

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

taskHeadersAndCssClasses is moved to TaskPagedTable.headers

@SparkQA
Copy link

SparkQA commented Jul 15, 2015

Test build #37366 has finished for PR 7399 at commit d9285f0.

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

@carsonwang
Copy link
Contributor

The REST API provides all the task information in JSON format. It seems it is easier to enable pagination on client side by consuming the JSON data. I was actually doing something like this. Is there any reason not consuming the JSON data on the UI?

@zsxwing
Copy link
Member Author

zsxwing commented Jul 16, 2015

The REST API provides all the task information in JSON format. It seems it is easier to enable pagination on client side by consuming the JSON data. I was actually doing something like this. Is there any reason not consuming the JSON data on the UI?

I chose this way because the current Spark UI doesn't use the REST API. In addition, we cannot write unit tests for JavaScript now, so it's better to use Scala.

stageData.hasBytesSpilled,
currentTime,
page = taskPage,
pageSize = 100, // Show 100 tasks at most in the table
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we make this something higher? Maybe 200?

Copy link
Contributor

Choose a reason for hiding this comment

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

Actually on second though I prefer 100.

@pwendell
Copy link
Contributor

I took a look at the UI and I like it. I did have two thoughts though:

  1. Can we make it so the headers, when hovering, show the link cursor? Otherwise it is not clear they are clickable.
  2. Can we also make the page links show the link cursor when hovering?
  3. To the left of the page selector, can you have the word "Page:" otherwise it looks like those are task numbers, since the header says "Tasks".

* < 1 2* 3 4 5 6 7 8 9 10 > >>
*
* This is the first group and the first page, so "<<" and "<" are hidden.
* 1 2* 3 4 5 6 7 8 9 10 > >>
Copy link
Contributor

Choose a reason for hiding this comment

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

Is this 1* 2 3 4 ?

@carsonwang
Copy link
Contributor

@zsxwing , I just left some minor comments.
Another problem: If a user selects to show additional metrics and sort by one metric like SchedulerDelay, the server will return a page without showing the additional metrics. We may need remember what additional metrics the user selects, and show these metrics when returning the page.

@zsxwing
Copy link
Member Author

zsxwing commented Jul 18, 2015

By the way, one more thing is that when I put in Page 111 and press Go when there are only 100 pages, it gives me the following error. We could have a more user-friendly error. This is minor and does not have to be fixed in this patch.

Maybe we should add an error page for the Spark UI? Now this is the default Jetty error page.

…f expand-dag-viz-arrow-true and expand-dag-viz-arrow-false
@andrewor14
Copy link
Contributor

Maybe we should add an error page for the Spark UI? Now this is the default Jetty error page.

I was thinking just displaying the normal stage page, but replace the task table with an informative message, something like "Page 111 is out of range. Please select a page number between 1 and 100." We probably don't need to add an error page.

If that's a lot of work feel free to file a separate JIRA and do it later.

@zsxwing
Copy link
Member Author

zsxwing commented Jul 19, 2015

@andrewor14 the latest commits addressed your comments. Here is the new screenshot:

pagination-100k-200

@SparkQA
Copy link

SparkQA commented Jul 19, 2015

Test build #37774 has finished for PR 7399 at commit c2f7f39.

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

@andrewor14
Copy link
Contributor

Hi @zsxwing thanks for implementing the changes. I tried this out locally and it looks like there are a few usability issues with the new features:

  • When I put in page 111 / 100, I see the following, which is good. However, all the buttons are now missing, and I can't "select a page number between 1 and 100" as the error message suggested me to do.
    screen shot 2015-07-19 at 12 28 37 pm
  • Also, if you look at the screenshot, there is no space between the error message and the bottom of the page. It would be good to add some padding there. This is a relatively minor UI issue.
  • When I tried to increase the number of tasks shown per page, it gives me the above error message. For instance, I was previously on page 99 / 100, and then I increased the number of tasks displayed from 100 to 200, then I got an error telling me to select a number between 1 and 50. I think we should just clear the "current page number" variable when we change the number of tasks to display.

@zsxwing
Copy link
Member Author

zsxwing commented Jul 20, 2015

@andrewor14 Here is the new screenshot for the error message:

screen shot 2015-07-20 at 2 36 48 pm

And if the user updates the page size, the page number will be reset to 1.

@zsxwing
Copy link
Member Author

zsxwing commented Jul 20, 2015

retest this please

@SparkQA
Copy link

SparkQA commented Jul 20, 2015

Test build #35 has finished for PR 7399 at commit 144f513.

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

@SparkQA
Copy link

SparkQA commented Jul 20, 2015

Test build #37816 has finished for PR 7399 at commit 144f513.

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

@andrewor14
Copy link
Contributor

Looks great!! I'll try this out later today again.

@andrewor14
Copy link
Contributor

LGTM, I'm merging this into master. Thanks @zsxwing.

@asfgit asfgit closed this in 4f7f1ee Jul 21, 2015
@zsxwing zsxwing deleted the task-table-pagination branch July 26, 2015 08:38
mingyukim pushed a commit to palantir/spark that referenced this pull request Sep 22, 2015
This PR adds pagination for the task table to solve the scalability issue of the stage page. Here is the initial screenshot:
<img width="1347" alt="pagination" src="https://cloud.githubusercontent.com/assets/1000778/8679669/9e63863c-2a8e-11e5-94e4-994febcd6717.png">
The task table only shows 100 tasks. There is a page navigation above the table. Users can click the page navigation or type the page number to jump to another page. The table can be sorted by clicking the headers. However, unlike previous implementation, the sorting work is done in the server now. So clicking a table column to sort needs to refresh the web page.

Author: zsxwing <zsxwing@gmail.com>

Closes apache#7399 from zsxwing/task-table-pagination and squashes the following commits:

144f513 [zsxwing] Display the page navigation when the page number is out of range
a3eee22 [zsxwing] Add extra space for the error message
54c5b84 [zsxwing] Reset page to 1 if the user changes the page size
c2f7f39 [zsxwing] Add a text field to let users fill the page size
bad52eb [zsxwing] Display user-friendly error messages
410586b [zsxwing] Scroll down to the tasks table if the url contains any sort column
a0746d1 [zsxwing] Use expand-dag-viz-arrow-job and expand-dag-viz-arrow-stage instead of expand-dag-viz-arrow-true and expand-dag-viz-arrow-false
b123f67 [zsxwing] Use localStorage to remember the user's actions and replay them when loading the page
894a342 [zsxwing] Show the link cursor when hovering for headers and page links and other minor fix
4d4fecf [zsxwing] Address Carson's comments
d9285f0 [zsxwing] Add comments and fix the style
74285fa [zsxwing] Merge branch 'master' into task-table-pagination
db6c859 [zsxwing] Task table pagination for the Stage page
sarutak pushed a commit that referenced this pull request May 23, 2020
### What changes were proposed in this pull request?
Add Pagination Support for structured streaming page. Now both tables `Active Queries` and `Completed Queries` will have pagination.
To implement pagination, pagination framework from #7399  is used.
* Also tables will only be shown if there is at least one entry in the table.

### Why are the changes needed?
* This will help users in analysing their structured streaming queries in much better way.
* Other Web UI pages support pagination in their table. So this will make web UI more consistent across pages.
* This can prevent potential OOM errors.

### Does this PR introduce _any_ user-facing change?
Yes. Both tables will support pagination.

### How was this patch tested?
Manually. I will add snapshots soon.

Closes #28485 from iRakson/SPARK-31642.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
sarutak pushed a commit that referenced this pull request Jun 7, 2020
### What changes were proposed in this pull request?
* Pagination Support is added to all tables of streaming page in spark web UI.
For adding pagination support, existing classes from #7399 were used.
* Earlier streaming page has two tables `Active Batches` and `Completed Batches`. Now, we will have three tables `Running Batches`, `Waiting Batches` and `Completed Batches`. If we have large number of waiting and running batches then keeping track in a single table is difficult. Also other pages have different table for different type type of data.
* Earlier empty tables were shown. Now only non-empty tables will be shown.
`Active Batches` table used to show details of waiting batches followed by running batches.

### Why are the changes needed?
Pagination will allow users to analyse the table in much better way. All spark web UI pages support pagination apart from streaming pages, so this will add consistency as well. Also it might fix the potential OOM errors that can arise.

### Does this PR introduce _any_ user-facing change?
Yes. `Active Batches` table is split into two tables `Running Batches` and `Waiting Batches`. Pagination Support is added to the all the tables. Every other functionality is unchanged.

### How was this patch tested?
Manually.

Before changes:
<img width="1667" alt="Screenshot 2020-05-03 at 7 07 14 PM" src="https://user-images.githubusercontent.com/15366835/80915680-8fb44b80-8d71-11ea-9957-c4a3769b8b67.png">

After Changes:
<img width="1669" alt="Screenshot 2020-05-03 at 6 51 22 PM" src="https://user-images.githubusercontent.com/15366835/80915694-a9ee2980-8d71-11ea-8fc5-246413a4951d.png">

Closes #28439 from iRakson/streamingPagination.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
iRakson added a commit to iRakson/spark that referenced this pull request Jun 8, 2020
### What changes were proposed in this pull request?
* Pagination Support is added to all tables of streaming page in spark web UI.
For adding pagination support, existing classes from apache#7399 were used.
* Earlier streaming page has two tables `Active Batches` and `Completed Batches`. Now, we will have three tables `Running Batches`, `Waiting Batches` and `Completed Batches`. If we have large number of waiting and running batches then keeping track in a single table is difficult. Also other pages have different table for different type type of data.
* Earlier empty tables were shown. Now only non-empty tables will be shown.
`Active Batches` table used to show details of waiting batches followed by running batches.

### Why are the changes needed?
Pagination will allow users to analyse the table in much better way. All spark web UI pages support pagination apart from streaming pages, so this will add consistency as well. Also it might fix the potential OOM errors that can arise.

### Does this PR introduce _any_ user-facing change?
Yes. `Active Batches` table is split into two tables `Running Batches` and `Waiting Batches`. Pagination Support is added to the all the tables. Every other functionality is unchanged.

### How was this patch tested?
Manually.

Before changes:
<img width="1667" alt="Screenshot 2020-05-03 at 7 07 14 PM" src="https://user-images.githubusercontent.com/15366835/80915680-8fb44b80-8d71-11ea-9957-c4a3769b8b67.png">

After Changes:
<img width="1669" alt="Screenshot 2020-05-03 at 6 51 22 PM" src="https://user-images.githubusercontent.com/15366835/80915694-a9ee2980-8d71-11ea-8fc5-246413a4951d.png">

Closes apache#28439 from iRakson/streamingPagination.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
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.

6 participants