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

ASF GitHub Bot commented on FLINK-9444:
---------------------------------------

Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6218#discussion_r199746426
  
    --- Diff: 
flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowSerializationSchema.java
 ---
    @@ -37,18 +43,42 @@
     import java.io.IOException;
     import java.io.ObjectInputStream;
     import java.io.ObjectOutputStream;
    +import java.math.BigDecimal;
    +import java.nio.ByteBuffer;
    +import java.sql.Date;
    +import java.sql.Time;
    +import java.sql.Timestamp;
    +import java.util.HashMap;
     import java.util.List;
    +import java.util.Map;
    +import java.util.TimeZone;
     
     /**
    - * Serialization schema that serializes {@link Row} over {@link 
SpecificRecord} into a Avro bytes.
    + * Serialization schema that serializes {@link Row} into Avro bytes.
    + *
    + * <p>Serializes objects that are represented in (nested) Flink rows. It 
support types that
    + * are compatible with Flink's Table & SQL API.
    + *
    + * <p>Note: Changes in this class need to be kept in sync with the 
corresponding runtime
    + * class {@link AvroRowDeserializationSchema} and schema converter {@link 
AvroSchemaConverter}.
      */
     public class AvroRowSerializationSchema implements 
SerializationSchema<Row> {
     
        /**
    -    * Avro record class.
    +    * Used for time conversions into SQL types.
    +    */
    +   private static final TimeZone LOCAL_TZ = TimeZone.getDefault();
    --- End diff --
    
    We are using this pattern at different places. E.g. 
`org.apache.flink.orc.OrcBatchReader`. The problem is that Java's SQL 
time/date/timestamp are a complete design fail. They are timezone specific. 
This adds/removes the local timezone from the timestamp. Such that the string 
representation of the produced `Timestamp` object is always correct.


> KafkaAvroTableSource failed to work for map and array fields
> ------------------------------------------------------------
>
>                 Key: FLINK-9444
>                 URL: https://issues.apache.org/jira/browse/FLINK-9444
>             Project: Flink
>          Issue Type: Bug
>          Components: Kafka Connector, Table API &amp; SQL
>    Affects Versions: 1.6.0
>            Reporter: Jun Zhang
>            Assignee: Jun Zhang
>            Priority: Blocker
>              Labels: patch, pull-request-available
>             Fix For: 1.6.0
>
>         Attachments: flink-9444.patch
>
>
> When some Avro schema has map/array fields and the corresponding TableSchema 
> declares *MapTypeInfo/ListTypeInfo* for these fields, an exception will be 
> thrown when registering the *KafkaAvroTableSource*, complaining like:
> Exception in thread "main" org.apache.flink.table.api.ValidationException: 
> Type Map<String, Integer> of table field 'event' does not match with type 
> GenericType<java.util.Map> of the field 'event' of the TableSource return 
> type.
>  at org.apache.flink.table.api.ValidationException$.apply(exceptions.scala:74)
>  at 
> org.apache.flink.table.sources.TableSourceUtil$$anonfun$validateTableSource$1.apply(TableSourceUtil.scala:92)
>  at 
> org.apache.flink.table.sources.TableSourceUtil$$anonfun$validateTableSource$1.apply(TableSourceUtil.scala:71)
>  at 
> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
>  at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
>  at 
> org.apache.flink.table.sources.TableSourceUtil$.validateTableSource(TableSourceUtil.scala:71)
>  at 
> org.apache.flink.table.plan.schema.StreamTableSourceTable.<init>(StreamTableSourceTable.scala:33)
>  at 
> org.apache.flink.table.api.StreamTableEnvironment.registerTableSourceInternal(StreamTableEnvironment.scala:124)
>  at 
> org.apache.flink.table.api.TableEnvironment.registerTableSource(TableEnvironment.scala:438)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to