Skip to content

[SPARK-26193][SQL] Implement shuffle write metrics in SQL #23207

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

Closed
wants to merge 11 commits into from

Conversation

xuanyuanking
Copy link
Member

@xuanyuanking xuanyuanking commented Dec 3, 2018

What changes were proposed in this pull request?

  1. Implement SQLShuffleWriteMetricsReporter on the SQL side as the customized ShuffleWriteMetricsReporter.
  2. Add shuffle write metrics to ShuffleExchangeExec, and use these metrics to create corresponding SQLShuffleWriteMetricsReporter in shuffle dependency.
  3. Rework on ShuffleMapTask to add new class named ShuffleWriteProcessor which control shuffle write process, we use sql shuffle write metrics by customizing a ShuffleWriteProcessor on SQL side.

How was this patch tested?

Add UT in SQLMetricsSuite.
Manually test locally, update screen shot to document attached in JIRA.

@xuanyuanking
Copy link
Member Author

test this please.

@SparkQA
Copy link

SparkQA commented Dec 3, 2018

Test build #99616 has finished for PR 23207 at commit 7c8e516.

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

@SparkQA
Copy link

SparkQA commented Dec 3, 2018

Test build #99617 has finished for PR 23207 at commit 7c8e516.

  • This patch fails from timeout after a configured wait of 400m.
  • This patch merges cleanly.
  • This patch adds no public classes.

@xuanyuanking
Copy link
Member Author

retest this please.

@SparkQA
Copy link

SparkQA commented Dec 4, 2018

Test build #99643 has finished for PR 23207 at commit 7c8e516.

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

@xuanyuanking
Copy link
Member Author

cc @cloud-fan @gatorsmile @rxin

@cloud-fan
Copy link
Contributor

Can you share some ideas about it? IMO shuffle write metrics is hard, as an RDD can have shuffle dependencies with multiple upstream RDDs. That said, in general the shuffle write metrics should belong to the upstream RDDs.

In Spark SQL, it's a little simpler, as the ShuffledRowRDD always have only one child, so it's reasonable to say that shuffle write metrics belong to ShuffledRowRDD.

That said, we need to design a not-so-general shuffle write metrics API in Spark core, which will only be used in Spark SQL.

@xuanyuanking
Copy link
Member Author

Thanks for your reply Wenchen, there's a sketch doc assigned in JIRA:https://docs.google.com/document/d/1DX0gLkpk_NCE5MwI1_m4gnA2rLdjDkynZ02u2VWDR-8/edit

IMO shuffle write metrics is hard, as an RDD can have shuffle dependencies with multiple upstream RDDs. That said, in general the shuffle write metrics should belong to the upstream RDDs.

That's right and that's also what I try to do at first, logically upstream operator trigger shuffle write, and first attempt implementation is also changed SparkPlan base class to achieve this.

In Spark SQL, it's a little simpler, as the ShuffledRowRDD always have only one child, so it's reasonable to say that shuffle write metrics belong to ShuffledRowRDD.

Yes, maybe this also the suggestion by Reynold, ShuffleExchangeExec has only one child, we can do a simplify on the implementation. But as the shuffle write metrics are updated by task inner, so the core module still need some changes.

@SparkQA
Copy link

SparkQA commented Dec 4, 2018

Test build #99677 has finished for PR 23207 at commit fcd62b3.

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

@SparkQA
Copy link

SparkQA commented Dec 4, 2018

Test build #99676 has finished for PR 23207 at commit ca6c407.

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

@SparkQA
Copy link

SparkQA commented Dec 5, 2018

Test build #99722 has finished for PR 23207 at commit cf35b9f.

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

@SparkQA
Copy link

SparkQA commented Dec 5, 2018

Test build #99736 has finished for PR 23207 at commit 76d1ca0.

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

@rxin
Copy link
Contributor

rxin commented Dec 6, 2018

@xuanyuanking can you separate the prs to rename read side metric and the write side change?

@xuanyuanking
Copy link
Member Author

Can we put the above in a closure and pass it into shuffle dependency? Then in SQL we just put the above in SQL using custom metrics.

