You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Zhenghua Gao (Jira)" <ji...@apache.org> on 2020/01/06 03:56:00 UTC

[jira] [Commented] (FLINK-15445) JDBC Table Source didn't work for Types with precision (or/and scale)

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

Zhenghua Gao commented on FLINK-15445:
--------------------------------------

[~jark] [~twalthr] I have opened a PR to let JDBC source support precision/scale, and a problem arises: Do we need to check whether the underlying database supports the data types defined in our DDL to avoid precision loss?  Some scenarios are listed as following:
 # the underlying DB supports DECIMAL(65, 30), which is out of range of Flink's decimal
 # User defines a table with DECIMAL(10, 4) in underlying db, and defines a table with DECIMAL(5, 2) in Flink
 # the underlying DB supports TIMESTAMP(6), and user defines a table with TIMESTAMP(9) in Flink
 # the precision or scale of the underlying DB varies between different versions

What do you think about this?

> JDBC Table Source didn't work for Types with precision (or/and scale)
> ---------------------------------------------------------------------
>
>                 Key: FLINK-15445
>                 URL: https://issues.apache.org/jira/browse/FLINK-15445
>             Project: Flink
>          Issue Type: Bug
>          Components: Connectors / JDBC
>    Affects Versions: 1.10.0
>            Reporter: Zhenghua Gao
>            Assignee: Zhenghua Gao
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.10.0
>
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> {code:java}
>  public class JDBCSourceExample {
>     public static void main(String[] args) throws Exception {
>         StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
>         env.setParallelism(1);
>         EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
>                 .useBlinkPlanner()
>                 .inStreamingMode()
>                 .build();
>         StreamTableEnvironment tableEnvironment = StreamTableEnvironment.create(env, envSettings);
>         String mysqlCurrencyDDL = "CREATE TABLE currency (\n" +
>                 "  currency_id BIGINT,\n" +
>                 "  currency_name STRING,\n" +
>                 "  rate DOUBLE,\n" +
>                 "  currency_time TIMESTAMP(3),\n" +
>                 "  country STRING,\n" +
>                 "  timestamp6 TIMESTAMP(6),\n" +
>                 "  time6 TIME(6),\n" +
>                 "  gdp DECIMAL(10, 4)\n" +
>                 ") WITH (\n" +
>                 "   'connector.type' = 'jdbc',\n" +
>                 "   'connector.url' = 'jdbc:mysql://localhost:3306/test',\n" +
>                 "   'connector.username' = 'root'," +
>                 "   'connector.table' = 'currency',\n" +
>                 "   'connector.driver' = 'com.mysql.jdbc.Driver',\n" +
>                 "   'connector.lookup.cache.max-rows' = '500', \n" +
>                 "   'connector.lookup.cache.ttl' = '10s',\n" +
>                 "   'connector.lookup.max-retries' = '3'" +
>                 ")";
>         tableEnvironment.sqlUpdate(mysqlCurrencyDDL);
>         String querySQL = "select * from currency";
>         tableEnvironment.toAppendStream(tableEnvironment.sqlQuery(querySQL), Row.class).print();
>         tableEnvironment.execute("JdbcExample");
>     }
> }{code}
>  
> Throws Exception:
>  
> {code:java}
> org.apache.flink.table.api.ValidationException: Type TIMESTAMP(6) of table field 'timestamp6_col' does not match with the physical type TIMESTAMP(3) of the 'timestamp6_col' field of the TableSource return type.org.apache.flink.table.api.ValidationException: Type TIMESTAMP(6) of table field 'timestamp6_col' does not match with the physical type TIMESTAMP(3) of the 'timestamp6_col' field of the TableSource return type.
>  at org.apache.flink.table.utils.TypeMappingUtils.lambda$checkPhysicalLogicalTypeCompatible$4(TypeMappingUtils.java:164) at org.apache.flink.table.utils.TypeMappingUtils$1.defaultMethod(TypeMappingUtils.java:277) at org.apache.flink.table.utils.TypeMappingUtils$1.defaultMethod(TypeMappingUtils.java:254) at org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor.visit(LogicalTypeDefaultVisitor.java:132) at org.apache.flink.table.types.logical.TimestampType.accept(TimestampType.java:151) at org.apache.flink.table.utils.TypeMappingUtils.checkIfCompatible(TypeMappingUtils.java:254) at org.apache.flink.table.utils.TypeMappingUtils.checkPhysicalLogicalTypeCompatible(TypeMappingUtils.java:160) at org.apache.flink.table.utils.TypeMappingUtils.lambda$computeInCompositeType$8(TypeMappingUtils.java:232) at java.util.stream.Collectors.lambda$toMap$58(Collectors.java:1321) at java.util.stream.ReduceOps$3ReducingSink.accept(ReduceOps.java:169) at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1382) at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481) at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471) at java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708) at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:499) at org.apache.flink.table.utils.TypeMappingUtils.computeInCompositeType(TypeMappingUtils.java:214) at org.apache.flink.table.utils.TypeMappingUtils.computePhysicalIndices(TypeMappingUtils.java:192) at org.apache.flink.table.utils.TypeMappingUtils.computePhysicalIndicesOrTimeAttributeMarkers(TypeMappingUtils.java:112) at org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecTableSourceScan.computeIndexMapping(StreamExecTableSourceScan.scala:211) at org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecTableSourceScan.translateToPlanInternal(StreamExecTableSourceScan.scala:107) at org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecTableSourceScan.translateToPlanInternal(StreamExecTableSourceScan.scala:62) at org.apache.flink.table.planner.plan.nodes.exec.ExecNode$class.translateToPlan(ExecNode.scala:58) at org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecTableSourceScan.translateToPlan(StreamExecTableSourceScan.scala:62) at org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecSink.translateToTransformation(StreamExecSink.scala:183) at org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecSink.translateToPlanInternal(StreamExecSink.scala:152) at org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecSink.translateToPlanInternal(StreamExecSink.scala:48) at org.apache.flink.table.planner.plan.nodes.exec.ExecNode$class.translateToPlan(ExecNode.scala:58) at org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecSink.translateToPlan(StreamExecSink.scala:48) at org.apache.flink.table.planner.delegation.StreamPlanner$$anonfun$translateToPlan$1.apply(StreamPlanner.scala:60) at org.apache.flink.table.planner.delegation.StreamPlanner$$anonfun$translateToPlan$1.apply(StreamPlanner.scala:59) 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.StreamPlanner.translateToPlan(StreamPlanner.scala:59) at org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:153) at org.apache.flink.table.api.java.internal.StreamTableEnvironmentImpl.toDataStream(StreamTableEnvironmentImpl.java:351) at org.apache.flink.table.api.java.internal.StreamTableEnvironmentImpl.toAppendStream(StreamTableEnvironmentImpl.java:259) at org.apache.flink.table.api.java.internal.StreamTableEnvironmentImpl.toAppendStream(StreamTableEnvironmentImpl.java:250) at org.apache.flink.api.java.io.jdbc.JDBCTableSourceITCase.testJDBCSource(JDBCTableSourceITCase.java:102) 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:50) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47) 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.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55) at org.junit.rules.RunRules.evaluate(RunRules.java:20) at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57) at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288) at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268) at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48) at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48) at org.junit.rules.RunRules.evaluate(RunRules.java:20) at org.junit.runners.ParentRunner.run(ParentRunner.java:363) at org.junit.runner.JUnitCore.run(JUnitCore.java:137) at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:68) at com.intellij.rt.execution.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:47) at com.intellij.rt.execution.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:242) at com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:70){code}
>  



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