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 9629C200D5A for ; Thu, 14 Dec 2017 17:03:19 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 94A57160C25; Thu, 14 Dec 2017 16:03:19 +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 8CBFD160C01 for ; Thu, 14 Dec 2017 17:03:18 +0100 (CET) Received: (qmail 57266 invoked by uid 500); 14 Dec 2017 16:03:17 -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 57257 invoked by uid 99); 14 Dec 2017 16:03:17 -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; Thu, 14 Dec 2017 16:03:17 +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 4418B1809DE for ; Thu, 14 Dec 2017 16:03:17 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -4.021 X-Spam-Level: X-Spam-Status: No, score=-4.021 tagged_above=-999 required=6.31 tests=[KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-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 eYGMPknoHszW for ; Thu, 14 Dec 2017 16:03:11 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with SMTP id 1054F5FB29 for ; Thu, 14 Dec 2017 16:03:10 +0000 (UTC) Received: (qmail 53518 invoked by uid 99); 14 Dec 2017 16:03:10 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 14 Dec 2017 16:03:10 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 17182F16CF; Thu, 14 Dec 2017 16:03:10 +0000 (UTC) From: fhueske To: issues@flink.incubator.apache.org Reply-To: issues@flink.incubator.apache.org References: In-Reply-To: Subject: [GitHub] flink pull request #4612: [FLINK-7452] [types] Add helper methods for all bu... Content-Type: text/plain Message-Id: <20171214160310.17182F16CF@git1-us-west.apache.org> Date: Thu, 14 Dec 2017 16:03:10 +0000 (UTC) archived-at: Thu, 14 Dec 2017 16:03:19 -0000 Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/4612#discussion_r156942641 --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeinfo/Types.java --- @@ -19,56 +19,422 @@ 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.tuple.Tuple; +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 org.apache.flink.types.Either; +import org.apache.flink.types.Row; +import org.apache.flink.types.Value; +import java.lang.reflect.Field; import java.math.BigDecimal; +import java.math.BigInteger; import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; +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 type information has to be + * supplied manually or would result in an inefficient type. + * + *

Please note that the Scala API and Table API provide more specialized Types classes. + * (See org.apache.flink.api.scala.Types and org.apache.flink.table.api.Types) + * + *

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 {@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 {@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 {@link java.lang.Short}. + * Does not support a null value. + */ + public static final TypeInformation SHORT = BasicTypeInfo.SHORT_TYPE_INFO; + + /** + * Returns type information for both a primitive int and {@link java.lang.Integer}. + * Does not support a null value. + */ + public static final TypeInformation INT = BasicTypeInfo.INT_TYPE_INFO; + + /** + * Returns type information for both a primitive long and {@link java.lang.Long}. + * Does not support a null value. + */ + public static final TypeInformation LONG = BasicTypeInfo.LONG_TYPE_INFO; + + /** + * Returns type information for both a primitive float and {@link java.lang.Float}. + * Does not support a null value. + */ + public static final TypeInformation FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO; + + /** + * Returns type information for both a primitive double and {@link java.lang.Double}. + * Does not support a null value. + */ + public static final TypeInformation DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO; + + /** + * Returns type information for both a primitive char and {@link java.lang.Character}. + * Does not support a null value. + */ + public static final TypeInformation CHAR = BasicTypeInfo.CHAR_TYPE_INFO; + + /** + * Returns type information for {@link java.math.BigDecimal}. Supports a null value. + */ + public static final TypeInformation BIG_DEC = BasicTypeInfo.BIG_DEC_TYPE_INFO; + + /** + * Returns type information for {@link java.math.BigInteger}. Supports a null value. + */ + public static final TypeInformation BIG_INT = BasicTypeInfo.BIG_INT_TYPE_INFO; + + /** + * Returns type information for {@link java.sql.Date}. Supports a null value. + */ + public static final TypeInformation SQL_DATE = SqlTimeTypeInfo.DATE; + + /** + * Returns type information for {@link java.sql.Time}. Supports a null value. + */ + public static final TypeInformation

A row is a fixed-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. + * The type of row fields cannot be automatically inferred; therefore, it is required to pass + * type information whenever a row is used. + * + *

The schema of rows can have up to Integer.MAX_VALUE fields, however, all row instances + * must have the same length otherwise serialization fails or information is lost. * - *

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 fixed-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. + * The type of row fields cannot be automatically inferred; therefore, it is required to pass + * type information whenever a row is used. + * + *

The schema of rows can have up to Integer.MAX_VALUE fields, however, all row instances + * must have the same length otherwise serialization fails or information is lost. + * *

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; a tuple does not support null values unless its field type supports nullability. --- End diff -- "a tuple does not support null values unless its field type supports nullability." -> "a tuple does not support null-valued fields unless the type of the field supports nullability." ---