You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "Michael Luckey (Jira)" <ji...@apache.org> on 2021/11/18 19:39:00 UTC

[jira] [Comment Edited] (BEAM-8101) Flakes in ParDoLifecycleTest.testTeardownCalledAfterExceptionInStartBundleStateful for Direct, Spark, Flink

    [ https://issues.apache.org/jira/browse/BEAM-8101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17446116#comment-17446116 ] 

Michael Luckey edited comment on BEAM-8101 at 11/18/21, 7:38 PM:
-----------------------------------------------------------------

So, after all this time i debugged again.

What does happen on these error cases, is that instead of
{noformat}
callStateVisited=[SETUP, START_BUNDLE, PROCESS_ELEMENT, FINISH_BUNDLE, TEARDOWN]
{noformat}
the order is (e.g.)
{noformat}
callStateVisited=[SETUP, TEARDOWN, START_BUNDLE, PROCESS_ELEMENT, FINISH_BUNDLE]
{noformat}
This essentially happens, because there are multiple instances of DoFn started (default 3?). If the very first fails - on that exception thrown in order to test that teardown is called, the executor service is shutdown which internally invalidates the cache of DoFns [1].

If one DoFn was pulled from the cache and is still processing, e.g. [2], there is - of course - a race condition. So, one could argue, that the assumption, the test makes are just 'to strong'. If - of course - we could live with the runner calling start/process/finish on an already torn down DoFn.

Any idea how we could/should handle that?

[1]
{noformat}
at org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingFn$DoFnInvoker.invokeTeardown(Unknown Source)
        at org.apache.beam.runners.direct.DoFnLifecycleManager$TeardownRemovedFnListener.onRemoval(DoFnLifecycleManager.java:116)
        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.clear(LocalCache.java:3215)
        at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache.clear(LocalCache.java:4270)
        at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$LocalManualCache.invalidateAll(LocalCache.java:4909)
        at org.apache.beam.runners.direct.DoFnLifecycleManager.removeAll(DoFnLifecycleManager.java:86)
        at org.apache.beam.runners.direct.DoFnLifecycleManagers.removeAllFromManagers(DoFnLifecycleManagers.java:32)
        at org.apache.beam.runners.direct.ParDoEvaluatorFactory.cleanup(ParDoEvaluatorFactory.java:95)
        at org.apache.beam.runners.direct.TransformEvaluatorRegistry.cleanup(TransformEvaluatorRegistry.java:185)
        at org.apache.beam.runners.direct.ExecutorServiceParallelExecutor.shutdownIfNecessary(ExecutorServiceParallelExecutor.java:334)
        at org.apache.beam.runners.direct.ExecutorServiceParallelExecutor.access$200(ExecutorServiceParallelExecutor.java:62)
        at org.apache.beam.runners.direct.ExecutorServiceParallelExecutor$2.run(ExecutorServiceParallelExecutor.java:191)
{noformat}
[2]
{noformat}
at org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingFn$DoFnInvoker.invokeStartBundle(Unknown Source)
        at org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner.startBundle(SimpleDoFnRunner.java:177)
        at org.apache.beam.repackaged.direct_java.runners.core.SimplePushbackSideInputDoFnRunner.startBundle(SimplePushbackSideInputDoFnRunner.java:72)
        at org.apache.beam.runners.direct.ParDoEvaluator.<init>(ParDoEvaluator.java:223)
        at org.apache.beam.runners.direct.ParDoEvaluator.create(ParDoEvaluator.java:179)
        at org.apache.beam.runners.direct.ParDoEvaluator.create(ParDoEvaluator.java:171)
        at org.apache.beam.runners.direct.ParDoEvaluatorFactory.createParDoEvaluator(ParDoEvaluatorFactory.java:152)
        at org.apache.beam.runners.direct.ParDoEvaluatorFactory.createEvaluator(ParDoEvaluatorFactory.java:123)
        at org.apache.beam.runners.direct.ParDoEvaluatorFactory.forApplication(ParDoEvaluatorFactory.java:81)
        at org.apache.beam.runners.direct.TransformEvaluatorRegistry.forApplication(TransformEvaluatorRegistry.java:178)
        at org.apache.beam.runners.direct.DirectTransformExecutor.run(DirectTransformExecutor.java:122)
        at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
{noformat}


was (Author: michel):
So, after all this time i debugged again.

What does happen on these error cases, is that instead of 
{noformat}
callStateVisited=[SETUP, START_BUNDLE, PROCESS_ELEMENT, FINISH_BUNDLE, TEARDOWN]
{noformat}
the order is
{noformat}
callStateVisited=[SETUP, TEARDOWN, START_BUNDLE, PROCESS_ELEMENT, FINISH_BUNDLE]
{noformat}

This essentially happens, because there are multiple instances of DoFn started (default 3?). If the very first fails - on that exception thrown in order to test that teardown is called, the executor service is shutdown which internally invalidates the cache of DoFns [1].

If one DoFn was pulled from the cache and is still processing, e.g. [2], there is - of course - a race condition. So, one could argue, that the assumption, the test makes are just 'to strong'. If - of course - we could live with the runner calling start/process/finish on an already torn down DoFn.

Any idea how we could/should handle that?


[1]
{noformat}
at org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingFn$DoFnInvoker.invokeTeardown(Unknown Source)
        at org.apache.beam.runners.direct.DoFnLifecycleManager$TeardownRemovedFnListener.onRemoval(DoFnLifecycleManager.java:116)
        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.clear(LocalCache.java:3215)
        at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache.clear(LocalCache.java:4270)
        at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$LocalManualCache.invalidateAll(LocalCache.java:4909)
        at org.apache.beam.runners.direct.DoFnLifecycleManager.removeAll(DoFnLifecycleManager.java:86)
        at org.apache.beam.runners.direct.DoFnLifecycleManagers.removeAllFromManagers(DoFnLifecycleManagers.java:32)
        at org.apache.beam.runners.direct.ParDoEvaluatorFactory.cleanup(ParDoEvaluatorFactory.java:95)
        at org.apache.beam.runners.direct.TransformEvaluatorRegistry.cleanup(TransformEvaluatorRegistry.java:185)
        at org.apache.beam.runners.direct.ExecutorServiceParallelExecutor.shutdownIfNecessary(ExecutorServiceParallelExecutor.java:334)
        at org.apache.beam.runners.direct.ExecutorServiceParallelExecutor.access$200(ExecutorServiceParallelExecutor.java:62)
        at org.apache.beam.runners.direct.ExecutorServiceParallelExecutor$2.run(ExecutorServiceParallelExecutor.java:191)
{noformat}

[2]
{noformat}
at org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingFn$DoFnInvoker.invokeStartBundle(Unknown Source)
        at org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner.startBundle(SimpleDoFnRunner.java:177)
        at org.apache.beam.repackaged.direct_java.runners.core.SimplePushbackSideInputDoFnRunner.startBundle(SimplePushbackSideInputDoFnRunner.java:72)
        at org.apache.beam.runners.direct.ParDoEvaluator.<init>(ParDoEvaluator.java:223)
        at org.apache.beam.runners.direct.ParDoEvaluator.create(ParDoEvaluator.java:179)
        at org.apache.beam.runners.direct.ParDoEvaluator.create(ParDoEvaluator.java:171)
        at org.apache.beam.runners.direct.ParDoEvaluatorFactory.createParDoEvaluator(ParDoEvaluatorFactory.java:152)
        at org.apache.beam.runners.direct.ParDoEvaluatorFactory.createEvaluator(ParDoEvaluatorFactory.java:123)
        at org.apache.beam.runners.direct.ParDoEvaluatorFactory.forApplication(ParDoEvaluatorFactory.java:81)
        at org.apache.beam.runners.direct.TransformEvaluatorRegistry.forApplication(TransformEvaluatorRegistry.java:178)
        at org.apache.beam.runners.direct.DirectTransformExecutor.run(DirectTransformExecutor.java:122)
        at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
{noformat}

> Flakes in ParDoLifecycleTest.testTeardownCalledAfterExceptionInStartBundleStateful for Direct, Spark, Flink
> -----------------------------------------------------------------------------------------------------------
>
>                 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, 2.31.0
>            Reporter: Jan Lukavský
>            Priority: P1
>              Labels: currently-failing, 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.20.1#820001)