[
https://issues.apache.org/jira/browse/BEAM-8101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17227572#comment-17227572
]
Kenneth Knowles commented on BEAM-8101:
---------------------------------------
Here is a deep link to the docs:
[https://beam.apache.org/documentation/programming-guide/#dofn]
It isn't too deep. Roughly it is {{Setup ProcessBundle* Teardown}} where we
actually don't care about the details of processing a bundle, for this bug.
Technically the sequence can fail at any step so teardown is sometimes
described as "optional". If we unpacked {{ProcessBundle}} we would also have to
annotate a lot of phases as "optional" too. But they are optional in totally
different ways, hence the nice diagram.
The meaningful bit about {{Teardown}} for this bug is that the runner is
expected to call it if at all possible. However, a user must treat it as best
effort and not put critical logic in it, because some failures are not
recoverable enough to get a change to call teardown.
The bug here appears to be specifically
{{testTeardownCalledAfterExceptionInStartBundleStateful}}. These tests are all
ValidatesRunner tests. The reason it showed up in precommit is that the
DirectRunner fails the ValidatesRunner. The fact that it showed up for other
runners could mean a few different things:
1. They share some code (like {{StatefulDoFnRunner}}?) that causes the problem.
2. The other runners don't really support Teardown at all and it just happens
to overlap.
3. The test is bad.
4. By coincidence all the runners fail to call teardown but only for stateful
DoFn, even though the bug isn't in any shared code.
> ParDoLifecycleTest is flaky
> ---------------------------
>
> Key: BEAM-8101
> URL: https://issues.apache.org/jira/browse/BEAM-8101
> Project: Beam
> Issue Type: Bug
> Components: runner-direct, runner-flink, runner-spark
> Affects Versions: 2.15.0, 2.22.0
> Reporter: Jan Lukavský
> Assignee: Michael Luckey
> Priority: P1
> Labels: flake, flaky, flaky-test
>
> Temporary fail from Jenkins:
> {code}
> java.lang.AssertionError: Function should have been torn down after exception
> Expected: is <TEARDOWN>
> but: was <START_BUNDLE>
> at org.hamcrest.MatcherAssert.assertThat(MatcherAssert.java:18)
> at
> org.apache.beam.sdk.transforms.ParDoLifecycleTest.lambda$validate$0(ParDoLifecycleTest.java:266)
> at
> java.util.concurrent.ConcurrentHashMap$ValuesView.forEach(ConcurrentHashMap.java:4707)
> at
> org.apache.beam.sdk.transforms.ParDoLifecycleTest.validate(ParDoLifecycleTest.java:264)
> at
> org.apache.beam.sdk.transforms.ParDoLifecycleTest.testTeardownCalledAfterExceptionInProcessElementStateful(ParDoLifecycleTest.java:253)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
> at
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
> at
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
> at
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
> at
> org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
> at
> org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:319)
> at
> org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
> at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:349)
> at
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
> at
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
> at org.junit.runners.ParentRunner$3.run(ParentRunner.java:314)
> at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
> at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:312)
> at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
> at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:292)
> at org.junit.runners.ParentRunner.run(ParentRunner.java:396)
> at
> org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.runTestClass(JUnitTestClassExecutor.java:110)
> at
> org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:58)
> at
> org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:38)
> at
> org.gradle.api.internal.tasks.testing.junit.AbstractJUnitTestClassProcessor.processTestClass(AbstractJUnitTestClassProcessor.java:62)
> at
> org.gradle.api.internal.tasks.testing.SuiteTestClassProcessor.processTestClass(SuiteTestClassProcessor.java:51)
> at sun.reflect.GeneratedMethodAccessor31.invoke(Unknown Source)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at
> org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
> at
> org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
> at
> org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:32)
> at
> org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:93)
> at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
> at
> org.gradle.api.internal.tasks.testing.worker.TestWorker.processTestClass(TestWorker.java:118)
> at sun.reflect.GeneratedMethodAccessor30.invoke(Unknown Source)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at
> org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
> at
> org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
> at
> org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:175)
> at
> org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:157)
> at
> org.gradle.internal.remote.internal.hub.MessageHub$Handler.run(MessageHub.java:404)
> at
> org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:63)
> at
> org.gradle.internal.concurrent.ManagedExecutorImpl$1.run(ManagedExecutorImpl.java:46)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at
> org.gradle.internal.concurrent.ThreadFactoryImpl$ManagedThreadRunnable.run(ThreadFactoryImpl.java:55)
> at java.lang.Thread.run(Thread.java:748)
> Standard Error
> Aug 26, 2019 2:47:16 PM org.apache.beam.runners.direct.ParDoEvaluatorFactory
> createParDoEvaluator
> SEVERE: Exception encountered while cleaning up in ParDo evaluator
> construction
> org.apache.beam.sdk.util.UserCodeException: java.lang.AssertionError:
> Expected to have a processing method throw an exception
> at
> org.apache.beam.sdk.util.UserCodeException.wrap(UserCodeException.java:34)
> at
> org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingFn$DoFnInvoker.invokeTeardown(Unknown
> Source)
> at
> org.apache.beam.runners.direct.DoFnLifecycleManager$TeardownRemovedFnListener.onRemoval(DoFnLifecycleManager.java:113)
> at
> org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache.processPendingNotifications(LocalCache.java:1809)
> at
> org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.runUnlockedCleanup(LocalCache.java:3462)
> at
> org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.postWriteCleanup(LocalCache.java:3438)
> at
> org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.remove(LocalCache.java:3072)
> at
> org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache.remove(LocalCache.java:4236)
> at
> org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$LocalManualCache.invalidate(LocalCache.java:4899)
> at
> org.apache.beam.runners.direct.DoFnLifecycleManager.remove(DoFnLifecycleManager.java:66)
> at
> org.apache.beam.runners.direct.ParDoEvaluatorFactory.createParDoEvaluator(ParDoEvaluatorFactory.java:168)
> at
> org.apache.beam.runners.direct.ParDoEvaluatorFactory.createEvaluator(ParDoEvaluatorFactory.java:121)
> at
> org.apache.beam.runners.direct.ParDoEvaluatorFactory.forApplication(ParDoEvaluatorFactory.java:79)
> at
> org.apache.beam.runners.direct.TransformEvaluatorRegistry.forApplication(TransformEvaluatorRegistry.java:169)
> at
> org.apache.beam.runners.direct.DirectTransformExecutor.run(DirectTransformExecutor.java:117)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.AssertionError: Expected to have a processing method
> throw an exception
> at org.junit.Assert.fail(Assert.java:89)
> at
> org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingFn.after(ParDoLifecycleTest.java:398)
> {code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)