[ https://issues.apache.org/jira/browse/FLINK-3599?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15473596#comment-15473596 ]
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_r77988402 --- Diff: flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java --- @@ -393,6 +443,18 @@ public void addComparatorField(int fieldId, TypeComparator<?> comparator) { keyFields.size() == fieldComparators.size(), "Number of key fields and field comparators is not equal."); + Tuple2<ArrayList<Integer>, Class> custCompKey = new Tuple2(keyFieldIds, getTypeClass()); + if (customComparators.containsKey(custCompKey)) { + return InstantiationUtil.instantiate(customComparators.get(custCompKey)); + } + + if (config.isCodeGenerationEnabled()) { + return new PojoComparatorGenerator<T>(keyFields.toArray(new Field[keyFields.size()]), + fieldComparators.toArray(new TypeComparator[fieldComparators.size()]), createSerializer + (config), getTypeClass(), keyFieldIds.toArray(new Integer[keyFields.size()]), config) --- End diff -- I think it's less verbose to use the parameterless toArray, and cast to the appropriate array type. > 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)