You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@beam.apache.org by Niel Markwick <ni...@google.com> on 2022/06/14 16:16:12 UTC

Null PCollection errors in v2.40 unit tests

Developing on master/HEAD, I have a bunch of unit tests failing locally due
to Null PCollections

java.lang.NullPointerException: Null PCollection
	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)


eg for :sdks:java:io:google-cloud-platform:test, 271 failures out of 1338
tests, all for this reason.

These tests all pass locally when I checkout the  2.39 release tag, and
they don't fail in Jenkins-CI pre/post-commit tests either.

Doing a git bisect gives the commit 4ffeae4d2b - Update all links to in
progress jiras to issues (#21749)
<https://github.com/apache/beam/commit/4ffeae4d2b800f2df36d2ea2eab549f2204d5691>
as
the one where the tests start to fail (They succeed in the parent b0d964c4
<https://github.com/apache/beam/commit/b0d964c43092994977b10272451acc24b4a62aad>),
however the only changes in that commit appear to be comments.

I can reliably repro this - checking out 4ffeae4d2b, and running
:sdks:java:io:google-cloud-platform:test -- tests fail, checking
out b0d964c4 and they pass.

Any hints on what could be going on here?






-- 
<https://cloud.google.com>
* •  **Niel Markwick*
* •  *Cloud Solutions Architect <https://cloud.google.com/docs/tutorials>
* •  *Google Belgium
* •  *nielm@google.com
* •  *+32 2 894 6771

Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie.
RPR: 0878.065.378

If you have received this communication by mistake, please don't forward it
to anyone else (it may contain confidential or privileged information),
please erase all copies of it, including all attachments, and please let
the sender know it went to the wrong person. Thanks

Re: Null PCollection errors in v2.40 unit tests

Posted by Niel Markwick <ni...@google.com>.
It's never the compiler/runtime env... Until it is :)

Thanks for the follow-up!

On Mon, 20 Jun 2022, 17:00 Kenneth Knowles, <ke...@apache.org> wrote:

> Just to bring this back to the thread, it turns out to be a JDK bug
> https://bugs.openjdk.org/browse/JDK-8225377 reported in 2019 by +Liam
> Miller-Cushon <cu...@google.com> who is now also a contributor to Beam
> :-)
>
> We may be able to work around it somehow.
>
>
> On Thu, Jun 16, 2022 at 3:18 PM Kenneth Knowles <ke...@apache.org> wrote:
>
>> Filed https://github.com/google/auto/issues/1338
>>
>> On Thu, Jun 16, 2022 at 7:53 AM Niel Markwick <ni...@google.com> wrote:
>>
>>> yes, its AutoValue and Gradle being weird.
>>>
>>> In a 'good' build, the AutoValue generated code constructor does not
>>> check for null
>>>
>>> AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle(
>>>     @Nullable PCollection<T> PCollection,
>>>     StructuralKey<?> key,
>>>     Iterable<WindowedValue<T>> elements,
>>>     Instant minimumTimestamp,
>>>     Instant synchronizedProcessingOutputWatermark) {
>>>   this.PCollection = PCollection;
>>>
>>>
>>> in a 'bad' build, the autovalue java *is re-generated*, but has an
>>> added nullness check:
>>>
>>> AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle(
>>>     PCollection<T> PCollection,
>>>     StructuralKey<?> key,
>>>     Iterable<WindowedValue<T>> elements,
>>>     Instant minimumTimestamp,
>>>     Instant synchronizedProcessingOutputWatermark) {
>>>   if (PCollection == null) {
>>>     throw new NullPointerException("Null PCollection");
>>>   }
>>>   this.PCollection = PCollection;
>>>
>>>
>>> and then if I --rerun-tasks it gets re-re-generated without the
>>> nullness check.
>>>
>>> I have confirmed that getPCollection()
>>> in runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedBundle.java
>>> does not change (!)
>>>
>>> To repro I do:
>>>
>>> git checkout 4ffeae4d2b800f2df36d2ea2eab549f2204d5691~1
>>> ./gradlew :runners:direct-java:compileJava
>>> less
>>> ./runners/direct-java/build/generated/sources/annotationProcessor/java/main/org/apache/beam/runners/direct/AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java
>>>
>>> git checkout 4ffeae4d2b800f2df36d2ea2eab549f2204d5691
>>> ./gradlew :runners:direct-java:compileJava
>>> less
>>> ./runners/direct-java/build/generated/sources/annotationProcessor/java/main/org/apache/beam/runners/direct/AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java
>>>
>>> I am not sure why the particular commit
>>> 4ffeae4d2b800f2df36d2ea2eab549f2204d5691 triggers it, but that commit while
>>> comments-only-changes does modify some gradle build files
>>>
>>> It's weird. But now I know to --rerun-tasks to get a clean(er) build
>>>
>>>
>>> --
>>> <https://cloud.google.com>
>>> * •  **Niel Markwick*
>>> * •  *Cloud Solutions Architect
>>> <https://cloud.google.com/docs/tutorials>
>>> * •  *Google Belgium
>>> * •  *nielm@google.com
>>> * •  *+32 2 894 6771
>>>
>>>
>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>
>>> If you have received this communication by mistake, please don't forward
>>> it to anyone else (it may contain confidential or privileged information),
>>> please erase all copies of it, including all attachments, and please let
>>> the sender know it went to the wrong person. Thanks
>>>
>>>
>>> On Wed, 15 Jun 2022 at 15:42, Kenneth Knowles <ke...@apache.org> wrote:
>>>
>>>> This looks to be clearly a bug in autovalue. The PCollection field is
>>>> clearly labeled as @Nullable. Autovalue generates the
>>>> `checkArgument(pcollection != null)` for things that are not
>>>> marked @Nullable. It should not do so here.
>>>>
>>>> The bugs sounds like it is stateful, where some global state in the
>>>> autovalue annotation processor changes depending on what tasks were run
>>>> prior to generating the code.
>>>>
>>>> You could dig further by reproducing the bad build and then opening up
>>>> the autovalue generated code (`find
>>>> runners/direct-java/src AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java`
>>>> ).
>>>>
>>>> Kenn
>>>>
>>>> On Wed, Jun 15, 2022 at 12:50 AM Niel Markwick <ni...@google.com>
>>>> wrote:
>>>>
>>>>> Thank you!  `--rerun-tasks` fixed it for me as well!
>>>>>
>>>>> Bizarre issue, but then to misquote Clarke, a sufficiently advanced
>>>>> build system is indistinguishable from magic!
>>>>> I had tried ./gradlew clean and rm ~/.gradle, but that obviously
>>>>> didn't do enough cleaning!
>>>>>
>>>>> Now to get back to what I was doing before I disappeared down this
>>>>> rabbit hole - diagnosing/fixing flakey tests!
>>>>>
>>>>> --
>>>>> <https://cloud.google.com>
>>>>> * •  **Niel Markwick*
>>>>> * •  *Cloud Solutions Architect
>>>>> <https://cloud.google.com/docs/tutorials>
>>>>> * •  *Google Belgium
>>>>> * •  *nielm@google.com
>>>>> * •  *+32 2 894 6771
>>>>>
>>>>>
>>>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>>>
>>>>> If you have received this communication by mistake, please don't
>>>>> forward it to anyone else (it may contain confidential or privileged
>>>>> information), please erase all copies of it, including all attachments, and
>>>>> please let the sender know it went to the wrong person. Thanks
>>>>>
>>>>>
>>>>> On Tue, 14 Jun 2022 at 23:27, Steve Niemitz <sn...@apache.org>
>>>>> wrote:
>>>>>
>>>>>> I had brought up a weird issues I was having with AutoValue awhile
>>>>>> ago that looks actually very similar to this:
>>>>>> https://lists.apache.org/thread/0sbkykop2gsw71jpf3ln6forbnwq3j4o
>>>>>>
>>>>>> I never got to the bottom of it, but `--rerun-tasks` always fixes it
>>>>>> for me.
>>>>>>
>>>>>>
>>>>>> On Tue, Jun 14, 2022 at 5:11 PM Danny McCormick <
>>>>>> dannymccormick@google.com> wrote:
>>>>>>
>>>>>>> It seems like this may be specifically caused by jumping around to
>>>>>>> different commits, and Evan's solution seems like the right one. I got a
>>>>>>> clean vm and did:
>>>>>>>
>>>>>>> sudo apt install git openjdk-11-jdk
>>>>>>> git clone https://github.com/apache/beam.git
>>>>>>> cd beam
>>>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>>>
>>>>>>> tests pass
>>>>>>>
>>>>>>>
>>>>>>> git checkout b0d964c430
>>>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>>>
>>>>>>> tests fail (this is the one we would expect to pass)
>>>>>>>
>>>>>>>
>>>>>>> git checkout 4ffeae4d
>>>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>>>
>>>>>>> tests fail
>>>>>>>
>>>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test --rerun-tasks
>>>>>>>
>>>>>>> tests passed (this is still on the "bad commit")
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Danny
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Tue, Jun 14, 2022 at 3:56 PM Evan Galpin <eg...@apache.org>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> I had this happen to me recently as well.  After `git bisecting`
>>>>>>>> led to confusing results, I ran my tests again via gradlew adding
>>>>>>>> `--rerun-tasks` to the command.  This is an expensive operation, but after
>>>>>>>> I ran that I was able to test again with expected results. YMMV
>>>>>>>>
>>>>>>>> Thanks,
>>>>>>>> Evan
>>>>>>>>
>>>>>>>>
>>>>>>>> On Tue, Jun 14, 2022 at 2:12 PM Niel Markwick <ni...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> I agree that it is very strange!
>>>>>>>>>
>>>>>>>>> I have also just repro'd it on the cleanest possible environment:
>>>>>>>>> a brand new GCE debian 11 VM...
>>>>>>>>>
>>>>>>>>> sudo apt install git openjdk-11-jdk
>>>>>>>>> git clone https://github.com/apache/beam.git
>>>>>>>>> cd beam
>>>>>>>>> git checkout b0d964c430
>>>>>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>>>>>
>>>>>>>>> tests pass
>>>>>>>>>
>>>>>>>>> git checkout 4ffeae4d
>>>>>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>>>>>
>>>>>>>>> tests fail.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> The test failure stack traces are pretty much identical - the only
>>>>>>>>> difference being the test being run.
>>>>>>>>>
>>>>>>>>> They all complain about a Null PCollection from the directRunner
>>>>>>>>> (a couple complain due to incorrect expected exceptions, or asserts in a
>>>>>>>>> finally block, but they are failing because of the Null PCollection)
>>>>>>>>>
>>>>>>>>> I am not sure but I think the common ground _could_ be that a side
>>>>>>>>> input is used in the failing tests.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> org.apache.beam.sdk.Pipeline$PipelineExecutionException: java.lang.NullPointerException: Null PCollection
>>>>>>>>> 	at app//org.apache.beam.sdk.Pipeline.run(Pipeline.java:329)
>>>>>>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:399)
>>>>>>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:335)
>>>>>>>>> 	at app//org.apache.beam.sdk.io.gcp.spanner.SpannerIOWriteTest.deadlineExceededFailsAfterRetries(SpannerIOWriteTest.java:734)
>>>>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>>>>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>>>>>>> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
>>>>>>>>> 	at app//org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
>>>>>>>>> 	at app//org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>>>>>>>>> 	at app//org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
>>>>>>>>> 	at app//org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>>>>>>>>> 	at app//org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
>>>>>>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:323)
>>>>>>>>> 	at app//org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:258)
>>>>>>>>> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>>>>>>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
>>>>>>>>> 	at app//org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
>>>>>>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
>>>>>>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
>>>>>>>>> 	at app//org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
>>>>>>>>> 	at app//org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
>>>>>>>>> 	at app//org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
>>>>>>>>> 	at app//org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
>>>>>>>>> 	at app//org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
>>>>>>>>> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>>>>>>>>> 	at app//org.junit.runners.ParentRunner.run(ParentRunner.java:413)
>>>>>>>>> 	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 java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>>>>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>>>>>>> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
>>>>>>>>> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
>>>>>>>>> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
>>>>>>>>> 	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:33)
>>>>>>>>> 	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:94)
>>>>>>>>> 	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
>>>>>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker$2.run(TestWorker.java:176)
>>>>>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.executeAndMaintainThreadName(TestWorker.java:129)
>>>>>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:100)
>>>>>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:60)
>>>>>>>>> 	at org.gradle.process.internal.worker.child.ActionExecutionWorker.execute(ActionExecutionWorker.java:56)
>>>>>>>>> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:133)
>>>>>>>>> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:71)
>>>>>>>>> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.run(GradleWorkerMain.java:69)
>>>>>>>>> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.main(GradleWorkerMain.java:74)
>>>>>>>>> Caused by: java.lang.NullPointerException: Null PCollection
>>>>>>>>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>>>>>>>> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$CommittedImmutableListBundle.create(ImmutableListBundleFactory.java:137)
>>>>>>>>> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$UncommittedImmutableListBundle.commit(ImmutableListBundleFactory.java:115)
>>>>>>>>> 	at org.apache.beam.runners.direct.BoundedReadEvaluatorFactory$InputProvider.getInitialInputs(BoundedReadEvaluatorFactory.java:224)
>>>>>>>>> 	at org.apache.beam.runners.direct.ReadEvaluatorFactory$InputProvider.getInitialInputs(ReadEvaluatorFactory.java:88)
>>>>>>>>> 	at org.apache.beam.runners.direct.RootProviderRegistry.getInitialInputs(RootProviderRegistry.java:80)
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> --
>>>>>>>>> <https://cloud.google.com>
>>>>>>>>> * •  **Niel Markwick*
>>>>>>>>> * •  *Cloud Solutions Architect
>>>>>>>>> <https://cloud.google.com/docs/tutorials>
>>>>>>>>> * •  *Google Belgium
>>>>>>>>> * •  *nielm@google.com
>>>>>>>>> * •  *+32 2 894 6771
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>>>>>>>
>>>>>>>>> If you have received this communication by mistake, please don't
>>>>>>>>> forward it to anyone else (it may contain confidential or privileged
>>>>>>>>> information), please erase all copies of it, including all attachments, and
>>>>>>>>> please let the sender know it went to the wrong person. Thanks
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Tue, 14 Jun 2022 at 19:17, Danny McCormick <
>>>>>>>>> dannymccormick@google.com> wrote:
>>>>>>>>>
>>>>>>>>>> Hey Niel,
>>>>>>>>>>
>>>>>>>>>> Unfortunately I can't repro on master, and I'm a little bit at a
>>>>>>>>>> loss as to what could be causing this since the changes are all string
>>>>>>>>>> changes (like you mentioned). I also don't see any Jenkins failures related
>>>>>>>>>> to this (I *think* the Java PostCommits
>>>>>>>>>> <https://ci-beam.apache.org/job/beam_PostCommit_Java/> should
>>>>>>>>>> catch this generally). Any chance you could share some logs from your
>>>>>>>>>> failing test run? I'm interested if there's a correlation between the
>>>>>>>>>> failing tests (e.g. if they all call a specific function) or if there's
>>>>>>>>>> additional info in the stacktrace.
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> Danny
>>>>>>>>>>
>>>>>>>>>> On Tue, Jun 14, 2022 at 12:16 PM Niel Markwick <ni...@google.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Developing on master/HEAD, I have a bunch of unit tests failing
>>>>>>>>>>> locally due to Null PCollections
>>>>>>>>>>>
>>>>>>>>>>> java.lang.NullPointerException: Null PCollection
>>>>>>>>>>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> eg for :sdks:java:io:google-cloud-platform:test, 271 failures
>>>>>>>>>>> out of 1338 tests, all for this reason.
>>>>>>>>>>>
>>>>>>>>>>> These tests all pass locally when I checkout the  2.39 release
>>>>>>>>>>> tag, and they don't fail in Jenkins-CI pre/post-commit tests either.
>>>>>>>>>>>
>>>>>>>>>>> Doing a git bisect gives the commit 4ffeae4d2b - Update all
>>>>>>>>>>> links to in progress jiras to issues (#21749)
>>>>>>>>>>> <https://github.com/apache/beam/commit/4ffeae4d2b800f2df36d2ea2eab549f2204d5691> as
>>>>>>>>>>> the one where the tests start to fail (They succeed in the parent
>>>>>>>>>>> b0d964c4
>>>>>>>>>>> <https://github.com/apache/beam/commit/b0d964c43092994977b10272451acc24b4a62aad>),
>>>>>>>>>>> however the only changes in that commit appear to be comments.
>>>>>>>>>>>
>>>>>>>>>>> I can reliably repro this - checking out 4ffeae4d2b, and running
>>>>>>>>>>> :sdks:java:io:google-cloud-platform:test -- tests fail, checking
>>>>>>>>>>> out b0d964c4 and they pass.
>>>>>>>>>>>
>>>>>>>>>>> Any hints on what could be going on here?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> --
>>>>>>>>>>> <https://cloud.google.com>
>>>>>>>>>>> * •  **Niel Markwick*
>>>>>>>>>>> * •  *Cloud Solutions Architect
>>>>>>>>>>> <https://cloud.google.com/docs/tutorials>
>>>>>>>>>>> * •  *Google Belgium
>>>>>>>>>>> * •  *nielm@google.com
>>>>>>>>>>> * •  *+32 2 894 6771
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>>>>>>>>>
>>>>>>>>>>> If you have received this communication by mistake, please don't
>>>>>>>>>>> forward it to anyone else (it may contain confidential or privileged
>>>>>>>>>>> information), please erase all copies of it, including all attachments, and
>>>>>>>>>>> please let the sender know it went to the wrong person. Thanks
>>>>>>>>>>>
>>>>>>>>>>

Re: Null PCollection errors in v2.40 unit tests

Posted by Kenneth Knowles <ke...@apache.org>.
Just to bring this back to the thread, it turns out to be a JDK bug
https://bugs.openjdk.org/browse/JDK-8225377 reported in 2019 by +Liam
Miller-Cushon <cu...@google.com> who is now also a contributor to Beam :-)

We may be able to work around it somehow.


On Thu, Jun 16, 2022 at 3:18 PM Kenneth Knowles <ke...@apache.org> wrote:

> Filed https://github.com/google/auto/issues/1338
>
> On Thu, Jun 16, 2022 at 7:53 AM Niel Markwick <ni...@google.com> wrote:
>
>> yes, its AutoValue and Gradle being weird.
>>
>> In a 'good' build, the AutoValue generated code constructor does not
>> check for null
>>
>> AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle(
>>     @Nullable PCollection<T> PCollection,
>>     StructuralKey<?> key,
>>     Iterable<WindowedValue<T>> elements,
>>     Instant minimumTimestamp,
>>     Instant synchronizedProcessingOutputWatermark) {
>>   this.PCollection = PCollection;
>>
>>
>> in a 'bad' build, the autovalue java *is re-generated*, but has an added
>> nullness check:
>>
>> AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle(
>>     PCollection<T> PCollection,
>>     StructuralKey<?> key,
>>     Iterable<WindowedValue<T>> elements,
>>     Instant minimumTimestamp,
>>     Instant synchronizedProcessingOutputWatermark) {
>>   if (PCollection == null) {
>>     throw new NullPointerException("Null PCollection");
>>   }
>>   this.PCollection = PCollection;
>>
>>
>> and then if I --rerun-tasks it gets re-re-generated without the nullness
>> check.
>>
>> I have confirmed that getPCollection()
>> in runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedBundle.java
>> does not change (!)
>>
>> To repro I do:
>>
>> git checkout 4ffeae4d2b800f2df36d2ea2eab549f2204d5691~1
>> ./gradlew :runners:direct-java:compileJava
>> less
>> ./runners/direct-java/build/generated/sources/annotationProcessor/java/main/org/apache/beam/runners/direct/AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java
>>
>> git checkout 4ffeae4d2b800f2df36d2ea2eab549f2204d5691
>> ./gradlew :runners:direct-java:compileJava
>> less
>> ./runners/direct-java/build/generated/sources/annotationProcessor/java/main/org/apache/beam/runners/direct/AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java
>>
>> I am not sure why the particular commit
>> 4ffeae4d2b800f2df36d2ea2eab549f2204d5691 triggers it, but that commit while
>> comments-only-changes does modify some gradle build files
>>
>> It's weird. But now I know to --rerun-tasks to get a clean(er) build
>>
>>
>> --
>> <https://cloud.google.com>
>> * •  **Niel Markwick*
>> * •  *Cloud Solutions Architect <https://cloud.google.com/docs/tutorials>
>> * •  *Google Belgium
>> * •  *nielm@google.com
>> * •  *+32 2 894 6771
>>
>>
>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>
>> If you have received this communication by mistake, please don't forward
>> it to anyone else (it may contain confidential or privileged information),
>> please erase all copies of it, including all attachments, and please let
>> the sender know it went to the wrong person. Thanks
>>
>>
>> On Wed, 15 Jun 2022 at 15:42, Kenneth Knowles <ke...@apache.org> wrote:
>>
>>> This looks to be clearly a bug in autovalue. The PCollection field is
>>> clearly labeled as @Nullable. Autovalue generates the
>>> `checkArgument(pcollection != null)` for things that are not
>>> marked @Nullable. It should not do so here.
>>>
>>> The bugs sounds like it is stateful, where some global state in the
>>> autovalue annotation processor changes depending on what tasks were run
>>> prior to generating the code.
>>>
>>> You could dig further by reproducing the bad build and then opening up
>>> the autovalue generated code (`find
>>> runners/direct-java/src AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java`
>>> ).
>>>
>>> Kenn
>>>
>>> On Wed, Jun 15, 2022 at 12:50 AM Niel Markwick <ni...@google.com> wrote:
>>>
>>>> Thank you!  `--rerun-tasks` fixed it for me as well!
>>>>
>>>> Bizarre issue, but then to misquote Clarke, a sufficiently advanced
>>>> build system is indistinguishable from magic!
>>>> I had tried ./gradlew clean and rm ~/.gradle, but that obviously didn't
>>>> do enough cleaning!
>>>>
>>>> Now to get back to what I was doing before I disappeared down this
>>>> rabbit hole - diagnosing/fixing flakey tests!
>>>>
>>>> --
>>>> <https://cloud.google.com>
>>>> * •  **Niel Markwick*
>>>> * •  *Cloud Solutions Architect
>>>> <https://cloud.google.com/docs/tutorials>
>>>> * •  *Google Belgium
>>>> * •  *nielm@google.com
>>>> * •  *+32 2 894 6771
>>>>
>>>>
>>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>>
>>>> If you have received this communication by mistake, please don't
>>>> forward it to anyone else (it may contain confidential or privileged
>>>> information), please erase all copies of it, including all attachments, and
>>>> please let the sender know it went to the wrong person. Thanks
>>>>
>>>>
>>>> On Tue, 14 Jun 2022 at 23:27, Steve Niemitz <sn...@apache.org>
>>>> wrote:
>>>>
>>>>> I had brought up a weird issues I was having with AutoValue awhile ago
>>>>> that looks actually very similar to this:
>>>>> https://lists.apache.org/thread/0sbkykop2gsw71jpf3ln6forbnwq3j4o
>>>>>
>>>>> I never got to the bottom of it, but `--rerun-tasks` always fixes it
>>>>> for me.
>>>>>
>>>>>
>>>>> On Tue, Jun 14, 2022 at 5:11 PM Danny McCormick <
>>>>> dannymccormick@google.com> wrote:
>>>>>
>>>>>> It seems like this may be specifically caused by jumping around to
>>>>>> different commits, and Evan's solution seems like the right one. I got a
>>>>>> clean vm and did:
>>>>>>
>>>>>> sudo apt install git openjdk-11-jdk
>>>>>> git clone https://github.com/apache/beam.git
>>>>>> cd beam
>>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>>
>>>>>> tests pass
>>>>>>
>>>>>>
>>>>>> git checkout b0d964c430
>>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>>
>>>>>> tests fail (this is the one we would expect to pass)
>>>>>>
>>>>>>
>>>>>> git checkout 4ffeae4d
>>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>>
>>>>>> tests fail
>>>>>>
>>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test --rerun-tasks
>>>>>>
>>>>>> tests passed (this is still on the "bad commit")
>>>>>>
>>>>>> Thanks,
>>>>>> Danny
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Tue, Jun 14, 2022 at 3:56 PM Evan Galpin <eg...@apache.org>
>>>>>> wrote:
>>>>>>
>>>>>>> I had this happen to me recently as well.  After `git bisecting` led
>>>>>>> to confusing results, I ran my tests again via gradlew adding
>>>>>>> `--rerun-tasks` to the command.  This is an expensive operation, but after
>>>>>>> I ran that I was able to test again with expected results. YMMV
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Evan
>>>>>>>
>>>>>>>
>>>>>>> On Tue, Jun 14, 2022 at 2:12 PM Niel Markwick <ni...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> I agree that it is very strange!
>>>>>>>>
>>>>>>>> I have also just repro'd it on the cleanest possible environment: a
>>>>>>>> brand new GCE debian 11 VM...
>>>>>>>>
>>>>>>>> sudo apt install git openjdk-11-jdk
>>>>>>>> git clone https://github.com/apache/beam.git
>>>>>>>> cd beam
>>>>>>>> git checkout b0d964c430
>>>>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>>>>
>>>>>>>> tests pass
>>>>>>>>
>>>>>>>> git checkout 4ffeae4d
>>>>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>>>>
>>>>>>>> tests fail.
>>>>>>>>
>>>>>>>>
>>>>>>>> The test failure stack traces are pretty much identical - the only
>>>>>>>> difference being the test being run.
>>>>>>>>
>>>>>>>> They all complain about a Null PCollection from the directRunner (a
>>>>>>>> couple complain due to incorrect expected exceptions, or asserts in a
>>>>>>>> finally block, but they are failing because of the Null PCollection)
>>>>>>>>
>>>>>>>> I am not sure but I think the common ground _could_ be that a side
>>>>>>>> input is used in the failing tests.
>>>>>>>>
>>>>>>>>
>>>>>>>> org.apache.beam.sdk.Pipeline$PipelineExecutionException: java.lang.NullPointerException: Null PCollection
>>>>>>>> 	at app//org.apache.beam.sdk.Pipeline.run(Pipeline.java:329)
>>>>>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:399)
>>>>>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:335)
>>>>>>>> 	at app//org.apache.beam.sdk.io.gcp.spanner.SpannerIOWriteTest.deadlineExceededFailsAfterRetries(SpannerIOWriteTest.java:734)
>>>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>>>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>>>>>> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
>>>>>>>> 	at app//org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
>>>>>>>> 	at app//org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>>>>>>>> 	at app//org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
>>>>>>>> 	at app//org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>>>>>>>> 	at app//org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
>>>>>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:323)
>>>>>>>> 	at app//org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:258)
>>>>>>>> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>>>>>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
>>>>>>>> 	at app//org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
>>>>>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
>>>>>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
>>>>>>>> 	at app//org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
>>>>>>>> 	at app//org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
>>>>>>>> 	at app//org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
>>>>>>>> 	at app//org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
>>>>>>>> 	at app//org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
>>>>>>>> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>>>>>>>> 	at app//org.junit.runners.ParentRunner.run(ParentRunner.java:413)
>>>>>>>> 	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 java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>>>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>>>>>> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
>>>>>>>> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
>>>>>>>> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
>>>>>>>> 	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:33)
>>>>>>>> 	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:94)
>>>>>>>> 	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
>>>>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker$2.run(TestWorker.java:176)
>>>>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.executeAndMaintainThreadName(TestWorker.java:129)
>>>>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:100)
>>>>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:60)
>>>>>>>> 	at org.gradle.process.internal.worker.child.ActionExecutionWorker.execute(ActionExecutionWorker.java:56)
>>>>>>>> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:133)
>>>>>>>> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:71)
>>>>>>>> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.run(GradleWorkerMain.java:69)
>>>>>>>> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.main(GradleWorkerMain.java:74)
>>>>>>>> Caused by: java.lang.NullPointerException: Null PCollection
>>>>>>>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>>>>>>> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$CommittedImmutableListBundle.create(ImmutableListBundleFactory.java:137)
>>>>>>>> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$UncommittedImmutableListBundle.commit(ImmutableListBundleFactory.java:115)
>>>>>>>> 	at org.apache.beam.runners.direct.BoundedReadEvaluatorFactory$InputProvider.getInitialInputs(BoundedReadEvaluatorFactory.java:224)
>>>>>>>> 	at org.apache.beam.runners.direct.ReadEvaluatorFactory$InputProvider.getInitialInputs(ReadEvaluatorFactory.java:88)
>>>>>>>> 	at org.apache.beam.runners.direct.RootProviderRegistry.getInitialInputs(RootProviderRegistry.java:80)
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> --
>>>>>>>> <https://cloud.google.com>
>>>>>>>> * •  **Niel Markwick*
>>>>>>>> * •  *Cloud Solutions Architect
>>>>>>>> <https://cloud.google.com/docs/tutorials>
>>>>>>>> * •  *Google Belgium
>>>>>>>> * •  *nielm@google.com
>>>>>>>> * •  *+32 2 894 6771
>>>>>>>>
>>>>>>>>
>>>>>>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>>>>>>
>>>>>>>> If you have received this communication by mistake, please don't
>>>>>>>> forward it to anyone else (it may contain confidential or privileged
>>>>>>>> information), please erase all copies of it, including all attachments, and
>>>>>>>> please let the sender know it went to the wrong person. Thanks
>>>>>>>>
>>>>>>>>
>>>>>>>> On Tue, 14 Jun 2022 at 19:17, Danny McCormick <
>>>>>>>> dannymccormick@google.com> wrote:
>>>>>>>>
>>>>>>>>> Hey Niel,
>>>>>>>>>
>>>>>>>>> Unfortunately I can't repro on master, and I'm a little bit at a
>>>>>>>>> loss as to what could be causing this since the changes are all string
>>>>>>>>> changes (like you mentioned). I also don't see any Jenkins failures related
>>>>>>>>> to this (I *think* the Java PostCommits
>>>>>>>>> <https://ci-beam.apache.org/job/beam_PostCommit_Java/> should
>>>>>>>>> catch this generally). Any chance you could share some logs from your
>>>>>>>>> failing test run? I'm interested if there's a correlation between the
>>>>>>>>> failing tests (e.g. if they all call a specific function) or if there's
>>>>>>>>> additional info in the stacktrace.
>>>>>>>>>
>>>>>>>>> Thanks,
>>>>>>>>> Danny
>>>>>>>>>
>>>>>>>>> On Tue, Jun 14, 2022 at 12:16 PM Niel Markwick <ni...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Developing on master/HEAD, I have a bunch of unit tests failing
>>>>>>>>>> locally due to Null PCollections
>>>>>>>>>>
>>>>>>>>>> java.lang.NullPointerException: Null PCollection
>>>>>>>>>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> eg for :sdks:java:io:google-cloud-platform:test, 271 failures out
>>>>>>>>>> of 1338 tests, all for this reason.
>>>>>>>>>>
>>>>>>>>>> These tests all pass locally when I checkout the  2.39 release
>>>>>>>>>> tag, and they don't fail in Jenkins-CI pre/post-commit tests either.
>>>>>>>>>>
>>>>>>>>>> Doing a git bisect gives the commit 4ffeae4d2b - Update all
>>>>>>>>>> links to in progress jiras to issues (#21749)
>>>>>>>>>> <https://github.com/apache/beam/commit/4ffeae4d2b800f2df36d2ea2eab549f2204d5691> as
>>>>>>>>>> the one where the tests start to fail (They succeed in the parent
>>>>>>>>>> b0d964c4
>>>>>>>>>> <https://github.com/apache/beam/commit/b0d964c43092994977b10272451acc24b4a62aad>),
>>>>>>>>>> however the only changes in that commit appear to be comments.
>>>>>>>>>>
>>>>>>>>>> I can reliably repro this - checking out 4ffeae4d2b, and running
>>>>>>>>>> :sdks:java:io:google-cloud-platform:test -- tests fail, checking
>>>>>>>>>> out b0d964c4 and they pass.
>>>>>>>>>>
>>>>>>>>>> Any hints on what could be going on here?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> --
>>>>>>>>>> <https://cloud.google.com>
>>>>>>>>>> * •  **Niel Markwick*
>>>>>>>>>> * •  *Cloud Solutions Architect
>>>>>>>>>> <https://cloud.google.com/docs/tutorials>
>>>>>>>>>> * •  *Google Belgium
>>>>>>>>>> * •  *nielm@google.com
>>>>>>>>>> * •  *+32 2 894 6771
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>>>>>>>>
>>>>>>>>>> If you have received this communication by mistake, please don't
>>>>>>>>>> forward it to anyone else (it may contain confidential or privileged
>>>>>>>>>> information), please erase all copies of it, including all attachments, and
>>>>>>>>>> please let the sender know it went to the wrong person. Thanks
>>>>>>>>>>
>>>>>>>>>

Re: Null PCollection errors in v2.40 unit tests

Posted by Kenneth Knowles <ke...@apache.org>.
Filed https://github.com/google/auto/issues/1338

On Thu, Jun 16, 2022 at 7:53 AM Niel Markwick <ni...@google.com> wrote:

> yes, its AutoValue and Gradle being weird.
>
> In a 'good' build, the AutoValue generated code constructor does not check
> for null
>
> AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle(
>     @Nullable PCollection<T> PCollection,
>     StructuralKey<?> key,
>     Iterable<WindowedValue<T>> elements,
>     Instant minimumTimestamp,
>     Instant synchronizedProcessingOutputWatermark) {
>   this.PCollection = PCollection;
>
>
> in a 'bad' build, the autovalue java *is re-generated*, but has an added
> nullness check:
>
> AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle(
>     PCollection<T> PCollection,
>     StructuralKey<?> key,
>     Iterable<WindowedValue<T>> elements,
>     Instant minimumTimestamp,
>     Instant synchronizedProcessingOutputWatermark) {
>   if (PCollection == null) {
>     throw new NullPointerException("Null PCollection");
>   }
>   this.PCollection = PCollection;
>
>
> and then if I --rerun-tasks it gets re-re-generated without the nullness
> check.
>
> I have confirmed that getPCollection()
> in runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedBundle.java
> does not change (!)
>
> To repro I do:
>
> git checkout 4ffeae4d2b800f2df36d2ea2eab549f2204d5691~1
> ./gradlew :runners:direct-java:compileJava
> less
> ./runners/direct-java/build/generated/sources/annotationProcessor/java/main/org/apache/beam/runners/direct/AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java
>
> git checkout 4ffeae4d2b800f2df36d2ea2eab549f2204d5691
> ./gradlew :runners:direct-java:compileJava
> less
> ./runners/direct-java/build/generated/sources/annotationProcessor/java/main/org/apache/beam/runners/direct/AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java
>
> I am not sure why the particular commit
> 4ffeae4d2b800f2df36d2ea2eab549f2204d5691 triggers it, but that commit while
> comments-only-changes does modify some gradle build files
>
> It's weird. But now I know to --rerun-tasks to get a clean(er) build
>
>
> --
> <https://cloud.google.com>
> * •  **Niel Markwick*
> * •  *Cloud Solutions Architect <https://cloud.google.com/docs/tutorials>
> * •  *Google Belgium
> * •  *nielm@google.com
> * •  *+32 2 894 6771
>
>
> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>
> If you have received this communication by mistake, please don't forward
> it to anyone else (it may contain confidential or privileged information),
> please erase all copies of it, including all attachments, and please let
> the sender know it went to the wrong person. Thanks
>
>
> On Wed, 15 Jun 2022 at 15:42, Kenneth Knowles <ke...@apache.org> wrote:
>
>> This looks to be clearly a bug in autovalue. The PCollection field is
>> clearly labeled as @Nullable. Autovalue generates the
>> `checkArgument(pcollection != null)` for things that are not
>> marked @Nullable. It should not do so here.
>>
>> The bugs sounds like it is stateful, where some global state in the
>> autovalue annotation processor changes depending on what tasks were run
>> prior to generating the code.
>>
>> You could dig further by reproducing the bad build and then opening up
>> the autovalue generated code (`find
>> runners/direct-java/src AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java`
>> ).
>>
>> Kenn
>>
>> On Wed, Jun 15, 2022 at 12:50 AM Niel Markwick <ni...@google.com> wrote:
>>
>>> Thank you!  `--rerun-tasks` fixed it for me as well!
>>>
>>> Bizarre issue, but then to misquote Clarke, a sufficiently advanced
>>> build system is indistinguishable from magic!
>>> I had tried ./gradlew clean and rm ~/.gradle, but that obviously didn't
>>> do enough cleaning!
>>>
>>> Now to get back to what I was doing before I disappeared down this
>>> rabbit hole - diagnosing/fixing flakey tests!
>>>
>>> --
>>> <https://cloud.google.com>
>>> * •  **Niel Markwick*
>>> * •  *Cloud Solutions Architect
>>> <https://cloud.google.com/docs/tutorials>
>>> * •  *Google Belgium
>>> * •  *nielm@google.com
>>> * •  *+32 2 894 6771
>>>
>>>
>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>
>>> If you have received this communication by mistake, please don't forward
>>> it to anyone else (it may contain confidential or privileged information),
>>> please erase all copies of it, including all attachments, and please let
>>> the sender know it went to the wrong person. Thanks
>>>
>>>
>>> On Tue, 14 Jun 2022 at 23:27, Steve Niemitz <sn...@apache.org> wrote:
>>>
>>>> I had brought up a weird issues I was having with AutoValue awhile ago
>>>> that looks actually very similar to this:
>>>> https://lists.apache.org/thread/0sbkykop2gsw71jpf3ln6forbnwq3j4o
>>>>
>>>> I never got to the bottom of it, but `--rerun-tasks` always fixes it
>>>> for me.
>>>>
>>>>
>>>> On Tue, Jun 14, 2022 at 5:11 PM Danny McCormick <
>>>> dannymccormick@google.com> wrote:
>>>>
>>>>> It seems like this may be specifically caused by jumping around to
>>>>> different commits, and Evan's solution seems like the right one. I got a
>>>>> clean vm and did:
>>>>>
>>>>> sudo apt install git openjdk-11-jdk
>>>>> git clone https://github.com/apache/beam.git
>>>>> cd beam
>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>
>>>>> tests pass
>>>>>
>>>>>
>>>>> git checkout b0d964c430
>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>
>>>>> tests fail (this is the one we would expect to pass)
>>>>>
>>>>>
>>>>> git checkout 4ffeae4d
>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>
>>>>> tests fail
>>>>>
>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test --rerun-tasks
>>>>>
>>>>> tests passed (this is still on the "bad commit")
>>>>>
>>>>> Thanks,
>>>>> Danny
>>>>>
>>>>>
>>>>>
>>>>> On Tue, Jun 14, 2022 at 3:56 PM Evan Galpin <eg...@apache.org>
>>>>> wrote:
>>>>>
>>>>>> I had this happen to me recently as well.  After `git bisecting` led
>>>>>> to confusing results, I ran my tests again via gradlew adding
>>>>>> `--rerun-tasks` to the command.  This is an expensive operation, but after
>>>>>> I ran that I was able to test again with expected results. YMMV
>>>>>>
>>>>>> Thanks,
>>>>>> Evan
>>>>>>
>>>>>>
>>>>>> On Tue, Jun 14, 2022 at 2:12 PM Niel Markwick <ni...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> I agree that it is very strange!
>>>>>>>
>>>>>>> I have also just repro'd it on the cleanest possible environment: a
>>>>>>> brand new GCE debian 11 VM...
>>>>>>>
>>>>>>> sudo apt install git openjdk-11-jdk
>>>>>>> git clone https://github.com/apache/beam.git
>>>>>>> cd beam
>>>>>>> git checkout b0d964c430
>>>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>>>
>>>>>>> tests pass
>>>>>>>
>>>>>>> git checkout 4ffeae4d
>>>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>>>
>>>>>>> tests fail.
>>>>>>>
>>>>>>>
>>>>>>> The test failure stack traces are pretty much identical - the only
>>>>>>> difference being the test being run.
>>>>>>>
>>>>>>> They all complain about a Null PCollection from the directRunner (a
>>>>>>> couple complain due to incorrect expected exceptions, or asserts in a
>>>>>>> finally block, but they are failing because of the Null PCollection)
>>>>>>>
>>>>>>> I am not sure but I think the common ground _could_ be that a side
>>>>>>> input is used in the failing tests.
>>>>>>>
>>>>>>>
>>>>>>> org.apache.beam.sdk.Pipeline$PipelineExecutionException: java.lang.NullPointerException: Null PCollection
>>>>>>> 	at app//org.apache.beam.sdk.Pipeline.run(Pipeline.java:329)
>>>>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:399)
>>>>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:335)
>>>>>>> 	at app//org.apache.beam.sdk.io.gcp.spanner.SpannerIOWriteTest.deadlineExceededFailsAfterRetries(SpannerIOWriteTest.java:734)
>>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>>>>> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
>>>>>>> 	at app//org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
>>>>>>> 	at app//org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>>>>>>> 	at app//org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
>>>>>>> 	at app//org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>>>>>>> 	at app//org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
>>>>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:323)
>>>>>>> 	at app//org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:258)
>>>>>>> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>>>>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
>>>>>>> 	at app//org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
>>>>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
>>>>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
>>>>>>> 	at app//org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
>>>>>>> 	at app//org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
>>>>>>> 	at app//org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
>>>>>>> 	at app//org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
>>>>>>> 	at app//org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
>>>>>>> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>>>>>>> 	at app//org.junit.runners.ParentRunner.run(ParentRunner.java:413)
>>>>>>> 	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 java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>>>>> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
>>>>>>> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
>>>>>>> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
>>>>>>> 	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:33)
>>>>>>> 	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:94)
>>>>>>> 	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
>>>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker$2.run(TestWorker.java:176)
>>>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.executeAndMaintainThreadName(TestWorker.java:129)
>>>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:100)
>>>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:60)
>>>>>>> 	at org.gradle.process.internal.worker.child.ActionExecutionWorker.execute(ActionExecutionWorker.java:56)
>>>>>>> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:133)
>>>>>>> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:71)
>>>>>>> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.run(GradleWorkerMain.java:69)
>>>>>>> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.main(GradleWorkerMain.java:74)
>>>>>>> Caused by: java.lang.NullPointerException: Null PCollection
>>>>>>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>>>>>> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$CommittedImmutableListBundle.create(ImmutableListBundleFactory.java:137)
>>>>>>> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$UncommittedImmutableListBundle.commit(ImmutableListBundleFactory.java:115)
>>>>>>> 	at org.apache.beam.runners.direct.BoundedReadEvaluatorFactory$InputProvider.getInitialInputs(BoundedReadEvaluatorFactory.java:224)
>>>>>>> 	at org.apache.beam.runners.direct.ReadEvaluatorFactory$InputProvider.getInitialInputs(ReadEvaluatorFactory.java:88)
>>>>>>> 	at org.apache.beam.runners.direct.RootProviderRegistry.getInitialInputs(RootProviderRegistry.java:80)
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>> <https://cloud.google.com>
>>>>>>> * •  **Niel Markwick*
>>>>>>> * •  *Cloud Solutions Architect
>>>>>>> <https://cloud.google.com/docs/tutorials>
>>>>>>> * •  *Google Belgium
>>>>>>> * •  *nielm@google.com
>>>>>>> * •  *+32 2 894 6771
>>>>>>>
>>>>>>>
>>>>>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>>>>>
>>>>>>> If you have received this communication by mistake, please don't
>>>>>>> forward it to anyone else (it may contain confidential or privileged
>>>>>>> information), please erase all copies of it, including all attachments, and
>>>>>>> please let the sender know it went to the wrong person. Thanks
>>>>>>>
>>>>>>>
>>>>>>> On Tue, 14 Jun 2022 at 19:17, Danny McCormick <
>>>>>>> dannymccormick@google.com> wrote:
>>>>>>>
>>>>>>>> Hey Niel,
>>>>>>>>
>>>>>>>> Unfortunately I can't repro on master, and I'm a little bit at a
>>>>>>>> loss as to what could be causing this since the changes are all string
>>>>>>>> changes (like you mentioned). I also don't see any Jenkins failures related
>>>>>>>> to this (I *think* the Java PostCommits
>>>>>>>> <https://ci-beam.apache.org/job/beam_PostCommit_Java/> should
>>>>>>>> catch this generally). Any chance you could share some logs from your
>>>>>>>> failing test run? I'm interested if there's a correlation between the
>>>>>>>> failing tests (e.g. if they all call a specific function) or if there's
>>>>>>>> additional info in the stacktrace.
>>>>>>>>
>>>>>>>> Thanks,
>>>>>>>> Danny
>>>>>>>>
>>>>>>>> On Tue, Jun 14, 2022 at 12:16 PM Niel Markwick <ni...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Developing on master/HEAD, I have a bunch of unit tests failing
>>>>>>>>> locally due to Null PCollections
>>>>>>>>>
>>>>>>>>> java.lang.NullPointerException: Null PCollection
>>>>>>>>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> eg for :sdks:java:io:google-cloud-platform:test, 271 failures out
>>>>>>>>> of 1338 tests, all for this reason.
>>>>>>>>>
>>>>>>>>> These tests all pass locally when I checkout the  2.39 release
>>>>>>>>> tag, and they don't fail in Jenkins-CI pre/post-commit tests either.
>>>>>>>>>
>>>>>>>>> Doing a git bisect gives the commit 4ffeae4d2b - Update all links
>>>>>>>>> to in progress jiras to issues (#21749)
>>>>>>>>> <https://github.com/apache/beam/commit/4ffeae4d2b800f2df36d2ea2eab549f2204d5691> as
>>>>>>>>> the one where the tests start to fail (They succeed in the parent
>>>>>>>>> b0d964c4
>>>>>>>>> <https://github.com/apache/beam/commit/b0d964c43092994977b10272451acc24b4a62aad>),
>>>>>>>>> however the only changes in that commit appear to be comments.
>>>>>>>>>
>>>>>>>>> I can reliably repro this - checking out 4ffeae4d2b, and running
>>>>>>>>> :sdks:java:io:google-cloud-platform:test -- tests fail, checking
>>>>>>>>> out b0d964c4 and they pass.
>>>>>>>>>
>>>>>>>>> Any hints on what could be going on here?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> --
>>>>>>>>> <https://cloud.google.com>
>>>>>>>>> * •  **Niel Markwick*
>>>>>>>>> * •  *Cloud Solutions Architect
>>>>>>>>> <https://cloud.google.com/docs/tutorials>
>>>>>>>>> * •  *Google Belgium
>>>>>>>>> * •  *nielm@google.com
>>>>>>>>> * •  *+32 2 894 6771
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>>>>>>>
>>>>>>>>> If you have received this communication by mistake, please don't
>>>>>>>>> forward it to anyone else (it may contain confidential or privileged
>>>>>>>>> information), please erase all copies of it, including all attachments, and
>>>>>>>>> please let the sender know it went to the wrong person. Thanks
>>>>>>>>>
>>>>>>>>

Re: Null PCollection errors in v2.40 unit tests

Posted by Niel Markwick <ni...@google.com>.
yes, its AutoValue and Gradle being weird.

In a 'good' build, the AutoValue generated code constructor does not check
for null

AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle(
    @Nullable PCollection<T> PCollection,
    StructuralKey<?> key,
    Iterable<WindowedValue<T>> elements,
    Instant minimumTimestamp,
    Instant synchronizedProcessingOutputWatermark) {
  this.PCollection = PCollection;


in a 'bad' build, the autovalue java *is re-generated*, but has an added
nullness check:

AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle(
    PCollection<T> PCollection,
    StructuralKey<?> key,
    Iterable<WindowedValue<T>> elements,
    Instant minimumTimestamp,
    Instant synchronizedProcessingOutputWatermark) {
  if (PCollection == null) {
    throw new NullPointerException("Null PCollection");
  }
  this.PCollection = PCollection;


and then if I --rerun-tasks it gets re-re-generated without the nullness
check.

I have confirmed that getPCollection()
in runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedBundle.java
does not change (!)

To repro I do:

git checkout 4ffeae4d2b800f2df36d2ea2eab549f2204d5691~1
./gradlew :runners:direct-java:compileJava
less
./runners/direct-java/build/generated/sources/annotationProcessor/java/main/org/apache/beam/runners/direct/AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java

git checkout 4ffeae4d2b800f2df36d2ea2eab549f2204d5691
./gradlew :runners:direct-java:compileJava
less
./runners/direct-java/build/generated/sources/annotationProcessor/java/main/org/apache/beam/runners/direct/AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java

I am not sure why the particular commit
4ffeae4d2b800f2df36d2ea2eab549f2204d5691 triggers it, but that commit while
comments-only-changes does modify some gradle build files

It's weird. But now I know to --rerun-tasks to get a clean(er) build


-- 
<https://cloud.google.com>
* •  **Niel Markwick*
* •  *Cloud Solutions Architect <https://cloud.google.com/docs/tutorials>
* •  *Google Belgium
* •  *nielm@google.com
* •  *+32 2 894 6771

Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie.
RPR: 0878.065.378

If you have received this communication by mistake, please don't forward it
to anyone else (it may contain confidential or privileged information),
please erase all copies of it, including all attachments, and please let
the sender know it went to the wrong person. Thanks


On Wed, 15 Jun 2022 at 15:42, Kenneth Knowles <ke...@apache.org> wrote:

> This looks to be clearly a bug in autovalue. The PCollection field is
> clearly labeled as @Nullable. Autovalue generates the
> `checkArgument(pcollection != null)` for things that are not
> marked @Nullable. It should not do so here.
>
> The bugs sounds like it is stateful, where some global state in the
> autovalue annotation processor changes depending on what tasks were run
> prior to generating the code.
>
> You could dig further by reproducing the bad build and then opening up the
> autovalue generated code (`find
> runners/direct-java/src AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java`
> ).
>
> Kenn
>
> On Wed, Jun 15, 2022 at 12:50 AM Niel Markwick <ni...@google.com> wrote:
>
>> Thank you!  `--rerun-tasks` fixed it for me as well!
>>
>> Bizarre issue, but then to misquote Clarke, a sufficiently advanced build
>> system is indistinguishable from magic!
>> I had tried ./gradlew clean and rm ~/.gradle, but that obviously didn't
>> do enough cleaning!
>>
>> Now to get back to what I was doing before I disappeared down this
>> rabbit hole - diagnosing/fixing flakey tests!
>>
>> --
>> <https://cloud.google.com>
>> * •  **Niel Markwick*
>> * •  *Cloud Solutions Architect <https://cloud.google.com/docs/tutorials>
>> * •  *Google Belgium
>> * •  *nielm@google.com
>> * •  *+32 2 894 6771
>>
>>
>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>
>> If you have received this communication by mistake, please don't forward
>> it to anyone else (it may contain confidential or privileged information),
>> please erase all copies of it, including all attachments, and please let
>> the sender know it went to the wrong person. Thanks
>>
>>
>> On Tue, 14 Jun 2022 at 23:27, Steve Niemitz <sn...@apache.org> wrote:
>>
>>> I had brought up a weird issues I was having with AutoValue awhile ago
>>> that looks actually very similar to this:
>>> https://lists.apache.org/thread/0sbkykop2gsw71jpf3ln6forbnwq3j4o
>>>
>>> I never got to the bottom of it, but `--rerun-tasks` always fixes it for
>>> me.
>>>
>>>
>>> On Tue, Jun 14, 2022 at 5:11 PM Danny McCormick <
>>> dannymccormick@google.com> wrote:
>>>
>>>> It seems like this may be specifically caused by jumping around to
>>>> different commits, and Evan's solution seems like the right one. I got a
>>>> clean vm and did:
>>>>
>>>> sudo apt install git openjdk-11-jdk
>>>> git clone https://github.com/apache/beam.git
>>>> cd beam
>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>
>>>> tests pass
>>>>
>>>>
>>>> git checkout b0d964c430
>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>
>>>> tests fail (this is the one we would expect to pass)
>>>>
>>>>
>>>> git checkout 4ffeae4d
>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>
>>>> tests fail
>>>>
>>>> ./gradlew :sdks:java:io:google-cloud-platform:test --rerun-tasks
>>>>
>>>> tests passed (this is still on the "bad commit")
>>>>
>>>> Thanks,
>>>> Danny
>>>>
>>>>
>>>>
>>>> On Tue, Jun 14, 2022 at 3:56 PM Evan Galpin <eg...@apache.org> wrote:
>>>>
>>>>> I had this happen to me recently as well.  After `git bisecting` led
>>>>> to confusing results, I ran my tests again via gradlew adding
>>>>> `--rerun-tasks` to the command.  This is an expensive operation, but after
>>>>> I ran that I was able to test again with expected results. YMMV
>>>>>
>>>>> Thanks,
>>>>> Evan
>>>>>
>>>>>
>>>>> On Tue, Jun 14, 2022 at 2:12 PM Niel Markwick <ni...@google.com>
>>>>> wrote:
>>>>>
>>>>>> I agree that it is very strange!
>>>>>>
>>>>>> I have also just repro'd it on the cleanest possible environment: a
>>>>>> brand new GCE debian 11 VM...
>>>>>>
>>>>>> sudo apt install git openjdk-11-jdk
>>>>>> git clone https://github.com/apache/beam.git
>>>>>> cd beam
>>>>>> git checkout b0d964c430
>>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>>
>>>>>> tests pass
>>>>>>
>>>>>> git checkout 4ffeae4d
>>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>>
>>>>>> tests fail.
>>>>>>
>>>>>>
>>>>>> The test failure stack traces are pretty much identical - the only
>>>>>> difference being the test being run.
>>>>>>
>>>>>> They all complain about a Null PCollection from the directRunner (a
>>>>>> couple complain due to incorrect expected exceptions, or asserts in a
>>>>>> finally block, but they are failing because of the Null PCollection)
>>>>>>
>>>>>> I am not sure but I think the common ground _could_ be that a side
>>>>>> input is used in the failing tests.
>>>>>>
>>>>>>
>>>>>> org.apache.beam.sdk.Pipeline$PipelineExecutionException: java.lang.NullPointerException: Null PCollection
>>>>>> 	at app//org.apache.beam.sdk.Pipeline.run(Pipeline.java:329)
>>>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:399)
>>>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:335)
>>>>>> 	at app//org.apache.beam.sdk.io.gcp.spanner.SpannerIOWriteTest.deadlineExceededFailsAfterRetries(SpannerIOWriteTest.java:734)
>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>>>> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
>>>>>> 	at app//org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
>>>>>> 	at app//org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>>>>>> 	at app//org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
>>>>>> 	at app//org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>>>>>> 	at app//org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
>>>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:323)
>>>>>> 	at app//org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:258)
>>>>>> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>>>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
>>>>>> 	at app//org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
>>>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
>>>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
>>>>>> 	at app//org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
>>>>>> 	at app//org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
>>>>>> 	at app//org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
>>>>>> 	at app//org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
>>>>>> 	at app//org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
>>>>>> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>>>>>> 	at app//org.junit.runners.ParentRunner.run(ParentRunner.java:413)
>>>>>> 	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 java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>>>>>> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>>>> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
>>>>>> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
>>>>>> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
>>>>>> 	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:33)
>>>>>> 	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:94)
>>>>>> 	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
>>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker$2.run(TestWorker.java:176)
>>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.executeAndMaintainThreadName(TestWorker.java:129)
>>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:100)
>>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:60)
>>>>>> 	at org.gradle.process.internal.worker.child.ActionExecutionWorker.execute(ActionExecutionWorker.java:56)
>>>>>> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:133)
>>>>>> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:71)
>>>>>> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.run(GradleWorkerMain.java:69)
>>>>>> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.main(GradleWorkerMain.java:74)
>>>>>> Caused by: java.lang.NullPointerException: Null PCollection
>>>>>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>>>>> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$CommittedImmutableListBundle.create(ImmutableListBundleFactory.java:137)
>>>>>> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$UncommittedImmutableListBundle.commit(ImmutableListBundleFactory.java:115)
>>>>>> 	at org.apache.beam.runners.direct.BoundedReadEvaluatorFactory$InputProvider.getInitialInputs(BoundedReadEvaluatorFactory.java:224)
>>>>>> 	at org.apache.beam.runners.direct.ReadEvaluatorFactory$InputProvider.getInitialInputs(ReadEvaluatorFactory.java:88)
>>>>>> 	at org.apache.beam.runners.direct.RootProviderRegistry.getInitialInputs(RootProviderRegistry.java:80)
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> --
>>>>>> <https://cloud.google.com>
>>>>>> * •  **Niel Markwick*
>>>>>> * •  *Cloud Solutions Architect
>>>>>> <https://cloud.google.com/docs/tutorials>
>>>>>> * •  *Google Belgium
>>>>>> * •  *nielm@google.com
>>>>>> * •  *+32 2 894 6771
>>>>>>
>>>>>>
>>>>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>>>>
>>>>>> If you have received this communication by mistake, please don't
>>>>>> forward it to anyone else (it may contain confidential or privileged
>>>>>> information), please erase all copies of it, including all attachments, and
>>>>>> please let the sender know it went to the wrong person. Thanks
>>>>>>
>>>>>>
>>>>>> On Tue, 14 Jun 2022 at 19:17, Danny McCormick <
>>>>>> dannymccormick@google.com> wrote:
>>>>>>
>>>>>>> Hey Niel,
>>>>>>>
>>>>>>> Unfortunately I can't repro on master, and I'm a little bit at a
>>>>>>> loss as to what could be causing this since the changes are all string
>>>>>>> changes (like you mentioned). I also don't see any Jenkins failures related
>>>>>>> to this (I *think* the Java PostCommits
>>>>>>> <https://ci-beam.apache.org/job/beam_PostCommit_Java/> should catch
>>>>>>> this generally). Any chance you could share some logs from your failing
>>>>>>> test run? I'm interested if there's a correlation between the failing tests
>>>>>>> (e.g. if they all call a specific function) or if there's additional info
>>>>>>> in the stacktrace.
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Danny
>>>>>>>
>>>>>>> On Tue, Jun 14, 2022 at 12:16 PM Niel Markwick <ni...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Developing on master/HEAD, I have a bunch of unit tests failing
>>>>>>>> locally due to Null PCollections
>>>>>>>>
>>>>>>>> java.lang.NullPointerException: Null PCollection
>>>>>>>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>>>>>>>
>>>>>>>>
>>>>>>>> eg for :sdks:java:io:google-cloud-platform:test, 271 failures out
>>>>>>>> of 1338 tests, all for this reason.
>>>>>>>>
>>>>>>>> These tests all pass locally when I checkout the  2.39 release tag,
>>>>>>>> and they don't fail in Jenkins-CI pre/post-commit tests either.
>>>>>>>>
>>>>>>>> Doing a git bisect gives the commit 4ffeae4d2b - Update all links
>>>>>>>> to in progress jiras to issues (#21749)
>>>>>>>> <https://github.com/apache/beam/commit/4ffeae4d2b800f2df36d2ea2eab549f2204d5691> as
>>>>>>>> the one where the tests start to fail (They succeed in the parent
>>>>>>>> b0d964c4
>>>>>>>> <https://github.com/apache/beam/commit/b0d964c43092994977b10272451acc24b4a62aad>),
>>>>>>>> however the only changes in that commit appear to be comments.
>>>>>>>>
>>>>>>>> I can reliably repro this - checking out 4ffeae4d2b, and running
>>>>>>>> :sdks:java:io:google-cloud-platform:test -- tests fail, checking
>>>>>>>> out b0d964c4 and they pass.
>>>>>>>>
>>>>>>>> Any hints on what could be going on here?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> --
>>>>>>>> <https://cloud.google.com>
>>>>>>>> * •  **Niel Markwick*
>>>>>>>> * •  *Cloud Solutions Architect
>>>>>>>> <https://cloud.google.com/docs/tutorials>
>>>>>>>> * •  *Google Belgium
>>>>>>>> * •  *nielm@google.com
>>>>>>>> * •  *+32 2 894 6771
>>>>>>>>
>>>>>>>>
>>>>>>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>>>>>>
>>>>>>>> If you have received this communication by mistake, please don't
>>>>>>>> forward it to anyone else (it may contain confidential or privileged
>>>>>>>> information), please erase all copies of it, including all attachments, and
>>>>>>>> please let the sender know it went to the wrong person. Thanks
>>>>>>>>
>>>>>>>

Re: Null PCollection errors in v2.40 unit tests

Posted by Kenneth Knowles <ke...@apache.org>.
This looks to be clearly a bug in autovalue. The PCollection field is
clearly labeled as @Nullable. Autovalue generates the
`checkArgument(pcollection != null)` for things that are not
marked @Nullable. It should not do so here.

The bugs sounds like it is stateful, where some global state in the
autovalue annotation processor changes depending on what tasks were run
prior to generating the code.

You could dig further by reproducing the bad build and then opening up the
autovalue generated code (`find
runners/direct-java/src
AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java`
).

Kenn

On Wed, Jun 15, 2022 at 12:50 AM Niel Markwick <ni...@google.com> wrote:

> Thank you!  `--rerun-tasks` fixed it for me as well!
>
> Bizarre issue, but then to misquote Clarke, a sufficiently advanced build
> system is indistinguishable from magic!
> I had tried ./gradlew clean and rm ~/.gradle, but that obviously didn't do
> enough cleaning!
>
> Now to get back to what I was doing before I disappeared down this
> rabbit hole - diagnosing/fixing flakey tests!
>
> --
> <https://cloud.google.com>
> * •  **Niel Markwick*
> * •  *Cloud Solutions Architect <https://cloud.google.com/docs/tutorials>
> * •  *Google Belgium
> * •  *nielm@google.com
> * •  *+32 2 894 6771
>
>
> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>
> If you have received this communication by mistake, please don't forward
> it to anyone else (it may contain confidential or privileged information),
> please erase all copies of it, including all attachments, and please let
> the sender know it went to the wrong person. Thanks
>
>
> On Tue, 14 Jun 2022 at 23:27, Steve Niemitz <sn...@apache.org> wrote:
>
>> I had brought up a weird issues I was having with AutoValue awhile ago
>> that looks actually very similar to this:
>> https://lists.apache.org/thread/0sbkykop2gsw71jpf3ln6forbnwq3j4o
>>
>> I never got to the bottom of it, but `--rerun-tasks` always fixes it for
>> me.
>>
>>
>> On Tue, Jun 14, 2022 at 5:11 PM Danny McCormick <
>> dannymccormick@google.com> wrote:
>>
>>> It seems like this may be specifically caused by jumping around to
>>> different commits, and Evan's solution seems like the right one. I got a
>>> clean vm and did:
>>>
>>> sudo apt install git openjdk-11-jdk
>>> git clone https://github.com/apache/beam.git
>>> cd beam
>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>
>>> tests pass
>>>
>>>
>>> git checkout b0d964c430
>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>
>>> tests fail (this is the one we would expect to pass)
>>>
>>>
>>> git checkout 4ffeae4d
>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>
>>> tests fail
>>>
>>> ./gradlew :sdks:java:io:google-cloud-platform:test --rerun-tasks
>>>
>>> tests passed (this is still on the "bad commit")
>>>
>>> Thanks,
>>> Danny
>>>
>>>
>>>
>>> On Tue, Jun 14, 2022 at 3:56 PM Evan Galpin <eg...@apache.org> wrote:
>>>
>>>> I had this happen to me recently as well.  After `git bisecting` led to
>>>> confusing results, I ran my tests again via gradlew adding `--rerun-tasks`
>>>> to the command.  This is an expensive operation, but after I ran that I was
>>>> able to test again with expected results. YMMV
>>>>
>>>> Thanks,
>>>> Evan
>>>>
>>>>
>>>> On Tue, Jun 14, 2022 at 2:12 PM Niel Markwick <ni...@google.com> wrote:
>>>>
>>>>> I agree that it is very strange!
>>>>>
>>>>> I have also just repro'd it on the cleanest possible environment: a
>>>>> brand new GCE debian 11 VM...
>>>>>
>>>>> sudo apt install git openjdk-11-jdk
>>>>> git clone https://github.com/apache/beam.git
>>>>> cd beam
>>>>> git checkout b0d964c430
>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>
>>>>> tests pass
>>>>>
>>>>> git checkout 4ffeae4d
>>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>>
>>>>> tests fail.
>>>>>
>>>>>
>>>>> The test failure stack traces are pretty much identical - the only
>>>>> difference being the test being run.
>>>>>
>>>>> They all complain about a Null PCollection from the directRunner (a
>>>>> couple complain due to incorrect expected exceptions, or asserts in a
>>>>> finally block, but they are failing because of the Null PCollection)
>>>>>
>>>>> I am not sure but I think the common ground _could_ be that a side
>>>>> input is used in the failing tests.
>>>>>
>>>>>
>>>>> org.apache.beam.sdk.Pipeline$PipelineExecutionException: java.lang.NullPointerException: Null PCollection
>>>>> 	at app//org.apache.beam.sdk.Pipeline.run(Pipeline.java:329)
>>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:399)
>>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:335)
>>>>> 	at app//org.apache.beam.sdk.io.gcp.spanner.SpannerIOWriteTest.deadlineExceededFailsAfterRetries(SpannerIOWriteTest.java:734)
>>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>>>>> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>>> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
>>>>> 	at app//org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
>>>>> 	at app//org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>>>>> 	at app//org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
>>>>> 	at app//org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>>>>> 	at app//org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
>>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:323)
>>>>> 	at app//org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:258)
>>>>> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
>>>>> 	at app//org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
>>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
>>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
>>>>> 	at app//org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
>>>>> 	at app//org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
>>>>> 	at app//org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
>>>>> 	at app//org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
>>>>> 	at app//org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
>>>>> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>>>>> 	at app//org.junit.runners.ParentRunner.run(ParentRunner.java:413)
>>>>> 	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 java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>>>>> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>>> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
>>>>> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
>>>>> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
>>>>> 	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:33)
>>>>> 	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:94)
>>>>> 	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker$2.run(TestWorker.java:176)
>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.executeAndMaintainThreadName(TestWorker.java:129)
>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:100)
>>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:60)
>>>>> 	at org.gradle.process.internal.worker.child.ActionExecutionWorker.execute(ActionExecutionWorker.java:56)
>>>>> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:133)
>>>>> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:71)
>>>>> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.run(GradleWorkerMain.java:69)
>>>>> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.main(GradleWorkerMain.java:74)
>>>>> Caused by: java.lang.NullPointerException: Null PCollection
>>>>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>>>> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$CommittedImmutableListBundle.create(ImmutableListBundleFactory.java:137)
>>>>> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$UncommittedImmutableListBundle.commit(ImmutableListBundleFactory.java:115)
>>>>> 	at org.apache.beam.runners.direct.BoundedReadEvaluatorFactory$InputProvider.getInitialInputs(BoundedReadEvaluatorFactory.java:224)
>>>>> 	at org.apache.beam.runners.direct.ReadEvaluatorFactory$InputProvider.getInitialInputs(ReadEvaluatorFactory.java:88)
>>>>> 	at org.apache.beam.runners.direct.RootProviderRegistry.getInitialInputs(RootProviderRegistry.java:80)
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> <https://cloud.google.com>
>>>>> * •  **Niel Markwick*
>>>>> * •  *Cloud Solutions Architect
>>>>> <https://cloud.google.com/docs/tutorials>
>>>>> * •  *Google Belgium
>>>>> * •  *nielm@google.com
>>>>> * •  *+32 2 894 6771
>>>>>
>>>>>
>>>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>>>
>>>>> If you have received this communication by mistake, please don't
>>>>> forward it to anyone else (it may contain confidential or privileged
>>>>> information), please erase all copies of it, including all attachments, and
>>>>> please let the sender know it went to the wrong person. Thanks
>>>>>
>>>>>
>>>>> On Tue, 14 Jun 2022 at 19:17, Danny McCormick <
>>>>> dannymccormick@google.com> wrote:
>>>>>
>>>>>> Hey Niel,
>>>>>>
>>>>>> Unfortunately I can't repro on master, and I'm a little bit at a loss
>>>>>> as to what could be causing this since the changes are all string changes
>>>>>> (like you mentioned). I also don't see any Jenkins failures related to this
>>>>>> (I *think* the Java PostCommits
>>>>>> <https://ci-beam.apache.org/job/beam_PostCommit_Java/> should catch
>>>>>> this generally). Any chance you could share some logs from your failing
>>>>>> test run? I'm interested if there's a correlation between the failing tests
>>>>>> (e.g. if they all call a specific function) or if there's additional info
>>>>>> in the stacktrace.
>>>>>>
>>>>>> Thanks,
>>>>>> Danny
>>>>>>
>>>>>> On Tue, Jun 14, 2022 at 12:16 PM Niel Markwick <ni...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Developing on master/HEAD, I have a bunch of unit tests failing
>>>>>>> locally due to Null PCollections
>>>>>>>
>>>>>>> java.lang.NullPointerException: Null PCollection
>>>>>>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>>>>>>
>>>>>>>
>>>>>>> eg for :sdks:java:io:google-cloud-platform:test, 271 failures out of
>>>>>>> 1338 tests, all for this reason.
>>>>>>>
>>>>>>> These tests all pass locally when I checkout the  2.39 release tag,
>>>>>>> and they don't fail in Jenkins-CI pre/post-commit tests either.
>>>>>>>
>>>>>>> Doing a git bisect gives the commit 4ffeae4d2b - Update all links
>>>>>>> to in progress jiras to issues (#21749)
>>>>>>> <https://github.com/apache/beam/commit/4ffeae4d2b800f2df36d2ea2eab549f2204d5691> as
>>>>>>> the one where the tests start to fail (They succeed in the parent
>>>>>>> b0d964c4
>>>>>>> <https://github.com/apache/beam/commit/b0d964c43092994977b10272451acc24b4a62aad>),
>>>>>>> however the only changes in that commit appear to be comments.
>>>>>>>
>>>>>>> I can reliably repro this - checking out 4ffeae4d2b, and running
>>>>>>> :sdks:java:io:google-cloud-platform:test -- tests fail, checking
>>>>>>> out b0d964c4 and they pass.
>>>>>>>
>>>>>>> Any hints on what could be going on here?
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>> <https://cloud.google.com>
>>>>>>> * •  **Niel Markwick*
>>>>>>> * •  *Cloud Solutions Architect
>>>>>>> <https://cloud.google.com/docs/tutorials>
>>>>>>> * •  *Google Belgium
>>>>>>> * •  *nielm@google.com
>>>>>>> * •  *+32 2 894 6771
>>>>>>>
>>>>>>>
>>>>>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>>>>>
>>>>>>> If you have received this communication by mistake, please don't
>>>>>>> forward it to anyone else (it may contain confidential or privileged
>>>>>>> information), please erase all copies of it, including all attachments, and
>>>>>>> please let the sender know it went to the wrong person. Thanks
>>>>>>>
>>>>>>

Re: Null PCollection errors in v2.40 unit tests

Posted by Niel Markwick <ni...@google.com>.
Thank you!  `--rerun-tasks` fixed it for me as well!

Bizarre issue, but then to misquote Clarke, a sufficiently advanced build
system is indistinguishable from magic!
I had tried ./gradlew clean and rm ~/.gradle, but that obviously didn't do
enough cleaning!

Now to get back to what I was doing before I disappeared down this
rabbit hole - diagnosing/fixing flakey tests!

-- 
<https://cloud.google.com>
* •  **Niel Markwick*
* •  *Cloud Solutions Architect <https://cloud.google.com/docs/tutorials>
* •  *Google Belgium
* •  *nielm@google.com
* •  *+32 2 894 6771

Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie.
RPR: 0878.065.378

If you have received this communication by mistake, please don't forward it
to anyone else (it may contain confidential or privileged information),
please erase all copies of it, including all attachments, and please let
the sender know it went to the wrong person. Thanks


On Tue, 14 Jun 2022 at 23:27, Steve Niemitz <sn...@apache.org> wrote:

> I had brought up a weird issues I was having with AutoValue awhile ago
> that looks actually very similar to this:
> https://lists.apache.org/thread/0sbkykop2gsw71jpf3ln6forbnwq3j4o
>
> I never got to the bottom of it, but `--rerun-tasks` always fixes it for
> me.
>
>
> On Tue, Jun 14, 2022 at 5:11 PM Danny McCormick <da...@google.com>
> wrote:
>
>> It seems like this may be specifically caused by jumping around to
>> different commits, and Evan's solution seems like the right one. I got a
>> clean vm and did:
>>
>> sudo apt install git openjdk-11-jdk
>> git clone https://github.com/apache/beam.git
>> cd beam
>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>
>> tests pass
>>
>>
>> git checkout b0d964c430
>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>
>> tests fail (this is the one we would expect to pass)
>>
>>
>> git checkout 4ffeae4d
>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>
>> tests fail
>>
>> ./gradlew :sdks:java:io:google-cloud-platform:test --rerun-tasks
>>
>> tests passed (this is still on the "bad commit")
>>
>> Thanks,
>> Danny
>>
>>
>>
>> On Tue, Jun 14, 2022 at 3:56 PM Evan Galpin <eg...@apache.org> wrote:
>>
>>> I had this happen to me recently as well.  After `git bisecting` led to
>>> confusing results, I ran my tests again via gradlew adding `--rerun-tasks`
>>> to the command.  This is an expensive operation, but after I ran that I was
>>> able to test again with expected results. YMMV
>>>
>>> Thanks,
>>> Evan
>>>
>>>
>>> On Tue, Jun 14, 2022 at 2:12 PM Niel Markwick <ni...@google.com> wrote:
>>>
>>>> I agree that it is very strange!
>>>>
>>>> I have also just repro'd it on the cleanest possible environment: a
>>>> brand new GCE debian 11 VM...
>>>>
>>>> sudo apt install git openjdk-11-jdk
>>>> git clone https://github.com/apache/beam.git
>>>> cd beam
>>>> git checkout b0d964c430
>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>
>>>> tests pass
>>>>
>>>> git checkout 4ffeae4d
>>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>>
>>>> tests fail.
>>>>
>>>>
>>>> The test failure stack traces are pretty much identical - the only
>>>> difference being the test being run.
>>>>
>>>> They all complain about a Null PCollection from the directRunner (a
>>>> couple complain due to incorrect expected exceptions, or asserts in a
>>>> finally block, but they are failing because of the Null PCollection)
>>>>
>>>> I am not sure but I think the common ground _could_ be that a side
>>>> input is used in the failing tests.
>>>>
>>>>
>>>> org.apache.beam.sdk.Pipeline$PipelineExecutionException: java.lang.NullPointerException: Null PCollection
>>>> 	at app//org.apache.beam.sdk.Pipeline.run(Pipeline.java:329)
>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:399)
>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:335)
>>>> 	at app//org.apache.beam.sdk.io.gcp.spanner.SpannerIOWriteTest.deadlineExceededFailsAfterRetries(SpannerIOWriteTest.java:734)
>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>>>> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
>>>> 	at app//org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
>>>> 	at app//org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>>>> 	at app//org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
>>>> 	at app//org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>>>> 	at app//org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
>>>> 	at app//org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:323)
>>>> 	at app//org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:258)
>>>> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
>>>> 	at app//org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
>>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
>>>> 	at app//org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
>>>> 	at app//org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
>>>> 	at app//org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
>>>> 	at app//org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
>>>> 	at app//org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
>>>> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>>>> 	at app//org.junit.runners.ParentRunner.run(ParentRunner.java:413)
>>>> 	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 java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>>>> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
>>>> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
>>>> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
>>>> 	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:33)
>>>> 	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:94)
>>>> 	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker$2.run(TestWorker.java:176)
>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.executeAndMaintainThreadName(TestWorker.java:129)
>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:100)
>>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:60)
>>>> 	at org.gradle.process.internal.worker.child.ActionExecutionWorker.execute(ActionExecutionWorker.java:56)
>>>> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:133)
>>>> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:71)
>>>> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.run(GradleWorkerMain.java:69)
>>>> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.main(GradleWorkerMain.java:74)
>>>> Caused by: java.lang.NullPointerException: Null PCollection
>>>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>>> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$CommittedImmutableListBundle.create(ImmutableListBundleFactory.java:137)
>>>> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$UncommittedImmutableListBundle.commit(ImmutableListBundleFactory.java:115)
>>>> 	at org.apache.beam.runners.direct.BoundedReadEvaluatorFactory$InputProvider.getInitialInputs(BoundedReadEvaluatorFactory.java:224)
>>>> 	at org.apache.beam.runners.direct.ReadEvaluatorFactory$InputProvider.getInitialInputs(ReadEvaluatorFactory.java:88)
>>>> 	at org.apache.beam.runners.direct.RootProviderRegistry.getInitialInputs(RootProviderRegistry.java:80)
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> --
>>>> <https://cloud.google.com>
>>>> * •  **Niel Markwick*
>>>> * •  *Cloud Solutions Architect
>>>> <https://cloud.google.com/docs/tutorials>
>>>> * •  *Google Belgium
>>>> * •  *nielm@google.com
>>>> * •  *+32 2 894 6771
>>>>
>>>>
>>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>>
>>>> If you have received this communication by mistake, please don't
>>>> forward it to anyone else (it may contain confidential or privileged
>>>> information), please erase all copies of it, including all attachments, and
>>>> please let the sender know it went to the wrong person. Thanks
>>>>
>>>>
>>>> On Tue, 14 Jun 2022 at 19:17, Danny McCormick <
>>>> dannymccormick@google.com> wrote:
>>>>
>>>>> Hey Niel,
>>>>>
>>>>> Unfortunately I can't repro on master, and I'm a little bit at a loss
>>>>> as to what could be causing this since the changes are all string changes
>>>>> (like you mentioned). I also don't see any Jenkins failures related to this
>>>>> (I *think* the Java PostCommits
>>>>> <https://ci-beam.apache.org/job/beam_PostCommit_Java/> should catch
>>>>> this generally). Any chance you could share some logs from your failing
>>>>> test run? I'm interested if there's a correlation between the failing tests
>>>>> (e.g. if they all call a specific function) or if there's additional info
>>>>> in the stacktrace.
>>>>>
>>>>> Thanks,
>>>>> Danny
>>>>>
>>>>> On Tue, Jun 14, 2022 at 12:16 PM Niel Markwick <ni...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Developing on master/HEAD, I have a bunch of unit tests failing
>>>>>> locally due to Null PCollections
>>>>>>
>>>>>> java.lang.NullPointerException: Null PCollection
>>>>>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>>>>>
>>>>>>
>>>>>> eg for :sdks:java:io:google-cloud-platform:test, 271 failures out of
>>>>>> 1338 tests, all for this reason.
>>>>>>
>>>>>> These tests all pass locally when I checkout the  2.39 release tag,
>>>>>> and they don't fail in Jenkins-CI pre/post-commit tests either.
>>>>>>
>>>>>> Doing a git bisect gives the commit 4ffeae4d2b - Update all links to
>>>>>> in progress jiras to issues (#21749)
>>>>>> <https://github.com/apache/beam/commit/4ffeae4d2b800f2df36d2ea2eab549f2204d5691> as
>>>>>> the one where the tests start to fail (They succeed in the parent
>>>>>> b0d964c4
>>>>>> <https://github.com/apache/beam/commit/b0d964c43092994977b10272451acc24b4a62aad>),
>>>>>> however the only changes in that commit appear to be comments.
>>>>>>
>>>>>> I can reliably repro this - checking out 4ffeae4d2b, and running
>>>>>> :sdks:java:io:google-cloud-platform:test -- tests fail, checking
>>>>>> out b0d964c4 and they pass.
>>>>>>
>>>>>> Any hints on what could be going on here?
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> --
>>>>>> <https://cloud.google.com>
>>>>>> * •  **Niel Markwick*
>>>>>> * •  *Cloud Solutions Architect
>>>>>> <https://cloud.google.com/docs/tutorials>
>>>>>> * •  *Google Belgium
>>>>>> * •  *nielm@google.com
>>>>>> * •  *+32 2 894 6771
>>>>>>
>>>>>>
>>>>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>>>>
>>>>>> If you have received this communication by mistake, please don't
>>>>>> forward it to anyone else (it may contain confidential or privileged
>>>>>> information), please erase all copies of it, including all attachments, and
>>>>>> please let the sender know it went to the wrong person. Thanks
>>>>>>
>>>>>

Re: Null PCollection errors in v2.40 unit tests

Posted by Steve Niemitz <sn...@apache.org>.
I had brought up a weird issues I was having with AutoValue awhile ago that
looks actually very similar to this:
https://lists.apache.org/thread/0sbkykop2gsw71jpf3ln6forbnwq3j4o

I never got to the bottom of it, but `--rerun-tasks` always fixes it for me.


On Tue, Jun 14, 2022 at 5:11 PM Danny McCormick <da...@google.com>
wrote:

> It seems like this may be specifically caused by jumping around to
> different commits, and Evan's solution seems like the right one. I got a
> clean vm and did:
>
> sudo apt install git openjdk-11-jdk
> git clone https://github.com/apache/beam.git
> cd beam
> ./gradlew :sdks:java:io:google-cloud-platform:test
>
> tests pass
>
>
> git checkout b0d964c430
> ./gradlew :sdks:java:io:google-cloud-platform:test
>
> tests fail (this is the one we would expect to pass)
>
>
> git checkout 4ffeae4d
> ./gradlew :sdks:java:io:google-cloud-platform:test
>
> tests fail
>
> ./gradlew :sdks:java:io:google-cloud-platform:test --rerun-tasks
>
> tests passed (this is still on the "bad commit")
>
> Thanks,
> Danny
>
>
>
> On Tue, Jun 14, 2022 at 3:56 PM Evan Galpin <eg...@apache.org> wrote:
>
>> I had this happen to me recently as well.  After `git bisecting` led to
>> confusing results, I ran my tests again via gradlew adding `--rerun-tasks`
>> to the command.  This is an expensive operation, but after I ran that I was
>> able to test again with expected results. YMMV
>>
>> Thanks,
>> Evan
>>
>>
>> On Tue, Jun 14, 2022 at 2:12 PM Niel Markwick <ni...@google.com> wrote:
>>
>>> I agree that it is very strange!
>>>
>>> I have also just repro'd it on the cleanest possible environment: a
>>> brand new GCE debian 11 VM...
>>>
>>> sudo apt install git openjdk-11-jdk
>>> git clone https://github.com/apache/beam.git
>>> cd beam
>>> git checkout b0d964c430
>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>
>>> tests pass
>>>
>>> git checkout 4ffeae4d
>>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>>
>>> tests fail.
>>>
>>>
>>> The test failure stack traces are pretty much identical - the only
>>> difference being the test being run.
>>>
>>> They all complain about a Null PCollection from the directRunner (a
>>> couple complain due to incorrect expected exceptions, or asserts in a
>>> finally block, but they are failing because of the Null PCollection)
>>>
>>> I am not sure but I think the common ground _could_ be that a side input
>>> is used in the failing tests.
>>>
>>>
>>> org.apache.beam.sdk.Pipeline$PipelineExecutionException: java.lang.NullPointerException: Null PCollection
>>> 	at app//org.apache.beam.sdk.Pipeline.run(Pipeline.java:329)
>>> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:399)
>>> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:335)
>>> 	at app//org.apache.beam.sdk.io.gcp.spanner.SpannerIOWriteTest.deadlineExceededFailsAfterRetries(SpannerIOWriteTest.java:734)
>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>>> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
>>> 	at app//org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
>>> 	at app//org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>>> 	at app//org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
>>> 	at app//org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>>> 	at app//org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
>>> 	at app//org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:323)
>>> 	at app//org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:258)
>>> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
>>> 	at app//org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
>>> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
>>> 	at app//org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
>>> 	at app//org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
>>> 	at app//org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
>>> 	at app//org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
>>> 	at app//org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
>>> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>>> 	at app//org.junit.runners.ParentRunner.run(ParentRunner.java:413)
>>> 	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 java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>>> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
>>> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
>>> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
>>> 	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:33)
>>> 	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:94)
>>> 	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker$2.run(TestWorker.java:176)
>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.executeAndMaintainThreadName(TestWorker.java:129)
>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:100)
>>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:60)
>>> 	at org.gradle.process.internal.worker.child.ActionExecutionWorker.execute(ActionExecutionWorker.java:56)
>>> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:133)
>>> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:71)
>>> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.run(GradleWorkerMain.java:69)
>>> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.main(GradleWorkerMain.java:74)
>>> Caused by: java.lang.NullPointerException: Null PCollection
>>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$CommittedImmutableListBundle.create(ImmutableListBundleFactory.java:137)
>>> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$UncommittedImmutableListBundle.commit(ImmutableListBundleFactory.java:115)
>>> 	at org.apache.beam.runners.direct.BoundedReadEvaluatorFactory$InputProvider.getInitialInputs(BoundedReadEvaluatorFactory.java:224)
>>> 	at org.apache.beam.runners.direct.ReadEvaluatorFactory$InputProvider.getInitialInputs(ReadEvaluatorFactory.java:88)
>>> 	at org.apache.beam.runners.direct.RootProviderRegistry.getInitialInputs(RootProviderRegistry.java:80)
>>>
>>>
>>>
>>>
>>>
>>>
>>> --
>>> <https://cloud.google.com>
>>> * •  **Niel Markwick*
>>> * •  *Cloud Solutions Architect
>>> <https://cloud.google.com/docs/tutorials>
>>> * •  *Google Belgium
>>> * •  *nielm@google.com
>>> * •  *+32 2 894 6771
>>>
>>>
>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>
>>> If you have received this communication by mistake, please don't forward
>>> it to anyone else (it may contain confidential or privileged information),
>>> please erase all copies of it, including all attachments, and please let
>>> the sender know it went to the wrong person. Thanks
>>>
>>>
>>> On Tue, 14 Jun 2022 at 19:17, Danny McCormick <da...@google.com>
>>> wrote:
>>>
>>>> Hey Niel,
>>>>
>>>> Unfortunately I can't repro on master, and I'm a little bit at a loss
>>>> as to what could be causing this since the changes are all string changes
>>>> (like you mentioned). I also don't see any Jenkins failures related to this
>>>> (I *think* the Java PostCommits
>>>> <https://ci-beam.apache.org/job/beam_PostCommit_Java/> should catch
>>>> this generally). Any chance you could share some logs from your failing
>>>> test run? I'm interested if there's a correlation between the failing tests
>>>> (e.g. if they all call a specific function) or if there's additional info
>>>> in the stacktrace.
>>>>
>>>> Thanks,
>>>> Danny
>>>>
>>>> On Tue, Jun 14, 2022 at 12:16 PM Niel Markwick <ni...@google.com>
>>>> wrote:
>>>>
>>>>> Developing on master/HEAD, I have a bunch of unit tests failing
>>>>> locally due to Null PCollections
>>>>>
>>>>> java.lang.NullPointerException: Null PCollection
>>>>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>>>>
>>>>>
>>>>> eg for :sdks:java:io:google-cloud-platform:test, 271 failures out of
>>>>> 1338 tests, all for this reason.
>>>>>
>>>>> These tests all pass locally when I checkout the  2.39 release tag,
>>>>> and they don't fail in Jenkins-CI pre/post-commit tests either.
>>>>>
>>>>> Doing a git bisect gives the commit 4ffeae4d2b - Update all links to
>>>>> in progress jiras to issues (#21749)
>>>>> <https://github.com/apache/beam/commit/4ffeae4d2b800f2df36d2ea2eab549f2204d5691> as
>>>>> the one where the tests start to fail (They succeed in the parent
>>>>> b0d964c4
>>>>> <https://github.com/apache/beam/commit/b0d964c43092994977b10272451acc24b4a62aad>),
>>>>> however the only changes in that commit appear to be comments.
>>>>>
>>>>> I can reliably repro this - checking out 4ffeae4d2b, and running
>>>>> :sdks:java:io:google-cloud-platform:test -- tests fail, checking
>>>>> out b0d964c4 and they pass.
>>>>>
>>>>> Any hints on what could be going on here?
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> <https://cloud.google.com>
>>>>> * •  **Niel Markwick*
>>>>> * •  *Cloud Solutions Architect
>>>>> <https://cloud.google.com/docs/tutorials>
>>>>> * •  *Google Belgium
>>>>> * •  *nielm@google.com
>>>>> * •  *+32 2 894 6771
>>>>>
>>>>>
>>>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>>>
>>>>> If you have received this communication by mistake, please don't
>>>>> forward it to anyone else (it may contain confidential or privileged
>>>>> information), please erase all copies of it, including all attachments, and
>>>>> please let the sender know it went to the wrong person. Thanks
>>>>>
>>>>

Re: Null PCollection errors in v2.40 unit tests

Posted by Danny McCormick <da...@google.com>.
It seems like this may be specifically caused by jumping around to
different commits, and Evan's solution seems like the right one. I got a
clean vm and did:

sudo apt install git openjdk-11-jdk
git clone https://github.com/apache/beam.git
cd beam
./gradlew :sdks:java:io:google-cloud-platform:test

tests pass


git checkout b0d964c430
./gradlew :sdks:java:io:google-cloud-platform:test

tests fail (this is the one we would expect to pass)


git checkout 4ffeae4d
./gradlew :sdks:java:io:google-cloud-platform:test

tests fail

./gradlew :sdks:java:io:google-cloud-platform:test --rerun-tasks

tests passed (this is still on the "bad commit")

Thanks,
Danny



On Tue, Jun 14, 2022 at 3:56 PM Evan Galpin <eg...@apache.org> wrote:

> I had this happen to me recently as well.  After `git bisecting` led to
> confusing results, I ran my tests again via gradlew adding `--rerun-tasks`
> to the command.  This is an expensive operation, but after I ran that I was
> able to test again with expected results. YMMV
>
> Thanks,
> Evan
>
>
> On Tue, Jun 14, 2022 at 2:12 PM Niel Markwick <ni...@google.com> wrote:
>
>> I agree that it is very strange!
>>
>> I have also just repro'd it on the cleanest possible environment: a brand
>> new GCE debian 11 VM...
>>
>> sudo apt install git openjdk-11-jdk
>> git clone https://github.com/apache/beam.git
>> cd beam
>> git checkout b0d964c430
>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>
>> tests pass
>>
>> git checkout 4ffeae4d
>> ./gradlew :sdks:java:io:google-cloud-platform:test
>>
>> tests fail.
>>
>>
>> The test failure stack traces are pretty much identical - the only
>> difference being the test being run.
>>
>> They all complain about a Null PCollection from the directRunner (a
>> couple complain due to incorrect expected exceptions, or asserts in a
>> finally block, but they are failing because of the Null PCollection)
>>
>> I am not sure but I think the common ground _could_ be that a side input
>> is used in the failing tests.
>>
>>
>> org.apache.beam.sdk.Pipeline$PipelineExecutionException: java.lang.NullPointerException: Null PCollection
>> 	at app//org.apache.beam.sdk.Pipeline.run(Pipeline.java:329)
>> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:399)
>> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:335)
>> 	at app//org.apache.beam.sdk.io.gcp.spanner.SpannerIOWriteTest.deadlineExceededFailsAfterRetries(SpannerIOWriteTest.java:734)
>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
>> 	at app//org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
>> 	at app//org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>> 	at app//org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
>> 	at app//org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>> 	at app//org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
>> 	at app//org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:323)
>> 	at app//org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:258)
>> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>> 	at app//org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
>> 	at app//org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
>> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
>> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
>> 	at app//org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
>> 	at app//org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
>> 	at app//org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
>> 	at app//org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
>> 	at app//org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
>> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
>> 	at app//org.junit.runners.ParentRunner.run(ParentRunner.java:413)
>> 	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 java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
>> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
>> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
>> 	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:33)
>> 	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:94)
>> 	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker$2.run(TestWorker.java:176)
>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.executeAndMaintainThreadName(TestWorker.java:129)
>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:100)
>> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:60)
>> 	at org.gradle.process.internal.worker.child.ActionExecutionWorker.execute(ActionExecutionWorker.java:56)
>> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:133)
>> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:71)
>> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.run(GradleWorkerMain.java:69)
>> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.main(GradleWorkerMain.java:74)
>> Caused by: java.lang.NullPointerException: Null PCollection
>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$CommittedImmutableListBundle.create(ImmutableListBundleFactory.java:137)
>> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$UncommittedImmutableListBundle.commit(ImmutableListBundleFactory.java:115)
>> 	at org.apache.beam.runners.direct.BoundedReadEvaluatorFactory$InputProvider.getInitialInputs(BoundedReadEvaluatorFactory.java:224)
>> 	at org.apache.beam.runners.direct.ReadEvaluatorFactory$InputProvider.getInitialInputs(ReadEvaluatorFactory.java:88)
>> 	at org.apache.beam.runners.direct.RootProviderRegistry.getInitialInputs(RootProviderRegistry.java:80)
>>
>>
>>
>>
>>
>>
>> --
>> <https://cloud.google.com>
>> * •  **Niel Markwick*
>> * •  *Cloud Solutions Architect <https://cloud.google.com/docs/tutorials>
>> * •  *Google Belgium
>> * •  *nielm@google.com
>> * •  *+32 2 894 6771
>>
>>
>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>
>> If you have received this communication by mistake, please don't forward
>> it to anyone else (it may contain confidential or privileged information),
>> please erase all copies of it, including all attachments, and please let
>> the sender know it went to the wrong person. Thanks
>>
>>
>> On Tue, 14 Jun 2022 at 19:17, Danny McCormick <da...@google.com>
>> wrote:
>>
>>> Hey Niel,
>>>
>>> Unfortunately I can't repro on master, and I'm a little bit at a loss as
>>> to what could be causing this since the changes are all string changes
>>> (like you mentioned). I also don't see any Jenkins failures related to this
>>> (I *think* the Java PostCommits
>>> <https://ci-beam.apache.org/job/beam_PostCommit_Java/> should catch
>>> this generally). Any chance you could share some logs from your failing
>>> test run? I'm interested if there's a correlation between the failing tests
>>> (e.g. if they all call a specific function) or if there's additional info
>>> in the stacktrace.
>>>
>>> Thanks,
>>> Danny
>>>
>>> On Tue, Jun 14, 2022 at 12:16 PM Niel Markwick <ni...@google.com> wrote:
>>>
>>>> Developing on master/HEAD, I have a bunch of unit tests failing locally
>>>> due to Null PCollections
>>>>
>>>> java.lang.NullPointerException: Null PCollection
>>>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>>>
>>>>
>>>> eg for :sdks:java:io:google-cloud-platform:test, 271 failures out of
>>>> 1338 tests, all for this reason.
>>>>
>>>> These tests all pass locally when I checkout the  2.39 release tag, and
>>>> they don't fail in Jenkins-CI pre/post-commit tests either.
>>>>
>>>> Doing a git bisect gives the commit 4ffeae4d2b - Update all links to
>>>> in progress jiras to issues (#21749)
>>>> <https://github.com/apache/beam/commit/4ffeae4d2b800f2df36d2ea2eab549f2204d5691> as
>>>> the one where the tests start to fail (They succeed in the parent
>>>> b0d964c4
>>>> <https://github.com/apache/beam/commit/b0d964c43092994977b10272451acc24b4a62aad>),
>>>> however the only changes in that commit appear to be comments.
>>>>
>>>> I can reliably repro this - checking out 4ffeae4d2b, and running
>>>> :sdks:java:io:google-cloud-platform:test -- tests fail, checking
>>>> out b0d964c4 and they pass.
>>>>
>>>> Any hints on what could be going on here?
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> --
>>>> <https://cloud.google.com>
>>>> * •  **Niel Markwick*
>>>> * •  *Cloud Solutions Architect
>>>> <https://cloud.google.com/docs/tutorials>
>>>> * •  *Google Belgium
>>>> * •  *nielm@google.com
>>>> * •  *+32 2 894 6771
>>>>
>>>>
>>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>>
>>>> If you have received this communication by mistake, please don't
>>>> forward it to anyone else (it may contain confidential or privileged
>>>> information), please erase all copies of it, including all attachments, and
>>>> please let the sender know it went to the wrong person. Thanks
>>>>
>>>

Re: Null PCollection errors in v2.40 unit tests

Posted by Evan Galpin <eg...@apache.org>.
I had this happen to me recently as well.  After `git bisecting` led to
confusing results, I ran my tests again via gradlew adding `--rerun-tasks`
to the command.  This is an expensive operation, but after I ran that I was
able to test again with expected results. YMMV

Thanks,
Evan


On Tue, Jun 14, 2022 at 2:12 PM Niel Markwick <ni...@google.com> wrote:

> I agree that it is very strange!
>
> I have also just repro'd it on the cleanest possible environment: a brand
> new GCE debian 11 VM...
>
> sudo apt install git openjdk-11-jdk
> git clone https://github.com/apache/beam.git
> cd beam
> git checkout b0d964c430
> ./gradlew :sdks:java:io:google-cloud-platform:test
>
> tests pass
>
> git checkout 4ffeae4d
> ./gradlew :sdks:java:io:google-cloud-platform:test
>
> tests fail.
>
>
> The test failure stack traces are pretty much identical - the only
> difference being the test being run.
>
> They all complain about a Null PCollection from the directRunner (a couple
> complain due to incorrect expected exceptions, or asserts in a finally
> block, but they are failing because of the Null PCollection)
>
> I am not sure but I think the common ground _could_ be that a side input
> is used in the failing tests.
>
>
> org.apache.beam.sdk.Pipeline$PipelineExecutionException: java.lang.NullPointerException: Null PCollection
> 	at app//org.apache.beam.sdk.Pipeline.run(Pipeline.java:329)
> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:399)
> 	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:335)
> 	at app//org.apache.beam.sdk.io.gcp.spanner.SpannerIOWriteTest.deadlineExceededFailsAfterRetries(SpannerIOWriteTest.java:734)
> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
> 	at app//org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
> 	at app//org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
> 	at app//org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
> 	at app//org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
> 	at app//org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
> 	at app//org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:323)
> 	at app//org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:258)
> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
> 	at app//org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
> 	at app//org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
> 	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
> 	at app//org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
> 	at app//org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
> 	at app//org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
> 	at app//org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
> 	at app//org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
> 	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
> 	at app//org.junit.runners.ParentRunner.run(ParentRunner.java:413)
> 	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 java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> 	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> 	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
> 	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
> 	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:33)
> 	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:94)
> 	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker$2.run(TestWorker.java:176)
> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.executeAndMaintainThreadName(TestWorker.java:129)
> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:100)
> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:60)
> 	at org.gradle.process.internal.worker.child.ActionExecutionWorker.execute(ActionExecutionWorker.java:56)
> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:133)
> 	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:71)
> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.run(GradleWorkerMain.java:69)
> 	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.main(GradleWorkerMain.java:74)
> Caused by: java.lang.NullPointerException: Null PCollection
> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$CommittedImmutableListBundle.create(ImmutableListBundleFactory.java:137)
> 	at org.apache.beam.runners.direct.ImmutableListBundleFactory$UncommittedImmutableListBundle.commit(ImmutableListBundleFactory.java:115)
> 	at org.apache.beam.runners.direct.BoundedReadEvaluatorFactory$InputProvider.getInitialInputs(BoundedReadEvaluatorFactory.java:224)
> 	at org.apache.beam.runners.direct.ReadEvaluatorFactory$InputProvider.getInitialInputs(ReadEvaluatorFactory.java:88)
> 	at org.apache.beam.runners.direct.RootProviderRegistry.getInitialInputs(RootProviderRegistry.java:80)
>
>
>
>
>
>
> --
> <https://cloud.google.com>
> * •  **Niel Markwick*
> * •  *Cloud Solutions Architect <https://cloud.google.com/docs/tutorials>
> * •  *Google Belgium
> * •  *nielm@google.com
> * •  *+32 2 894 6771
>
>
> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>
> If you have received this communication by mistake, please don't forward
> it to anyone else (it may contain confidential or privileged information),
> please erase all copies of it, including all attachments, and please let
> the sender know it went to the wrong person. Thanks
>
>
> On Tue, 14 Jun 2022 at 19:17, Danny McCormick <da...@google.com>
> wrote:
>
>> Hey Niel,
>>
>> Unfortunately I can't repro on master, and I'm a little bit at a loss as
>> to what could be causing this since the changes are all string changes
>> (like you mentioned). I also don't see any Jenkins failures related to this
>> (I *think* the Java PostCommits
>> <https://ci-beam.apache.org/job/beam_PostCommit_Java/> should catch this
>> generally). Any chance you could share some logs from your failing test
>> run? I'm interested if there's a correlation between the failing tests
>> (e.g. if they all call a specific function) or if there's additional info
>> in the stacktrace.
>>
>> Thanks,
>> Danny
>>
>> On Tue, Jun 14, 2022 at 12:16 PM Niel Markwick <ni...@google.com> wrote:
>>
>>> Developing on master/HEAD, I have a bunch of unit tests failing locally
>>> due to Null PCollections
>>>
>>> java.lang.NullPointerException: Null PCollection
>>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>>
>>>
>>> eg for :sdks:java:io:google-cloud-platform:test, 271 failures out of
>>> 1338 tests, all for this reason.
>>>
>>> These tests all pass locally when I checkout the  2.39 release tag, and
>>> they don't fail in Jenkins-CI pre/post-commit tests either.
>>>
>>> Doing a git bisect gives the commit 4ffeae4d2b - Update all links to in
>>> progress jiras to issues (#21749)
>>> <https://github.com/apache/beam/commit/4ffeae4d2b800f2df36d2ea2eab549f2204d5691> as
>>> the one where the tests start to fail (They succeed in the parent
>>> b0d964c4
>>> <https://github.com/apache/beam/commit/b0d964c43092994977b10272451acc24b4a62aad>),
>>> however the only changes in that commit appear to be comments.
>>>
>>> I can reliably repro this - checking out 4ffeae4d2b, and running
>>> :sdks:java:io:google-cloud-platform:test -- tests fail, checking
>>> out b0d964c4 and they pass.
>>>
>>> Any hints on what could be going on here?
>>>
>>>
>>>
>>>
>>>
>>>
>>> --
>>> <https://cloud.google.com>
>>> * •  **Niel Markwick*
>>> * •  *Cloud Solutions Architect
>>> <https://cloud.google.com/docs/tutorials>
>>> * •  *Google Belgium
>>> * •  *nielm@google.com
>>> * •  *+32 2 894 6771
>>>
>>>
>>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>>
>>> If you have received this communication by mistake, please don't forward
>>> it to anyone else (it may contain confidential or privileged information),
>>> please erase all copies of it, including all attachments, and please let
>>> the sender know it went to the wrong person. Thanks
>>>
>>

Re: Null PCollection errors in v2.40 unit tests

Posted by Niel Markwick <ni...@google.com>.
I agree that it is very strange!

I have also just repro'd it on the cleanest possible environment: a brand
new GCE debian 11 VM...

sudo apt install git openjdk-11-jdk
git clone https://github.com/apache/beam.git
cd beam
git checkout b0d964c430
./gradlew :sdks:java:io:google-cloud-platform:test

tests pass

git checkout 4ffeae4d
./gradlew :sdks:java:io:google-cloud-platform:test

tests fail.


The test failure stack traces are pretty much identical - the only
difference being the test being run.

They all complain about a Null PCollection from the directRunner (a couple
complain due to incorrect expected exceptions, or asserts in a finally
block, but they are failing because of the Null PCollection)

I am not sure but I think the common ground _could_ be that a side input is
used in the failing tests.


org.apache.beam.sdk.Pipeline$PipelineExecutionException:
java.lang.NullPointerException: Null PCollection
	at app//org.apache.beam.sdk.Pipeline.run(Pipeline.java:329)
	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:399)
	at app//org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:335)
	at app//org.apache.beam.sdk.io.gcp.spanner.SpannerIOWriteTest.deadlineExceededFailsAfterRetries(SpannerIOWriteTest.java:734)
	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native
Method)
	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
	at app//org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
	at app//org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
	at app//org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
	at app//org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
	at app//org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
	at app//org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:323)
	at app//org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:258)
	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
	at app//org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
	at app//org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
	at app//org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
	at app//org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
	at app//org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
	at app//org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
	at app//org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
	at app//org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
	at app//org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
	at app//org.junit.runners.ParentRunner.run(ParentRunner.java:413)
	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 java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native
