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

[BEAM-10549] Improve runtime type checking performance for the Python SDK #12352

Conversation

saavan-google
Copy link
Contributor

@saavan-google saavan-google commented Jul 23, 2020

This PR creates a new flag, performance_runtime_type_check which provides faster runtime type checking by sampling values.

Tests and documentation have been added to reflect this change.


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).
  • Format the pull request title like [BEAM-XXX] Fixes bug in ApproximateQuantiles, where you replace BEAM-XXX with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
  • 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.

Post-Commit Tests Status (on master branch)

Lang SDK Dataflow Flink Samza Spark Twister2
Go Build Status --- Build Status --- Build Status ---
Java Build Status Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status Build Status
Build Status
Build Status
Build Status
Python Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
--- Build Status ---
XLang Build Status --- Build Status --- Build Status ---

Pre-Commit Tests Status (on master branch)

--- Java Python Go Website
Non-portable Build Status Build Status
Build Status
Build Status Build Status
Portable --- Build Status --- ---

See .test-infra/jenkins/README for trigger phrase, status and link of all Jenkins jobs.

@saavan-google
Copy link
Contributor Author

Retest this please

1 similar comment
@saavan-google
Copy link
Contributor Author

Retest this please

@saavan-google
Copy link
Contributor Author

R: @udim
R: @robertwb

PTAL

Copy link
Contributor

@robertwb robertwb left a comment

Choose a reason for hiding this comment

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

This'll be a lot faster .

Most of the comments revolve around pushing the figuring-out-what-to-check logic earlier, so the code in opcounters just becomes a dumb checker. E.g. we should be able to construct the map {"Input to Full/Transform/Label": int, "Output of Another/Map(fn_name)": float} in the visitor and then pass that around. (This should be easier to unit test as well.)

sdks/python/apache_beam/runners/common.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/options/pipeline_options.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/runners/worker/bundle_processor.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/runners/worker/operations.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/typehints/typecheck.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/pipeline.py Show resolved Hide resolved
sdks/python/apache_beam/runners/worker/opcounters.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/runners/worker/opcounters.py Outdated Show resolved Hide resolved
Copy link
Contributor

@robertwb robertwb left a comment

Choose a reason for hiding this comment

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

As discussed in person, let's push the construction of a {source_str: TypeConstraint} all the way up to the visitor.

sdks/python/apache_beam/runners/worker/bundle_processor.py Outdated Show resolved Hide resolved
@saavan-google
Copy link
Contributor Author

As discussed in person, let's push the construction of a {source_str: TypeConstraint} all the way up to the visitor.

Just pushed these changes - PTAL

I kept get_perf_runtime_type_hints because it's merely responsible for unpickling the function to get the constructed type hints dict

sdks/python/apache_beam/runners/common.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/runners/worker/opcounters.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/runners/worker/opcounters.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/runners/worker/operations.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/typehints/typecheck.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/typehints/typecheck.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/typehints/typecheck.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/typehints/typecheck.py Outdated Show resolved Hide resolved
@robertwb
Copy link
Contributor

As discussed in person, let's push the construction of a {source_str: TypeConstraint} all the way up to the visitor.

Just pushed these changes - PTAL

I kept get_perf_runtime_type_hints because it's merely responsible for unpickling the function to get the constructed type hints dict

The problem with this function is that it's called on all sorts of operations, even those without type constraints, when it applies to only one kind of operation. This pattern (and the heavy use of hasattr) suggests it'd be better to make this a method on Operation, say with default of return None.

@saavan-google
Copy link
Contributor Author

The problem with this function is that it's called on all sorts of operations, even those without type constraints, when it applies to only one kind of operation. This pattern (and the heavy use of hasattr) suggests it'd be better to make this a method on Operation, say with default of return None.

That makes sense, just pushed fixes with these design changes.

PTAL @robertwb

Copy link
Contributor

@robertwb robertwb left a comment

Choose a reason for hiding this comment

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

Sorry I wasn't able to get to this earlier. This is looking really good, only minor comments at this point.

Also, could you run the microbenchmark before and after your change with all typechecking disabled and post the results to verify those timing remain unchanged?

sdks/python/apache_beam/examples/snippets/snippets.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/runners/worker/operations.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/transforms/ptransform.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/transforms/core.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/typehints/typecheck.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/typehints/typecheck.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/typehints/typecheck.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/typehints/typecheck.py Outdated Show resolved Hide resolved
@saavan-google
Copy link
Contributor Author

