From issues-return-166727-archive-asf-public=cust-asf.ponee.io@flink.apache.org Tue May 15 13:37:05 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 60EFA180634 for ; Tue, 15 May 2018 13:37:05 +0200 (CEST) Received: (qmail 65836 invoked by uid 500); 15 May 2018 11:37:04 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 65827 invoked by uid 99); 15 May 2018 11:37:04 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 15 May 2018 11:37:04 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id F21511807D8 for ; Tue, 15 May 2018 11:37:03 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -110.311 X-Spam-Level: X-Spam-Status: No, score=-110.311 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, T_RP_MATCHES_RCVD=-0.01, USER_IN_DEF_SPF_WL=-7.5, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id RrLFFso6nLYX for ; Tue, 15 May 2018 11:37:02 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id C4A065FB29 for ; Tue, 15 May 2018 11:37:01 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id D073AE0F4F for ; Tue, 15 May 2018 11:37:00 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 3C1E9217A7 for ; Tue, 15 May 2018 11:37:00 +0000 (UTC) Date: Tue, 15 May 2018 11:37:00 +0000 (UTC) From: "Sergey Nuyanzin (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-9341) Oracle: "Type is not supported: Date" MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/FLINK-9341?page=3Dcom.atlassian= .jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=3D1647= 5693#comment-16475693 ]=20 Sergey Nuyanzin commented on FLINK-9341: ---------------------------------------- Hello please sorry if I'm wrong however working on another activity I was able t= o generate the same trace As I understand it does not matter Oracle DB or whatever else in use. It l= ooks like Flink waits for _java.sql.Date_ and fails if receive _java.util.D= ate_ but tells us only with simple names i.e. Date. From my point of view i= t makes sense to specify typename in exception e.g. here _org.apache.flink.= table.calcite.FlinkTypeFactory#typeInfoToSqlTypeName_ instead of {code:java} case _@t =3D> throw TableException(s"Type is not supported: $t"){code} should be {code:java} case _@t =3D> throw TableException(s"Type is not supported: ${t.getTypeClass.getT= ypeName}"){code} and in other places (another option could be use such typename in _org.apac= he.flink.api.common.typeinfo.BasicTypeInfo#toString_ but it is public) [~fhueske] could you please comment such findings? If you are ok I could pr= oceed 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 =3D ExecutionEnvironment.getExecutionEnvir= onment(); BatchTableEnvironment tEnv =3D TableEnvironment.getTableEnvironment= (env); DataSet inputUD =3D env.fromElements( new UtilDate(new java.util.Date(System.currentTimeMillis())= ), new UtilDate(new java.util.Date(System.currentTimeMillis())= )); DataSet inputSD =3D 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 =3D tEnv.sqlQuery("SELECT CURRENT_DATE as sqlDat= e FROM TEST_SQL_DATE"); Table tableUtilDate =3D tEnv.sqlQuery("SELECT CURRENT_DATE as utilD= ate FROM TEST_UTIL_DATE"); DataSet resultSqlDate =3D tEnv.toDataSet(tableSqlDate, Sql= Date.class); DataSet utilDate =3D tEnv.toDataSet(tableUtilDate, UtilDa= te.class); resultSqlDate.print(); utilDate.print(); } public static class SqlDate { public java.sql.Date sqlDate; public SqlDate() {} public SqlDate(java.sql.Date sqlDate) { this.sqlDate =3D 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 =3D utilDate; } @Override public String toString() { return "UtilDate " + utilDate; } } } {code} =C2=A0 > 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=C2=A0"Type is not supported: Date". This happens with as si= mple a query as > {code:java} > SELECT SYSDATE FROM DUAL{code} > =C2=A0Stack trace: > {noformat} > Caused by: org.apache.flink.table.api.TableException: Type is not support= ed: Date > =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.flink.table.api.= TableException$.apply(exceptions.scala:53) ~[flink-table_2.11-1.4.2.jar:1.4= .2] > =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.flink.table.calc= ite.FlinkTypeFactory$.org$apache$flink$table$calcite$FlinkTypeFactory$$type= InfoToSqlTypeName(FlinkTypeFactory.scala:336) ~[flink-table_2.11-1.4.2.jar:= 1.4.2] > =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.flink.table.calc= ite.FlinkTypeFactory.createTypeFromTypeInfo(FlinkTypeFactory.scala:68) ~[fl= ink-table_2.11-1.4.2.jar:1.4.2] > =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.flink.table.calc= ite.FlinkTypeFactory$$anonfun$buildLogicalRowType$1.apply(FlinkTypeFactory.= scala:198) ~[flink-table_2.11-1.4.2.jar:1.4.2] > =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.flink.table.calc= ite.FlinkTypeFactory$$anonfun$buildLogicalRowType$1.apply(FlinkTypeFactory.= scala:195) ~[flink-table_2.11-1.4.2.jar:1.4.2] > =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at scala.collection.mutable.Re= sizableArray$class.foreach(ResizableArray.scala:59) ~[scala-library-2.11.11= .jar:na] > =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at scala.collection.mutable.Ar= rayBuffer.foreach(ArrayBuffer.scala:48) ~[scala-library-2.11.11.jar:na] > =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.flink.table.calc= ite.FlinkTypeFactory.buildLogicalRowType(FlinkTypeFactory.scala:195) ~[flin= k-table_2.11-1.4.2.jar:1.4.2] > =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.flink.table.plan= .schema.InlineTable.getRowType(InlineTable.scala:105) ~[flink-table_2.11-1.= 4.2.jar:1.4.2] > =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.flink.table.api.= TableEnvironment.scanInternal(TableEnvironment.scala:499) ~[flink-table_2.1= 1-1.4.2.jar:1.4.2] > =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.flink.table.api.= TableEnvironment.scan(TableEnvironment.scala:485) ~[flink-table_2.11-1.4.2.= jar:1.4.2] > =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.flink.table.api.= java.BatchTableEnvironment.fromDataSet(BatchTableEnvironment.scala:61) ~[fl= ink-table_2.11-1.4.2.jar:1.4.2] > =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 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)