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

ASF GitHub Bot commented on FLINK-6059:
---------------------------------------

Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3546#discussion_r113740954
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
 ---
    @@ -498,6 +505,10 @@ abstract class TableEnvironment(val config: 
TableConfig) {
         TableEnvironment.validateType(inputType)
    --- End diff --
    
    The `validateType()` method is also used to check the output type, i.e., 
the type of a DataSet or DataStream which is created from a Table. I think we 
should allow `GenericType<Row>` in that case.
    Hence, I would keep the `validateType()` method as it is.


> Reject DataSet<Row> and DataStream<Row> without RowTypeInformation
> ------------------------------------------------------------------
>
>                 Key: FLINK-6059
>                 URL: https://issues.apache.org/jira/browse/FLINK-6059
>             Project: Flink
>          Issue Type: Bug
>          Components: Table API & SQL
>    Affects Versions: 1.2.0, 1.3.0
>            Reporter: Fabian Hueske
>            Assignee: Fabian Hueske
>
> It is not possible to automatically extract proper type information for 
> {{Row}} because it is not typed with generics and holds values in an 
> {{Object[]}}.
> Consequently is handled as {{GenericType<Row>}} unless a {{RowTypeInfo}} is 
> explicitly specified.
> This can lead to unexpected behavior when converting a {{DataSet<Row>}} or 
> {{DataStream<Row>}} into a Table. If the data set or data stream has a 
> {{GenericType<Row>}}, the rows are treated as atomic type and converted into 
> a single field.
> I think we should reject input types of {{GenericType<Row>}} when converting 
> data sets and data streams and request a proper {{RowTypeInfo}}. 



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to