Skip to content

Comments

Use input type in coder inference for MapElements and FlatMapElements#757

Merged
asfgit merged 1 commit intoapache:masterfrom
kennknowles:Map-FlatMap-TypeDescriptor
Aug 5, 2016
Merged

Use input type in coder inference for MapElements and FlatMapElements#757
asfgit merged 1 commit intoapache:masterfrom
kennknowles:Map-FlatMap-TypeDescriptor

Conversation

@kennknowles
Copy link
Member

@kennknowles kennknowles commented Jul 29, 2016

Be sure to do all of the following to help us incorporate your contribution
quickly and easily:

  • Make sure the PR title is formatted like:
    [BEAM-<Jira issue #>] Description of pull request
  • Make sure tests pass via mvn clean verify. (Even better, enable
    Travis-CI on your fork and ensure the whole test matrix passes).
  • Replace <Jira issue #> in the title with the actual Jira issue
    number, if there is one.
  • If this contribution is large, please file an Apache
    Individual Contributor License Agreement.

It seems that PR #756 triggered some very strange serialization issues with regards to lambda (https://builds.apache.org/job/beam_PreCommit_MavenVerify/2610/console). To isolate them, I went through the conflicts and peeled this commit off the top. Description follows:

Previously, the input TypeDescriptor was unknown, so we would fail to infer a coder for things like MapElements.of(SimpleFunction<T, T>) even if the input PCollection provided a coder for T.

Now, the input type is plumbed appropriately and the coder is inferred.

This required internal changes to explicitly support good display data (tests fail with a naive refactor). While doing this, I just added display data to SimpleFunction by analogy with DoFn.

@kennknowles kennknowles force-pushed the Map-FlatMap-TypeDescriptor branch 2 times, most recently from 055654b to 2a3d894 Compare July 29, 2016 04:20
@kennknowles
Copy link
Member Author

A bit of flavor on the motivation by this change, which might otherwise look bigger than it needs to be:

  • ParDo.apply always clones its argument immediately via serialization to support the (very questionable IMO) use case of a user who wants to repeatedly mutate their DoFn while re-using it, and have each of the re-uses keep the value at the time of apply. To me, this contradicts the general semantics of Java...
  • But anyhow in that case, transient fields end up null so I had to make the types non-transient.
  • But even though TypeDescriptor is serializable, TypeVariable is not (this is why they were transient in the first place). And this fix is specifically for cases where both the input type and output type contain type variables.
  • So I "swapped the polarity" on whether the underlying transform stashes a SimpleFunction or a SerializableFunction plus type descriptors.
  • That broke display data for a minute, since it looked directly at the classes, and in the process of fixing that, I just added display data to the functions. The fix didn't actually require this latter bit.

So this is my best approach to getting a good TypeDescriptor for both input and output without every having to serialize them.

@kennknowles
Copy link
Member Author

R: @bjchambers AND @swegner

@swegner
Copy link
Contributor

swegner commented Jul 29, 2016

Looks like these same change appears in #756; I've added comments there.

@kennknowles kennknowles force-pushed the Map-FlatMap-TypeDescriptor branch 2 times, most recently from 5271795 to 9777ade Compare July 29, 2016 17:39
@kennknowles
Copy link
Member Author

Thanks to your comments in #756, I've fixed this one up. They are now disjoint, but this one will have to merge first.

@Override
public List<Integer> apply(Integer input) {
return Collections.emptyList();
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you update these tests to also verify display data items from the SimpleFunction are registered? i.e., add a populateDisplayData() override to add some display data and make sure it gets wired through.

Copy link
Contributor

Choose a reason for hiding this comment

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

Ah, nevermind I see you moved that the a separate PR.

@swegner
Copy link
Contributor

swegner commented Jul 29, 2016

Aside from comment about additional testing, LGTM

@kennknowles
Copy link
Member Author

Pinging @bjchambers for binding LGTM

@kennknowles
Copy link
Member Author

Travis failure is timeout.

@kennknowles kennknowles force-pushed the Map-FlatMap-TypeDescriptor branch from aa7524d to 71e7348 Compare July 30, 2016 02:53
@kennknowles
Copy link
Member Author

Rebased to kick Travis.

pipeline.run();
}

@Test
Copy link
Contributor

Choose a reason for hiding this comment

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

It seems like there should be tests for the "propagate input coder to output" logic in at least a few diffferent cases T -> T, T -> Iterable, etc.

Copy link
Member Author

Choose a reason for hiding this comment

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

Done

@kennknowles kennknowles force-pushed the Map-FlatMap-TypeDescriptor branch from 71e7348 to 714a218 Compare August 2, 2016 18:19
* A {@link SimpleFunction} to test that the coder registry can propagate coders
* that are bound to type variables.
*/
private static class PolymorphicSimpleFunction<T> extends SimpleFunction<T, Iterable<T>> {
Copy link
Contributor

Choose a reason for hiding this comment

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

We could also play with a few other simple functions:

SimpleFunction<T, T>
SimpleFunction<Iterable, T> <- wouldn't expect this to work?

@bjchambers
Copy link
Contributor

Jenkins is red.

@kennknowles kennknowles force-pushed the Map-FlatMap-TypeDescriptor branch from 714a218 to db57ce1 Compare August 2, 2016 18:25
@kennknowles
Copy link
Member Author

Jenkins rightly took issue with a poorly place linebreak. Fixed.

@kennknowles kennknowles force-pushed the Map-FlatMap-TypeDescriptor branch 2 times, most recently from 1ec5770 to 00bbb0a Compare August 4, 2016 17:12
@kennknowles
Copy link
Member Author

Rebased onto the new DoFn. PTAL.

@kennknowles kennknowles force-pushed the Map-FlatMap-TypeDescriptor branch 2 times, most recently from f8533ad to 4ac5caf Compare August 5, 2016 03:18
Previously, the input TypeDescriptor was unknown, so we would fail
to infer a coder for things like MapElements.of(SimpleFunction<T, T>)
even if the input PCollection provided a coder for T.

Now, the input type is plumbed appropriately and the coder is inferred.
@asfgit asfgit merged commit 4ac5caf into apache:master Aug 5, 2016
asfgit pushed a commit that referenced this pull request Aug 5, 2016
@kennknowles kennknowles deleted the Map-FlatMap-TypeDescriptor branch November 10, 2016 03:10
pl04351820 pushed a commit to pl04351820/beam that referenced this pull request Dec 20, 2023
* docs: Minor formatting
chore: Update gapic-generator-python to v1.11.5
build: Update rules_python to 0.24.0

PiperOrigin-RevId: 563436317

Source-Link: googleapis/googleapis@42fd37b

Source-Link: https://github.com/googleapis/googleapis-gen/commit/280264ca02fb9316b4237a96d0af1a2343a81a56
Copy-Tag: eyJwIjoiLmdpdGh1Yi8uT3dsQm90LnlhbWwiLCJoIjoiMjgwMjY0Y2EwMmZiOTMxNmI0MjM3YTk2ZDBhZjFhMjM0M2E4MWE1NiJ9

* 🦉 Updates from OwlBot post-processor

See https://github.com/googleapis/repo-automation-bots/blob/main/packages/owl-bot/README.md

---------

Co-authored-by: Owl Bot <gcf-owl-bot[bot]@users.noreply.github.com>
Co-authored-by: Victor Chudnovsky <vchudnov@google.com>
Co-authored-by: Daniel Sanche <sanche@google.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants