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 B207A200D22 for ; Fri, 15 Sep 2017 15:39:12 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id B0AA91609CF; Fri, 15 Sep 2017 13:39:12 +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 AE2EB1609D3 for ; Fri, 15 Sep 2017 15:39:11 +0200 (CEST) Received: (qmail 42299 invoked by uid 500); 15 Sep 2017 13:39:10 -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 42137 invoked by uid 99); 15 Sep 2017 13:39:10 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 15 Sep 2017 13:39:10 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 903931A0EE2 for ; Fri, 15 Sep 2017 13:39:09 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -100.001 X-Spam-Level: X-Spam-Status: No, score=-100.001 tagged_above=-999 required=6.31 tests=[RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, URIBL_BLOCKED=0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id Arn53XDv-rPb for ; Fri, 15 Sep 2017 13:39:06 +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 8DCDC5FBDF for ; Fri, 15 Sep 2017 13:39:06 +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 55C22E0F23 for ; Fri, 15 Sep 2017 13:39:04 +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 9CBFA253A2 for ; Fri, 15 Sep 2017 13:39:01 +0000 (UTC) Date: Fri, 15 Sep 2017 13:39:01 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-7452) Add helper methods for all built-in Flink types to Types MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Fri, 15 Sep 2017 13:39:12 -0000 [ https://issues.apache.org/jira/browse/FLINK-7452?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16167878#comment-16167878 ] ASF GitHub Bot commented on FLINK-7452: --------------------------------------- Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/4612#discussion_r139112027 --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeinfo/Types.java --- @@ -19,56 +19,408 @@ package org.apache.flink.api.common.typeinfo; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.functions.InvalidTypesException; +import org.apache.flink.api.java.typeutils.EitherTypeInfo; +import org.apache.flink.api.java.typeutils.EnumTypeInfo; +import org.apache.flink.api.java.typeutils.GenericTypeInfo; +import org.apache.flink.api.java.typeutils.ListTypeInfo; +import org.apache.flink.api.java.typeutils.MapTypeInfo; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.PojoField; +import org.apache.flink.api.java.typeutils.PojoTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.api.java.typeutils.ValueTypeInfo; -import java.math.BigDecimal; -import java.sql.Date; -import java.sql.Time; -import java.sql.Timestamp; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; /** - * This class gives access to the type information of the most most common types. + * This class gives access to the type information of the most common types for which Flink + * has built-in serializers and comparators. + * + *

In many cases, Flink tries to analyze generic signatures of functions to determine return + * types automatically. This class is intended for cases where the extraction is not possible + * (or inefficient) as well as cases where type information has to be supplied manually. + * + *

Depending on the API you are using (e.g. Scala API or Table API), there might be a more + * specialized Types class. + * + *

A more convenient alternative might be a {@link TypeHint}. + * + * @see TypeInformation#of(Class) specify type information based on a class that will be analyzed + * @see TypeInformation#of(TypeHint) specify type information based on a {@link TypeHint} */ @PublicEvolving public class Types { - public static final BasicTypeInfo STRING = BasicTypeInfo.STRING_TYPE_INFO; - public static final BasicTypeInfo BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO; - public static final BasicTypeInfo BYTE = BasicTypeInfo.BYTE_TYPE_INFO; - public static final BasicTypeInfo SHORT = BasicTypeInfo.SHORT_TYPE_INFO; - public static final BasicTypeInfo INT = BasicTypeInfo.INT_TYPE_INFO; - public static final BasicTypeInfo LONG = BasicTypeInfo.LONG_TYPE_INFO; - public static final BasicTypeInfo FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO; - public static final BasicTypeInfo DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO; - public static final BasicTypeInfo DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO; + /** + * Returns type information for {@link java.lang.Void}. Does not support a null value. + */ + public static final TypeInformation VOID = BasicTypeInfo.VOID_TYPE_INFO; + + /** + * Returns type information for {@link java.lang.String}. Supports a null value. + */ + public static final TypeInformation STRING = BasicTypeInfo.STRING_TYPE_INFO; + + /** + * Returns type information for both a primitive byte and a + * wrapped {@link java.lang.Byte}. Does not support a null value. + */ + public static final TypeInformation BYTE = BasicTypeInfo.BYTE_TYPE_INFO; + + /** + * Returns type information for both a primitive boolean and a + * wrapped {@link java.lang.Boolean}. Does not support a null value. + */ + public static final TypeInformation BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO; + + /** + * Returns type information for both a primitive short and a + * wrapped {@link java.lang.Short}. Does not support a null value. + */ + public static final TypeInformation SHORT = BasicTypeInfo.SHORT_TYPE_INFO; - public static final SqlTimeTypeInfo SQL_DATE = SqlTimeTypeInfo.DATE; - public static final SqlTimeTypeInfo

A row is a variable-length, null-aware composite type for storing multiple values in a + * deterministic field order. Every field can be null independent of the field's type. + * Fields of a row are untyped; therefore, it is required to pass type information whenever a row is used. * - *

This method is a shortcut to {@code new RowTypeInfo(types)}. + *

This method generates type information with fields of the given types; the fields have + * the default names (f0, f1, f2 ..). * * @param types The types of the row fields, e.g., Types.STRING, Types.INT */ - public static RowTypeInfo ROW(TypeInformation... types) { + public static TypeInformation ROW(TypeInformation... types) { return new RowTypeInfo(types); } /** - * Generates a RowTypeInfo with fields of the given types and with given names. - * + * Returns type information for {@link org.apache.flink.types.Row} with fields of the given types and + * with given names. A row must not be null. + * + *

A row is a variable-length, null-aware composite type for storing multiple values in a + * deterministic field order. Every field can be null independent of the field's type. + * Fields of a row are untyped; therefore, it is required to pass type information whenever a row is used. + * *

Example use: {@code ROW_NAMED(new String[]{"name", "number"}, Types.STRING, Types.INT)}. - * - *

This method is identical to {@code new RowTypeInfo(types, names)}. * * @param fieldNames array of field names * @param types array of field types */ - public static RowTypeInfo ROW_NAMED(String[] fieldNames, TypeInformation... types) { + public static TypeInformation ROW_NAMED(String[] fieldNames, TypeInformation... types) { return new RowTypeInfo(types, fieldNames); } + + /** + * Returns type information for subclasses of Flink's {@link org.apache.flink.api.java.tuple.Tuple} + * (namely {@link org.apache.flink.api.java.tuple.Tuple0} till {@link org.apache.flink.api.java.tuple.Tuple25}) + * with fields of the given types. A tuple must not be null. + * + *

A tuple is a fixed-length composite type for storing multiple values in a + * deterministic field order. Fields of a tuple are typed. Tuples are the most efficient composite + * type; therefore, a tuple does not support null values unless its field type supports nullability. + * + * @param types The types of the tuple fields, e.g., Types.STRING, Types.INT + */ + public static TypeInformation TUPLE(TypeInformation... types) { + return new TupleTypeInfo<>(types); + } + + /** + * Returns type information for typed subclasses of Flink's {@link org.apache.flink.api.java.tuple.Tuple}. + * Typed subclassed are classes that extend {@link org.apache.flink.api.java.tuple.Tuple0} till + * {@link org.apache.flink.api.java.tuple.Tuple25} to provide types for all fields and might add --- End diff -- Explicitly mention that no new member fields may be added. > Add helper methods for all built-in Flink types to Types > -------------------------------------------------------- > > Key: FLINK-7452 > URL: https://issues.apache.org/jira/browse/FLINK-7452 > Project: Flink > Issue Type: Improvement > Components: Type Serialization System > Reporter: Timo Walther > Assignee: Timo Walther > > Sometimes it is very difficult to provide `TypeInformation` manually, in case some extraction fails or is not available. {{TypeHint}}s should be the preferred way but this methods can ensure correct types. > I propose to add all built-in Flink types to the {{Types}}. Such as: > {code} > Types.POJO(MyPojo.class) > Types.POJO(Map) > Types.GENERIC(Object.class) > Types.TUPLE(TypeInformation, ...) > Types.MAP(TypeInformation, TypeInformation) > {code} > The methods should validate that the returned type is exactly the requested type. And especially in case of POJO should help creating {{PojoTypeInfo}}. > Once this is in place, we can deprecate the {{TypeInfoParser}}. -- This message was sent by Atlassian JIRA (v6.4.14#64029)