[ https://issues.apache.org/jira/browse/BEAM-3219?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16268064#comment-16268064 ]
ASF GitHub Bot commented on BEAM-3219: -------------------------------------- kennknowles closed pull request #4153: [BEAM-3219] DataflowRunner: delegate @Setup and @Teardown in stateful ParDo URL: https://github.com/apache/beam/pull/4153 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java index d7e9d06ef4c..6330e79181c 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java @@ -36,6 +36,7 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.ParDo.MultiOutput; import org.apache.beam.sdk.transforms.ParDo.SingleOutput; +import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -315,12 +316,22 @@ public void processElement(ProcessContext c) { return underlyingDoFn; } + @Setup + public void setup() { + DoFnInvokers.invokerFor(underlyingDoFn).invokeSetup(); + } + @ProcessElement public void processElement(final ProcessContext c, final BoundedWindow window) { throw new UnsupportedOperationException( "BatchStatefulDoFn.ProcessElement should never be invoked"); } + @Teardown + public void teardown() { + DoFnInvokers.invokerFor(underlyingDoFn).invokeTeardown(); + } + @Override public TypeDescriptor<OutputT> getOutputTypeDescriptor() { return underlyingDoFn.getOutputTypeDescriptor(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java index 849b874852d..77fea826108 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java @@ -27,9 +27,14 @@ import java.io.Serializable; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.UsesStatefulParDo; import org.apache.beam.sdk.testing.ValidatesRunner; +import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; @@ -151,6 +156,21 @@ public void testFnCallSequenceMulti() { p.run(); } + @Test + @Category({ValidatesRunner.class, UsesStatefulParDo.class}) + public void testFnCallSequenceStateful() { + PCollectionList.of(p.apply("Impolite", Create.of(KV.of("a", 1), KV.of("b", 2), KV.of("a", 4)))) + .and( + p.apply( + "Polite", Create.of(KV.of("b", 3), KV.of("a", 5), KV.of("c", 6), KV.of("c", 7)))) + .apply(Flatten.<KV<String, Integer>>pCollections()) + .apply( + ParDo.of(new CallSequenceEnforcingStatefulFn<String, Integer>()) + .withOutputTags(new TupleTag<KV<String, Integer>>() {}, TupleTagList.empty())); + + p.run(); + } + private static class CallSequenceEnforcingFn<T> extends DoFn<T, T> { private boolean setupCalled = false; private int startBundleCalls = 0; @@ -204,25 +224,13 @@ public void after() { } } - @Test - @Category(NeedsRunner.class) - public void testTeardownCalledAfterExceptionInSetup() { - ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.SETUP); - p - .apply(Create.of(1, 2, 3)) - .apply(ParDo.of(fn)); - try { - p.run(); - fail("Pipeline should have failed with an exception"); - } catch (Exception e) { - assertThat( - "Function should have been torn down after exception", - ExceptionThrowingOldFn.teardownCalled.get(), - is(true)); - } - } - + private static class CallSequenceEnforcingStatefulFn<K, V> + extends CallSequenceEnforcingDoFn<KV<K, V>> { + private static final String STATE_ID = "foo"; + @StateId(STATE_ID) + private final StateSpec<ValueState<String>> valueSpec = StateSpecs.value(); + } @Test @Category(NeedsRunner.class) ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > DataflowRunner: @Setup not called for batch stateful DoFn > --------------------------------------------------------- > > Key: BEAM-3219 > URL: https://issues.apache.org/jira/browse/BEAM-3219 > Project: Beam > Issue Type: Bug > Components: runner-dataflow > Affects Versions: 2.0.0, 2.1.0, 2.2.0 > Reporter: Kenneth Knowles > Assignee: Kenneth Knowles > > Reported on u...@beam.apache.org: > https://lists.apache.org/thread.html/33c0038508308b69527dab5c3d27cf17ed65fdd9f5d7f6a17cf6d794@%3Cuser.beam.apache.org%3E -- This message was sent by Atlassian JIRA (v6.4.14#64029)