This is the same as the issue with Create. Inferring a coder based on the
class of values is fragile, because coders are invariant.
PCollection<T> input = ...
Key k = ...;
PCollection<KV<Object, T>> pc = input.apply(WithKeys.of((Object)
subclassOfKey))
// a PCollection with a Coder<Key> that might have arbitrary Object
elements
Flatten.pCollections(pc, someOtherPcWithOtherKeyTypes);
I just checked the code and Flatten does not reject heterogeneous coders.
We went through this for Create and IIRC determined to remove value-based
coder inference. I can't recall if we actually broke "type" safety in the
process; I haven't typed this in and tried to get a crash.
Void is a special case that is safe, but the reflective mechanism does not
actually have access to the fact that you want it to be a Void.
Kenn
On Tue, Sep 26, 2017 at 8:41 AM, Lukasz Cwik <[email protected]>
wrote:
> The issue is that WithKeys uses the objects class for inferring the coder.
> In this case it would be safe to assume that we can use the VoidCoder if
> the user wants a null key. Filed
> https://issues.apache.org/jira/browse/BEAM-2989
>
>
>
> On Tue, Sep 26, 2017 at 7:37 AM, Reuven Lax <[email protected]>
> wrote:
>
> > If I have a PCollection<String> pc and I write
> >
> > pc.apply(WithKeys.<Void, String>of((Void) null))
> > .apply(GroupByKey.<Void, String>create())
> >
> > I'll get the following failure
> >
> > java.lang.IllegalStateException: Unable to return a default Coder for
> > <transform> [PCollection]. Correct one of the following root causes:
> > No Coder has been manually specified; you may do so using .setCoder().
> > Inferring a Coder from the CoderRegistry failed: Cannot provide coder
> for
> > parameterized type org.apache.beam.sdk.values.KV<K, V>: Unable to
> provide
> > a
> > Coder for K.
> >
> > I can fix this by adding the following after WithKeys.
> >
> > .setCoder(KvCoder.of(VoidCoder.of(), StringUtf8Coder.of()))
> >
> >
> > Why can't Beam infer a coder here? String, Void, and KV should all be
> > registered in the CoderRegistry.
> >
> >
> > Reuven
> >
>