Method)
	at java.base@11.0.13/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at java.base@11.0.13/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.base@11.0.13/java.lang.reflect.Method.invoke(Method.java:566)
	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:33)
	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:94)
	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
	at org.gradle.api.internal.tasks.testing.worker.TestWorker$2.run(TestWorker.java:176)
	at org.gradle.api.internal.tasks.testing.worker.TestWorker.executeAndMaintainThreadName(TestWorker.java:129)
	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:100)
	at org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:60)
	at org.gradle.process.internal.worker.child.ActionExecutionWorker.execute(ActionExecutionWorker.java:56)
	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:133)
	at org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:71)
	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.run(GradleWorkerMain.java:69)
	at app//worker.org.gradle.process.internal.worker.GradleWorkerMain.main(GradleWorkerMain.java:74)
Caused by: java.lang.NullPointerException: Null PCollection
	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
	at org.apache.beam.runners.direct.ImmutableListBundleFactory$CommittedImmutableListBundle.create(ImmutableListBundleFactory.java:137)
	at org.apache.beam.runners.direct.ImmutableListBundleFactory$UncommittedImmutableListBundle.commit(ImmutableListBundleFactory.java:115)
	at org.apache.beam.runners.direct.BoundedReadEvaluatorFactory$InputProvider.getInitialInputs(BoundedReadEvaluatorFactory.java:224)
	at org.apache.beam.runners.direct.ReadEvaluatorFactory$InputProvider.getInitialInputs(ReadEvaluatorFactory.java:88)
	at org.apache.beam.runners.direct.RootProviderRegistry.getInitialInputs(RootProviderRegistry.java:80)






