You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Sergey Nuyanzin (JIRA)" <ji...@apache.org> on 2018/04/29 20:42:00 UTC

[jira] [Comment Edited] (FLINK-8255) Key expressions on named row types do not work

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

Sergey Nuyanzin edited comment on FLINK-8255 at 4/29/18 8:41 PM:
-----------------------------------------------------------------

A little bit research shows that it is something related to class Hierarchy: RowTypeInfo, TupleTypeInfoBase, TupleTypeInfo. Both RowTypeInfo and TupleTypeInfo are ancent of TupleTypeInfoBase. At the same time in e.g. org.apache.flink.streaming.util.typeutils.FieldAccessorFactory there are some checks with casting {code:java}	...
 else if (typeInfo.isTupleType()) {
			TupleTypeInfoBase tupleTypeInfo = (TupleTypeInfoBase) typeInfo;
...{code}
As RowTypeInfo and TupleTypeInfo are in parallel hierarchy branches => casting will fail for RowTypeInfo. At the same time it looks like there is nothing special related to TupleTypeInfo => casting to TupleTypeInfoBase is enough. 
Based on finding usages of FieldAccessorFactory's methods with specified casting there could be added 2 more test-case which are also fails with the similar ClassCastException
{code:java}
		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();

		TypeInformation[] types = new TypeInformation[]{Types.INT, Types.INT};

		String[] fieldNames = new String[]{"id", "value"};
		RowTypeInfo rowTypeInfo = new RowTypeInfo(types, fieldNames);

		UnsortedGrouping groupDs = env.fromCollection(Collections.singleton(new Row(2)), rowTypeInfo).groupBy(0);

		groupDs.maxBy(1);
{code}
and one more almost the same however with .minBy in the last line

one of the possible fixes: usage casting to TupleTypeInfoBase rther than to TupleTypeInfo (I'm not sure that changing hierarchy could be an option).
such fix for the mentioned 3 cases is available here https://github.com/apache/flink/compare/master...snuyanzin:FLINK-8255_Key_expressions_on_named_row_types_do_not_work

at the same time it looks like there still could be issues, e.g.
org.apache.flink.api.java.DataSet#minBy
org.apache.flink.api.java.DataSet#maxBy
org.apache.flink.streaming.util.typeutils.FieldAccessor.RecursiveTupleFieldAccessor#RecursiveTupleFieldAccessor 
also have such casting however at the moment I do not have any idea about test where it could fail


was (Author: sergey nuyanzin):
A little bit research shows that it is something related to class Hierarchy: RowTypeInfo, TupleTypeInfoBase, TupleTypeInfo. Both RowTypeInfo and TupleTypeInfo are ancent of TupleTypeInfoBase. At the same time in e.g. org.apache.flink.streaming.util.typeutils.FieldAccessorFactory there are some checks with casting {code:java}	...
 else if (typeInfo.isTupleType()) {
			TupleTypeInfoBase tupleTypeInfo = (TupleTypeInfoBase) typeInfo;
...{code}
As RowTypeInfo and TupleTypeInfo are in parallel hierarchy branches => casting will fail for RowTypeInfo. At the same time it looks like there is nothing special related to TupleTypeInfo => casting to TupleTypeInfoBase is enough. 
Based on finding usages of FieldAccessorFactory's methods with specified casting there could be added 2 more test-case which are also fails with the similar ClastCastException
{code:java}
		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();

		TypeInformation[] types = new TypeInformation[]{Types.INT, Types.INT};

		String[] fieldNames = new String[]{"id", "value"};
		RowTypeInfo rowTypeInfo = new RowTypeInfo(types, fieldNames);

		UnsortedGrouping groupDs = env.fromCollection(Collections.singleton(new Row(2)), rowTypeInfo).groupBy(0);

		groupDs.maxBy(1);
{code}
and one more almost the same however with .minBy in the last line

one of the possible fixes: usage casting to TupleTypeInfoBase rther than to TupleTypeInfo (I'm not sure that changing hierarchy could be an option).
such fix for the mentioned 3 cases is available here https://github.com/apache/flink/compare/master...snuyanzin:FLINK-8255_Key_expressions_on_named_row_types_do_not_work

at the same time it looks like there still could be issues, e.g.
org.apache.flink.api.java.DataSet#minBy
org.apache.flink.api.java.DataSet#maxBy
org.apache.flink.streaming.util.typeutils.FieldAccessor.RecursiveTupleFieldAccessor#RecursiveTupleFieldAccessor 
also have such casting however at the moment I do not have any idea about test where it could fail

> Key expressions on named row types do not work
> ----------------------------------------------
>
>                 Key: FLINK-8255
>                 URL: https://issues.apache.org/jira/browse/FLINK-8255
>             Project: Flink
>          Issue Type: Bug
>          Components: DataSet API, DataStream API
>    Affects Versions: 1.4.0, 1.5.0
>            Reporter: Timo Walther
>            Priority: Major
>
> The following program fails with a {{ClassCastException}}. It seems that key expressions and rows are not tested well. We should add more tests for them.
> {code}
> final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
> TypeInformation[] types = new TypeInformation[] {Types.INT, Types.INT};
> String[] fieldNames = new String[]{"id", "value"};
> RowTypeInfo rowTypeInfo = new RowTypeInfo(types, fieldNames);
> env.fromCollection(Collections.singleton(new Row(2)), rowTypeInfo)
> .keyBy("id").sum("value").print();
> env.execute("Streaming WordCount");
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)