[
https://issues.apache.org/jira/browse/BEAM-440?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15760233#comment-15760233
]
Kenneth Knowles commented on BEAM-440:
--------------------------------------
The root issue here is actually worse. There are two facets.
1. {{Create}} based on the value is unsafe in general. (full disclosure: I
implemented this unsafe "feature"). It adds a coder that is safe for _those
values only_ but not for their static type.
{code}
PCollection<Object> allStrings = pipeline.apply("Create strings",
Create.<Object>of("hello"));
{code}
The coder inference infrastructure reasonably assumes that
{{allStrings.getCoder()}} returns a coder that can handle all values of type
{{Object}}, which is as false in this example as it is for an empty {{Create}}.
Transforms like {{MapElements<Object, Object>}} will propagate the coder for
{{Object}} which will break if any non-string is in the output (which obviously
it could be).
2. {{Flatten}} quite explicitly assumes the same thing and grabs the first
coder it sees, so when you flatten a bunch of {{PCollection<Object>}} together
that have been made with {{Create}} you are in trouble.
So for this issue, we should probably just remove
https://github.com/apache/incubator-beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java#L496.
At the time we added it, it seemed handy to infer a coder based on the values,
since we didn't have any other information to go on due to the limits of Java
reflection, but the idea doesn't work without further validation which requires
as much user input as just setting the coder or type descriptor anyhow.
> Create.values() returns a type-unsafe Coder
> -------------------------------------------
>
> Key: BEAM-440
> URL: https://issues.apache.org/jira/browse/BEAM-440
> Project: Beam
> Issue Type: Bug
> Components: sdk-java-core
> Reporter: Daniel Halperin
> Labels: newbie, starter
>
> {{Create.values()}} with no arguments will default to a {{VoidCoder}}, unless
> one is set later with {{setCoder(Coder)}}.
> Although it will encode its input correctly, this seems like a bad choice in
> many cases. E.g., with {{Flatten}}:
> {code}
> PCollection<KV<SomeClass, Integer>> initial = p.apply("First",
> Create.<KV<SomeClass, Integer>>of());
> PCollection<KV<SomeClass, Integer>> second =
> p.apply("Second", Create.of("a", "b")).apply(ParDo.of(new
> MyAvroDoFn()));
> PCollectionList
> .of(initial).and(second)
> .apply(Flatten.<KV<SomeClass, Integer>>pCollections());
> {code}
> This crashes trying to cast a KV from "Second" to a Void.class.
> 1. Suggest throwing a warning in {{getDefaultOutputCoder}} when defaulting to
> {{VoidCoder}} for an empty elements list. Should this be an error?
> 2. Suggest adding something like {{Create.empty(TypeDescriptor)}} to handle
> this case properly.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)