[
https://issues.apache.org/jira/browse/FLINK-9341?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16475693#comment-16475693
]
Sergey Nuyanzin commented on FLINK-9341:
----------------------------------------
Hello
please sorry if I'm wrong however working on another activity I was able to
generate the same trace
As I understand it does not matter Oracle DB or whatever else in use. It looks
like Flink waits for _java.sql.Date_ and fails if receive _java.util.Date_ but
tells us only with simple names i.e. Date. From my point of view it makes sense
to specify typename in exception e.g. here
_org.apache.flink.table.calcite.FlinkTypeFactory#typeInfoToSqlTypeName_ instead
of
{code:java}
case _@t =>
throw TableException(s"Type is not supported: $t"){code}
should be
{code:java}
case _@t =>
throw TableException(s"Type is not supported:
${t.getTypeClass.getTypeName}"){code}
and in other places (another option could be use such typename in
_org.apache.flink.api.common.typeinfo.BasicTypeInfo#toString_ but it is public)
[~fhueske] could you please comment such findings? If you are ok I could
proceed with this issue.
I was able do generate other similar examples
here it is a code snippet to reproduce it
{code:java}
import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.table.api.Table;
import org.apache.flink.table.api.TableEnvironment;
import org.apache.flink.table.api.java.BatchTableEnvironment;
public class Case2 {
public static void main(String[] args) throws Exception {
ExecutionEnvironment env =
ExecutionEnvironment.getExecutionEnvironment();
BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env);
DataSet<UtilDate> inputUD = env.fromElements(
new UtilDate(new java.util.Date(System.currentTimeMillis())),
new UtilDate(new java.util.Date(System.currentTimeMillis())));
DataSet<SqlDate> inputSD = env.fromElements(
new SqlDate(new java.sql.Date(System.currentTimeMillis())),
new SqlDate(new java.sql.Date(System.currentTimeMillis())));
tEnv.registerDataSet("TEST_UTIL_DATE", inputUD, "utilDate");
tEnv.registerDataSet("TEST_SQL_DATE", inputSD, "sqlDate");
Table tableSqlDate = tEnv.sqlQuery("SELECT CURRENT_DATE as sqlDate FROM
TEST_SQL_DATE");
Table tableUtilDate = tEnv.sqlQuery("SELECT CURRENT_DATE as utilDate
FROM TEST_UTIL_DATE");
DataSet<SqlDate> resultSqlDate = tEnv.toDataSet(tableSqlDate,
SqlDate.class);
DataSet<UtilDate> utilDate = tEnv.toDataSet(tableUtilDate,
UtilDate.class);
resultSqlDate.print();
utilDate.print();
}
public static class SqlDate {
public java.sql.Date sqlDate;
public SqlDate() {}
public SqlDate(java.sql.Date sqlDate) {
this.sqlDate = sqlDate;
}
@Override
public String toString() {
return "SQL_DATE " + sqlDate;
}
}
public static class UtilDate {
public java.util.Date utilDate;
public UtilDate() {}
public UtilDate(java.util.Date utilDate) {
this.utilDate = utilDate;
}
@Override
public String toString() {
return "UtilDate " + utilDate;
}
}
}
{code}
> Oracle: "Type is not supported: Date"
> -------------------------------------
>
> Key: FLINK-9341
> URL: https://issues.apache.org/jira/browse/FLINK-9341
> Project: Flink
> Issue Type: Bug
> Components: Table API & SQL
> Affects Versions: 1.4.2
> Reporter: Ken Geis
> Priority: Major
>
> When creating a Table from an Oracle JDBCInputFormat with a date column, I
> get the error "Type is not supported: Date". This happens with as simple a
> query as
> {code:java}
> SELECT SYSDATE FROM DUAL{code}
> Stack trace:
> {noformat}
> Caused by: org.apache.flink.table.api.TableException: Type is not supported:
> Date
> at
> org.apache.flink.table.api.TableException$.apply(exceptions.scala:53)
> ~[flink-table_2.11-1.4.2.jar:1.4.2]
> at
> org.apache.flink.table.calcite.FlinkTypeFactory$.org$apache$flink$table$calcite$FlinkTypeFactory$$typeInfoToSqlTypeName(FlinkTypeFactory.scala:336)
> ~[flink-table_2.11-1.4.2.jar:1.4.2]
> at
> org.apache.flink.table.calcite.FlinkTypeFactory.createTypeFromTypeInfo(FlinkTypeFactory.scala:68)
> ~[flink-table_2.11-1.4.2.jar:1.4.2]
> at
> org.apache.flink.table.calcite.FlinkTypeFactory$$anonfun$buildLogicalRowType$1.apply(FlinkTypeFactory.scala:198)
> ~[flink-table_2.11-1.4.2.jar:1.4.2]
> at
> org.apache.flink.table.calcite.FlinkTypeFactory$$anonfun$buildLogicalRowType$1.apply(FlinkTypeFactory.scala:195)
> ~[flink-table_2.11-1.4.2.jar:1.4.2]
> at
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> ~[scala-library-2.11.11.jar:na]
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> ~[scala-library-2.11.11.jar:na]
> at
> org.apache.flink.table.calcite.FlinkTypeFactory.buildLogicalRowType(FlinkTypeFactory.scala:195)
> ~[flink-table_2.11-1.4.2.jar:1.4.2]
> at
> org.apache.flink.table.plan.schema.InlineTable.getRowType(InlineTable.scala:105)
> ~[flink-table_2.11-1.4.2.jar:1.4.2]
> at
> org.apache.flink.table.api.TableEnvironment.scanInternal(TableEnvironment.scala:499)
> ~[flink-table_2.11-1.4.2.jar:1.4.2]
> at
> org.apache.flink.table.api.TableEnvironment.scan(TableEnvironment.scala:485)
> ~[flink-table_2.11-1.4.2.jar:1.4.2]
> at
> org.apache.flink.table.api.java.BatchTableEnvironment.fromDataSet(BatchTableEnvironment.scala:61)
> ~[flink-table_2.11-1.4.2.jar:1.4.2]
> at
> org.apache.flink.table.api.java.BatchTableEnvironment$fromDataSet$0.call(Unknown
> Source) ~[na:na]
> (at my code...)
> {noformat}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)