xiaojin.wy created FLINK-23271: ---------------------------------- Summary: RuntimeException: while resolving method 'booleanValue' in class class java.math.BigDecimal Key: FLINK-23271 URL: https://issues.apache.org/jira/browse/FLINK-23271 Project: Flink Issue Type: Bug Components: Table SQL / Planner Affects Versions: 1.14.0 Reporter: xiaojin.wy
*--sql--* CREATE TABLE database3_t0( c0 DECIMAL , c1 SMALLINT ) WITH ( 'connector' = 'filesystem', 'path' = 'hdfs:///tmp/database3_t0.csv', 'format' = 'csv' ); INSERT OVERWRITE database8_t0(c0, c1) VALUES(2113554022, cast(-22975 as SMALLINT)), (1570419395, cast(-26858 as SMALLINT)), (-1569861129, cast(-20143 as SMALLINT)); SELECT database8_t0.c0 AS ref0 FROM database8_t0 WHERE CAST (0.10915913549909961 AS BOOLEAN; After excuting the sql, you will find the error: java.lang.RuntimeException: while resolving method 'booleanValue' in class class java.math.BigDecimal at org.apache.calcite.linq4j.tree.Expressions.call(Expressions.java:424) at org.apache.calcite.linq4j.tree.Expressions.call(Expressions.java:435) at org.apache.calcite.linq4j.tree.Expressions.unbox(Expressions.java:1453) at org.apache.calcite.adapter.enumerable.EnumUtils.convert(EnumUtils.java:398) at org.apache.calcite.adapter.enumerable.EnumUtils.convert(EnumUtils.java:326) at org.apache.calcite.adapter.enumerable.RexToLixTranslator.translateCast(RexToLixTranslator.java:538) at org.apache.calcite.adapter.enumerable.RexImpTable$CastImplementor.implementSafe(RexImpTable.java:2450) at org.apache.calcite.adapter.enumerable.RexImpTable$AbstractRexCallImplementor.genValueStatement(RexImpTable.java:2894) at org.apache.calcite.adapter.enumerable.RexImpTable$AbstractRexCallImplementor.implement(RexImpTable.java:2859) at org.apache.calcite.adapter.enumerable.RexToLixTranslator.visitCall(RexToLixTranslator.java:1084) at org.apache.calcite.adapter.enumerable.RexToLixTranslator.visitCall(RexToLixTranslator.java:90) at org.apache.calcite.rex.RexCall.accept(RexCall.java:174) at org.apache.calcite.adapter.enumerable.RexToLixTranslator.visitLocalRef(RexToLixTranslator.java:970) at org.apache.calcite.adapter.enumerable.RexToLixTranslator.visitLocalRef(RexToLixTranslator.java:90) at org.apache.calcite.rex.RexLocalRef.accept(RexLocalRef.java:75) at org.apache.calcite.adapter.enumerable.RexToLixTranslator.translate(RexToLixTranslator.java:237) at org.apache.calcite.adapter.enumerable.RexToLixTranslator.translate(RexToLixTranslator.java:231) at org.apache.calcite.adapter.enumerable.RexToLixTranslator.translateList(RexToLixTranslator.java:818) at org.apache.calcite.adapter.enumerable.RexToLixTranslator.translateProjects(RexToLixTranslator.java:198) at org.apache.calcite.rex.RexExecutorImpl.compile(RexExecutorImpl.java:90) at org.apache.calcite.rex.RexExecutorImpl.compile(RexExecutorImpl.java:66) at org.apache.calcite.rex.RexExecutorImpl.reduce(RexExecutorImpl.java:128) at org.apache.calcite.rex.RexSimplify.simplifyCast(RexSimplify.java:2101) at org.apache.calcite.rex.RexSimplify.simplify(RexSimplify.java:326) at org.apache.calcite.rex.RexSimplify.simplifyUnknownAs(RexSimplify.java:287) at org.apache.calcite.rex.RexSimplify.simplify(RexSimplify.java:262) at org.apache.flink.table.planner.plan.utils.FlinkRexUtil$.simplify(FlinkRexUtil.scala:224) 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.BatchCommonSubGraphBasedOptimizer.optimizeTree(BatchCommonSubGraphBasedOptimizer.scala:87) at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer.org$apache$flink$table$planner$plan$optimize$BatchCommonSubGraphBasedOptimizer$$optimizeBlock(BatchCommonSubGraphBasedOptimizer.scala:58) at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer$$anonfun$doOptimize$1.apply(BatchCommonSubGraphBasedOptimizer.scala:46) at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer$$anonfun$doOptimize$1.apply(BatchCommonSubGraphBasedOptimizer.scala:46) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer.doOptimize(BatchCommonSubGraphBasedOptimizer.scala:46) at org.apache.flink.table.planner.plan.optimize.CommonSubGraphBasedOptimizer.optimize(CommonSubGraphBasedOptimizer.scala:93) at org.apache.flink.table.planner.delegation.PlannerBase.optimize(PlannerBase.scala:310) at org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:172) at org.apache.flink.table.api.internal.TableEnvironmentImpl.translate(TableEnvironmentImpl.java:1658) at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:747) at org.apache.flink.table.api.internal.StatementSetImpl.execute(StatementSetImpl.java:100) at org.apache.flink.table.planner.runtime.batch.sql.TableSourceITCase.testTableXiaojin(TableSourceITCase.scala:436) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.apache.flink.util.TestNameProvider$1.evaluate(TestNameProvider.java:45) at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:61) at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54) at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54) at org.junit.rules.RunRules.evaluate(RunRules.java:20) at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) at org.junit.runners.ParentRunner.run(ParentRunner.java:413) at org.junit.runner.JUnitCore.run(JUnitCore.java:137) at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:68) at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:33) at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:230) at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:58) Caused by: java.lang.NoSuchMethodException: java.math.BigDecimal.booleanValue() at java.lang.Class.getMethod(Class.java:1786) at org.apache.calcite.linq4j.tree.Expressions.call(Expressions.java:421) ... 112 more -- This message was sent by Atlassian Jira (v8.3.4#803005)