Skip to content

Commit

Permalink
[FLINK-5481] [core] Followups to the typeinfo/Types utility
Browse files Browse the repository at this point in the history
  - Move new 'Types' class to 'common' scope, not 'java api'.
  - Make the method names the same in Scala and Java
  • Loading branch information
StephanEwen committed Apr 20, 2017
1 parent eb4db53 commit c607b9a
Show file tree
Hide file tree
Showing 2 changed files with 19 additions and 17 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -16,21 +16,20 @@
* limitations under the License.
*/

package org.apache.flink.api.java.typeutils;
package org.apache.flink.api.common.typeinfo;

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 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 enumerates all supported types of
* the BasicTypeInfo, SqlTimeTypeInfo and RowTypeInfo for creation simplifying
* 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;
Expand All @@ -48,25 +47,28 @@ public class Types {
public static final SqlTimeTypeInfo<Timestamp> SQL_TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP;

/**
* Generates RowTypeInfo with default names (f1, f2 ..).
* same as new RowTypeInfo(types)
* 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 of Row fields. e.g. ROW(Types.STRING, Types.INT)
* @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 RowTypeInfo.
* same as new RowTypeInfo(types, names)
*
* e.g. ROW(new String[]{"name", "number"}, Types.STRING, Types.INT)
* 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(String[] fieldNames, TypeInformation<?>... types) {
public static RowTypeInfo ROW_NAMED(String[] fieldNames, TypeInformation<?>... types) {
return new RowTypeInfo(types, fieldNames);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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: _*)
}
}

0 comments on commit c607b9a

Please sign in to comment.