Your email was actually what made me notice this! :D

I haven't been able to reproduce the NPE you found (also on 2.37) but that
certainly doesn't mean it's not a bug.



On Tue, Mar 22, 2022 at 5:23 PM Niel Markwick <ni...@google.com> wrote:

> I have also seen this with Java beam 2.36.0 and 2.37.0, again with large
> groups...
>
> The behaviour is that some elements become null, which then causes either
> coder problems for the primitive types (as seen in Claire's stack trace),
> or downstream issues for custom types which are not expecting null
> elements...
>
> I have a relatively simple Java reproducer creating 200,000 KVs and doing
> a CoGroupByKey with a single KV. This runs fine on DirectRunner, and on
> Dataflow with 2.35.0, but fails on Dataflow on 2.36.0 and 2.47.0 with the
> same error: "org.apache.beam.sdk.coders.CoderException: cannot encode a
> null String" in the CoGroupByKey step
>
>     Pipeline p =
> Pipeline.create(PipelineOptionsFactory.fromArgs(args).create());
>     String[] bigarray = new String[200000];
>     Arrays.fill(bigarray, "Hello World");
>     PCollection<KV<Integer, String>> right =
>         p.apply(Create.of(Arrays.asList(bigarray)))
>             .apply(
>                 MapElements.into(new TypeDescriptor<KV<Integer, String>>()
> {})
>                     .via(input -> KV.of(100, input)));
>     PCollection<KV<Integer, String>> left = p.apply(Create.of(KV.of(100,
> "goodbye world")));
>
>     TupleTag<String> leftTag = new TupleTag<String>();
>     TupleTag<String> rightTag = new TupleTag<String>();
>
>     PCollection<KV<Integer, CoGbkResult>> joinedResult =
>         KeyedPCollectionTuple.of(leftTag, left)
>             .and(rightTag, right)
>             .apply("Join By Key", CoGroupByKey.<Integer>create());
>
>     joinedResult.apply(
>         "Report",
>         MapElements.into(TypeDescriptor.of(Void.class))
>             .via(
>                 (KV<Integer, CoGbkResult> element) -> {
>                   Iterable<String> leftValues =
> element.getValue().getAll(leftTag);
>                   Iterable<String> rightValues =
> element.getValue().getAll(rightTag);
>
>                   System.out.println("Key = " + element.getKey());
>                   System.out.println("left size= " +
> Iterables.size(leftValues));
>                   System.out.println("right size= " +
> Iterables.size(rightValues));
>                   return (Void) null;
>                 }));
>
>     p.run().waitUntilFinish();
>
>
> On Fri, 18 Mar 2022, 16:01 Reuven Lax, <re...@google.com> wrote:
>
>>
>> On Fri, Mar 18, 2022, 8:28 AM Claire McGinty <claire.d.mcgi...@gmail.com>
>> wrote:
>>
>>> To add more investigative context, we discovered that these jobs succeed
>>> when run with Dataflow Prime; it's just Beam 2.36.0 + Dataflow V1 that
>>> produces this regression. Unfortunately it's not feasible to upgrade our
>>> whole fleet to Dataflow Prime right now, so we've created an internal
>>> Google support ticket for this, too.
>>>
>>> - Claire
>>>
>>> On Tue, Mar 15, 2022 at 12:22 PM Claire McGinty <
>>> claire.d.mcgi...@gmail.com> wrote:
>>>
>>>> Hi! No, there aren't null strings in the data--it would have thrown a
>>>> NPE much earlier if that were the case, and likely failed when run with
>>>> Beam 2.35.0 also, unless null-handling changed too?
>>>>
>>>> - Claire
>>>>
>>>> On Tue, Mar 15, 2022 at 12:19 PM Reuven Lax <re...@google.com> wrote:
>>>>
>>>>> Is it expected to have null strings in your data?
>>>>>
>>>>> On Tue, Mar 15, 2022 at 9:06 AM Claire McGinty <
>>>>> claire.d.mcgi...@gmail.com> wrote:
>>>>>
>>>>>> Hi Beam devs!
>>>>>>
>>>>>> I wanted to surface a possible regression with CoGroupByKeys in this
>>>>>> transform. Our organization (which primarily uses Scio) has had several
>>>>>> pipelines start failing after upgrading from Beam 2.35.0 to Beam
>>>>>> 2.36.0, specifically during cogroup operations with large (>10,000) key
>>>>>> groups.
>>>>>>
>>>>>> We initially saw this problem using Scio join
>>>>>> <https://github.com/spotify/scio/blob/be7166c58450bd59ae5d0048d39ef3ea0d5ed107/scio-core/src/main/scala/com/spotify/scio/util/ArtisanJoin.scala#L84-L112>
>>>>>>  operations,
>>>>>> but I was able to reproduce using public datasets & Beam PTransforms
>>>>>> directly (job code link
>>>>>> <https://github.com/clairemcginty/beam-test/blob/main/src/main/scala/clairem/data/JoinTest.scala#L32>;
>>>>>> the repo README includes instructions to run it if anyone is interested).
>>>>>> This job throws the following error when run in Dataflow with Beam 
>>>>>> 2.36.0:
>>>>>>
>>>>>> Caused by: java.lang.IllegalArgumentException: Unable to encode
>>>>>> element
>>>>>> '[org.apache.beam.sdk.transforms.join.CoGbkResult$TagIterable@33dd2cbb,
>>>>>> org.apache.beam.sdk.transforms.join.CoGbkResult$TagIterable@26edfb82]'
>>>>>> with coder
>>>>>> 'org.apache.beam.sdk.transforms.join.CoGbkResult$CoGbkResultCoder@346927'.
>>>>>> org.apache.beam.sdk.coders.Coder.getEncodedElementByteSize(Coder.java:300)
>>>>>>
>>>>>> org.apache.beam.sdk.coders.Coder.registerByteSizeObserver(Coder.java:291)
>>>>>>
>>>>>> org.apache.beam.sdk.coders.KvCoder.registerByteSizeObserver(KvCoder.java:130)
>>>>>>
>>>>>> org.apache.beam.sdk.coders.KvCoder.registerByteSizeObserver(KvCoder.java:37)
>>>>>>
>>>>>> org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder.registerByteSizeObserver(WindowedValue.java:642)
>>>>>>
>>>>>> org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder.registerByteSizeObserver(WindowedValue.java:558)
>>>>>>
>>>>>> org.apache.beam.runners.dataflow.worker.IntrinsicMapTaskExecutorFactory$ElementByteSizeObservableCoder.registerByteSizeObserver(IntrinsicMapTaskExecutorFactory.java:403)
>>>>>>
>>>>>> org.apache.beam.runners.dataflow.worker.util.common.worker.OutputObjectAndByteCounter.update(OutputObjectAndByteCounter.java:128)
>>>>>>
>>>>>> org.apache.beam.runners.dataflow.worker.DataflowOutputCounter.update(DataflowOutputCounter.java:67)
>>>>>>
>>>>>> org.apache.beam.runners.dataflow.worker.util.common.worker.OutputReceiver.process(OutputReceiver.java:43)
>>>>>>
>>>>>> org.apache.beam.runners.dataflow.worker.SimpleParDoFn$1.output(SimpleParDoFn.java:285)
>>>>>>
>>>>>> org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner.outputWindowedValue(SimpleDoFnRunner.java:268)
>>>>>>
>>>>>> org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner.access$900(SimpleDoFnRunner.java:84)
>>>>>>
>>>>>> org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$DoFnProcessContext.output(SimpleDoFnRunner.java:416)
>>>>>>
>>>>>> org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$DoFnProcessContext.output(SimpleDoFnRunner.java:404)
>>>>>>
>>>>>> org.apache.beam.sdk.transforms.join.CoGroupByKey$ConstructCoGbkResultFn.processElement(CoGroupByKey.java:192)
>>>>>> Caused by: org.apache.beam.sdk.coders.CoderException: cannot encode a
>>>>>> null String
>>>>>> org.apache.beam.sdk.coders.StringUtf8Coder.encode(StringUtf8Coder.java:74)
>>>>>>
>>>>>> To troubleshoot, I first tried downgrading to Beam 2.35.0; the job
>>>>>> then ran successfully. Next, I tried running the job using Beam 2.36.0, 
>>>>>> but
>>>>>> with 2.35.0's version of CoGbkResult
>>>>>> <https://github.com/apache/beam/blob/v2.35.0/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java>
>>>>>>  on
>>>>>> the classpath; it ran successfully then, too. I'm wondering if it's 
>>>>>> related
>>>>>> to the lazy-iterator improvements introduced in BEAM-13541
>>>>>> <https://github.com/apache/beam/pull/16354>? I wasn't able to
>>>>>> reproduce it with a unit test yet, unfortunately; maybe it's unique to 
>>>>>> the
>>>>>> way Dataflow lazily-loads large CoGbkResults.
>>>>>>
>>>>>> I'm continuing to try to reproduce this on a smaller scale, but
>>>>>> wanted to flag it here for now; it's preventing our pipeline fleet from
>>>>>> upgrading to Beam 2.36.0. Any insight would be appreciated!
>>>>>>
>>>>>> Thanks,
>>>>>> Claire
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>

Reply via email to