[ https://issues.apache.org/jira/browse/FLINK-20942?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17272647#comment-17272647 ]
Danny Chen edited comment on FLINK-20942 at 1/27/21, 8:05 AM: -------------------------------------------------------------- I have fired a fix in CALCITE-4479 but i have no idea how to fix quickly in Flink side, should we copy the {{RexLiteral}} then ? The class is huge .. was (Author: danny0405): I have fired a fix in CALCITE-4479 but i have no idea how to fix quickly in Flink side, should we copy the {{RexLiteral }} then ? The class is huge .. > Digest of FLOAT literals throws UnsupportedOperationException > ------------------------------------------------------------- > > Key: FLINK-20942 > URL: https://issues.apache.org/jira/browse/FLINK-20942 > Project: Flink > Issue Type: Bug > Components: Table SQL / Planner > Reporter: Timo Walther > Priority: Critical > Fix For: 1.13.0, 1.12.2 > > > The recent refactoring of Calcite's digests might have caused a regression > for FLOAT literals. {{org.apache.calcite.rex.RexLiteral#appendAsJava}} throws > a UnsupportedOperationException for the following query: > {code} > def main(args: Array[String]): Unit = { > val env = StreamExecutionEnvironment.getExecutionEnvironment > val source = env.fromElements( > (1.0f, 11.0f, 12.0f), > (2.0f, 21.0f, 22.0f), > (3.0f, 31.0f, 32.0f), > (4.0f, 41.0f, 42.0f), > (5.0f, 51.0f, 52.0f) > ) > val settings = EnvironmentSettings.newInstance() > .inStreamingMode() > .useBlinkPlanner() > .build() > val tEnv = StreamTableEnvironment.create(env, settings) > tEnv.createTemporaryView("myTable", source, $("id"), $("f1"), $("f2")) > val query = > """ > |select * from myTable where id in (1.0, 2.0, 3.0) > |""".stripMargin > tEnv.executeSql(query).print() > } > {code} > Stack trace: > {code} > Exception in thread "main" java.lang.UnsupportedOperationException: class > org.apache.calcite.sql.type.SqlTypeName: FLOAT > at org.apache.calcite.util.Util.needToImplement(Util.java:1075) > at org.apache.calcite.rex.RexLiteral.appendAsJava(RexLiteral.java:703) > at org.apache.calcite.rex.RexLiteral.toJavaString(RexLiteral.java:408) > at org.apache.calcite.rex.RexLiteral.computeDigest(RexLiteral.java:276) > at org.apache.calcite.rex.RexLiteral.<init>(RexLiteral.java:223) > at org.apache.calcite.rex.RexLiteral.toLiteral(RexLiteral.java:737) > at > org.apache.calcite.rex.RexLiteral.lambda$printSarg$4(RexLiteral.java:710) > at > org.apache.calcite.util.RangeSets$Printer.singleton(RangeSets.java:397) > at org.apache.calcite.util.RangeSets.forEach(RangeSets.java:237) > at org.apache.calcite.util.Sarg.lambda$printTo$0(Sarg.java:110) > at org.apache.calcite.linq4j.Ord.forEach(Ord.java:157) > at org.apache.calcite.util.Sarg.printTo(Sarg.java:106) > at org.apache.calcite.rex.RexLiteral.printSarg(RexLiteral.java:709) > at > org.apache.calcite.rex.RexLiteral.lambda$appendAsJava$1(RexLiteral.java:652) > at org.apache.calcite.util.Util.asStringBuilder(Util.java:2502) > at org.apache.calcite.rex.RexLiteral.appendAsJava(RexLiteral.java:651) > at org.apache.calcite.rex.RexLiteral.toJavaString(RexLiteral.java:408) > at org.apache.calcite.rex.RexLiteral.computeDigest(RexLiteral.java:276) > at org.apache.calcite.rex.RexLiteral.<init>(RexLiteral.java:223) > at org.apache.calcite.rex.RexBuilder.makeLiteral(RexBuilder.java:971) > at > org.apache.calcite.rex.RexBuilder.makeSearchArgumentLiteral(RexBuilder.java:1066) > at > org.apache.calcite.rex.RexSimplify$SargCollector.fix(RexSimplify.java:2786) > at > org.apache.calcite.rex.RexSimplify.lambda$simplifyOrs$6(RexSimplify.java:1843) > at java.util.ArrayList.forEach(ArrayList.java:1257) > at org.apache.calcite.rex.RexSimplify.simplifyOrs(RexSimplify.java:1843) > at org.apache.calcite.rex.RexSimplify.simplifyOr(RexSimplify.java:1817) > at org.apache.calcite.rex.RexSimplify.simplify(RexSimplify.java:313) > at > org.apache.calcite.rex.RexSimplify.simplifyUnknownAs(RexSimplify.java:282) > at org.apache.calcite.rex.RexSimplify.simplify(RexSimplify.java:257) > at > org.apache.flink.table.planner.plan.utils.FlinkRexUtil$.simplify(FlinkRexUtil.scala:213) > at > org.apache.flink.table.planner.plan.rules.logical.SimplifyFilterConditionRule.simplify(SimplifyFilterConditionRule.scala:63) > at > org.apache.flink.table.planner.plan.rules.logical.SimplifyFilterConditionRule.onMatch(SimplifyFilterConditionRule.scala:46) > 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.FlinkGroupProgram$$anonfun$optimize$1$$anonfun$apply$1.apply(FlinkGroupProgram.scala:63) > at > org.apache.flink.table.planner.plan.optimize.program.FlinkGroupProgram$$anonfun$optimize$1$$anonfun$apply$1.apply(FlinkGroupProgram.scala:60) > at > scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157) > at > scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157) > at scala.collection.Iterator$class.foreach(Iterator.scala:891) > at scala.collection.AbstractIterator.foreach(Iterator.scala:1334) > at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) > at scala.collection.AbstractIterable.foreach(Iterable.scala:54) > at > scala.collection.TraversableOnce$class.foldLeft(TraversableOnce.scala:157) > at scala.collection.AbstractTraversable.foldLeft(Traversable.scala:104) > at > org.apache.flink.table.planner.plan.optimize.program.FlinkGroupProgram$$anonfun$optimize$1.apply(FlinkGroupProgram.scala:60) > at > org.apache.flink.table.planner.plan.optimize.program.FlinkGroupProgram$$anonfun$optimize$1.apply(FlinkGroupProgram.scala:55) > at > scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157) > at > scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157) > at scala.collection.immutable.Range.foreach(Range.scala:160) > at > scala.collection.TraversableOnce$class.foldLeft(TraversableOnce.scala:157) > at scala.collection.AbstractTraversable.foldLeft(Traversable.scala:104) > at > org.apache.flink.table.planner.plan.optimize.program.FlinkGroupProgram.optimize(FlinkGroupProgram.scala:55) > at > org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram$$anonfun$optimize$1.apply(FlinkChainedProgram.scala:62) > at > org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram$$anonfun$optimize$1.apply(FlinkChainedProgram.scala:58) > at > scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157) > at > scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157) > at scala.collection.Iterator$class.foreach(Iterator.scala:891) > at scala.collection.AbstractIterator.foreach(Iterator.scala:1334) > at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) > at scala.collection.AbstractIterable.foreach(Iterable.scala:54) > at > scala.collection.TraversableOnce$class.foldLeft(TraversableOnce.scala:157) > at scala.collection.AbstractTraversable.foldLeft(Traversable.scala:104) > at > org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram.optimize(FlinkChainedProgram.scala:57) > 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:79) > at > org.apache.flink.table.planner.plan.optimize.CommonSubGraphBasedOptimizer.optimize(CommonSubGraphBasedOptimizer.scala:77) > at > org.apache.flink.table.planner.delegation.PlannerBase.optimize(PlannerBase.scala:286) > at > org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:165) > at > org.apache.flink.table.api.internal.TableEnvironmentImpl.translate(TableEnvironmentImpl.java:1329) > at > org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:707) > at > org.apache.flink.table.api.internal.TableEnvironmentImpl.executeOperation(TableEnvironmentImpl.java:1107) > at > org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:666) > at > org.apache.flink.table.examples.scala.basics.WordCountTable$.main(WordCountTable.scala:59) > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005)