twalthr commented on a change in pull request #8500: [FLINK-12254][table-common] More preparation for using the new type system URL: https://github.com/apache/flink/pull/8500#discussion_r286464896
########## File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/LegacyTypeInfoDataTypeConverter.java ########## @@ -0,0 +1,347 @@ +/* + * 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.table.types.utils; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.common.typeutils.CompositeType; +import org.apache.flink.api.java.typeutils.MapTypeInfo; +import org.apache.flink.api.java.typeutils.MultisetTypeInfo; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.types.AtomicDataType; +import org.apache.flink.table.types.CollectionDataType; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.FieldsDataType; +import org.apache.flink.table.types.KeyValueDataType; +import org.apache.flink.table.types.logical.LegacyTypeInformationType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampKind; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.TypeInformationAnyType; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.table.typeutils.TimeIntervalTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import java.util.HashMap; +import java.util.Map; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +/** + * Converter between {@link TypeInformation} and {@link DataType} that reflects the behavior before + * Flink 1.9. The conversion is a 1:1 mapping that allows back-and-forth conversion. + * + * <p>This converter only exists to still support deprecated methods that take or return {@link TypeInformation}. + * Some methods will still support type information in the future, however, the future type information + * support will integrate nicer with the new type stack. This converter reflects the old behavior that includes: + * + * <p>Use old {@code java.sql.*} time classes for time data types. + * + * <p>Only support millisecond precision for timestamps or day-time intervals. + * + * <p>Do not support fractional seconds for the time type. + * + * <p>Let variable precision and scale for decimal types pass through the planner. + * + * <p>Let POJOs, case classes, and tuples pass through the planner. + * + * <p>Inconsistent nullability. Most types are nullable even though type information does not support it. + */ +@Internal +public final class LegacyTypeInfoDataTypeConverter { + + private static final Map<TypeInformation<?>, DataType> typeInfoDataTypeMap = new HashMap<>(); + private static final Map<DataType, TypeInformation<?>> dataTypeTypeInfoMap = new HashMap<>(); + static { + addMapping(Types.STRING, DataTypes.STRING().bridgedTo(String.class)); + addMapping(Types.BOOLEAN, DataTypes.BOOLEAN().bridgedTo(Boolean.class)); + addMapping(Types.BYTE, DataTypes.TINYINT().bridgedTo(Byte.class)); + addMapping(Types.SHORT, DataTypes.SMALLINT().bridgedTo(Short.class)); + addMapping(Types.INT, DataTypes.INT().bridgedTo(Integer.class)); + addMapping(Types.LONG, DataTypes.BIGINT().bridgedTo(Long.class)); + addMapping(Types.FLOAT, DataTypes.FLOAT().bridgedTo(Float.class)); + addMapping(Types.DOUBLE, DataTypes.DOUBLE().bridgedTo(Double.class)); + addMapping(Types.BIG_DEC, createLegacyType(LogicalTypeRoot.DECIMAL, Types.BIG_DEC)); + addMapping(Types.SQL_DATE, DataTypes.DATE().bridgedTo(java.sql.Date.class)); + addMapping(Types.SQL_TIME, DataTypes.TIME(0).bridgedTo(java.sql.Time.class)); + addMapping(Types.SQL_TIMESTAMP, DataTypes.TIMESTAMP(3).bridgedTo(java.sql.Timestamp.class)); + addMapping( + TimeIntervalTypeInfo.INTERVAL_MONTHS, + DataTypes.INTERVAL(DataTypes.MONTH()).bridgedTo(Integer.class)); + addMapping( + TimeIntervalTypeInfo.INTERVAL_MILLIS, + DataTypes.INTERVAL(DataTypes.SECOND(3)).bridgedTo(Long.class)); + addMapping( + PrimitiveArrayTypeInfo.BOOLEAN_PRIMITIVE_ARRAY_TYPE_INFO, + DataTypes.ARRAY(DataTypes.BOOLEAN().notNull().bridgedTo(boolean.class)).bridgedTo(boolean[].class)); + addMapping( + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, + DataTypes.BYTES().bridgedTo(byte[].class)); + addMapping( + PrimitiveArrayTypeInfo.SHORT_PRIMITIVE_ARRAY_TYPE_INFO, + DataTypes.ARRAY(DataTypes.SMALLINT().notNull().bridgedTo(short.class)).bridgedTo(short[].class)); + addMapping( + PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO, + DataTypes.ARRAY(DataTypes.INT().notNull().bridgedTo(int.class)).bridgedTo(int[].class)); + addMapping( + PrimitiveArrayTypeInfo.LONG_PRIMITIVE_ARRAY_TYPE_INFO, + DataTypes.ARRAY(DataTypes.BIGINT().notNull().bridgedTo(long.class)).bridgedTo(long[].class)); + addMapping( + PrimitiveArrayTypeInfo.FLOAT_PRIMITIVE_ARRAY_TYPE_INFO, + DataTypes.ARRAY(DataTypes.FLOAT().notNull().bridgedTo(float.class)).bridgedTo(float[].class)); + addMapping( + PrimitiveArrayTypeInfo.DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO, + DataTypes.ARRAY(DataTypes.DOUBLE().notNull().bridgedTo(double.class)).bridgedTo(double[].class)); + } + + private static void addMapping(TypeInformation<?> typeInfo, DataType dataType) { + Preconditions.checkArgument(!typeInfoDataTypeMap.containsKey(typeInfo)); + typeInfoDataTypeMap.put(typeInfo, dataType); + dataTypeTypeInfoMap.put(dataType, typeInfo); + } + + public static DataType toDataType(TypeInformation<?> typeInfo) { + // time indicators first as their hashCode/equals is shared with those of regular timestamps + if (typeInfo instanceof TimeIndicatorTypeInfo) { + return convertToTimeAttributeType((TimeIndicatorTypeInfo) typeInfo); + } + + final DataType foundDataType = typeInfoDataTypeMap.get(typeInfo); + if (foundDataType != null) { + return foundDataType; + } + + if (typeInfo instanceof RowTypeInfo) { + return convertToRowType((RowTypeInfo) typeInfo); + } + + else if (typeInfo instanceof ObjectArrayTypeInfo) { + return convertToArrayType( + typeInfo.getTypeClass(), + ((ObjectArrayTypeInfo) typeInfo).getComponentInfo()); + } + + else if (typeInfo instanceof BasicArrayTypeInfo) { + return convertToArrayType( + typeInfo.getTypeClass(), + ((BasicArrayTypeInfo) typeInfo).getComponentInfo()); + } + + else if (typeInfo instanceof MultisetTypeInfo) { + return convertToMultisetType(((MultisetTypeInfo) typeInfo).getElementTypeInfo()); + } + + else if (typeInfo instanceof MapTypeInfo) { + return convertToMapType((MapTypeInfo) typeInfo); + } + + else if (typeInfo instanceof CompositeType) { + return createLegacyType(LogicalTypeRoot.STRUCTURED_TYPE, typeInfo); + } + + return DataTypes.ANY(typeInfo); + } + + public static TypeInformation<?> toLegacyTypeInfo(DataType dataType) { + // time indicators first as their hashCode/equals is shared with those of regular timestamps + if (canConvertToTimeAttributeTypeInfo(dataType)) { + return convertToTimeAttributeTypeInfo((TimestampType) dataType.getLogicalType()); + } + + final TypeInformation<?> foundTypeInfo = dataTypeTypeInfoMap.get(dataType); + if (foundTypeInfo != null) { + return foundTypeInfo; + } + + if (canConvertToRowTypeInfo(dataType)) { + return convertToRowTypeInfo((FieldsDataType) dataType); + } + + else if (canConvertToObjectArrayTypeInfo(dataType)) { + return convertToObjectArrayTypeInfo((CollectionDataType) dataType); + } + + else if (canConvertToMultisetTypeInfo(dataType)) { + return convertToMultisetTypeInfo((CollectionDataType) dataType); + } + + else if (canConvertToMapTypeInfo(dataType)) { + return convertToMapTypeInfo((KeyValueDataType) dataType); + } + + else if (canConvertToLegacyTypeInfo(dataType)) { + return convertToLegacyTypeInfo(dataType); + } + + else if (canConvertToAnyTypeInfo(dataType)) { + return convertToAnyTypeInfo(dataType); + } + + throw new TableException( + String.format( + "Unsupported conversion from data type '%s' to type information. Only data types " + + "that originated from type information fully support a reverse conversion.", + dataType)); + } + + private static DataType createLegacyType(LogicalTypeRoot typeRoot, TypeInformation<?> typeInfo) { + return new AtomicDataType(new LegacyTypeInformationType<>(typeRoot, typeInfo)) + .bridgedTo(typeInfo.getTypeClass()); + } + + private static DataType convertToTimeAttributeType(TimeIndicatorTypeInfo timeIndicatorTypeInfo) { + final TimestampKind kind; + if (timeIndicatorTypeInfo.isEventTime()) { + kind = TimestampKind.ROWTIME; + } else { + kind = TimestampKind.PROCTIME; + } + return new AtomicDataType(new TimestampType(true, kind, 3)) + .bridgedTo(java.sql.Timestamp.class); + } + + private static boolean canConvertToTimeAttributeTypeInfo(DataType dataType) { + return dataType.getLogicalType().getTypeRoot() == LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE && + dataTypeTypeInfoMap.containsKey(dataType) && // checks precision and conversion + ((TimestampType) dataType.getLogicalType()).getKind() != TimestampKind.REGULAR; + } + + private static TypeInformation<?> convertToTimeAttributeTypeInfo(TimestampType timestampType) { + if (timestampType.getKind() == TimestampKind.ROWTIME) { + return TimeIndicatorTypeInfo.ROWTIME_INDICATOR; + } else { + return TimeIndicatorTypeInfo.PROCTIME_INDICATOR; + } + } + + private static DataType convertToRowType(RowTypeInfo rowTypeInfo) { + final String[] fieldNames = rowTypeInfo.getFieldNames(); + final DataTypes.Field[] fields = IntStream.range(0, rowTypeInfo.getArity()) + .mapToObj(i -> { + final String fieldName = fieldNames[i]; Review comment: Thanks, my inspection was disabled by accident. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services
