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

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

GitHub user StephanEwen opened a pull request:

    https://github.com/apache/flink/pull/4943

    [FLINK-6022] [avro] Use Avro to serialize Avro in flight and in State 

    ## What is the purpose of the change
    
    This changes Avro types to be serialized with a proper Avro serializer. The 
Avro serializer efficiently handles both Specific Records (generated by Avro) 
and Avro-reflection-based serialization.
    
    In order to maintain backwards compatibility, Avro type info generates 
actually a wrapping serializer that falls back to a Pojo (or Kryo) serializer 
when being reconfigured from an old snapshot.
    
    ## Brief change log
    
      - Adds a proper Avro type serializers
      - Adds a backwards-compatible Avro serializer that falls back to 
Pojo/Kryo on old snapshots
      - Adds a bunch of test
    
    ## Verifying this change
    
      -  Using Avro specific record types in the program and enjoying nice 
performant execution ;-)
      - Using Avro for Flink state and getting it serialized via Avro, allowing 
a schema upgrade of state
      - Running the added unit tests
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (yes / **no)**
      - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (yes / **no)**
      - The serializers: **(yes** / no / don't know)
      - The runtime per-record code paths (performance sensitive): (yes / 
**no** / don't know)
      - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (yes / **no)**
      - If yes, how is the feature documented? (**not applicable** / docs / 
JavaDocs / not documented)


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/StephanEwen/incubator-flink use_proper_avro

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/4943.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4943
    
----
commit dd05a3bf3471702ac8c9129d2d80f2feeca0f949
Author: Stephan Ewen <se...@apache.org>
Date:   2017-11-03T13:47:33Z

    [FLINK-6022] [avro] Use Avro to serialize Avro in flight and in State
    
    This falls back to the original serializer (Pojo / Kryo) in cases where
    an old snapshot is resumed.

----


> 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
>            Priority: Major
>
> 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