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

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

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

    https://github.com/apache/flink/pull/2211#discussion_r77987079
  
    --- Diff: 
flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java 
---
    @@ -70,10 +77,41 @@
        private static final Pattern PATTERN_NESTED_FIELDS = 
Pattern.compile(REGEX_NESTED_FIELDS);
        private static final Pattern PATTERN_NESTED_FIELDS_WILDCARD = 
Pattern.compile(REGEX_NESTED_FIELDS_WILDCARD);
     
    +   private static final Map<Class<?>, Class<? extends TypeSerializer>> 
customSerializers = new HashMap<>();
    +   private static final Map<Tuple2<ArrayList<Integer>, Class>, Class<? 
extends TypeComparator>> customComparators =
    +           new HashMap<>();
    +
        private final PojoField[] fields;
        
        private final int totalFields;
     
    +   /**
    +    * Register a custom serializer for a type. The precedence of the 
serializers
    +    * is the following (highest to lowest): Kryo, Avro, Custom, Generated, 
Flink.
    +    * The chosen serializer will be the first one from the list that is 
turned on.
    +    *
    +    */
    +   public static <C, S extends TypeSerializer<C>> void 
registerCustomSerializer(Class<C> clazz, Class<S> ser) {
    +           Constructor<?>[] ctors = ser.getConstructors();
    +           assert ctors.length == 1;
    +           assert ctors[0].getParameterTypes().length == 0;
    --- End diff --
    
    Is it documented somewhere that custom serializers have to have these 
properties?
    
    Also, these shouldn't be asserts, but throw exceptions instead. (I think 
asserts are generally for internal consistency stuff, i.e., they should fire 
only when you have a bug, and not when some stuff that a user gave us is not in 
the right form.)


> GSoC: Code Generation in Serializers
> ------------------------------------
>
>                 Key: FLINK-3599
>                 URL: https://issues.apache.org/jira/browse/FLINK-3599
>             Project: Flink
>          Issue Type: Improvement
>          Components: Type Serialization System
>            Reporter: Márton Balassi
>            Assignee: Gabor Horvath
>              Labels: gsoc2016, mentor
>
> The current implementation of the serializers can be a
> performance bottleneck in some scenarios. These performance problems were
> also reported on the mailing list recently [1].
> E.g. the PojoSerializer uses reflection for accessing the fields, which is 
> slow [2].
> For the complete proposal see [3].
> [1] 
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/Tuple-performance-and-the-curious-JIT-compiler-td10666.html
> [2] 
> https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java#L369
> [3] 
> https://docs.google.com/document/d/1VC8lCeErx9kI5lCMPiUn625PO0rxR-iKlVqtt3hkVnk



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to