Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 13740200C2A for ; Tue, 14 Feb 2017 17:09:48 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 1232A160B52; Tue, 14 Feb 2017 16:09:48 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 376F7160B6D for ; Tue, 14 Feb 2017 17:09:47 +0100 (CET) Received: (qmail 63229 invoked by uid 500); 14 Feb 2017 16:09:46 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 63203 invoked by uid 99); 14 Feb 2017 16:09:46 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 14 Feb 2017 16:09:46 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id D31BD186144 for ; Tue, 14 Feb 2017 16:09:45 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -1.198 X-Spam-Level: X-Spam-Status: No, score=-1.198 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, KAM_LAZY_DOMAIN_SECURITY=1, RP_MATCHES_RCVD=-2.999, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id bK-q-NHl7I6h for ; Tue, 14 Feb 2017 16:09:44 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id 5BC2860F56 for ; Tue, 14 Feb 2017 16:09:44 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id F19EAE07DD for ; Tue, 14 Feb 2017 16:09:42 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 1566824146 for ; Tue, 14 Feb 2017 16:09:42 +0000 (UTC) Date: Tue, 14 Feb 2017 16:09:42 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-5790) Use list types when ListStateDescriptor extends StateDescriptor MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Tue, 14 Feb 2017 16:09:48 -0000 [ 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 The type of the elements in the list. + */ + + +@Public +public final class ListTypeInfo extends TypeInformation> { + + private final TypeInformation elementTypeInfo; + + public ListTypeInfo(Class elementTypeClass) { + this.elementTypeInfo = TypeExtractor.createTypeInfo(elementTypeClass); + } + + public ListTypeInfo(TypeInformation elementTypeInfo) { + this.elementTypeInfo = elementTypeInfo; + } + + public TypeInformation 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> getTypeClass() { + return (Class>)(Class)List.class; + } + + @Override + public boolean isKeyType() { + return false; + } + + @Override + public TypeSerializer> createSerializer(ExecutionConfig config) { + TypeSerializer 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`. > 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)