[
https://issues.apache.org/jira/browse/FLINK-16693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17065840#comment-17065840
]
Jark Wu commented on FLINK-16693:
---------------------------------
Hi [~twalthr], I think it is caused by FLINK-14645. However, it's not a bug but
an incompatible change. As the release note said,
{quote}The TableSchema derived from properties now bridges to the default
conversion class, i.e. LocalDateTime/LocalDate/LocalTime, instead of
java.sql.Timestamp/Date/Time. All the connectors consume/produce the
java.sql.Timestamp/Date/Time classes should adapt to the new conversion
classes.{quote}
You shouldn't call {{TableSchema#getFieldTypes()}} which is deprecated since
1.9. Instead, you should using the following code to get a RowTypeInfo with
Timestamp type in it.
{code:java}
DataType dataType = DataTypeUtils.transform(schema.toRowDataType(),
TypeTransformations.timeToSqlTypes());
RowTypeInfo rowTypeInfo = (RowTypeInfo)
TypeConversions.fromDataTypeToLegacyInfo(dataType);
{code}
[1]:
https://github.com/pravega/flink-connectors/blob/278aab3e62d0609afe6e40d9d5b1daebd5789d36/src/main/java/io/pravega/connectors/flink/FlinkPravegaTableFactoryBase.java#L233
> Legacy planner incompatible with Timestamp backed by LocalDateTime
> ------------------------------------------------------------------
>
> Key: FLINK-16693
> URL: https://issues.apache.org/jira/browse/FLINK-16693
> Project: Flink
> Issue Type: Bug
> Components: Table SQL / Legacy Planner
> Affects Versions: 1.10.0
> Reporter: Paul Lin
> Priority: Major
>
> 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 functions are still backed by java.sql.Timestamp.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)