PTAL @robertwb

@udim
Copy link
Member

udim commented Aug 17, 2020

Run Python PreCommit

Copy link
Contributor

@robertwb robertwb left a comment

Choose a reason for hiding this comment

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

Just a couple of cosmetic fixes and pending the comparison of benchmarks to master and this'll be ready to merge.

sdks/python/apache_beam/examples/snippets/snippets.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/typehints/typecheck.py Outdated Show resolved Hide resolved
@saavan-google saavan-google force-pushed the BEAM-10549-improve-runtime-type-checking-performance branch from fd1f391 to 68b4ce8 Compare August 19, 2020 20:57
@saavan-google
Copy link
Contributor Author

run python precommit

1 similar comment
@saavan-google
Copy link
Contributor Author

run python precommit

@saavan-google
Copy link
Contributor Author

retest this please

@saavan-google
Copy link
Contributor Author

run python unit tests

Copy link
Contributor

@robertwb robertwb left a comment

Choose a reason for hiding this comment

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

Thanks! I've reviewed the benchmarks as well; they seem to be in parameters now.

@robertwb robertwb merged commit c9bed1e into apache:master Aug 21, 2020
TobKed pushed a commit to damgadbot/beam that referenced this pull request Aug 27, 2020
… SDK (apache#12352)

* Add performance runtime type check flag

* Add PerformanceTypeCheckVisitor which stores type hints in DoFns

* Store the type hints in the OperationCounters class

* Runtime type checking works with input values

* Runtime type checking works with output values

* Declare type_check function in cython and reduce error handling code

* Throw error if both flags are on

* Use producer and consumer Operations' type hints

* Store in _runtime_type_hints instead of _type_hints

* Simplify extraction process of type hints

* Rename variables

* Use correct type hints in broken tests

* Add equivalent tests for both runtime type checking flags without duplication

* Remove accidental changes

* Fixup: apply YAPF

* Remove redundant test

* Fix cython error

* Fix cython error

* Fix cython error

* Fix update_counters_finish

* Fix bugs, and py2 incompatability

* Fixup: apply YAPF

* Fix line-too-long error

* For testing purposes, default to performance_runtime_type_check when both flags are on

* Fix import order

* Fixup: apply YAPF

* Fix bug where CombineFns could get type hints

* Fix ConsumerSet init bug

* Ignore all combine classes, and fix iteration bug

* Add full labels to transforms

* Provide more descriptive error message

* Fix bug where non-iterables wouldn't be type-checked

* Store argspec

* Compute type hints in visitor instead of opcounters for perf gains, and to get the main input kwarg

* Fixup: apply YAPF

* Fixup: apply YAPF

* Fix import order

* Fix lint errors

* Use param name in error string

* Fixup: apply YAPF

* Fix lint

* Fix bug where inspect can fail

* Check for dict, str, unicode, bytes types in OutputProcessor

* Fixup: apply YAPF

* Make feature Python 3 only

* Update tests to be compatible with both flags

* Add tests

* Add more tests

* Fix lint errors

* Add more tests

* Leave performance_runtime_type_check off by default

* Fix lint

* Add website documentation

* Use the same value in tests because sampling is non-deterministic

* Update a test

* Remove accidental changes

* PR Fixes

* Stage 1 Refactor

* Refactor Stage 2

* Refactor Stage 3

* Refactor Stage 4

* Refactor Stage 5

* Fixup: apply YAPF

* Add comments for how to type-check side inputs, and lint fixes

* Use mega type hint

* PR Fixes: fix None bug, empty type hints, and move label transformation to visitor

* Fix some PR comments

* Add new method to PTransform: _add_type_constraint_from_consumer

* Get runtime type hints only from DoOperation instead of all operations

* Fix lint errors

* Fix lint and cython error

* Fix test that was written incorrectly

* Add microbenchmark

* Fix benchmark & lint errors

* Use averaging in benchmark

* Remove iterable check in common.py for performance concerns

* Add TODO for fixing error message strings

* Update documentation strings

* Update test and move annotations to a comment

* Make logic easier to read and document type hints for _add_type_constraint_from_consumer

* Remove ParDo prefix and fix bug

* Remove unnecessary try/except and optimize a selector statement

* Use type hints for combine functions and add a TODO for removing the isinstance(transform, ParDo) check in the visitor

* Extract type hints for all transform types, not just ParDo's

* Fix lint errors

* Fix type hint annotation docstrings

* Add test and fix type hint docstring

* Turn the perf RTC flag on for testing purposes

* Add try/catch for getfullargspec error

* Fix IndexError

* Turn flag off

* Use cleaner syntax

* Rebase onto master

* Fix test

* Remove unneeded code.

* Fix snippets bug

* Remove accidental changes

* Add TODO

* Remove expensive isinstance check

* Remove unused import

* Fix TODO

Co-authored-by: Robert Bradshaw <robertwb@gmail.com>
ibzib pushed a commit to ibzib/beam that referenced this pull request Sep 30, 2020
… SDK (apache#12352)

* Add performance runtime type check flag

* Add PerformanceTypeCheckVisitor which stores type hints in DoFns

* Store the type hints in the OperationCounters class

* Runtime type checking works with input values

* Runtime type checking works with output values

* Declare type_check function in cython and reduce error handling code

* Throw error if both flags are on

* Use producer and consumer Operations' type hints

* Store in _runtime_type_hints instead of _type_hints

* Simplify extraction process of type hints

* Rename variables

* Use correct type hints in broken tests

* Add equivalent tests for both runtime type checking flags without duplication

* Remove accidental changes

* Fixup: apply YAPF

* Remove redundant test

* Fix cython error

* Fix cython error

* Fix cython error

* Fix update_counters_finish

* Fix bugs, and py2 incompatability

* Fixup: apply YAPF

* Fix line-too-long error

* For testing purposes, default to performance_runtime_type_check when both flags are on

* Fix import order

* Fixup: apply YAPF

* Fix bug where CombineFns could get type hints

* Fix ConsumerSet init bug

* Ignore all combine classes, and fix iteration bug

* Add full labels to transforms

* Provide more descriptive error message

* Fix bug where non-iterables wouldn't be type-checked

* Store argspec

* Compute type hints in visitor instead of opcounters for perf gains, and to get the main input kwarg

* Fixup: apply YAPF

* Fixup: apply YAPF

* Fix import order

* Fix lint errors

* Use param name in error string

* Fixup: apply YAPF

* Fix lint

* Fix bug where inspect can fail

* Check for dict, str, unicode, bytes types in OutputProcessor

* Fixup: apply YAPF

* Make feature Python 3 only

* Update tests to be compatible with both flags

* Add tests

* Add more tests

* Fix lint errors

* Add more tests

* Leave performance_runtime_type_check off by default

* Fix lint

* Add website documentation

* Use the same value in tests because sampling is non-deterministic

* Update a test

* Remove accidental changes

* PR Fixes

* Stage 1 Refactor

* Refactor Stage 2

* Refactor Stage 3

* Refactor Stage 4

* Refactor Stage 5

* Fixup: apply YAPF

* Add comments for how to type-check side inputs, and lint fixes

* Use mega type hint

* PR Fixes: fix None bug, empty type hints, and move label transformation to visitor

* Fix some PR comments

* Add new method to PTransform: _add_type_constraint_from_consumer

* Get runtime type hints only from DoOperation instead of all operations

* Fix lint errors

* Fix lint and cython error

* Fix test that was written incorrectly

* Add microbenchmark

* Fix benchmark & lint errors

* Use averaging in benchmark

* Remove iterable check in common.py for performance concerns

* Add TODO for fixing error message strings

* Update documentation strings

* Update test and move annotations to a comment

* Make logic easier to read and document type hints for _add_type_constraint_from_consumer

* Remove ParDo prefix and fix bug

* Remove unnecessary try/except and optimize a selector statement

* Use type hints for combine functions and add a TODO for removing the isinstance(transform, ParDo) check in the visitor

* Extract type hints for all transform types, not just ParDo's

* Fix lint errors

* Fix type hint annotation docstrings

* Add test and fix type hint docstring

* Turn the perf RTC flag on for testing purposes

* Add try/catch for getfullargspec error

* Fix IndexError

* Turn flag off

* Use cleaner syntax

* Rebase onto master

* Fix test

* Remove unneeded code.

* Fix snippets bug

* Remove accidental changes

* Add TODO

* Remove expensive isinstance check

* Remove unused import

* Fix TODO

Co-authored-by: Robert Bradshaw <robertwb@gmail.com>
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.

4 participants