[
https://issues.apache.org/jira/browse/FLINK-8255?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16475411#comment-16475411
]
ASF GitHub Bot commented on FLINK-8255:
---------------------------------------
Github user snuyanzin commented on the issue:
https://github.com/apache/flink/pull/5961
Hello @fhueske
Thank you for your review
As you proposed I tried to use
org.apache.flink.api.java.typeutils.RowTypeInfo#isTupleType
```java
public boolean isTupleType() {
return false;
}
```
however after that these tests started to fail
org.apache.flink.table.api.batch.ExplainTest#testJoinWithoutExtended
org.apache.flink.table.api.batch.ExplainTest#testJoinWithExtended
like
```
testJoinWithoutExtended(org.apache.flink.table.api.batch.ExplainTest) Time
elapsed: 0.037 sec <<< ERROR!
org.apache.flink.api.common.InvalidProgramException: Specifying keys via
field positions is only valid for tuple data types. Type: Row(a: Integer, b:
String)
at
org.apache.flink.api.common.operators.Keys$ExpressionKeys.<init>(Keys.java:232)
at
org.apache.flink.api.common.operators.Keys$ExpressionKeys.<init>(Keys.java:223)
at
org.apache.flink.api.java.operators.JoinOperator$JoinOperatorSets.where(JoinOperator.java:901)
at
org.apache.flink.table.plan.nodes.dataset.DataSetJoin.addInnerJoin(DataSetJoin.scala:243)
at
org.apache.flink.table.plan.nodes.dataset.DataSetJoin.translateToPlan(DataSetJoin.scala:170)
at
org.apache.flink.table.plan.nodes.dataset.DataSetCalc.translateToPlan(DataSetCalc.scala:91)
at
org.apache.flink.table.api.BatchTableEnvironment.translate(BatchTableEnvironment.scala:422)
at
org.apache.flink.table.api.BatchTableEnvironment.explain(BatchTableEnvironment.scala:249)
at
org.apache.flink.table.api.BatchTableEnvironment.explain(BatchTableEnvironment.scala:275)
at
org.apache.flink.table.api.batch.ExplainTest.testJoinWithoutExtended(ExplainTest.scala:72)
```
that is why I decided to use instanceof TupleTypeInfo check as anyway next
line there is a cast to this type
could you please have a look at it and tell if it is acceptable or not?
> 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
> Assignee: Sergey Nuyanzin
> 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)