-- 
<https://cloud.google.com>
* •  **Niel Markwick*
* •  *Cloud Solutions Architect <https://cloud.google.com/docs/tutorials>
* •  *Google Belgium
* •  *nielm@google.com
* •  *+32 2 894 6771

Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie.
RPR: 0878.065.378

If you have received this communication by mistake, please don't forward it
to anyone else (it may contain confidential or privileged information),
please erase all copies of it, including all attachments, and please let
the sender know it went to the wrong person. Thanks


On Tue, 14 Jun 2022 at 19:17, Danny McCormick <da...@google.com>
wrote:

> Hey Niel,
>
> Unfortunately I can't repro on master, and I'm a little bit at a loss as
> to what could be causing this since the changes are all string changes
> (like you mentioned). I also don't see any Jenkins failures related to this
> (I *think* the Java PostCommits
> <https://ci-beam.apache.org/job/beam_PostCommit_Java/> should catch this
> generally). Any chance you could share some logs from your failing test
> run? I'm interested if there's a correlation between the failing tests
> (e.g. if they all call a specific function) or if there's additional info
> in the stacktrace.
>
> Thanks,
> Danny
>
> On Tue, Jun 14, 2022 at 12:16 PM Niel Markwick <ni...@google.com> wrote:
>
>> Developing on master/HEAD, I have a bunch of unit tests failing locally
>> due to Null PCollections
>>
>> java.lang.NullPointerException: Null PCollection
>> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>>
>>
>> eg for :sdks:java:io:google-cloud-platform:test, 271 failures out of 1338
>> tests, all for this reason.
>>
>> These tests all pass locally when I checkout the  2.39 release tag, and
>> they don't fail in Jenkins-CI pre/post-commit tests either.
>>
>> Doing a git bisect gives the commit 4ffeae4d2b - Update all links to in
>> progress jiras to issues (#21749)
>> <https://github.com/apache/beam/commit/4ffeae4d2b800f2df36d2ea2eab549f2204d5691> as
>> the one where the tests start to fail (They succeed in the parent
>> b0d964c4
>> <https://github.com/apache/beam/commit/b0d964c43092994977b10272451acc24b4a62aad>),
>> however the only changes in that commit appear to be comments.
>>
>> I can reliably repro this - checking out 4ffeae4d2b, and running
>> :sdks:java:io:google-cloud-platform:test -- tests fail, checking
>> out b0d964c4 and they pass.
>>
>> Any hints on what could be going on here?
>>
>>
>>
>>
>>
>>
>> --
>> <https://cloud.google.com>
>> * •  **Niel Markwick*
>> * •  *Cloud Solutions Architect <https://cloud.google.com/docs/tutorials>
>> * •  *Google Belgium
>> * •  *nielm@google.com
>> * •  *+32 2 894 6771
>>
>>
>> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>>
>> If you have received this communication by mistake, please don't forward
>> it to anyone else (it may contain confidential or privileged information),
>> please erase all copies of it, including all attachments, and please let
>> the sender know it went to the wrong person. Thanks
>>
>

Re: Null PCollection errors in v2.40 unit tests

Posted by Danny McCormick <da...@google.com>.
Hey Niel,

Unfortunately I can't repro on master, and I'm a little bit at a loss as to
what could be causing this since the changes are all string changes (like
you mentioned). I also don't see any Jenkins failures related to this (I
*think* the Java PostCommits
<https://ci-beam.apache.org/job/beam_PostCommit_Java/> should catch this
generally). Any chance you could share some logs from your failing test
run? I'm interested if there's a correlation between the failing tests
(e.g. if they all call a specific function) or if there's additional info
in the stacktrace.

Thanks,
Danny

On Tue, Jun 14, 2022 at 12:16 PM Niel Markwick <ni...@google.com> wrote:

> Developing on master/HEAD, I have a bunch of unit tests failing locally
> due to Null PCollections
>
> java.lang.NullPointerException: Null PCollection
> 	at org.apache.beam.runners.direct.AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.<init>(AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle.java:29)
>
>
> eg for :sdks:java:io:google-cloud-platform:test, 271 failures out of 1338
> tests, all for this reason.
>
> These tests all pass locally when I checkout the  2.39 release tag, and
> they don't fail in Jenkins-CI pre/post-commit tests either.
>
> Doing a git bisect gives the commit 4ffeae4d2b - Update all links to in
> progress jiras to issues (#21749)
> <https://github.com/apache/beam/commit/4ffeae4d2b800f2df36d2ea2eab549f2204d5691> as
> the one where the tests start to fail (They succeed in the parent b0d964c4
> <https://github.com/apache/beam/commit/b0d964c43092994977b10272451acc24b4a62aad>),
> however the only changes in that commit appear to be comments.
>
> I can reliably repro this - checking out 4ffeae4d2b, and running
> :sdks:java:io:google-cloud-platform:test -- tests fail, checking
> out b0d964c4 and they pass.
>
> Any hints on what could be going on here?
>
>
>
>
>
>
> --
> <https://cloud.google.com>
> * •  **Niel Markwick*
> * •  *Cloud Solutions Architect <https://cloud.google.com/docs/tutorials>
> * •  *Google Belgium
> * •  *nielm@google.com
> * •  *+32 2 894 6771
>
>
> Google Belgium NV/SA, Steenweg op Etterbeek 180, 1040 Brussel, Belgie. RPR: 0878.065.378
>
> If you have received this communication by mistake, please don't forward
> it to anyone else (it may contain confidential or privileged information),
> please erase all copies of it, including all attachments, and please let
> the sender know it went to the wrong person. Thanks
>