-
Notifications
You must be signed in to change notification settings - Fork 4.2k
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
Deduplicate identical environments in a pipeline. #22308
Conversation
Codecov Report
@@ Coverage Diff @@
## master #22308 +/- ##
==========================================
- Coverage 74.26% 74.20% -0.06%
==========================================
Files 702 703 +1
Lines 93019 93110 +91
==========================================
+ Hits 69076 69088 +12
- Misses 22676 22755 +79
Partials 1267 1267
Flags with carried forward coverage won't be shown. Click here to find out more.
Help us with your feedback. Take ten seconds to tell us how you rate us. |
Assigning reviewers. If you would like to opt out of this review, comment R: @ryanthompson591 for label python. Available commands:
The PR bot will only process comments in the main thread (not review comments). |
This is especially useful when external transforms from the same environment are used multiple times.
R: @chamikaramj |
Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control |
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.
files_by_hash = {} | ||
for env_id, env in proto.components.environments.items(): | ||
# First deduplicate any file dependencies by their hash. | ||
for dep in env.dependencies: |
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.
Probably we should sort dependencies before de-duping ?
Otherwise different environments with the same normalized set of files might end up picking different files based on the order.
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.
It's not really well defined whether dependencies are ordered (e.g. the order of jar files matters for class resolution) so I'm preserving the order here just in case.
dep.type_payload) | ||
if file_payload.sha256: | ||
if file_payload.sha256 in files_by_hash: | ||
file_payload.path = files_by_hash[file_payload.sha256] |
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.
FYI, we de-duplicate artifacts again before staging:
if type_payload.sha256 and type_payload.sha256 in staged_hashes: |
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.
Yeah. We could probably clean that up a lot once Runner v1 goes away.
}, | ||
environments={ | ||
'e1': beam_runner_api_pb2.Environment( | ||
dependencies=[file_artifact('a1', 'x', 'a')]), |
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.
Add a test for multiple-artifacts ?
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.
Done.
'e2': beam_runner_api_pb2.Environment( | ||
dependencies=[file_artifact('a2', 'x', 'a')]), | ||
'e3': beam_runner_api_pb2.Environment( | ||
dependencies=[file_artifact('a3', 'y', 'a')]), |
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.
Probably also should add a negative test for the case where artifacts are the same but other properties (for example, capabilities) are different.
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.
Done.
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. LGTM.
Run Python PreCommit |
Run Python_PVR_Flink PreCommit |
Container build failed pulling licenses. |
Run Python_PVR_Flink PreCommit |
This is especially useful when external transforms from the same environment are used multiple times.
This is especially useful when external transforms from the same environment
are used multiple times.
Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
R: @username
).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.