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

Michael Luckey commented on BEAM-8101:
--------------------------------------

Sorry for the slow response cycles.

Here the issue enhanced with stack traces to track what is calling teardown 
resp lifecycle methods. This case fails because FINISH bundle is called 
concurrently


{noformat}
>> @StartBundle  701821764 1635355537765871 on 
>> Thread[direct-runner-worker,5,main]
>> @StartBundle  2048943483 1635355537764616 on 
>> Thread[direct-runner-worker,5,main]
<< @StartBundle  2048943483 1635355538030297 on 
Thread[direct-runner-worker,5,main]
<< @StartBundle  701821764 1635355538024086 on 
Thread[direct-runner-worker,5,main]
>> @ProcessElement  701821764 1635355538965232 on 
>> Thread[direct-runner-worker,5,main]
>> @ProcessElement  2048943483 1635355538965799 on 
>> Thread[direct-runner-worker,5,main]
<< @ProcessElement  701821764 1635355539076834 on 
Thread[direct-runner-worker,5,main]
<< @ProcessElement  2048943483 1635355539090794 on 
Thread[direct-runner-worker,5,main]
>> @FinishBundle  2048943483 1635355541635988 on 
>> Thread[direct-runner-worker,5,main]
>> @FinishBundle  701821764 1635355541635988 on 
>> Thread[direct-runner-worker,5,main]
>> @Teardown  701821764 1635355542059723 on Thread[direct-runner-worker,5,main]
>> @Teardown  2048943483 1635355542060234 on Thread[direct-runner-worker,5,main]
<< @Teardown  2048943483 1635355542202009 on Thread[direct-runner-worker,5,main]
<< @Teardown  701821764 1635355542201432 on Thread[direct-runner-worker,5,main]
testTeardownCalledAfterExceptionInFinishBundleStateful validating now 
{701821764=DelayedCallStateTracker{latch=java.util.concurrent.CountDownLatch@4986c59d[Count
 = 0], callState=TEARDOWN, callStateVisited=[SETUP, START_BUNDLE, 
PROCESS_ELEMENT, FINISH_BUNDLE, TEARDOWN]}, 
2048943483=DelayedCallStateTracker{latch=java.util.concurrent.CountDownLatch@d2aa355[Count
 = 0], callState=TEARDOWN, callStateVisited=[SETUP, START_BUNDLE, 
PROCESS_ELEMENT, FINISH_BUNDLE, TEARDOWN]}}
>> @StartBundle  470547472 1635355543922464 on 
>> Thread[direct-runner-worker,5,main]
>> @StartBundle  461909200 1635355543922155 on 
>> Thread[direct-runner-worker,5,main]
>> @Teardown  461909200 1635355543851521 on Thread[direct-runner-worker,5,main]
<< @StartBundle  470547472 1635355544019543 on 
Thread[direct-runner-worker,5,main]
<< @StartBundle  461909200 1635355544026384 on 
Thread[direct-runner-worker,5,main]
<< @Teardown  461909200 1635355544345618 on Thread[direct-runner-worker,5,main]
>> @Teardown  470547472 1635355544468740 on Thread[direct-runner-worker,5,main]
>> @ProcessElement  461909200 1635355544492043 on 
>> Thread[direct-runner-worker,5,main]
>> @FinishBundle  470547472 1635355544491199 on 
>> Thread[direct-runner-worker,5,main]
<< @Teardown  470547472 1635355544863639 on Thread[direct-runner-worker,5,main]
failing on 470547472
call state retraction at 1635355544889553 on 470547472: TEARDOWN -> 
FINISH_BUNDLE
-------------------------------------
java.lang.Exception: SETUP
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$DelayedCallStateTracker.<init>(ParDoLifecycleTest.java:377)
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$DelayedCallStateTracker.<init>(ParDoLifecycleTest.java:363)
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingFn.initCallState(ParDoLifecycleTest.java:514)
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingFn.before(ParDoLifecycleTest.java:448)
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingStatefulFn$DoFnInvoker.invokeSetup(Unknown
 Source)
        at 
org.apache.beam.sdk.transforms.reflect.DoFnInvokers.tryInvokeSetupFor(DoFnInvokers.java:52)
        at 
org.apache.beam.runners.direct.DoFnLifecycleManager$DeserializingCacheLoader.load(DoFnLifecycleManager.java:104)
        at 
org.apache.beam.runners.direct.DoFnLifecycleManager$DeserializingCacheLoader.load(DoFnLifecycleManager.java:91)
        at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3528)
        at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2277)
        at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2154)
        at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2044)
        at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache.get(LocalCache.java:3952)
        at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:3974)
        at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4958)
        at 
org.apache.beam.runners.direct.DoFnLifecycleManager.get(DoFnLifecycleManager.java:61)
        at 
org.apache.beam.runners.direct.ParDoEvaluatorFactory.createEvaluator(ParDoEvaluatorFactory.java:129)
        at 
org.apache.beam.runners.direct.StatefulParDoEvaluatorFactory.createEvaluator(StatefulParDoEvaluatorFactory.java:105)
        at 
org.apache.beam.runners.direct.StatefulParDoEvaluatorFactory.forApplication(StatefulParDoEvaluatorFactory.java:85)
        at 
org.apache.beam.runners.direct.TransformEvaluatorRegistry.forApplication(TransformEvaluatorRegistry.java:169)
        at 
