dawidwys commented on a change in pull request #8933: [FLINK-12968][table-common] Add an utility for finding a common type from a set of types URL: https://github.com/apache/flink/pull/8933#discussion_r300665456
########## File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeGeneralization.java ########## @@ -0,0 +1,631 @@ +/* + * 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.logical.utils; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.CharType; +import org.apache.flink.table.types.logical.DayTimeIntervalType; +import org.apache.flink.table.types.logical.DayTimeIntervalType.DayTimeResolution; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.MultisetType; +import org.apache.flink.table.types.logical.NullType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.table.types.logical.YearMonthIntervalType; +import org.apache.flink.table.types.logical.YearMonthIntervalType.YearMonthResolution; +import org.apache.flink.table.types.logical.ZonedTimestampType; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; + +import java.util.AbstractList; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.flink.table.types.logical.DayTimeIntervalType.DayTimeResolution.DAY; +import static org.apache.flink.table.types.logical.DayTimeIntervalType.DayTimeResolution.DAY_TO_HOUR; +import static org.apache.flink.table.types.logical.DayTimeIntervalType.DayTimeResolution.DAY_TO_MINUTE; +import static org.apache.flink.table.types.logical.DayTimeIntervalType.DayTimeResolution.DAY_TO_SECOND; +import static org.apache.flink.table.types.logical.DayTimeIntervalType.DayTimeResolution.HOUR; +import static org.apache.flink.table.types.logical.DayTimeIntervalType.DayTimeResolution.HOUR_TO_MINUTE; +import static org.apache.flink.table.types.logical.DayTimeIntervalType.DayTimeResolution.HOUR_TO_SECOND; +import static org.apache.flink.table.types.logical.DayTimeIntervalType.DayTimeResolution.MINUTE; +import static org.apache.flink.table.types.logical.DayTimeIntervalType.DayTimeResolution.MINUTE_TO_SECOND; +import static org.apache.flink.table.types.logical.DayTimeIntervalType.DayTimeResolution.SECOND; +import static org.apache.flink.table.types.logical.LogicalTypeFamily.APPROXIMATE_NUMERIC; +import static org.apache.flink.table.types.logical.LogicalTypeFamily.BINARY_STRING; +import static org.apache.flink.table.types.logical.LogicalTypeFamily.CHARACTER_STRING; +import static org.apache.flink.table.types.logical.LogicalTypeFamily.DATETIME; +import static org.apache.flink.table.types.logical.LogicalTypeFamily.EXACT_NUMERIC; +import static org.apache.flink.table.types.logical.LogicalTypeFamily.INTERVAL; +import static org.apache.flink.table.types.logical.LogicalTypeFamily.NUMERIC; +import static org.apache.flink.table.types.logical.LogicalTypeFamily.TIME; +import static org.apache.flink.table.types.logical.LogicalTypeFamily.TIMESTAMP; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.ANY; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.ARRAY; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.DATE; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.DOUBLE; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.INTERVAL_DAY_TIME; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.INTERVAL_YEAR_MONTH; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.MULTISET; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.NULL; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.ROW; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.VARBINARY; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.VARCHAR; +import static org.apache.flink.table.types.logical.YearMonthIntervalType.YearMonthResolution.MONTH; +import static org.apache.flink.table.types.logical.YearMonthIntervalType.YearMonthResolution.YEAR; +import static org.apache.flink.table.types.logical.YearMonthIntervalType.YearMonthResolution.YEAR_TO_MONTH; +import static org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsImplicitCast; +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getLength; +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision; +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getScale; +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasFamily; +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasRoot; + +/** + * Utilities for finding a common, more general {@link LogicalType} for a given set of types. If such + * a type exists, all given types can be casted to this more general type. + * + * <p>This class aims to be compatible with the SQL standard. It is inspired by Apache Calcite's + * {@code SqlTypeFactoryImpl#leastRestrictive} method. + */ +@Internal +public final class LogicalTypeGeneralization { + + // mappings for interval generalization + private static final Map<YearMonthResolution, List<YearMonthResolution>> YEAR_MONTH_RES_TO_BOUNDARIES = new HashMap<>(); + private static final Map<List<YearMonthResolution>, YearMonthResolution> YEAR_MONTH_BOUNDARIES_TO_RES = new HashMap<>(); + static { + addYearMonthMapping(YEAR, YEAR); + addYearMonthMapping(MONTH, MONTH); + addYearMonthMapping(YEAR_TO_MONTH, YEAR, MONTH); + } + + private static final Map<DayTimeResolution, List<DayTimeResolution>> DAY_TIME_RES_TO_BOUNDARIES = new HashMap<>(); + private static final Map<List<DayTimeResolution>, DayTimeResolution> DAY_TIME_BOUNDARIES_TO_RES = new HashMap<>(); + static { + addDayTimeMapping(DAY, DAY); + addDayTimeMapping(DAY_TO_HOUR, DAY, HOUR); + addDayTimeMapping(DAY_TO_MINUTE, DAY, MINUTE); + addDayTimeMapping(DAY_TO_SECOND, DAY, SECOND); + addDayTimeMapping(HOUR, HOUR); + addDayTimeMapping(HOUR_TO_MINUTE, HOUR, MINUTE); + addDayTimeMapping(HOUR_TO_SECOND, HOUR, SECOND); + addDayTimeMapping(MINUTE, MINUTE); + addDayTimeMapping(MINUTE_TO_SECOND, MINUTE, SECOND); + addDayTimeMapping(SECOND, SECOND); + } + + private static void addYearMonthMapping(YearMonthResolution to, YearMonthResolution... boundaries) { + final List<YearMonthResolution> boundariesList = Arrays.asList(boundaries); + YEAR_MONTH_RES_TO_BOUNDARIES.put(to, boundariesList); + YEAR_MONTH_BOUNDARIES_TO_RES.put(boundariesList, to); + } + + private static void addDayTimeMapping(DayTimeResolution to, DayTimeResolution... boundaries) { + final List<DayTimeResolution> boundariesList = Arrays.asList(boundaries); + DAY_TIME_RES_TO_BOUNDARIES.put(to, boundariesList); + DAY_TIME_BOUNDARIES_TO_RES.put(boundariesList, to); + } + + /** + * Returns the most common type of a set of types. It determines a type to which all given types + * can be casted. + * + * <p>For example: {@code [INT, BIGINT, DECIMAL(2, 2)]} would lead to {@code DECIMAL(21, 2)}. + */ + public static Optional<LogicalType> findCommonType(List<LogicalType> types) { + Preconditions.checkArgument(types.size() > 0, "List of types must not be empty."); + + // collect statistics first + boolean hasAnyType = false; + boolean hasNullType = false; + boolean hasNullableTypes = false; + for (LogicalType type : types) { + final LogicalTypeRoot typeRoot = type.getTypeRoot(); + if (typeRoot == ANY) { + hasAnyType = true; + } else if (typeRoot == NULL) { + hasNullType = true; + } + if (type.isNullable()) { + hasNullableTypes = true; + } + } + + final List<LogicalType> normalizedTypes = types.stream() + .map(t -> t.copy(true)) + .collect(Collectors.toList()); + + LogicalType foundType = findCommonNullableType(normalizedTypes, hasAnyType, hasNullType); + if (foundType == null) { + foundType = findCommonCastableType(normalizedTypes); + } + + if (foundType != null) { + final LogicalType typeWithNullability = foundType.copy(hasNullableTypes); + return Optional.of(typeWithNullability); + } + return Optional.empty(); + } + + private static @Nullable LogicalType findCommonCastableType(List<LogicalType> normalizedTypes) { + LogicalType resultType = normalizedTypes.get(0); + + for (LogicalType type : normalizedTypes) { + final LogicalTypeRoot typeRoot = type.getTypeRoot(); + + // NULL does not affect the result of this loop + if (typeRoot == NULL) { + continue; + } + + if (supportsImplicitCast(resultType, type)) { + resultType = type; + } else { + if (!supportsImplicitCast(type, resultType)) { + return null; + } + } + } + + return resultType; + } + + @SuppressWarnings("ConstantConditions") + private static @Nullable LogicalType findCommonNullableType( + List<LogicalType> normalizedTypes, + boolean hasAnyType, + boolean hasNullType) { + + // all ANY types must be equal + if (hasAnyType) { + return findExactlySameType(normalizedTypes); + } + + LogicalType resultType = null; + + for (LogicalType type : normalizedTypes) { + final LogicalTypeRoot typeRoot = type.getTypeRoot(); + + // NULL does not affect the result of this loop + if (typeRoot == NULL) { + continue; + } + + // if result type is still null, consider the current type as a potential + // result type candidate + if (resultType == null) { + resultType = type; + } + + // find special patterns + final LogicalType patternType = findCommonTypePattern(resultType, type); + if (patternType != null) { + resultType = patternType; + continue; + } + + // for types of family CONSTRUCTED + if (typeRoot == ARRAY) { + return findCommonArrayType(normalizedTypes); + } else if (typeRoot == MULTISET) { + return findCommonMultisetType(normalizedTypes); + } else if (typeRoot == ROW) { + return findCommonRowType(normalizedTypes); + } Review comment: missing `MAP` typeRoot ---------------------------------------------------------------- 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
