Martijn Visser created FLINK-24735:
--------------------------------------
Summary: SQL client crashes with `Cannot add expression of
different type to set`
Key: FLINK-24735
URL: https://issues.apache.org/jira/browse/FLINK-24735
Project: Flink
Issue Type: Bug
Components: Table SQL / Client
Affects Versions: 1.14.0
Reporter: Martijn Visser
Assignee: Shengkai Fang
Reproductions steps:
1. Download airports.csv from https://www.kaggle.com/usdot/flight-delays
2. Start Flink SQL client and create table
{code:sql}
CREATE TABLE `airports` (
`IATA_CODE` CHAR(3),
`AIRPORT` STRING,
`CITY` STRING,
`STATE` CHAR(2),
`COUNTRY` CHAR(3),
`LATITUDE` DOUBLE NULL,
`LONGITUDE` DOUBLE NULL,
PRIMARY KEY (`IATA_CODE`) NOT ENFORCED
) WITH (
'connector' = 'filesystem',
'path' =
'file:///flink-sql-cookbook/other-builtin-functions/04_override_table_options/airports.csv',
'format' = 'csv'
);
{code}
3. Run the following SQL statement:
{code:sql}
SELECT * FROM `airports` /*+ OPTIONS('csv.ignore-parse-errors'='true') */ WHERE
COALESCE(`IATA_CODE`, `AIRPORT`) IS NULL;
{code}
Stacktrace:
{code:bash}
Exception in thread "main" org.apache.flink.table.client.SqlClientException:
Unexpected exception. This is a bug. Please consider filing an issue.
at
org.apache.flink.table.client.SqlClient.startClient(SqlClient.java:201)
at org.apache.flink.table.client.SqlClient.main(SqlClient.java:161)
Caused by: java.lang.AssertionError: Cannot add expression of different type to
set:
set type is RecordType(CHAR(3) CHARACTER SET "UTF-16LE" NOT NULL IATA_CODE,
VARCHAR(2147483647) CHARACTER SET "UTF-16LE" AIRPORT, VARCHAR(2147483647)
CHARACTER SET "UTF-16LE" CITY, CHAR(2) CHARACTER SET "UTF-16LE" STATE, CHAR(3)
CHARACTER SET "UTF-16LE" COUNTRY, DOUBLE LATITUDE, DOUBLE LONGITUDE) NOT NULL
expression type is RecordType(CHAR(3) CHARACTER SET "UTF-16LE" IATA_CODE,
VARCHAR(2147483647) CHARACTER SET "UTF-16LE" AIRPORT, VARCHAR(2147483647)
CHARACTER SET "UTF-16LE" CITY, CHAR(2) CHARACTER SET "UTF-16LE" STATE, CHAR(3)
CHARACTER SET "UTF-16LE" COUNTRY, DOUBLE LATITUDE, DOUBLE LONGITUDE) NOT NULL
set is rel#426:LogicalProject.NONE.any.None:
0.[NONE].[NONE](input=HepRelVertex#425,inputs=0..6)
expression is LogicalProject(IATA_CODE=[null:CHAR(3) CHARACTER SET "UTF-16LE"],
AIRPORT=[$1], CITY=[$2], STATE=[$3], COUNTRY=[$4], LATITUDE=[$5],
LONGITUDE=[$6])
LogicalFilter(condition=[IS NULL(CAST($0):VARCHAR(2147483647) CHARACTER SET
"UTF-16LE")])
LogicalTableScan(table=[[default_catalog, default_database, airports]],
hints=[[[OPTIONS inheritPath:[] options:{csv.ignore-parse-errors=true}]]])
at
org.apache.calcite.plan.RelOptUtil.verifyTypeEquivalence(RelOptUtil.java:381)
at
org.apache.calcite.plan.hep.HepRuleCall.transformTo(HepRuleCall.java:58)
at
org.apache.calcite.plan.RelOptRuleCall.transformTo(RelOptRuleCall.java:268)
at
org.apache.calcite.plan.RelOptRuleCall.transformTo(RelOptRuleCall.java:283)
at
org.apache.calcite.rel.rules.ReduceExpressionsRule$ProjectReduceExpressionsRule.onMatch(ReduceExpressionsRule.java:310)
at
org.apache.calcite.plan.AbstractRelOptPlanner.fireRule(AbstractRelOptPlanner.java:333)
at org.apache.calcite.plan.hep.HepPlanner.applyRule(HepPlanner.java:542)
at
org.apache.calcite.plan.hep.HepPlanner.applyRules(HepPlanner.java:407)
at
org.apache.calcite.plan.hep.HepPlanner.executeInstruction(HepPlanner.java:243)
at
org.apache.calcite.plan.hep.HepInstruction$RuleInstance.execute(HepInstruction.java:127)
at
org.apache.calcite.plan.hep.HepPlanner.executeProgram(HepPlanner.java:202)
at
org.apache.calcite.plan.hep.HepPlanner.findBestExp(HepPlanner.java:189)
at
org.apache.flink.table.planner.plan.optimize.program.FlinkHepProgram.optimize(FlinkHepProgram.scala:69)
at
org.apache.flink.table.planner.plan.optimize.program.FlinkHepRuleSetProgram.optimize(FlinkHepRuleSetProgram.scala:87)
at
org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram.$anonfun$optimize$1(FlinkChainedProgram.scala:62)
at
scala.collection.TraversableOnce.$anonfun$foldLeft$1(TraversableOnce.scala:156)
at
scala.collection.TraversableOnce.$anonfun$foldLeft$1$adapted(TraversableOnce.scala:156)
at scala.collection.Iterator.foreach(Iterator.scala:937)
at scala.collection.Iterator.foreach$(Iterator.scala:937)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1425)
at scala.collection.IterableLike.foreach(IterableLike.scala:70)
at scala.collection.IterableLike.foreach$(IterableLike.scala:69)
at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
at scala.collection.TraversableOnce.foldLeft(TraversableOnce.scala:156)
at scala.collection.TraversableOnce.foldLeft$(TraversableOnce.scala:154)
at scala.collection.AbstractTraversable.foldLeft(Traversable.scala:104)
at
org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram.optimize(FlinkChainedProgram.scala:58)
at
org.apache.flink.table.planner.plan.optimize.StreamCommonSubGraphBasedOptimizer.optimizeTree(StreamCommonSubGraphBasedOptimizer.scala:163)
at
org.apache.flink.table.planner.plan.optimize.StreamCommonSubGraphBasedOptimizer.doOptimize(StreamCommonSubGraphBasedOptimizer.scala:81)
at
org.apache.flink.table.planner.plan.optimize.CommonSubGraphBasedOptimizer.optimize(CommonSubGraphBasedOptimizer.scala:77)
at
org.apache.flink.table.planner.delegation.PlannerBase.optimize(PlannerBase.scala:300)
at
org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:183)
at
org.apache.flink.table.api.internal.TableEnvironmentImpl.translate(TableEnvironmentImpl.java:1665)
at
org.apache.flink.table.api.internal.TableEnvironmentImpl.executeQueryOperation(TableEnvironmentImpl.java:805)
at
org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:1274)
at
org.apache.flink.table.client.gateway.local.LocalExecutor.lambda$executeOperation$3(LocalExecutor.java:209)
at
org.apache.flink.table.client.gateway.context.ExecutionContext.wrapClassLoader(ExecutionContext.java:88)
at
org.apache.flink.table.client.gateway.local.LocalExecutor.executeOperation(LocalExecutor.java:209)
at
org.apache.flink.table.client.gateway.local.LocalExecutor.executeQuery(LocalExecutor.java:231)
at
org.apache.flink.table.client.cli.CliClient.callSelect(CliClient.java:532)
at
org.apache.flink.table.client.cli.CliClient.callOperation(CliClient.java:423)
at
org.apache.flink.table.client.cli.CliClient.lambda$executeStatement$1(CliClient.java:332)
at java.util.Optional.ifPresent(Optional.java:159)
at
org.apache.flink.table.client.cli.CliClient.executeStatement(CliClient.java:325)
at
org.apache.flink.table.client.cli.CliClient.executeInteractive(CliClient.java:297)
at
org.apache.flink.table.client.cli.CliClient.executeInInteractiveMode(CliClient.java:221)
at org.apache.flink.table.client.SqlClient.openCli(SqlClient.java:151)
at org.apache.flink.table.client.SqlClient.start(SqlClient.java:95)
at
org.apache.flink.table.client.SqlClient.startClient(SqlClient.java:187)
... 1 more
{code}
Any exception/error should be wrapped into a SqlExecutionException (in the
executor) or SqlClientException in the highest layer, which is not happening in
this case.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)