Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/4612#discussion_r139113352 --- 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. + * + * <p>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. + * + * <p>Depending on the API you are using (e.g. Scala API or Table API), there might be a more + * specialized <code>Types</code> class. + * + * <p>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> STRING = BasicTypeInfo.STRING_TYPE_INFO; - public static final BasicTypeInfo<Boolean> BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO; - public static final BasicTypeInfo<Byte> BYTE = BasicTypeInfo.BYTE_TYPE_INFO; - public static final BasicTypeInfo<Short> SHORT = BasicTypeInfo.SHORT_TYPE_INFO; - public static final BasicTypeInfo<Integer> INT = BasicTypeInfo.INT_TYPE_INFO; - public static final BasicTypeInfo<Long> LONG = BasicTypeInfo.LONG_TYPE_INFO; - public static final BasicTypeInfo<Float> FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO; - public static final BasicTypeInfo<Double> DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO; - public static final BasicTypeInfo<BigDecimal> 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 <code>byte</code> 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 <code>boolean</code> 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 <code>short</code> 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<Date> SQL_DATE = SqlTimeTypeInfo.DATE; - public static final SqlTimeTypeInfo<Time> SQL_TIME = SqlTimeTypeInfo.TIME; - public static final SqlTimeTypeInfo<Timestamp> SQL_TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP; + /** + * Returns type information for both a primitive <code>int</code> and a + * wrapped {@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 <code>long</code> and a + * wrapped {@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 <code>float</code> and a + * wrapped {@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 <code>double</code> and a + * wrapped {@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 <code>char</code> and a + * wrapped {@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; /** - * Generates a RowTypeInfo with fields of the given types. - * The fields have the default names (f0, f1, f2 ..). + * 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<?> SQL_TIME = SqlTimeTypeInfo.TIME; + + /** + * Returns type information for {@link java.sql.Timestamp}. Supports a null value. + */ + public static final TypeInformation<?> SQL_TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP; + + /** + * Returns type information for {@link org.apache.flink.types.Row} with fields of the given types. + * A row itself must not be null. + * + * <p>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. * - * <p>This method is a shortcut to {@code new RowTypeInfo(types)}. + * <p>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. + * + * <p>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. + * * <p>Example use: {@code ROW_NAMED(new String[]{"name", "number"}, Types.STRING, Types.INT)}. - * - * <p>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. + * + * <p>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 + * additional getters and setters for better readability. A tuple must not be null. + * + * <p>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 a field type supports nullability. + * + * <p>Types for all fields of the tuple can be defined in a hierarchy of subclasses. + * + * <p>If Flink's type analyzer is unable to extract a tuple type information with + * type information for all fields, an {@link org.apache.flink.api.common.functions.InvalidTypesException} + * is thrown. + * + * <p>Example use: + * <pre> + * {@code + * class MyTuple extends Tuple2<Integer, String> { + * + * public int getId() { return f0; } + * + * public String getName() { return f1; } + * } + * } + * + * Types.TUPLE(MyTuple.class) + * </pre> + * + * @param tupleSubclass A subclass of {@link org.apache.flink.api.java.tuple.Tuple0} till + * {@link org.apache.flink.api.java.tuple.Tuple25} that defines all field types and + * does not add any additional fields + */ + public static TypeInformation<?> TUPLE(Class<?> tupleSubclass) { + final TypeInformation<?> ti = TypeExtractor.createTypeInfo(tupleSubclass); + if (ti instanceof TupleTypeInfo) { + return ti; + } + throw new InvalidTypesException("Tuple type expected but was: " + ti); + } + + /** + * Returns type information for a POJO (Plain Old Java Object). + * + * <p>A POJO class is public and standalone (no non-static inner class). It has a public no-argument + * constructor. All non-static, non-transient fields in the class (and all superclasses) are either public + * (and non-final) or have a public getter and a setter method that follows the Java beans naming + * conventions for getters and setters. + * + * <p>A POJO is a fixed-length, null-aware composite type with non-deterministic field order. Every field --- End diff -- At which point is the field order important for users? Isn't it only relevant if you are dealing with the serialization format or when you want to map fields to a different type (as we do in the Table API)?
---