flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From fhueske <...@git.apache.org>
Subject [GitHub] flink pull request #4612: [FLINK-7452] [types] Add helper methods for all bu...
Date Thu, 14 Dec 2017 16:03:10 GMT
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.
    + *
    + * <p>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.
    + *
    + * <p>Please note that the Scala API and Table API provide more specialized Types
classes.
    + * (See <code>org.apache.flink.api.scala.Types</code> and <code>org.apache.flink.table.api.Types</code>)
    + *
    + * <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> VOID = BasicTypeInfo.VOID_TYPE_INFO;
    +
    +	/**
    +	 * Returns type information for {@link java.lang.String}. Supports a null value.
    +	 */
    +	public static final TypeInformation<String> STRING = BasicTypeInfo.STRING_TYPE_INFO;
    +
    +	/**
    +	 * Returns type information for both a primitive <code>byte</code> and {@link
java.lang.Byte}.
    +	 * Does not support a null value.
    +	 */
    +	public static final TypeInformation<Byte> BYTE = BasicTypeInfo.BYTE_TYPE_INFO;
    +
    +	/**
    +	 * Returns type information for both a primitive <code>boolean</code> and
{@link java.lang.Boolean}.
    +	 * Does not support a null value.
    +	 */
    +	public static final TypeInformation<Boolean> BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO;
    +
    +	/**
    +	 * Returns type information for both a primitive <code>short</code> and
{@link java.lang.Short}.
    +	 * Does not support a null value.
    +	 */
    +	public static final TypeInformation<Short> SHORT = BasicTypeInfo.SHORT_TYPE_INFO;
    +
    +	/**
    +	 * Returns type information for both a primitive <code>int</code> and {@link
java.lang.Integer}.
    +	 * Does not support a null value.
    +	 */
    +	public static final TypeInformation<Integer> INT = BasicTypeInfo.INT_TYPE_INFO;
    +
    +	/**
    +	 * Returns type information for both a primitive <code>long</code> and {@link
java.lang.Long}.
    +	 * Does not support a null value.
    +	 */
    +	public static final TypeInformation<Long> LONG = BasicTypeInfo.LONG_TYPE_INFO;
    +
    +	/**
    +	 * Returns type information for both a primitive <code>float</code> and
{@link java.lang.Float}.
    +	 * Does not support a null value.
    +	 */
    +	public static final TypeInformation<Float> FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO;
    +
    +	/**
    +	 * Returns type information for both a primitive <code>double</code> and
{@link java.lang.Double}.
    +	 * Does not support a null value.
    +	 */
    +	public static final TypeInformation<Double> DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO;
    +
    +	/**
    +	 * Returns type information for both a primitive <code>char</code> and {@link
java.lang.Character}.
    +	 * Does not support a null value.
    +	 */
    +	public static final TypeInformation<Character> CHAR = BasicTypeInfo.CHAR_TYPE_INFO;
    +
    +	/**
    +	 * Returns type information for {@link java.math.BigDecimal}. Supports a null value.
    +	 */
    +	public static final TypeInformation<BigDecimal> BIG_DEC = BasicTypeInfo.BIG_DEC_TYPE_INFO;
    +
    +	/**
    +	 * Returns type information for {@link java.math.BigInteger}. Supports a null value.
    +	 */
    +	public static final TypeInformation<BigInteger> BIG_INT = BasicTypeInfo.BIG_INT_TYPE_INFO;
    +
    +	/**
    +	 * Returns type information for {@link java.sql.Date}. Supports a null value.
    +	 */
    +	public static final TypeInformation<Date> SQL_DATE = SqlTimeTypeInfo.DATE;
    +
    +	/**
    +	 * Returns type information for {@link java.sql.Time}. Supports a null value.
    +	 */
    +	public static final TypeInformation<Time> SQL_TIME = SqlTimeTypeInfo.TIME;
     
    -	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 {@link java.sql.Timestamp}. Supports a null value.
    +	 */
    +	public static final TypeInformation<Timestamp> SQL_TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP;
     
     	/**
    -	 * Generates a RowTypeInfo with fields of the given types.
    -	 * The fields have the default names (f0, f1, f2 ..).
    +	 * 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 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.
    +	 *
    +	 * <p>The schema of rows can have up to <code>Integer.MAX_VALUE</code>
fields, however, all row instances
    +	 * must have the same length otherwise serialization fails or information is lost.
     	 * 
    -	 * <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> 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 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.
    +	 *
    +	 * <p>The schema of rows can have up to <code>Integer.MAX_VALUE</code>
fields, however, all row instances
    +	 * must have the same length otherwise serialization fails or information is lost.
    +	 *
     	 * <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> 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; 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."


---

Mime
View raw message