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

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_r78001809
  
    --- Diff: flink-core/src/main/resources/PojoSerializerTemplate.ftl ---
    @@ -0,0 +1,372 @@
    +/*
    + * 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 ${packageName};
    +import java.io.IOException;
    +import java.io.ObjectInputStream;
    +import java.io.ObjectOutputStream;
    +import java.lang.reflect.Modifier;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.LinkedHashMap;
    +import java.util.LinkedHashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import org.apache.flink.api.common.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +public final class ${className} extends TypeSerializer {
    +   private static byte IS_NULL = 1;
    +   private static byte NO_SUBCLASS = 2;
    +   private static byte IS_SUBCLASS = 4;
    +   private static byte IS_TAGGED_SUBCLASS = 8;
    +   private int numFields;
    +   private ExecutionConfig executionConfig;
    +   private Map<Class, TypeSerializer> subclassSerializerCache;
    +   private final Map<Class, Integer> registeredClasses;
    +   private final TypeSerializer[] registeredSerializers;
    +   Class clazz;
    +   <#list members as m>
    +   ${m}
    +   </#list>
    +   public ${className}(Class clazz, TypeSerializer[] serializerFields, 
ExecutionConfig e) {
    +           this.clazz = clazz;
    +           executionConfig = e;
    +           this.numFields = serializerFields.length;
    +           LinkedHashSet<Class> registeredPojoTypes = 
executionConfig.getRegisteredPojoTypes();
    +           subclassSerializerCache = new HashMap<Class, TypeSerializer>();
    +           List<Class> cleanedTaggedClasses = new 
ArrayList<Class>(registeredPojoTypes.size());
    +           for (Class registeredClass: registeredPojoTypes) {
    +                   if (registeredClass.equals(clazz)) {
    +                           continue;
    +                   }
    +                   if (!clazz.isAssignableFrom(registeredClass)) {
    +                           continue;
    +                   }
    +                   cleanedTaggedClasses.add(registeredClass);
    +           }
    +           this.registeredClasses = new LinkedHashMap<Class, 
Integer>(cleanedTaggedClasses.size());
    +           registeredSerializers = new 
TypeSerializer[cleanedTaggedClasses.size()];
    +           int id = 0;
    +           for (Class registeredClass: cleanedTaggedClasses) {
    +                   this.registeredClasses.put(registeredClass, id);
    +                   TypeInformation typeInfo = 
TypeExtractor.createTypeInfo(registeredClass);
    +                   registeredSerializers[id] = 
typeInfo.createSerializer(executionConfig);
    +                   id++;
    +           }
    +           <#list initMembers as m>
    +           ${m}
    +           </#list>
    +   }
    +   private TypeSerializer getSubclassSerializer(Class subclass) {
    +           TypeSerializer result = 
(TypeSerializer)subclassSerializerCache.get(subclass);
    +           if (result == null) {
    +                   TypeInformation typeInfo = 
TypeExtractor.createTypeInfo(subclass);
    +                   result = typeInfo.createSerializer(executionConfig);
    +                   subclassSerializerCache.put(subclass, result);
    +           }
    +           return result;
    +   }
    +   public boolean isImmutableType() { return false; }
    +   public ${className} duplicate() {
    +           boolean stateful = false;
    +           TypeSerializer[] duplicateFieldSerializers = new 
TypeSerializer[numFields];
    +           <#list duplicateSerializers as ds>
    +           ${ds}
    +           </#list>
    +           if (stateful) {
    +                   return new ${className}(clazz, 
duplicateFieldSerializers, executionConfig);
    +           } else {
    +                   return this;
    +           }
    +   }
    +   public ${typeName} createInstance() {
    +           if (clazz.isInterface() || 
Modifier.isAbstract(clazz.getModifiers())) {
    +                   return null;
    +           }
    +           try {
    +                   ${typeName} t = (${typeName})clazz.newInstance();
    --- End diff --
    
    You can generate `new ${typeName}` here, which avoids reflection. Also, the 
above checks can be performed at code generation time.


> 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