You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by "Leonard Xu (Jira)" <ji...@apache.org> on 2019/12/24 03:01:00 UTC

[jira] [Created] (FLINK-15366) Dimension table do not support computed column

Leonard Xu created FLINK-15366:
----------------------------------

             Summary: Dimension table do not support computed column
                 Key: FLINK-15366
                 URL: https://issues.apache.org/jira/browse/FLINK-15366
             Project: Flink
          Issue Type: Bug
          Components: Table SQL / Planner
    Affects Versions: 1.10.0
            Reporter: Leonard Xu


In Flink 1.10, we bring computed column feature, but I found dimension table do not support this feature.

public static final String mysqlCurrencyDDL = "CREATE TABLE currency (\n" +
 " currency_id BIGINT,\n" +
 " currency_name STRING,\n" +
 " rate DECIMAL(38, 4),\n" +
 " currency_time TIMESTAMP(3),\n" +
 " country STRING,\n" +
 " timestamp6 TIMESTAMP(6),\n" +
 " currency_next as currency_id + 1,\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'" +
 ")";

 

//
{code:java}
// 
Exception in thread "main" java.lang.ClassCastException: org.apache.calcite.rel.logical.LogicalProject cannot be cast to org.apache.calcite.rel.core.TableScanException in thread "main" java.lang.ClassCastException: org.apache.calcite.rel.logical.LogicalProject cannot be cast to org.apache.calcite.rel.core.TableScan at org.apache.calcite.sql2rel.SqlToRelConverter.snapshotTemporalTable(SqlToRelConverter.java:2438) at org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2062) at org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2005) at org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2085) at org.apache.calcite.sql2rel.SqlToRelConverter.convertSelectImpl(SqlToRelConverter.java:646) at org.apache.calcite.sql2rel.SqlToRelConverter.convertSelect(SqlToRelConverter.java:627) at org.apache.calcite.sql2rel.SqlToRelConverter.convertQueryRecursive(SqlToRelConverter.java:3181) at org.apache.calcite.sql2rel.SqlToRelConverter.convertQuery(SqlToRelConverter.java:563) at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerImpl.scala:148) at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerImpl.scala:135) at org.apache.flink.table.planner.operations.SqlToOperationConverter.toQueryOperation(SqlToOperationConverter.java:523) at org.apache.flink.table.planner.operations.SqlToOperationConverter.convertSqlQuery(SqlToOperationConverter.java:437) at org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:154) at org.apache.flink.table.planner.operations.SqlToOperationConverter.convertSqlInsert(SqlToOperationConverter.java:343) at org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:142) at org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:66) at org.apache.flink.table.api.internal.TableEnvironmentImpl.sqlUpdate(TableEnvironmentImpl.java:484) at job.KafkaJoinJdbc2Jdbc.main(KafkaJoinJdbc2Jdbc.java:59)
Process finished with exit code 1

{code}
 



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