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

Deduplicate identical environments in a pipeline. #22308

Merged
merged 3 commits into from
Jul 22, 2022

Conversation

robertwb
Copy link
Contributor

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:

  • Choose reviewer(s) and mention them in a comment (R: @username).
  • Mention the appropriate issue in your description (for example: 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, comment fixes #<ISSUE NUMBER> instead.
  • Update CHANGES.md with noteworthy changes.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

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)

Build python source distribution and wheels
Python tests
Java tests

See CI.md for more information about GitHub Actions CI.

@codecov
Copy link

codecov bot commented Jul 16, 2022

Codecov Report

Merging #22308 (508a991) into master (67e6726) will decrease coverage by 0.05%.
The diff coverage is 100.00%.

@@            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              
Flag Coverage Δ
python 83.54% <100.00%> (-0.10%) ⬇️

Flags with carried forward coverage won't be shown. Click here to find out more.

Impacted Files Coverage Δ
sdks/python/apache_beam/pipeline.py 92.84% <100.00%> (+0.41%) ⬆️
...ache_beam/runners/portability/local_job_service.py 82.42% <100.00%> (+0.14%) ⬆️
sdks/python/apache_beam/utils/interactive_utils.py 95.12% <0.00%> (-2.44%) ⬇️
sdks/python/apache_beam/io/source_test_utils.py 88.01% <0.00%> (-1.39%) ⬇️
...hon/apache_beam/runners/direct/test_stream_impl.py 93.28% <0.00%> (-0.75%) ⬇️
...eam/runners/portability/fn_api_runner/execution.py 92.44% <0.00%> (-0.65%) ⬇️
sdks/python/apache_beam/runners/direct/executor.py 96.46% <0.00%> (-0.55%) ⬇️
...dks/python/apache_beam/options/pipeline_options.py 94.34% <0.00%> (-0.42%) ⬇️
.../runners/portability/fn_api_runner/translations.py 92.65% <0.00%> (-0.34%) ⬇️
...eam/runners/interactive/interactive_environment.py 91.71% <0.00%> (-0.31%) ⬇️
... and 7 more

Help us with your feedback. Take ten seconds to tell us how you rate us.

@github-actions
Copy link
Contributor

Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer:

R: @ryanthompson591 for label python.

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

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.
@robertwb
Copy link
Contributor Author

R: @chamikaramj

@github-actions
Copy link
Contributor

Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control

Copy link
Contributor

@chamikaramj chamikaramj left a 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:
Copy link
Contributor

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.

Copy link
Contributor Author

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]
Copy link
Contributor

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:

Copy link
Contributor Author

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')]),
Copy link
Contributor

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 ?

Copy link
Contributor Author

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')]),
Copy link
Contributor

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.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

Copy link
Contributor

@chamikaramj chamikaramj left a comment

Choose a reason for hiding this comment

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

Thanks. LGTM.

@robertwb
Copy link
Contributor Author

Run Python PreCommit

@robertwb
Copy link
Contributor Author

Run Python_PVR_Flink PreCommit

@robertwb
Copy link
Contributor Author

Container build failed pulling licenses.

@robertwb
Copy link
Contributor Author

Run Python_PVR_Flink PreCommit

@robertwb robertwb merged commit df99d6a into apache:master Jul 22, 2022
grufino pushed a commit to grufino/beam that referenced this pull request Jul 30, 2022
This is especially useful when external transforms from the same environment
are used multiple times.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants