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-2898] Support Impulse transforms in Flink batch runner #4783

Merged
merged 2 commits into from
Apr 13, 2018

Conversation

bsidhom
Copy link
Contributor

@bsidhom bsidhom commented Mar 1, 2018

Fusion will be done via GreedyPipelineFuser, which does not currently support sourceless Read transforms. Instead, reads must happen via Impulse followed by a ParDo.

This change adds impulse support to the batch runner and a new ValidatesRunner ImpulseTest.


Follow this checklist to help us incorporate your contribution quickly and easily:

  • Make sure there is a JIRA issue filed for the change (usually before you start working on it). Trivial changes like typos do not require a JIRA issue. Your pull request should address just this issue, without pulling in other changes.
  • Format the pull request title like [BEAM-XXX] Fixes bug in ApproximateQuantiles, where you replace BEAM-XXX with the appropriate JIRA issue.
  • Write a pull request description that is detailed enough to understand:
    • What the pull request does
    • Why it does it
    • How it does it
    • Why this approach
  • Each commit in the pull request should have a meaningful subject line and body.
  • Run mvn clean verify to make sure basic checks pass. A more thorough check will be performed on your pull request automatically.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

@bsidhom
Copy link
Contributor Author

bsidhom commented Mar 1, 2018

This depends on #477, #4717, and #4776 and should be submitted after those.

@bsidhom
Copy link
Contributor Author

bsidhom commented Mar 1, 2018

R: @tgroh

* safely use an unwrapped SerializableCoder because
* {@link SerializableCoder#structuralValue(Serializable)} assumes that coded elements support
* object equality. By default, Coders compare equality by serialized bytes, which we want in
* this case. It is usually safe to depend on coded representation here because we only compare
Copy link
Member

Choose a reason for hiding this comment

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

"usually safe..." isn't really necessary.

Probably worth noting that we usually expect Source equals methods to not be defined.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed.

/**
* Fuses a {@link Pipeline} into a collection of {@link ExecutableStage}s.
*
* <p>This fuser expects each PTransform to have exactly one input. This means that pipelines must
Copy link
Member

Choose a reason for hiding this comment

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

Not true; Flatten is permitted, as is ParDo with side inputs.

However, it does expect all transforms with no inputs to not have an associated environment, which is what forces this requirement.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed.


@Override
public WindowedValue<byte[]> nextRecord(WindowedValue<byte[]> windowedValue) {
hasOutput = false;
Copy link
Member

Choose a reason for hiding this comment

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

This name really confused me for a while. You're using it like "This split/format has output within it, which has not been produced" and I read it as "This split/format has produced output", which, of course, mean the opposite things.

'containsOutput' or 'availableOutput', perhaps?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, I actually struggled to come up with a succinct yet descriptive name. Apparently this failed in the end... ;)

Updated to availableOutput and added a comment for clarity.

@Test
@Category({ValidatesRunner.class, UsesImpulse.class})
public void testImpulseRead() {
PCollection<Integer> result = p.apply(JavaReadViaImpulse.bounded(Source.of(1, 2, 3)));
Copy link
Member

Choose a reason for hiding this comment

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

why not just a ParDo?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Will pipeline construction always insert an Impulse in that case? Or do I need to insert one myself? I want to ensure that runners can actually handle Impulse proper nodes.

Copy link
Member

Choose a reason for hiding this comment

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

What I mean is just insert an Impulse then a ParDo after it, instead of wrapping a Source

Copy link
Contributor Author

Choose a reason for hiding this comment

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

OK, I've added another test with a direct Impulse -> ParDo. Let me know if this looks good and I'll delete the source-based test.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This seeks to work fine and tests what we want to test, so I've gone ahead and removed the source-based test.

Copy link
Contributor Author

@bsidhom bsidhom left a comment

Choose a reason for hiding this comment

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

I've added a raw Impulse test and rebased.

@Test
@Category({ValidatesRunner.class, UsesImpulse.class})
public void testImpulseRead() {
PCollection<Integer> result = p.apply(JavaReadViaImpulse.bounded(Source.of(1, 2, 3)));
Copy link
Contributor Author

Choose a reason for hiding this comment

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

OK, I've added another test with a direct Impulse -> ParDo. Let me know if this looks good and I'll delete the source-based test.

@tgroh
Copy link
Member

tgroh commented Mar 15, 2018

You'll need to rebase

* object equality. By default, Coders compare equality by serialized bytes, which we want in
* this case. It is usually safe to depend on coded representation here because we only compare
* objects on bundle commit, which compares serializations of the same object instance.
* object equality. Sources in general do not support object equality. By default, Coders compare
Copy link
Member

Choose a reason for hiding this comment

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

Though with #4817 I'm no longer hugely concerned about the coder stuff. I wouldn't revert it, however - certainly, the StructuralValue is better, even if it's not going to be used anywhere relevant, and the comment seems sufficient for the time being.

* use Impulse/ParDo transformations rather than read nodes. The utilities in
* {@link org.apache.beam.runners.core.construction.JavaReadViaImpulse} can be used to translate
* non-compliant pipelines.
* <p>This fuser expects each PTransform which has no inputs to have an associated environment.
Copy link
Member

Choose a reason for hiding this comment

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

s/an associated environment/no associated environment

* {@link org.apache.beam.runners.core.construction.JavaReadViaImpulse} can be used to translate
* non-compliant pipelines.
* <p>This fuser expects each PTransform which has no inputs to have an associated environment.
* This means that pipelines must use Impulse/ParDo transformations rather than read nodes. The
Copy link
Member

Choose a reason for hiding this comment

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

"This means that pipelines must be rooted at Impulse, or other runner-executed primitive transforms, instead of primitive Read nodes."

@@ -69,6 +69,15 @@ private GreedyPipelineFuser(Pipeline p) {
fusePipeline(groupSiblings(rootConsumers));
}

/**
* Fuses a {@link Pipeline} into a collection of {@link ExecutableStage}s.
Copy link
Member

Choose a reason for hiding this comment

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

{@link ExecutableStage ExecutableStages} is our normal style.

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.

* must be rooted at Impulse, or other runner-executed primitive transforms, instead of primitive
* Read nodes. The utilities in
* {@link org.apache.beam.runners.core.construction.JavaReadViaImpulse} can be used to translate
* non-compliant pipelines.
Copy link
Member

Choose a reason for hiding this comment

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

This does kind of have an associated TODO for unbounded reads; https://issues.apache.org/jira/browse/BEAM-3859 is the (just-authored) issue to link against.

'can be used to translate non-compliant pipelines -> can be used to convert bounded pipelines using the Read primitive.'

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.

@@ -106,6 +149,13 @@ public void testOutputCoder() {
equalTo(BigEndianIntegerCoder.of()));
}

private static void assertNotReadTransform(PTransform<?, ?> transform) {
Copy link
Member

Choose a reason for hiding this comment

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

This could be inlined into the anonymous subclass that you'er traversing the pipeline with

Copy link
Contributor Author

Choose a reason for hiding this comment

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

As above, this is already submitted, though I could change it here if you want.

PCollection<Long> input = p.apply(Read.from(source));
PAssert.that(input).containsInAnyOrder(0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L);
p.replaceAll(Collections.singletonList(JavaReadViaImpulse.boundedOverride()));
p.traverseTopologically(new Pipeline.PipelineVisitor() {
Copy link
Member

Choose a reason for hiding this comment

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

extend PipelineVisitor.Defaults, and get rid of enterPipeline, visitValue, and leave*

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is actually already in master and was just one of the dependent changes.


@Override
public WindowedValue<byte[]> nextRecord(WindowedValue<byte[]> windowedValue) {
availableOutput = false;
Copy link
Member

Choose a reason for hiding this comment

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

This could be a checkArgument before you mutate the state (minor preference, no strong feelings)

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.

@@ -66,6 +66,7 @@ dependencies {
shadow library.java.junit
shadow "org.tukaani:xz:1.5"
shadowTest project(":model:fn-execution").sourceSets.test.output
shadowTest project(path: ":runners:core-construction-java", configuration: "shadow")
Copy link
Member

Choose a reason for hiding this comment

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

What introduced this edge?

This also can't be represented in maven, and right now that worries me (though hopefully less soon)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This isn't needed anymore. Thanks for pointing it out.

@bsidhom
Copy link
Contributor Author

bsidhom commented Apr 2, 2018

Review comments addressed. PTAL.

@tgroh
Copy link
Member

tgroh commented Apr 11, 2018

run java precommit

1 similar comment
@bsidhom
Copy link
Contributor Author

bsidhom commented Apr 12, 2018

run java precommit

@bsidhom
Copy link
Contributor Author

bsidhom commented Apr 13, 2018

It looks like the precommits that are actually being run are passing.

@tgroh tgroh merged commit 007553f into apache:master Apr 13, 2018
@bsidhom bsidhom deleted the impulse-flink branch April 13, 2018 19:01
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.

None yet

2 participants