flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Till Rohrmann (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-3444) env.fromElements relies on the first input element for determining the DataSet/DataStream type
Date Wed, 27 Apr 2016 13:14:13 GMT

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

Till Rohrmann commented on FLINK-3444:
--------------------------------------

I'm wondering why the above described problem was solved by introducing an overloaded method
{{fromElements}} which takes as the first parameter the {{Class<T>}} of the elements?
The problem is that if you do something like {{env.fromElements(SubClass.class, new ParentClass())}}
then the vararg only variant of {{fromElements}} will be called with type {{Object}} as result
(since Object is the super type of {{Class<SubClass>}} and {{ParentClass}}. The reason
is that {{ParentClass}} is not a subtype of {{SubClass}} and thus the new {{fromElements}}
method is not applicable.

Wouldn't it have been better to automatically infer the super type of all given elements?
Usually the {{fromElements}} method is only called for very few elements and, thus, it should
be feasible to iterate over all elements to extract the common super type. 

> env.fromElements relies on the first input element for determining the DataSet/DataStream
type
> ----------------------------------------------------------------------------------------------
>
>                 Key: FLINK-3444
>                 URL: https://issues.apache.org/jira/browse/FLINK-3444
>             Project: Flink
>          Issue Type: Bug
>          Components: DataSet API, DataStream API
>    Affects Versions: 0.10.0, 1.0.0
>            Reporter: Vasia Kalavri
>             Fix For: 1.1.0
>
>
> The {{fromElements}} method of the {{ExecutionEnvironment}} and {{StreamExecutionEnvironment}}
determines the DataSet/DataStream type by extracting the type of the first input element.
> This is problematic if the first element is a subtype of another element in the collection.
> For example, the following
> {code}
> DataStream<Event> input = env.fromElements(new Event(1, "a"), new SubEvent(2, "b"));
> {code}
> succeeds, while the following
> {code}
> DataStream<Event> input = env.fromElements(new SubEvent(1, "a"), new Event(2, "b"));
> {code}
> fails with "java.lang.IllegalArgumentException: The elements in the collection are not
all subclasses of SubEvent".



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message