[ 
https://issues.apache.org/jira/browse/FLINK-19629?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

shizhengchao updated FLINK-19629:
---------------------------------
    Description: 
create table tableA (
  name  STRING,
  hobly MAP<STRING, STRING>,
  phone STRING
) with (
  'connector' = 'kafka-0.11',
  'topic' = 'ShizcTest',
  'properties.bootstrap.servers' = 'localhost:9092',
  'properties.group.id' = 'ShizcTest',
  'scan.startup.mode' = 'earliest-offset',
  'format' = 'avro'
);
if hobly have an null value like this:

{"name": "shizc", "hobly": {"key1":null}, "phone": "1104564"}

cause an NullPointException:

{code:java}
java.io.IOException: Failed to deserialize Avro record.
        at 
org.apache.flink.formats.avro.AvroRowDataDeserializationSchema.deserialize(AvroRowDataDeserializationSchema.java:150)
        at 
org.apache.flink.formats.avro.AvroRowDataDeserializationSchema.deserialize(AvroRowDataDeserializationSchema.java:75)
        at 
org.apache.flink.api.common.serialization.DeserializationSchema.deserialize(DeserializationSchema.java:81)
        at 
org.apache.flink.streaming.connectors.kafka.internals.KafkaDeserializationSchemaWrapper.deserialize(KafkaDeserializationSchemaWrapper.java:56)
        at 
org.apache.flink.streaming.connectors.kafka.internal.Kafka010Fetcher.runFetchLoop(Kafka010Fetcher.java:147)
        at 
org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase.run(FlinkKafkaConsumerBase.java:755)
        at 
org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:100)
        at 
org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:63)
        at 
org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:201)
Caused by: java.lang.NullPointerException: null
        at 
org.apache.flink.formats.avro.AvroRowDataDeserializationSchema.lambda$createConverter$57e941b$5(AvroRowDataDeserializationSchema.java:252)
        at 
org.apache.flink.formats.avro.AvroRowDataDeserializationSchema.lambda$createMapConverter$7941d275$1(AvroRowDataDeserializationSchema.java:315)
        at 
org.apache.flink.formats.avro.AvroRowDataDeserializationSchema.lambda$createNullableConverter$c3bac5d8$1(AvroRowDataDeserializationSchema.java:221)
        at 
org.apache.flink.formats.avro.AvroRowDataDeserializationSchema.lambda$createRowConverter$80d8b6bd$1(AvroRowDataDeserializationSchema.java:206)
        at 
org.apache.flink.formats.avro.AvroRowDataDeserializationSchema.deserialize(AvroRowDataDeserializationSchema.java:148)
        ... 8 common frames omitted
{code}


  was:
the docs Connectors/Table & SQL Connectors/Formats/Avro:
 In addition to the types listed above, Flink supports reading/writing nullable 
types. Flink maps nullable types to Avro union(something, null), where 
something is the Avro type converted from Flink type.

avro have no union type, should be unions:
 Avro unions(something, null)

by the way, an example should be provided that reading/writing nullable types, 
such as this:
{code:java}
CREATE TABLE user_behavior (
  behavior STRING NULL
) WITH (
 'connector' = 'kafka',
 'topic' = 'user_behavior',
 'properties.bootstrap.servers' = 'localhost:9092',
 'properties.group.id' = 'testGroup',
 'format' = 'avro'
)
{code}


> Avro format cause NullPointException,as null value in MAP type's  value type
> ----------------------------------------------------------------------------
>
>                 Key: FLINK-19629
>                 URL: https://issues.apache.org/jira/browse/FLINK-19629
>             Project: Flink
>          Issue Type: Bug
>          Components: Formats (JSON, Avro, Parquet, ORC, SequenceFile)
>    Affects Versions: 1.11.2
>            Reporter: shizhengchao
>            Priority: Critical
>             Fix For: 1.12.0
>
>
> create table tableA (
>   name  STRING,
>   hobly MAP<STRING, STRING>,
>   phone STRING
> ) with (
>   'connector' = 'kafka-0.11',
>   'topic' = 'ShizcTest',
>   'properties.bootstrap.servers' = 'localhost:9092',
>   'properties.group.id' = 'ShizcTest',
>   'scan.startup.mode' = 'earliest-offset',
>   'format' = 'avro'
> );
> if hobly have an null value like this:
> {"name": "shizc", "hobly": {"key1":null}, "phone": "1104564"}
> cause an NullPointException:
> {code:java}
> java.io.IOException: Failed to deserialize Avro record.
>       at 
> org.apache.flink.formats.avro.AvroRowDataDeserializationSchema.deserialize(AvroRowDataDeserializationSchema.java:150)
>       at 
> org.apache.flink.formats.avro.AvroRowDataDeserializationSchema.deserialize(AvroRowDataDeserializationSchema.java:75)
>       at 
> org.apache.flink.api.common.serialization.DeserializationSchema.deserialize(DeserializationSchema.java:81)
>       at 
> org.apache.flink.streaming.connectors.kafka.internals.KafkaDeserializationSchemaWrapper.deserialize(KafkaDeserializationSchemaWrapper.java:56)
>       at 
> org.apache.flink.streaming.connectors.kafka.internal.Kafka010Fetcher.runFetchLoop(Kafka010Fetcher.java:147)
>       at 
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase.run(FlinkKafkaConsumerBase.java:755)
>       at 
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:100)
>       at 
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:63)
>       at 
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:201)
> Caused by: java.lang.NullPointerException: null
>       at 
> org.apache.flink.formats.avro.AvroRowDataDeserializationSchema.lambda$createConverter$57e941b$5(AvroRowDataDeserializationSchema.java:252)
>       at 
> org.apache.flink.formats.avro.AvroRowDataDeserializationSchema.lambda$createMapConverter$7941d275$1(AvroRowDataDeserializationSchema.java:315)
>       at 
> org.apache.flink.formats.avro.AvroRowDataDeserializationSchema.lambda$createNullableConverter$c3bac5d8$1(AvroRowDataDeserializationSchema.java:221)
>       at 
> org.apache.flink.formats.avro.AvroRowDataDeserializationSchema.lambda$createRowConverter$80d8b6bd$1(AvroRowDataDeserializationSchema.java:206)
>       at 
> org.apache.flink.formats.avro.AvroRowDataDeserializationSchema.deserialize(AvroRowDataDeserializationSchema.java:148)
>       ... 8 common frames omitted
> {code}



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

Reply via email to