[ 
https://issues.apache.org/jira/browse/FLINK-17091?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17105183#comment-17105183
 ] 

Dawid Wysakowicz commented on FLINK-17091:
------------------------------------------

Good point that {{Timestamp}} type does not support {{long}} as the conversion 
class.  Actually I should've said that the {{AvroRowSerializationSchema}} in 
current implementation should return 
{{DataTypes.Timestamp().bridgedTo(java.sql.Timestamp.class)}}.

I don't think the problem lays in {{DescriptorProperties}}. What the 
{{TableSourceFactory}} receives is a requested logical schema (that's the case 
e.g. from DDL). It should be the responsibility of the source+schema to enrich 
that with the bridging classes the schema actually produces for those logical 
types. By the way I think this mechanism will be completely revamped in FLIP-95

I don't think adding a conversion between  
{{LocalDateTime/java.sql.Timestamp}}, {{long }} is correct. There is a reason 
why {{TIMESTAMP}} does not have that conversion. The correct type in this case 
would be {{TIMESTAMP_WITH_LOCAL_TIME_ZONE}}.

> AvroRow(De)SerializationSchema doesn't support new Timestamp conversion 
> classes
> -------------------------------------------------------------------------------
>
>                 Key: FLINK-17091
>                 URL: https://issues.apache.org/jira/browse/FLINK-17091
>             Project: Flink
>          Issue Type: Bug
>          Components: Formats (JSON, Avro, Parquet, ORC, SequenceFile)
>    Affects Versions: 1.10.0
>            Reporter: Paul Lin
>            Priority: Major
>
> AvroRow(De)SerializationSchema doesn't know how to convert the new physical 
> classes of Timestamp (eg. java.time.Date) to/from Avro's int/long based 
> timestamp. Currently, when encountering objects of the new physical classes, 
> AvroRow(De)SerializationSchema just ignores them and passes them to Avro's 
> GenericDatumWriter/Reader, which leads to ClassCastException thrown by 
> GenericDatumWriter/Reader. See 
> [AvroRowSerializationSchema|https://github.com/apache/flink/blob/master/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowSerializationSchema.java#L251].
> To fix this problem, we should support LocalTime/LocalDate/LocalDateTime 
> conversion to int/long in AvroRowSerializationSchema, and support int/long 
> conversion to LocalTime/LocalDate/LocalDateTime based on logical 
> types(Types.LOCAL_TIME/Types.LOCAL_DATE/Types.LOCAL_DATE_TIME) in 
> AvroRowDeserializationSchema.



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

Reply via email to