[
https://issues.apache.org/jira/browse/FLINK-3599?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15366478#comment-15366478
]
ASF GitHub Bot commented on FLINK-3599:
---------------------------------------
Github user greghogan commented on a diff in the pull request:
https://github.com/apache/flink/pull/2211#discussion_r69952548
--- Diff:
flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java
---
@@ -70,10 +75,21 @@
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<Class<?>, Class<? extends TypeComparator>>
customComparators = new HashMap<>();
+
private final PojoField[] fields;
private final int totalFields;
+ public static <C, S extends TypeSerializer<C>> void
registerCustomSerializer(Class<C> c, Class<S> s) {
+ customSerializers.put(c, s);
--- End diff --
Should we also do a preconditions check here (and in
`registerCustomComparator`) that `s` has a single public constructor? Is the
assertion message obvious in `createSerializer`?
> 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)