-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Update element_type inference (default_type_hints) for batched DoFns with yields_batches/yields_elements #22198
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
Update element_type inference (default_type_hints) for batched DoFns with yields_batches/yields_elements #22198
Conversation
|
FYI @robertwb this fixes a bug in Batched DoFns. I'm OOO next week and would like to get this in before the release cut the following Wednesday. |
Codecov Report
@@ Coverage Diff @@
## master #22198 +/- ##
==========================================
- Coverage 74.22% 74.20% -0.03%
==========================================
Files 702 703 +1
Lines 92937 93095 +158
==========================================
+ Hits 68985 69077 +92
- Misses 22685 22751 +66
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). |
|
Run Python PreCommit |
1 similar comment
|
Run Python PreCommit |
| | beam.Map(lambda x: x * 3)) | ||
| | beam.ParDo(ArrayProduceDoFn())) | ||
|
|
||
| self.assertEqual(pc.element_type, np.int64) |
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.
This is up to you, my rule of thumb is smaller concise tests for unit tests.
Like:
def test_pardo_maintains_type_hint():
with self.create_pipeline() as p:
pc = (
p
| beam.Create(np.array([1, 2, 3], dtype=np.int64)).with_output_types(
np.int64)
| beam.ParDo(ArrayProduceDoFn()))
self.assertEqual(pc.element_type, np.int64)
And then end the test there.
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.
Fair point, I initially added the test here for expediency, but now I have the concise test in batch_dofn_test. I'll drop this change.
|
|
||
| class MismatchedBatchProducingDoFn(beam.DoFn): | ||
| """A DoFn that produces batches from both process and process_batch, with | ||
| mismatched types. Should yield a construction time error when applied.""" |
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.
took me three looks to realize the mismatch was one produces floats and the other ints. Maybe just slightly amend the comment like this:
...with mismatched types (one has floats the other ints)....
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, done
| batch_fn_type_hints = typehints.decorators.IOTypeHints.from_callable( | ||
| self.process_batch) | ||
|
|
||
| if fn_type_hints is not None: |
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.
This nested logic is hard to read and a little messy and hard to follow.
For example here and below we have "if fn_type_hints is not None:"
I had to write this a little to really wrap my head around it. I don't know if any parts of my rewrite are useful.
# default type hints can come from one of two places. The process type hints or the process batch type hints.
# also the process might be set up to yield batches or to yield elements
has_fn_type_hints = fn_type_hints is not None
has_batch_type_hints = batch_fn_type_hints is not None
yields_batches = self._process_yields_batches
yeilds_elements = self._process_yields_elements
if has_fn_type_hints:
if yields_elements and has_batch_type_hints:
# because both batch and element type hints are defined, raise an exception if they are not equal.
self.validate_batch_and_fn_typehints_same()
fn_type_hints = fn_type_hints.with_output_types_from(
batch_fn_type_hints) # TODO: comment why this is done.
elif yields_batches:
# Because the process method produces batches, don't use its output typehints.
fn_type_hints = fn_type_hints.with_output_types_from(
typehints.decorators.IOTypeHints.empty())
elif yields_elements and has_batch_type_hints:
# process_batch produces elements, grab its output typehint
fn_type_hints = batch_fn_type_hints.with_input_types_from(
typehints.decorators.IOTypeHints.empty()) # TODO comment why does this use with_input_types_from
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 agreed this logic was not pretty. I think part of the issue is that it's has to deal with both None, and the possibility of IOTypeHints.empty() in both sets of typehints. I rewrote it to collapse those two possibilities, and also separated concerns a bit, first we deal with process_yields_batches, then deal with process_batch_yields_elements. WDYT?
Can you clarify the "TODO comment why does this use with_input_types_from"?
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.
I was a little concerned rewriting this way could break some esoteric use-case, but I ran all our internal tests and it looks good.
|
Reminder, please take a look at this pr: @ryanthompson591 |
TheNeuralBit
left a comment
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 for the feedback!
| | beam.Map(lambda x: x * 3)) | ||
| | beam.ParDo(ArrayProduceDoFn())) | ||
|
|
||
| self.assertEqual(pc.element_type, np.int64) |
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.
Fair point, I initially added the test here for expediency, but now I have the concise test in batch_dofn_test. I'll drop this change.
|
|
||
| class MismatchedBatchProducingDoFn(beam.DoFn): | ||
| """A DoFn that produces batches from both process and process_batch, with | ||
| mismatched types. Should yield a construction time error when applied.""" |
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, done
| batch_fn_type_hints = typehints.decorators.IOTypeHints.from_callable( | ||
| self.process_batch) | ||
|
|
||
| if fn_type_hints is not None: |
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 agreed this logic was not pretty. I think part of the issue is that it's has to deal with both None, and the possibility of IOTypeHints.empty() in both sets of typehints. I rewrote it to collapse those two possibilities, and also separated concerns a bit, first we deal with process_yields_batches, then deal with process_batch_yields_elements. WDYT?
Can you clarify the "TODO comment why does this use with_input_types_from"?
|
assign to next reviewer |
|
R: @tvalentyn could you take a look? It looks like Ryan will be out through next week and I'd like to get this in for the release cut. |
|
Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control |
|
I checked that |
|
Thanks for checking on that! I ran a TGP and everything looks good. |
Fixes #22197
This updates
DoFn.default_type_hintsto inspect the output type annotation for the appropriate method (process, or process_batch) depending on the yields_batches/yields_elements decorators.GitHub Actions Tests Status (on master branch)
See CI.md for more information about GitHub Actions CI.