-
Notifications
You must be signed in to change notification settings - Fork 4.3k
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 Dataset runner] Break linage of dataset to reduce Spark planning overhead in case of large query plans #25187
Conversation
Run Spark Runner Tpcds Tests |
3 similar comments
Run Spark Runner Tpcds Tests |
Run Spark Runner Tpcds Tests |
Run Spark Runner Tpcds Tests |
ecd030c
to
00cd938
Compare
R: @aromanenko-dev |
Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control |
@aromanenko-dev I'll be in vacations tonight, can you take care of this review ? |
@echauchot Sure, I'll take a look |
@aromanenko-dev kind ping :) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks, looks fine for me, only several notes, ptal
TRANSFORM_TRANSLATORS.put(Impulse.class, new ImpulseTranslatorBatch()); | ||
TRANSFORM_TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslatorBatch<>()); | ||
TRANSFORM_TRANSLATORS.put(Combine.Globally.class, new CombineGloballyTranslatorBatch<>()); | ||
TRANSFORM_TRANSLATORS.put(Impulse.class, new ImpulseTranslatorBatch(0)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
How all this and below complexity factors were estimated?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That's mostly just relative factors comparing what the translator adds to the DAG, no exact science needed here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Does it depend on a translated pipeline?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No, it doesn't ... this is only about a single PTransform (and it's translation) looked at in isolation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Then it could be just hardcoded as a constant value in every transform translator class and I don't see a reason to have a dedicated constructor for this.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That's the way i implemented it initially... I personally find it easier to have an overview of all values in a single place, but happy to revert.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ok, done @aromanenko-dev
...n/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java
Show resolved
Hide resolved
...n/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java
Show resolved
Hide resolved
...n/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java
Outdated
Show resolved
Hide resolved
...n/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java
Show resolved
Hide resolved
...n/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java
Outdated
Show resolved
Hide resolved
@aromanenko-dev pls have another look |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
Feel free to self-merge it once tests are green
Run Spark StructuredStreaming ValidatesRunner |
1 similar comment
Run Spark StructuredStreaming ValidatesRunner |
Thanks @aromanenko-dev , have to look into the vr tests, not sure what's going on :( |
🤯 wth, below the output of the following statements, somehow dataset.persist(storageLevel);
System.out.println("\nContent of persisted dataset (1st eval)");
dataset.foreach(printValue);
System.out.println("\nContent of persisted dataset (2nd eval)");
dataset.foreach(printValue);
System.out.println("\nContent of rdd (1st eval)");
dataset.rdd().foreach(printValue);
System.out.println("\nContent of rdd (2nd eval)");
dataset.rdd().foreach(printValue);
System.out.println("\nContent of persisted rdd (1st eval)");
dataset.rdd().persist().foreach(printValue);
System.out.println("\nContent of persisted rdd (2nd eval)");
dataset.rdd().persist().foreach(printValue);
|
@mosche Are you sure that it's caused by your changes? |
@aromanenko-dev It's triggered by this optional part https://github.com/apache/beam/pull/25187/files#diff-4df56f442668d45bf7269e0bc379e95298b178c2f3072c72f30c1c0c296caed9R301-R304 I could simply remove that and keep caching as dataset. But there's also other places where caching is done on the RDD rather than the dataset if storage level is MEMORY_ONLY. It's a bit concerning to now know why / how this bug is triggered :( |
Tracked this down to #25296, will fix the bug first and then get back to this PR. |
…ng overhead in case of large complex query plans (relates to apache#24710 and apache#23845)
3116d46
to
39f241a
Compare
Run Spark StructuredStreaming ValidatesRunner |
Run Java PreCommit |
Run Spark Runner Tpcds Tests |
1 similar comment
Run Spark Runner Tpcds Tests |
Hi @mosche and @aromanenko-dev Either this PR or #25297 likely caused SparkStructuredStreaming Batch Load test failing since Feb 2: https://ci-beam.apache.org/view/LoadTests/job/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch/1016/ error message is java.lang.StackOverflowError. It is an infinite loop of recursion. are there actions taken? Update: excluded #25297 because reverting that PR the StackOverflowError is still seen: https://ci-beam.apache.org/view/LoadTests/job/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch_PR/5/ |
Thanks @Abacn , having a look |
fixed here #25732 |
#24711 removed some unnecessary caching when processing MultiParDos with additional but unconsumed outputs. With storage level "Memory Only" caching was done via RDDs. This also had a positive side effect as it breaks linage of the dataset. This is particularly beneficial with complex query plans as generated by TCPDS query 83 (due to #24314).
With caching removed, performance dropped for query 83.
See http://metrics.beam.apache.org/d/tkqc0AdGk2/tpc-ds-spark-classic-new-sql?orgId=1&viewPanel=38&from=1670799600000&to=1673305199000
This change tracks a rough complexity estimate of the datasets and breaks linage where necessary by converting it to an RDD and back again.
Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
addresses #123
), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, commentfixes #<ISSUE NUMBER>
instead.CHANGES.md
with noteworthy changes.See the Contributor Guide for more tips on how to make review process smoother.
To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md
GitHub Actions Tests Status (on master branch)
See CI.md for more information about GitHub Actions CI.