Skip to content

[BEAM-498] Replace ParDo with MapElements and FlatMapElements where possible#756

Closed
kennknowles wants to merge 1 commit intoapache:masterfrom
kennknowles:map-flatmap
Closed

[BEAM-498] Replace ParDo with MapElements and FlatMapElements where possible#756
kennknowles wants to merge 1 commit intoapache:masterfrom
kennknowles:map-flatmap

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.

The commits ended up having fairly separate topics, but can be reviewed individually or as a medium-sized change.

  1. The first commit replaces ParDo with MapElements and FlatMapElements where it is easy to do so.
  2. While debugging, I noticed that DoFn used a less-powerful form of TypeDescriptor and switched trivially to the enhanced version.
  3. The root cause of issues with MapElements and FlatMapElements was a lack of use of the input type descriptor. Making it available involved a moderate refactor. In the process I broke some tests to do with display data and fixed them plus enhancements to display data for SimpleFunction.

If reviewers insist, I can try to alter this commit history.

R: @bjchambers AND @swegner

@kennknowles kennknowles changed the title Replace ParDo with MapElements and FlatMapElements where possible [BEAM-498] Replace ParDo with MapElements and FlatMapElements where possible Jul 29, 2016
private final SerializableFunction<InputT, OutputT> fn;
private final transient TypeDescriptor<OutputT> outputType;
private final SimpleFunction<InputT, OutputT> fn;
private final Class<?> fnClass; // for display data purposes
Copy link
Contributor

Choose a reason for hiding this comment

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

Careful, storing Class<?> instances can be problematic. Anonymous classes backed by Java 8 lambdas will explode during serialization. See CombineJava8Test#testLambdaSerialization(). In Combine.java, we store the DisplayData.Item<?> instead.

Recommend writing a Java8 test for this class as well. I wonder: would be it be easy to write some FindBugs-like static analysis that would catch Serializable classes with non-transient Class<?> instance fields?

Copy link
Member Author

Choose a reason for hiding this comment

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

Thank you! I did not identify the cause of the failure, but this was certainly the issue. Whatever I store does have to be serialized and non-transient, because of the unfortunate issue with ParDo. Your idea solved it. I'm going to drop the commit from this PR, please review #757 that is focused on just that.

@swegner
Copy link
Contributor

swegner commented Jul 29, 2016

This is small enough to review as two commits, but I'd recommend fixing up the history on merge to separate out the changes in two commits.

@kennknowles
Copy link
Member Author

R: -@swegner love the feedback but this PR may or may not remain interesting to you

@kennknowles
Copy link
Member Author

PTAL. Rebased and fixed up.

*/
public class MapElements<InputT, OutputT>
extends PTransform<PCollection<InputT>, PCollection<OutputT>> {
extends PTransform<PCollection<? extends InputT>, PCollection<OutputT>> {
Copy link
Contributor

Choose a reason for hiding this comment

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

Were the changes to MapElements not in a different PR already?

Copy link
Member Author

Choose a reason for hiding this comment

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

This particular change was not. There were cases that could not be ported without it, since the new generic makes it the same type as ParDo.

@kennknowles kennknowles force-pushed the map-flatmap branch 2 times, most recently from 0e3b412 to 85e3da9 Compare August 8, 2016 20:54
@kennknowles
Copy link
Member Author

This actually looks like a Jenkins race condition or some such. The errors in that build are not part of this PR. Rebasing to kick...

@kennknowles kennknowles force-pushed the map-flatmap branch 2 times, most recently from ac329b1 to 6befb9c Compare August 10, 2016 02:30
There are a number of places in the Java SDK where we use
ParDo.of(DoFn) when MapElements or other higher-level
composites are applicable and readable. This change
alters a number of those.
@kennknowles
Copy link
Member Author

R: -@bjchambers I had added you since you were very in-the-loop on both the new DoFn and display data, but now this PR has nothing to do with those and I've sent more than enough your way.

R: @aljoscha care to take a look? The tests have been passing for a while, but with some Travis timeouts. I just rebased to force Travis to run again in its new, faster, configuration.

@bjchambers
Copy link
Contributor

LGTM

@asfgit asfgit closed this in d93ef2e Aug 17, 2016
pl04351820 pushed a commit to pl04351820/beam that referenced this pull request Dec 20, 2023
* chore(deps): update all dependencies

* 🦉 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: 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.

3 participants