You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by Paul Lam <pa...@gmail.com> on 2020/03/19 08:26:51 UTC

SQL Timetamp types incompatible after migration to 1.10

Hi,

Recently I upgraded a simple application that inserts static data into a table from 1.9.0 to 1.10.0, and 
encountered a timestamp type incompatibility problem during the table sink validation.

The SQL is like:
```
insert into kafka.test.tbl_a # schema: (user_name STRING, user_id INT, login_time TIMESTAMP)
select ("ann", 1000, TIMESTAMP "2019-12-30 00:00:00")
```

And the error thrown:
```
Field types of query result and registered TableSink `kafka`.`test`.`tbl_a` do not match.
      Query result schema: [EXPR$0: String, EXPR$1: Integer, EXPR$2: Timestamp]
      TableSink schema:    [user_name: String, user_id: Integer, login_time: LocalDateTime]
```

After some digging, I found the root cause might be that since FLINK-14645 timestamp fields 
defined via TableFactory had been bridged to LocalDateTime, but timestamp literals are 
still backed by java.sql.Timestamp.

Is my reasoning correct? And is there any workaround? Thanks a lot!

Best,
Paul Lam


Re: SQL Timetamp types incompatible after migration to 1.10

Posted by Jark Wu <im...@gmail.com>.
Hi Brian,

Could you share the full exception stack of  `Unsupport cast from
LocalDateTime to Long` in the PR?

In 1.10 DDL, the conversion class or TypeInformation for TIMESTAMP becomes
`LocalDateTime`. Maybe your prolem is related to this change?
If the connector doesn't support `LocalDateTime`, you should bridge back to
sql Timestamp, see CsvTableSource [1].

