You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "godfrey he (Jira)" <ji...@apache.org> on 2022/04/25 07:03:00 UTC

[jira] [Commented] (FLINK-27369) COALESCE('1', CAST(NULL as varchar)) throws expression type mismatch

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

godfrey he commented on FLINK-27369:
------------------------------------

The reason UnreachableCoalesceArgumentsRemoveRexShuttle will return the first non-null operand of RexCall, which type may be different from the type of the original RexCall. The solution is UnreachableCoalesceArgumentsRemoveRexShuttle should make sure the type of new RexNode must be same with the type of original RexCall.

> COALESCE('1', CAST(NULL as varchar)) throws expression type mismatch
> --------------------------------------------------------------------
>
>                 Key: FLINK-27369
>                 URL: https://issues.apache.org/jira/browse/FLINK-27369
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Planner
>    Affects Versions: 1.15.0
>            Reporter: Jark Wu
>            Assignee: godfrey he
>            Priority: Critical
>             Fix For: 1.15.0
>
>
> {code}
> Flink SQL> SELECT
> >     COALESCE('1', cast(NULL as varchar)),
> >     COALESCE('4', cast(NULL as varchar), cast(NULL as varchar), cast(NULL as varchar));
> 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(VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL EXPR$0, VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL EXPR$1) NOT NULL
> expression type is RecordType(CHAR(1) CHARACTER SET "UTF-16LE" NOT NULL EXPR$0, CHAR(1) CHARACTER SET "UTF-16LE" NOT NULL EXPR$1) NOT NULL
> set is rel#910:LogicalProject.NONE.any.None: 0.[NONE].[NONE](input=HepRelVertex#909,exprs=[COALESCE(_UTF-16LE'1', null:VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), COALESCE(_UTF-16LE'4', null:VARCHAR(2147483647) CHARACTER SET "UTF-16LE", null:VARCHAR(2147483647) CHARACTER SET "UTF-16LE", null:VARCHAR(2147483647) CHARACTER SET "UTF-16LE")])
> expression is LogicalProject(EXPR$0=[_UTF-16LE'1'], EXPR$1=[_UTF-16LE'4'])
>   LogicalValues(tuples=[[{ 0 }]])
> 	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.flink.table.planner.plan.rules.logical.RemoveUnreachableCoalesceArgumentsRule.onMatch(RemoveUnreachableCoalesceArgumentsRule.java:71)
> 	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:164)
> 	at org.apache.flink.table.planner.plan.optimize.StreamCommonSubGraphBasedOptimizer.doOptimize(StreamCommonSubGraphBasedOptimizer.scala:82)
> 	at org.apache.flink.table.planner.plan.optimize.CommonSubGraphBasedOptimizer.optimize(CommonSubGraphBasedOptimizer.scala:77)
> 	at org.apache.flink.table.planner.delegation.PlannerBase.optimize(PlannerBase.scala:303)
> 	at org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:179)
> 	at org.apache.flink.table.api.internal.TableEnvironmentImpl.translate(TableEnvironmentImpl.java:1656)
> 	at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeQueryOperation(TableEnvironmentImpl.java:828)
> 	at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:1317)
> 	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:561)
> 	at org.apache.flink.table.client.cli.CliClient.callOperation(CliClient.java:446)
> 	at org.apache.flink.table.client.cli.CliClient.executeOperation(CliClient.java:373)
> 	at org.apache.flink.table.client.cli.CliClient.getAndExecuteStatements(CliClient.java:330)
> 	at org.apache.flink.table.client.cli.CliClient.executeInteractive(CliClient.java:281)
> 	at org.apache.flink.table.client.cli.CliClient.executeInInteractiveMode(CliClient.java:229)
> 	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)
> {code}



--
This message was sent by Atlassian Jira
(v8.20.7#820007)