[ https://issues.apache.org/jira/browse/FLINK-5481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15930054#comment-15930054 ]
ASF GitHub Bot commented on FLINK-5481: --------------------------------------- Github user StephanEwen commented on a diff in the pull request: https://github.com/apache/flink/pull/3127#discussion_r106664092 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala --- @@ -17,29 +17,51 @@ */ package org.apache.flink.table.api -import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo} +import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.typeutils.TimeIntervalTypeInfo +import org.apache.flink.api.java.typeutils.{Types => JTypes} /** * This class enumerates all supported types of the Table API. */ -object Types { +object Types extends JTypes { - val STRING = BasicTypeInfo.STRING_TYPE_INFO - val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO + val STRING = JTypes.STRING + val BOOLEAN = JTypes.BOOLEAN - val BYTE = BasicTypeInfo.BYTE_TYPE_INFO - val SHORT = BasicTypeInfo.SHORT_TYPE_INFO - val INT = BasicTypeInfo.INT_TYPE_INFO - val LONG = BasicTypeInfo.LONG_TYPE_INFO - val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO - val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO - val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO + val BYTE = JTypes.BYTE + val SHORT = JTypes.SHORT + val INT = JTypes.INT + val LONG = JTypes.LONG + val FLOAT = JTypes.FLOAT + val DOUBLE = JTypes.DOUBLE + val DECIMAL = JTypes.DECIMAL - val DATE = SqlTimeTypeInfo.DATE - val TIME = SqlTimeTypeInfo.TIME - val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP + val SQL_DATE = JTypes.SQL_DATE + val SQL_TIME = JTypes.SQL_TIME + val SQL_TIMESTAMP = JTypes.SQL_TIMESTAMP val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS + /** + * Generates RowTypeInfo with default names (f1, f2 ..). + * same as ``new RowTypeInfo(types)`` + * + * @param types of Row fields. e.g. ROW(Types.STRING, Types.INT) + */ + def ROW[T](types: TypeInformation[_]*)(implicit m: Manifest[T]) = { --- End diff -- Can you explain what the manifest is needed for? > Simplify Row creation > --------------------- > > Key: FLINK-5481 > URL: https://issues.apache.org/jira/browse/FLINK-5481 > Project: Flink > Issue Type: Bug > Components: DataSet API, Table API & SQL > Affects Versions: 1.2.0 > Reporter: Anton Solovev > Assignee: Anton Solovev > Priority: Trivial > > When we use {{ExecutionEnvironment#fromElements(X... data)}} it takes first > element of {{data}} to define a type. If first Row in collection has wrong > number of fields (there are nulls) TypeExtractor returns not RowTypeInfo, but > GenericType<Row> -- This message was sent by Atlassian JIRA (v6.3.15#6346)