Regarding to the batch table support in 1.10 blink planner, in blink
planner, batch and streaming are unified.
For sources, you can use `StreamTableSourceFactory` and returns bounded
`StreamTableSource` (via StreamTableSource#isBounded()) to support batch
source.
For sinks, you can take JDBCTableSourceSinkFactory [2] as an exmaple.

Best,
Jark

[1]:
https://github.com/apache/flink/blob/master/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSource.java#L308
[2]:
https://github.com/apache/flink/blob/master/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceSinkFactory.java#L72

On Thu, 19 Mar 2020 at 19:32, <B....@dell.com> wrote:

> Hi Jark,
>
>
>
> I saw this mail and found this is a similar issue I raised to the
> community several days ago.[1] Can you have a look to see if it’s the same
> issue as this.
>
>
>
> If yes, there is a further question. From the Pravega connector side, the
> issue is raised in our Batch Table API which means users using the
> BatchTableEnvironment to create tables. Currently, BatchTableEnvironment
> does not support Blink planner. Any suggestions on how we can support Batch
> Tables in Flink 1.10?
>
>
>
> [1]
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Need-help-on-timestamp-type-conversion-for-Table-API-on-Pravega-Connector-td33660.html
>
>
>
> Best Regards,
>
> Brian
>
>
>
> *From:* Jark Wu <im...@gmail.com>
> *Sent:* Thursday, March 19, 2020 17:14
> *To:* Paul Lam
> *Cc:* user
> *Subject:* Re: SQL Timetamp types incompatible after migration to 1.10
>
>
>
> [EXTERNAL EMAIL]
>
> Hi Paul,
>
>
>
> Are you using old planner? Did you try blink planner? I guess it maybe a
> bug in old planner which doesn't work well on new types.
>
>
>
> Best,
>
> Jark
>
>
>
> On Thu, 19 Mar 2020 at 16:27, Paul Lam <pa...@gmail.com> wrote:
>
> Hi,
>
>
>
> Recently I upgraded a simple application that inserts static data into a
> table from 1.9.0 to 1.10.0, and
>
> encountered a timestamp type incompatibility problem during the table sink
> validation.
>
>
>
> The SQL is like:
>
> ```
>
> insert into kafka.test.tbl_a # schema: (user_name STRING, user_id INT,
> login_time TIMESTAMP)
>
> select ("ann", 1000, TIMESTAMP "2019-12-30 00:00:00")
>
> ```
>
>
>
> And the error thrown:
>
> ```
>
> Field types of query result and registered TableSink
> `kafka`.`test`.`tbl_a` do not match.
>
> Query result schema: [EXPR$0: String, EXPR$1: Integer, EXPR$2: Timestamp]
>
> TableSink schema: [user_name: String, user_id: Integer, login_time:
> LocalDateTime]
>
> ```
>
>
>
> After some digging, I found the root cause might be that since FLINK-14645
> timestamp fields
>
> defined via TableFactory had been bridged to LocalDateTime, but timestamp
> literals are
>
> still backed by java.sql.Timestamp.
>
>
>
> Is my reasoning correct? And is there any workaround? Thanks a lot!
>
>
>
> Best,
>
> Paul Lam
>
>
>
>

RE: SQL Timetamp types incompatible after migration to 1.10

Posted by B....@dell.com.
Hi Jark,

I saw this mail and found this is a similar issue I raised to the community several days ago.[1] Can you have a look to see if it’s the same issue as this.

If yes, there is a further question. From the Pravega connector side, the issue is raised in our Batch Table API which means users using the BatchTableEnvironment to create tables. Currently, BatchTableEnvironment does not support Blink planner. Any suggestions on how we can support Batch Tables in Flink 1.10?

[1] http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Need-help-on-timestamp-type-conversion-for-Table-API-on-Pravega-Connector-td33660.html

Best Regards,
Brian

From: Jark Wu <im...@gmail.com>
Sent: Thursday, March 19, 2020 17:14
To: Paul Lam
Cc: user
Subject: Re: SQL Timetamp types incompatible after migration to 1.10


[EXTERNAL EMAIL]
Hi Paul,

Are you using old planner? Did you try blink planner? I guess it maybe a bug in old planner which doesn't work well on new types.

Best,
Jark

On Thu, 19 Mar 2020 at 16:27, Paul Lam <pa...@gmail.com>> wrote:
Hi,

Recently I upgraded a simple application that inserts static data into a table from 1.9.0 to 1.10.0, and
encountered a timestamp type incompatibility problem during the table sink validation.

The SQL is like:
```
insert into kafka.test.tbl_a # schema: (user_name STRING, user_id INT, login_time TIMESTAMP)
select ("ann", 1000, TIMESTAMP "2019-12-30 00:00:00")
```

And the error thrown:
```
Field types of query result and registered TableSink `kafka`.`test`.`tbl_a` do not match.
Query result schema: [EXPR$0: String, EXPR$1: Integer, EXPR$2: Timestamp]
TableSink schema: [user_name: String, user_id: Integer, login_time: LocalDateTime]
```

After some digging, I found the root cause might be that since FLINK-14645 timestamp fields
defined via TableFactory had been bridged to LocalDateTime, but timestamp literals are
still backed by java.sql.Timestamp.

Is my reasoning correct? And is there any workaround? Thanks a lot!

Best,
Paul Lam


Re: SQL Timetamp types incompatible after migration to 1.10

Posted by Paul Lam <pa...@gmail.com>.
Filed an issue to track this problem. [1]

[1] https://issues.apache.org/jira/browse/FLINK-16693 <https://issues.apache.org/jira/browse/FLINK-16693>

Best,
Paul Lam

> 在 2020年3月20日,17:17,Paul Lam <pa...@gmail.com> 写道:
> 
> Hi Jark,
> 
> Sorry for my late reply. 
> 
> Yes, I’m using the old planner. I’ve tried the blink planner, and it works well.
> 
> We would like to switch to the blink planner, but we’ve developed some custom features on the old planner, 
> so it would take some time to port the codes. So I might give a try to fix the old planner if it’s not too involved. 
> 
> Best,
> Paul Lam
> 
>> 在 2020年3月19日,17:13,Jark Wu <imjark@gmail.com <ma...@gmail.com>> 写道:
>> 
>> Hi Paul,
>> 
>> Are you using old planner? Did you try blink planner? I guess it maybe a bug in old planner which doesn't work well on new types.
>> 
>> Best,
>> Jark
>> 
>> On Thu, 19 Mar 2020 at 16:27, Paul Lam <paullin3280@gmail.com <ma...@gmail.com>> wrote:
>> Hi,
>> 
>> Recently I upgraded a simple application that inserts static data into a table from 1.9.0 to 1.10.0, and 
>> encountered a timestamp type incompatibility problem during the table sink validation.
>> 
>> The SQL is like:
>> ```
>> insert into kafka.test.tbl_a # schema: (user_name STRING, user_id INT, login_time TIMESTAMP)
>> select ("ann", 1000, TIMESTAMP "2019-12-30 00:00:00")
>> ```
>> 
>> And the error thrown:
>> ```
>> Field types of query result and registered TableSink `kafka`.`test`.`tbl_a` do not match.
>>       Query result schema: [EXPR$0: String, EXPR$1: Integer, EXPR$2: Timestamp]
>>       TableSink schema:    [user_name: String, user_id: Integer, login_time: LocalDateTime]
>> ```
>> 
>> After some digging, I found the root cause might be that since FLINK-14645 timestamp fields 
>> defined via TableFactory had been bridged to LocalDateTime, but timestamp literals are 
>> still backed by java.sql.Timestamp.
>> 
>> Is my reasoning correct? And is there any workaround? Thanks a lot!
>> 
>> Best,
>> Paul Lam
>> 
> 


Re: SQL Timetamp types incompatible after migration to 1.10

Posted by Paul Lam <pa...@gmail.com>.
Hi Jark,

Sorry for my late reply. 

Yes, I’m using the old planner. I’ve tried the blink planner, and it works well.

We would like to switch to the blink planner, but we’ve developed some custom features on the old planner, 
so it would take some time to port the codes. So I might give a try to fix the old planner if it’s not too involved. 

Best,
Paul Lam

> 在 2020年3月19日,17:13,Jark Wu <im...@gmail.com> 写道:
> 
> Hi Paul,
> 
> Are you using old planner? Did you try blink planner? I guess it maybe a bug in old planner which doesn't work well on new types.
> 
> Best,
> Jark
> 
> On Thu, 19 Mar 2020 at 16:27, Paul Lam <paullin3280@gmail.com <ma...@gmail.com>> wrote:
> Hi,
> 
> Recently I upgraded a simple application that inserts static data into a table from 1.9.0 to 1.10.0, and 
> encountered a timestamp type incompatibility problem during the table sink validation.
> 
> The SQL is like:
> ```
> insert into kafka.test.tbl_a # schema: (user_name STRING, user_id INT, login_time TIMESTAMP)
> select ("ann", 1000, TIMESTAMP "2019-12-30 00:00:00")
> ```
> 
> And the error thrown:
> ```
> Field types of query result and registered TableSink `kafka`.`test`.`tbl_a` do not match.
>       Query result schema: [EXPR$0: String, EXPR$1: Integer, EXPR$2: Timestamp]
>       TableSink schema:    [user_name: String, user_id: Integer, login_time: LocalDateTime]
> ```
> 
> After some digging, I found the root cause might be that since FLINK-14645 timestamp fields 
> defined via TableFactory had been bridged to LocalDateTime, but timestamp literals are 
> still backed by java.sql.Timestamp.
> 
> Is my reasoning correct? And is there any workaround? Thanks a lot!
> 
> Best,
> Paul Lam
> 


Re: SQL Timetamp types incompatible after migration to 1.10

Posted by Jark Wu <im...@gmail.com>.
Hi Paul,

Are you using old planner? Did you try blink planner? I guess it maybe a
bug in old planner which doesn't work well on new types.

Best,
Jark

On Thu, 19 Mar 2020 at 16:27, Paul Lam <pa...@gmail.com> wrote:

> Hi,
>
> Recently I upgraded a simple application that inserts static data into a
> table from 1.9.0 to 1.10.0, and
> encountered a timestamp type incompatibility problem during the table sink
> validation.
>
> The SQL is like:
> ```
> insert into kafka.test.tbl_a # schema: (user_name STRING, user_id INT,
> login_time TIMESTAMP)
> select ("ann", 1000, TIMESTAMP "2019-12-30 00:00:00")
> ```
>
> And the error thrown:
> ```
> Field types of query result and registered TableSink
> `kafka`.`test`.`tbl_a` do not match.
> Query result schema: [EXPR$0: String, EXPR$1: Integer, EXPR$2: Timestamp]
> TableSink schema: [user_name: String, user_id: Integer, login_time:
> LocalDateTime]
> ```
>
> After some digging, I found the root cause might be that since FLINK-14645
> timestamp fields
> defined via TableFactory had been bridged to LocalDateTime, but timestamp
> literals are
> still backed by java.sql.Timestamp.
>
> Is my reasoning correct? And is there any workaround? Thanks a lot!
>
> Best,
> Paul Lam
>
>