[
https://issues.apache.org/jira/browse/FLINK-3599?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15473725#comment-15473725
]
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_r77995157
--- Diff:
flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenTypeComparatorProxy.java
---
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.typeutils.runtime;
+
+import org.apache.flink.api.common.typeutils.CompositeTypeComparator;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.util.InstantiationUtil;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.lang.reflect.Constructor;
+import java.util.List;
+
+public class GenTypeComparatorProxy<T> extends CompositeTypeComparator<T>
implements java.io.Serializable {
+ private final String code;
+ private final String name;
+ private final Class<T> clazz;
+ private final TypeComparator<Object>[] comparators;
+ private final TypeSerializer<T> serializer;
+
+ transient private CompositeTypeComparator<T> impl = null;
+
+ private void compile() {
+ try {
+ assert impl == null;
+ Class<?> comparatorClazz =
InstantiationUtil.compile(clazz.getClassLoader(), name, code);
+ Constructor<?>[] ctors =
comparatorClazz.getConstructors();
+ assert ctors.length == 1;
+ impl = (CompositeTypeComparator<T>)
ctors[0].newInstance(new Object[]{comparators, serializer, clazz});
+ } catch (Exception e) {
+ throw new RuntimeException("Unable to generate
serializer: " + name, e);
+ }
+ }
+
+ public GenTypeComparatorProxy(Class<T> clazz, String name, String
code,TypeComparator<Object>[] comparators,
+
TypeSerializer<T> serializer) {
+ this.name = name;
+ this.code = code;
+ this.clazz = clazz;
+ this.comparators = comparators;
+ this.serializer = serializer;
+ compile();
+ }
+
+ @SuppressWarnings("unchecked")
--- End diff --
Not needed if `comparators` has `<?>`
> 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)