Caizhi Weng created FLINK-22870:
-----------------------------------
Summary: Grouping sets + case when + constant string throws
AssertionError
Key: FLINK-22870
URL: https://issues.apache.org/jira/browse/FLINK-22870
Project: Flink
Issue Type: Bug
Components: Table SQL / Planner
Affects Versions: 1.14.0, 1.13.2
Reporter: Caizhi Weng
Add the following case to {{org.apache.flink.table.api.TableEnvironmentITCase}}
to reproduce this issue.
{code:scala}
@Test
def myTest2(): Unit = {
tEnv.executeSql(
"""
|create temporary table my_source(
| a INT
|) WITH (
| 'connector' = 'values'
|)
|""".stripMargin)
tEnv.executeSql(
"""
|create temporary view my_view as select a, 'test' as b from my_source
|""".stripMargin)
tEnv.executeSql(
"""
|create temporary view my_view2 as select
| a,
| case when GROUPING(b) = 1 then 'test2' else b end as b
|from my_view
|group by grouping sets(
| (),
| (a),
| (b),
| (a, b)
|)
|""".stripMargin)
System.out.println(tEnv.explainSql(
"""
|select a, b from my_view2
|""".stripMargin))
}
{code}
The exception stack is
{code}
java.lang.AssertionError: Conversion to relational algebra failed to preserve
datatypes:
validated type:
RecordType(INTEGER a, VARCHAR(5) CHARACTER SET "UTF-16LE" NOT NULL b) NOT NULL
converted type:
RecordType(INTEGER a, VARCHAR(5) CHARACTER SET "UTF-16LE" b) NOT NULL
rel:
LogicalProject(a=[$0], b=[CASE(=($2, 1), _UTF-16LE'test2':VARCHAR(5) CHARACTER
SET "UTF-16LE", CAST($1):VARCHAR(5) CHARACTER SET "UTF-16LE")])
LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]],
agg#0=[GROUPING($1)])
LogicalProject(a=[$0], b=[_UTF-16LE'test'])
LogicalTableScan(table=[[default_catalog, default_database, my_source]])
at
org.apache.calcite.sql2rel.SqlToRelConverter.checkConvertedType(SqlToRelConverter.java:467)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertQuery(SqlToRelConverter.java:582)
at
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$rel(FlinkPlannerImpl.scala:177)
at
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerImpl.scala:169)
at
org.apache.flink.table.planner.operations.SqlToOperationConverter.toQueryOperation(SqlToOperationConverter.java:1048)
at
org.apache.flink.table.planner.operations.SqlToOperationConverter.convertViewQuery(SqlToOperationConverter.java:897)
at
org.apache.flink.table.planner.operations.SqlToOperationConverter.convertCreateView(SqlToOperationConverter.java:864)
at
org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:259)
at
org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:101)
at
org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:730)
at
org.apache.flink.table.api.TableEnvironmentITCase.myTest2(TableEnvironmentITCase.scala:148)
{code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)