Yea, the commit of a780b70 achieve this by adding ShuffleWriteProcessor abstract.
And the read metrics rename reverted in 7d104eb, will do it and display change in another pr.

@SparkQA
Copy link

SparkQA commented Dec 6, 2018

Test build #99782 has finished for PR 23207 at commit d5ee249.

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

@cloud-fan
Copy link
Contributor

the code looks much cleaner now!

@xuanyuanking
Copy link
Member Author

the code looks much cleaner now!

Sorry for the original rush and code, I should and will pay more attention on coding clean and more discussion on optional implementation.

@SparkQA
Copy link

SparkQA commented Dec 7, 2018

Test build #99805 has finished for PR 23207 at commit 6378a3d.

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

@xuanyuanking
Copy link
Member Author

retest this please.

Copy link
Contributor

@cloud-fan cloud-fan left a comment

Choose a reason for hiding this comment

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

LGTM except a few comments. cc @rxin to take another look

@SparkQA
Copy link

SparkQA commented Dec 7, 2018

Test build #99817 has finished for PR 23207 at commit 6378a3d.

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

@SparkQA
Copy link

SparkQA commented Dec 7, 2018

Test build #99825 has finished for PR 23207 at commit bc2c4f1.

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

@cloud-fan
Copy link
Contributor

thanks, merging to master!

@asfgit asfgit closed this in 877f82c Dec 9, 2018
@xuanyuanking
Copy link
Member Author

Thanks all for your review.

asfgit pushed a commit that referenced this pull request Dec 12, 2018
…anges

## What changes were proposed in this pull request?
Follow up pr for #23207, include following changes:

- Rename `SQLShuffleMetricsReporter` to `SQLShuffleReadMetricsReporter` to make it match with write side naming.
- Display text changes for read side for naming consistent.
- Rename function in `ShuffleWriteProcessor`.
- Delete `private[spark]` in execution package.

## How was this patch tested?

Existing tests.

Closes #23286 from xuanyuanking/SPARK-26193-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
holdenk pushed a commit to holdenk/spark that referenced this pull request Jan 5, 2019
…anges

## What changes were proposed in this pull request?
Follow up pr for apache#23207, include following changes:

- Rename `SQLShuffleMetricsReporter` to `SQLShuffleReadMetricsReporter` to make it match with write side naming.
- Display text changes for read side for naming consistent.
- Rename function in `ShuffleWriteProcessor`.
- Delete `private[spark]` in execution package.

## How was this patch tested?

Existing tests.

Closes apache#23286 from xuanyuanking/SPARK-26193-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
jackylee-ch pushed a commit to jackylee-ch/spark that referenced this pull request Feb 18, 2019
## What changes were proposed in this pull request?

1. Implement `SQLShuffleWriteMetricsReporter` on the SQL side as the customized `ShuffleWriteMetricsReporter`.
2. Add shuffle write metrics to `ShuffleExchangeExec`, and use these metrics to create corresponding `SQLShuffleWriteMetricsReporter` in shuffle dependency.
3. Rework on `ShuffleMapTask` to add new class named `ShuffleWriteProcessor` which control shuffle write process, we use sql shuffle write metrics by customizing a ShuffleWriteProcessor on SQL side.

## How was this patch tested?
Add UT in SQLMetricsSuite.
Manually test locally, update screen shot to document attached in JIRA.

Closes apache#23207 from xuanyuanking/SPARK-26193.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
jackylee-ch pushed a commit to jackylee-ch/spark that referenced this pull request Feb 18, 2019
…anges

## What changes were proposed in this pull request?
Follow up pr for apache#23207, include following changes:

- Rename `SQLShuffleMetricsReporter` to `SQLShuffleReadMetricsReporter` to make it match with write side naming.
- Display text changes for read side for naming consistent.
- Rename function in `ShuffleWriteProcessor`.
- Delete `private[spark]` in execution package.

## How was this patch tested?

Existing tests.

Closes apache#23286 from xuanyuanking/SPARK-26193-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.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