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

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

Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/4943
  
    I updated this PR with the following proposed solution:
    
      1. Avro is always part of the user code space, and hence will be loaded 
into the user code classloader. This solves multiple problems:
         - It allows users to use a different Avro version compared to the 
version in a spoiled classpath (for example when Hadoop depenencies pull in 
another Avro version).
         - It means that Avro class and schema caching is done per user code 
classloader, not JVM wide. That prevents "X cannot be cast to X" exceptions.
    
      2. By default, a new pure Avro serializer is used for Avro types (in 
flight and in state). Since Avro class format changed, having compatibility 
activated by default is not possible anyways.
    
      3. If a user wants to recover an old savepoint that happend to have Avro 
types serialized as PoJos, they need to do the following;
         - Set the Avro version to the same version that they had when creating 
the savepoint. Since Avro now lives in user code space, that is possible 
without changing/rebuilding Flink.
         - Pass a Flag to the `AvroTypeInfo` to use a backwards compatible 
Serializer: `new AvroTypeInfo(myType, true)`.


> Improve support for Avro GenericRecord
> --------------------------------------
>
>                 Key: FLINK-6022
>                 URL: https://issues.apache.org/jira/browse/FLINK-6022
>             Project: Flink
>          Issue Type: Improvement
>          Components: Type Serialization System
>            Reporter: Robert Metzger
>
> Currently, Flink is serializing the schema for each Avro GenericRecord in the 
> stream.
> This leads to a lot of overhead over the wire/disk + high serialization costs.
> Therefore, I'm proposing to improve the support for GenericRecord in Flink by 
> shipping the schema to each serializer  through the AvroTypeInformation.
> Then, we can only support GenericRecords with the same type per stream, but 
> the performance will be much better.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to