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

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

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

    https://github.com/apache/flink/pull/3305#discussion_r101070858
  
    --- Diff: 
flink-core/src/main/java/org/apache/flink/api/java/typeutils/ListTypeInfo.java 
---
    @@ -0,0 +1,116 @@
    +/*
    + * 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;
    +
    +import org.apache.flink.annotation.Public;
    +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.common.typeutils.base.ListSerializer;
    +
    +import java.util.List;
    +
    +/**
    + * A {@link TypeInformation} for the list types of the Java API.
    + *
    + * @param <T> The type of the elements in the list.
    + */
    +
    +
    +@Public
    +public final class ListTypeInfo<T> extends TypeInformation<List<T>> {
    +
    +   private final TypeInformation<T> elementTypeInfo;
    +
    +   public ListTypeInfo(Class<T> elementTypeClass) {
    +           this.elementTypeInfo = 
TypeExtractor.createTypeInfo(elementTypeClass);
    +   }
    +
    +   public ListTypeInfo(TypeInformation<T> elementTypeInfo) {
    +           this.elementTypeInfo = elementTypeInfo;
    +   }
    +
    +   public TypeInformation<T> getElementTypeInfo() {
    +           return elementTypeInfo;
    +   }
    +
    +   @Override
    +   public boolean isBasicType() {
    +           return false;
    +   }
    +
    +   @Override
    +   public boolean isTupleType() {
    +           return false;
    +   }
    +
    +   @Override
    +   public int getArity() {
    +           return 0;
    +   }
    +
    +   @Override
    +   public int getTotalFields() {
    +           return elementTypeInfo.getTotalFields();
    +   }
    +
    +   @SuppressWarnings("unchecked")
    +   @Override
    +   public Class<List<T>> getTypeClass() {
    +           return (Class<List<T>>)(Class<?>)List.class;
    +   }
    +
    +   @Override
    +   public boolean isKeyType() {
    +           return false;
    +   }
    +
    +   @Override
    +   public TypeSerializer<List<T>> createSerializer(ExecutionConfig config) 
{
    +           TypeSerializer<T> elementTypeSerializer = 
elementTypeInfo.createSerializer(config);
    +           return new ListSerializer<>(elementTypeSerializer);
    +   }
    +
    +   @Override
    +   public String toString() {
    +           return null;
    --- End diff --
    
    I think here we should return a more meaningful value. Something like 
`List<elementType.toString>`.


> Use list types when ListStateDescriptor extends StateDescriptor
> ---------------------------------------------------------------
>
>                 Key: FLINK-5790
>                 URL: https://issues.apache.org/jira/browse/FLINK-5790
>             Project: Flink
>          Issue Type: Improvement
>            Reporter: Xiaogang Shi
>            Assignee: Xiaogang Shi
>
> Flink keeps the state serializer in {{StateDescriptor}}, but it's the 
> serializer of list elements  that is put in {{ListStateDescriptor}}. The 
> implementation is a little confusing. Some backends need to construct the 
> state serializer with the element serializer by themselves.
> We should use an {{ArrayListSerializer}}, which is composed of the serializer 
> of the element, in the {{ListStateDescriptor}}. It helps the backend to avoid 
> constructing the state serializer.
> If a backend needs customized serialization of the state (e.g. 
> {{RocksDBStateBackend}}), it still can obtain the element serializer from the 
> {{ArrayListSerializer}}.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to