Repository: flink Updated Branches: refs/heads/master 52f8b33e5 -> c607b9a22
[FLINK-5481] [core] Followups to the typeinfo/Types utility - Move new 'Types' class to 'common' scope, not 'java api'. - Make the method names the same in Scala and Java Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c607b9a2 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c607b9a2 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c607b9a2 Branch: refs/heads/master Commit: c607b9a22a88162f64b8cb3bd2b191fa2f4f7cdb Parents: eb4db53 Author: Stephan Ewen <se...@apache.org> Authored: Wed Apr 19 20:05:21 2017 +0200 Committer: Stephan Ewen <se...@apache.org> Committed: Thu Apr 20 10:52:37 2017 +0200 ---------------------------------------------------------------------- .../apache/flink/api/common/typeinfo/Types.java | 74 ++++++++++++++++++++ .../apache/flink/api/java/typeutils/Types.java | 72 ------------------- .../org/apache/flink/table/api/Types.scala | 6 +- 3 files changed, 77 insertions(+), 75 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flink/blob/c607b9a2/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/Types.java ---------------------------------------------------------------------- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/Types.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/Types.java new file mode 100644 index 0000000..dcbb645 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/Types.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.typeinfo; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.java.typeutils.RowTypeInfo; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; + +/** + * This class gives access to the type information of the most most common types. + */ +@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; + + 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; + + /** + * Generates a RowTypeInfo with fields of the given types. + * The fields have the default names (f1, f2 ..). + * + * <p>This method is a shortcut to {@code new RowTypeInfo(types)}. + * + * @param types The types of the row fields, e.g., Types.STRING, Types.INT + */ + public static RowTypeInfo ROW(TypeInformation<?>... types) { + return new RowTypeInfo(types); + } + + /** + * Generates a RowTypeInfo with fields of the given types and with given names. + * + * <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) { + return new RowTypeInfo(types, fieldNames); + } +} http://git-wip-us.apache.org/repos/asf/flink/blob/c607b9a2/flink-core/src/main/java/org/apache/flink/api/java/typeutils/Types.java ---------------------------------------------------------------------- diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/Types.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/Types.java deleted file mode 100644 index 5159cde..0000000 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/Types.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.api.java.typeutils; - -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; - -import java.math.BigDecimal; -import java.sql.Date; -import java.sql.Time; -import java.sql.Timestamp; - -/** - * This class enumerates all supported types of - * the BasicTypeInfo, SqlTimeTypeInfo and RowTypeInfo for creation simplifying - */ -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; - - 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; - - /** - * Generates RowTypeInfo with default names (f1, f2 ..). - * same as new RowTypeInfo(types) - * - * @param types of Row fields. e.g. ROW(Types.STRING, Types.INT) - */ - public static RowTypeInfo ROW(TypeInformation<?>... types) { - return new RowTypeInfo(types); - } - - /** - * Generates RowTypeInfo. - * same as new RowTypeInfo(types, names) - * - * e.g. ROW(new String[]{"name", "number"}, Types.STRING, Types.INT) - * - * @param fieldNames array of field names - * @param types array of field types - */ - public static RowTypeInfo ROW(String[] fieldNames, TypeInformation<?>... types) { - return new RowTypeInfo(types, fieldNames); - } -} http://git-wip-us.apache.org/repos/asf/flink/blob/c607b9a2/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala ---------------------------------------------------------------------- diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala index b0cb338..262a452 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala @@ -17,9 +17,9 @@ */ package org.apache.flink.table.api -import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.common.typeinfo.{Types, TypeInformation} import org.apache.flink.table.typeutils.TimeIntervalTypeInfo -import org.apache.flink.api.java.typeutils.{Types => JTypes} +import org.apache.flink.api.common.typeinfo.{Types => JTypes} /** * This class enumerates all supported types of the Table API. @@ -62,6 +62,6 @@ object Types extends JTypes { def ROW_NAMED(fields: (String, TypeInformation[_])*) = { val names = fields.toList.map(_._1).toArray val types = fields.toList.map(_._2) - JTypes.ROW(names, types: _*) + JTypes.ROW_NAMED(names, types: _*) } }