org.apache.beam.runners.direct.DirectTransformExecutor.run(DirectTransformExecutor.java:117)
        at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at java.base/java.lang.Thread.run(Thread.java:834)
java.lang.Exception: START_BUNDLE
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$DelayedCallStateTracker.update(ParDoLifecycleTest.java:382)
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingFn.updateCallState(ParDoLifecycleTest.java:525)
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingFn.preBundle(ParDoLifecycleTest.java:461)
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingStatefulFn$DoFnInvoker.invokeStartBundle(Unknown
 Source)
        at 
org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner.startBundle(SimpleDoFnRunner.java:172)
        at 
org.apache.beam.repackaged.direct_java.runners.core.StatefulDoFnRunner.startBundle(StatefulDoFnRunner.java:124)
        at 
org.apache.beam.repackaged.direct_java.runners.core.SimplePushbackSideInputDoFnRunner.startBundle(SimplePushbackSideInputDoFnRunner.java:71)
        at 
org.apache.beam.runners.direct.ParDoEvaluator.<init>(ParDoEvaluator.java:219)
        at 
org.apache.beam.runners.direct.ParDoEvaluator.create(ParDoEvaluator.java:175)
        at 
org.apache.beam.runners.direct.ParDoEvaluator.create(ParDoEvaluator.java:167)
        at 
org.apache.beam.runners.direct.ParDoEvaluatorFactory.createParDoEvaluator(ParDoEvaluatorFactory.java:150)
        at 
org.apache.beam.runners.direct.ParDoEvaluatorFactory.createEvaluator(ParDoEvaluatorFactory.java:121)
        at 
org.apache.beam.runners.direct.StatefulParDoEvaluatorFactory.createEvaluator(StatefulParDoEvaluatorFactory.java:105)
        at 
org.apache.beam.runners.direct.StatefulParDoEvaluatorFactory.forApplication(StatefulParDoEvaluatorFactory.java:85)
        at 
org.apache.beam.runners.direct.TransformEvaluatorRegistry.forApplication(TransformEvaluatorRegistry.java:169)
        at 
org.apache.beam.runners.direct.DirectTransformExecutor.run(DirectTransformExecutor.java:117)
        at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at java.base/java.lang.Thread.run(Thread.java:834)
java.lang.Exception: TEARDOWN
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$DelayedCallStateTracker.update(ParDoLifecycleTest.java:382)
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingFn.updateCallState(ParDoLifecycleTest.java:525)
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingFn.after(ParDoLifecycleTest.java:508)
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingStatefulFn$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.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:85)
        at 
org.apache.beam.runners.direct.DoFnLifecycleManagers.removeAllFromManagers(DoFnLifecycleManagers.java:32)
        at 
org.apache.beam.runners.direct.ParDoEvaluatorFactory.cleanup(ParDoEvaluatorFactory.java:93)
        at 
org.apache.beam.runners.direct.StatefulParDoEvaluatorFactory.cleanup(StatefulParDoEvaluatorFactory.java:91)
        at 
org.apache.beam.runners.direct.TransformEvaluatorRegistry.cleanup(TransformEvaluatorRegistry.java:176)
        at 
org.apache.beam.runners.direct.ExecutorServiceParallelExecutor.shutdownIfNecessary(ExecutorServiceParallelExecutor.java:331)
        at 
org.apache.beam.runners.direct.ExecutorServiceParallelExecutor.access$200(ExecutorServiceParallelExecutor.java:59)
        at 
org.apache.beam.runners.direct.ExecutorServiceParallelExecutor$2.run(ExecutorServiceParallelExecutor.java:188)
        at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at java.base/java.lang.Thread.run(Thread.java:834)
java.lang.Exception: FINISH_BUNDLE
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$DelayedCallStateTracker.update(ParDoLifecycleTest.java:382)
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingFn.updateCallState(ParDoLifecycleTest.java:525)
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingFn.postBundle(ParDoLifecycleTest.java:485)
        at 
org.apache.beam.sdk.transforms.ParDoLifecycleTest$ExceptionThrowingStatefulFn$DoFnInvoker.invokeFinishBundle(Unknown
 Source)
        at 
org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner.finishBundle(SimpleDoFnRunner.java:237)
        at 
org.apache.beam.repackaged.direct_java.runners.core.StatefulDoFnRunner.finishBundle(StatefulDoFnRunner.java:129)
        at 
org.apache.beam.repackaged.direct_java.runners.core.SimplePushbackSideInputDoFnRunner.finishBundle(SimplePushbackSideInputDoFnRunner.java:124)
        at 
org.apache.beam.runners.direct.ParDoEvaluator.finishBundle(ParDoEvaluator.java:265)
        at 
org.apache.beam.runners.direct.DoFnLifecycleManagerRemovingTransformEvaluator.finishBundle(DoFnLifecycleManagerRemovingTransformEvaluator.java:73)
        at 
org.apache.beam.runners.direct.StatefulParDoEvaluatorFactory$StatefulParDoEvaluator.finishBundle(StatefulParDoEvaluatorFactory.java:203)
        at 
org.apache.beam.runners.direct.DirectTransformExecutor.finishBundle(DirectTransformExecutor.java:188)
        at 
org.apache.beam.runners.direct.DirectTransformExecutor.run(DirectTransformExecutor.java:126)
        at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at java.base/java.lang.Thread.run(Thread.java:834)
{noformat}


Hopefully that clarifies this a bit. 


> 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)

Reply via email to