[
https://issues.apache.org/jira/browse/FLINK-5750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16536822#comment-16536822
]
ASF GitHub Bot commented on FLINK-5750:
---------------------------------------
Github user AlexanderKoltsov commented on the issue:
https://github.com/apache/flink/pull/6287
The only thing that I didn't do is: I didn't add unit test for stream
environment.
During this test I got exception.
### Test in org.apache.flink.table.runtime.stream.sql package:
@Test
def testValuesWithCast(): Unit = {
val env = StreamExecutionEnvironment.getExecutionEnvironment
val tEnv = TableEnvironment.getTableEnvironment(env)
StreamITCase.clear
val sqlQuery = "VALUES (1, cast(1 as BIGINT) )," +
"(2, cast(2 as BIGINT))," +
"(3, cast(3 as BIGINT))"
val results = tEnv.sqlQuery(sqlQuery).toRetractStream[Row]
results.addSink(new StreamITCase.RetractingSink)
env.execute()
val expected = Seq(
"1,1\n2,2\n3,3"
)
assertEquals(expected.sorted, StreamITCase.retractedResults.sorted)
}
### Output:
org.apache.flink.table.api.TableException: Cannot generate a valid
execution plan for the given query:
FlinkLogicalUnion(all=[true])
FlinkLogicalCalc(expr#0=[{inputs}], expr#1=[1], expr#2=[1], EXPR$0=[$t1],
EXPR$1=[$t2])
FlinkLogicalValues(tuples=[[{ 0 }]])
FlinkLogicalCalc(expr#0=[{inputs}], expr#1=[2], expr#2=[2], EXPR$0=[$t1],
EXPR$1=[$t2])
FlinkLogicalValues(tuples=[[{ 0 }]])
FlinkLogicalCalc(expr#0=[{inputs}], expr#1=[3], expr#2=[3], EXPR$0=[$t1],
EXPR$1=[$t2])
FlinkLogicalValues(tuples=[[{ 0 }]])
This exception indicates that the query uses an unsupported SQL feature.
Please check the documentation for the set of currently supported SQL
features.
at
org.apache.flink.table.api.TableEnvironment.runVolcanoPlanner(TableEnvironment.scala:274)
at
org.apache.flink.table.api.StreamTableEnvironment.optimize(StreamTableEnvironment.scala:731)
at
org.apache.flink.table.api.StreamTableEnvironment.translate(StreamTableEnvironment.scala:778)
at
org.apache.flink.table.api.scala.StreamTableEnvironment.toRetractStream(StreamTableEnvironment.scala:254)
at
org.apache.flink.table.api.scala.StreamTableEnvironment.toRetractStream(StreamTableEnvironment.scala:234)
at
org.apache.flink.table.api.scala.TableConversions.toRetractStream(TableConversions.scala:189)
> Incorrect translation of n-ary Union
> ------------------------------------
>
> Key: FLINK-5750
> URL: https://issues.apache.org/jira/browse/FLINK-5750
> Project: Flink
> Issue Type: Bug
> Components: Table API & SQL
> Affects Versions: 1.2.0, 1.3.4, 1.5.0, 1.4.2, 1.6.0
> Reporter: Anton Mushin
> Assignee: Alexander Koltsov
> Priority: Critical
> Labels: pull-request-available
>
> Calcite's union operator is supports more than two input relations. However,
> Flink's translation rules only consider the first two relations because we
> assumed that Calcite's union is binary.
> This problem exists for batch and streaming queries.
> It seems that Calcite only generates non-binary Unions in rare cases
> ({{(SELECT * FROM t) UNION ALL (SELECT * FROM t) UNION ALL (SELECT * FROM
> t)}} results in two binary union operators) but the problem definitely needs
> to be fixed.
> The following query can be used to validate the problem.
> {code:java}
> @Test
> public void testValuesWithCast() throws Exception {
> ExecutionEnvironment env =
> ExecutionEnvironment.getExecutionEnvironment();
> BatchTableEnvironment tableEnv =
> TableEnvironment.getTableEnvironment(env, config());
> String sqlQuery = "VALUES (1, cast(1 as BIGINT) )," +
> "(2, cast(2 as BIGINT))," +
> "(3, cast(3 as BIGINT))";
> String sqlQuery2 = "VALUES (1,1)," +
> "(2, 2)," +
> "(3, 3)";
> Table result = tableEnv.sql(sqlQuery);
> DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
> List<Row> results = resultSet.collect();
> Table result2 = tableEnv.sql(sqlQuery2);
> DataSet<Row> resultSet2 = tableEnv.toDataSet(result2,
> Row.class);
> List<Row> results2 = resultSet2.collect();
> String expected = "1,1\n2,2\n3,3";
> compareResultAsText(results2, expected);
> compareResultAsText(results, expected);
> }
> {code}
> AR for {{results}} variable
> {noformat}
> java.lang.AssertionError: Different elements in arrays: expected 3 elements
> and received 2
> expected: [1,1, 2,2, 3,3]
> received: [1,1, 2,2]
> Expected :3
> Actual :2
> {noformat}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)