xiaojin.wy created FLINK-23288:
----------------------------------

             Summary: Inserting (1.378593404E9) (0.6047707965147558) to a 
double type, it will generate CodeGenException
                 Key: FLINK-23288
                 URL: https://issues.apache.org/jira/browse/FLINK-23288
             Project: Flink
          Issue Type: Improvement
          Components: Table SQL / Runtime
    Affects Versions: 1.14.0
            Reporter: xiaojin.wy


CREATE TABLE database5_t0(
`c0` DOUBLE , `c1` INTEGER , `c2` STRING
) WITH (
 'connector' = 'filesystem',
'path' = 'hdfs:///tmp/database5_t0.csv',   
 'format' = 'csv'
)
INSERT OVERWRITE database5_t0(c0, c1, c2) VALUES(1.378593404E9, 1336919677, 
'1969-12-31 20:29:41'), (0.6047707965147558, 1336919677, '1970-01-06 03:36:50')

*After excuting that, will generate this errors, but mysql,pg,sqlite won`t have 
the error:*

org.apache.flink.table.planner.codegen.CodeGenException: Incompatible types of 
expression and result type. 
Expression[GeneratedExpression(((org.apache.flink.table.data.DecimalData) 
decimal$3),false,,DECIMAL(17, 16) NOT NULL,Some(0.6047707965147558))] type is 
[DECIMAL(17, 16) NOT NULL], result type is [DOUBLE NOT NULL]

        at 
org.apache.flink.table.planner.codegen.ExprCodeGenerator$$anonfun$generateResultExpression$1.apply(ExprCodeGenerator.scala:312)
        at 
org.apache.flink.table.planner.codegen.ExprCodeGenerator$$anonfun$generateResultExpression$1.apply(ExprCodeGenerator.scala:300)
        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.codegen.ExprCodeGenerator.generateResultExpression(ExprCodeGenerator.scala:300)
        at 
org.apache.flink.table.planner.codegen.ExprCodeGenerator.generateResultExpression(ExprCodeGenerator.scala:256)
        at 
org.apache.flink.table.planner.codegen.ValuesCodeGenerator$$anonfun$1.apply(ValuesCodeGenerator.scala:44)
        at 
org.apache.flink.table.planner.codegen.ValuesCodeGenerator$$anonfun$1.apply(ValuesCodeGenerator.scala:43)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
        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.TraversableLike$class.map(TraversableLike.scala:234)
        at scala.collection.AbstractTraversable.map(Traversable.scala:104)
        at 
org.apache.flink.table.planner.codegen.ValuesCodeGenerator$.generatorInputFormat(ValuesCodeGenerator.scala:43)
        at 
org.apache.flink.table.planner.codegen.ValuesCodeGenerator.generatorInputFormat(ValuesCodeGenerator.scala)
        at 
org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecValues.translateToPlanInternal(CommonExecValues.java:50)
        at 
org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:171)
        at 
org.apache.flink.table.planner.plan.nodes.exec.ExecEdge.translateToPlan(ExecEdge.java:247)
        at 
org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSink.translateToPlanInternal(BatchExecSink.java:58)
        at 
org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:171)
        at 
org.apache.flink.table.planner.delegation.BatchPlanner$$anonfun$translateToPlan$1.apply(BatchPlanner.scala:81)
        at 
org.apache.flink.table.planner.delegation.BatchPlanner$$anonfun$translateToPlan$1.apply(BatchPlanner.scala:80)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
        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.TraversableLike$class.map(TraversableLike.scala:234)
        at scala.collection.AbstractTraversable.map(Traversable.scala:104)
        at 
org.apache.flink.table.planner.delegation.BatchPlanner.translateToPlan(BatchPlanner.scala:80)
        at 
org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:174)
        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:483)
        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)





--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to