You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/05/27 03:50:20 UTC

[GitHub] [flink] dawidwys commented on a change in pull request #12331: [FLINK-17936][table] Introduce new type inference for AS

dawidwys commented on a change in pull request #12331:
URL: https://github.com/apache/flink/pull/12331#discussion_r430217975



##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/RootArgumentTypeStrategy.java
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.inference.strategies;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.functions.FunctionDefinition;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.ArgumentTypeStrategy;
+import org.apache.flink.table.types.inference.CallContext;
+import org.apache.flink.table.types.inference.Signature.Argument;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeCasts;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.Optional;
+
+import static org.apache.flink.table.types.logical.LogicalTypeFamily.APPROXIMATE_NUMERIC;
+import static org.apache.flink.table.types.logical.LogicalTypeFamily.EXACT_NUMERIC;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.BINARY;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.CHAR;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE;
+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;
+
+/**
+ * Strategy for an argument that corresponds to a given {@link LogicalTypeRoot} and nullability.
+ *
+ * <p>Implicit casts will be inserted if possible.
+ */
+@Internal
+public final class RootArgumentTypeStrategy implements ArgumentTypeStrategy {
+
+	private final LogicalTypeRoot expectedRoot;
+
+	private final boolean expectedNullability;
+
+	public RootArgumentTypeStrategy(LogicalTypeRoot expectedRoot, boolean expectedNullability) {
+		this.expectedRoot = Preconditions.checkNotNull(expectedRoot);
+		this.expectedNullability = expectedNullability;
+	}
+
+	@Override
+	public Optional<DataType> inferArgumentType(CallContext callContext, int argumentPos, boolean throwOnFailure) {
+		final DataType actualDataType = callContext.getArgumentDataTypes().get(argumentPos);
+		final LogicalType actualType = actualDataType.getLogicalType();
+
+		if (!expectedNullability && actualType.isNullable()) {
+			if (throwOnFailure) {
+				throw callContext.newValidationError(
+					"Unsupported argument type. Expected nullable type of root '%s' but actual type was '%s'.",
+					expectedRoot,
+					actualType);
+			}
+			return Optional.empty();
+		}
+
+		return findDataType(
+			callContext,
+			throwOnFailure,
+			actualDataType,
+			expectedRoot,
+			expectedNullability);
+	}
+
+	@Override
+	public Argument getExpectedArgument(FunctionDefinition functionDefinition, int argumentPos) {
+		// "< ... >" to indicate that this is not a type
+		if (!expectedNullability) {
+			return Argument.of("<" + expectedRoot + " NOT NULL>");
+		}
+		return Argument.of("<" + expectedRoot + ">");
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+		RootArgumentTypeStrategy that = (RootArgumentTypeStrategy) o;
+		return expectedNullability == that.expectedNullability &&
+			expectedRoot == that.expectedRoot;
+	}
+
+	@Override
+	public int hashCode() {
+		return Objects.hash(expectedRoot, expectedNullability);
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * Finds a data type that is close to the given data type in terms of nullability and conversion
+	 * class but of the given logical root.
+	 *
+	 * <p>This method is shared with {@link FamilyArgumentTypeStrategy}.
+	 */
+	static Optional<DataType> findDataType(
+			CallContext callContext,
+			boolean throwOnFailure,
+			DataType actualDataType,
+			LogicalTypeRoot expectedRoot,
+			boolean expectedNullability) {
+		final LogicalType actualType = actualDataType.getLogicalType();
+		return Optional.ofNullable(findDataTypeOfRoot(actualDataType, expectedRoot))
+			// set nullability
+			.map(newDataType -> {
+				if (expectedNullability) {
+					return newDataType.nullable();
+				} else {
+					return newDataType.notNull();
+				}
+			})
+			// preserve bridging class if possible
+			.map(newDataType -> {
+				final Class<?> clazz = actualDataType.getConversionClass();
+				final LogicalType newType = newDataType.getLogicalType();
+				if (newType.supportsInputConversion(clazz) || newType.supportsOutputConversion(clazz)) {

Review comment:
       Shouldn't we check only the `supportsOutputConversion`? UDF arguments are kind of "outputs" not "inputs".

##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/RootArgumentTypeStrategy.java
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.inference.strategies;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.functions.FunctionDefinition;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.ArgumentTypeStrategy;
+import org.apache.flink.table.types.inference.CallContext;
+import org.apache.flink.table.types.inference.Signature.Argument;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeCasts;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.Optional;
+
+import static org.apache.flink.table.types.logical.LogicalTypeFamily.APPROXIMATE_NUMERIC;
+import static org.apache.flink.table.types.logical.LogicalTypeFamily.EXACT_NUMERIC;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.BINARY;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.CHAR;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE;
+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;
+
+/**
+ * Strategy for an argument that corresponds to a given {@link LogicalTypeRoot} and nullability.
+ *
+ * <p>Implicit casts will be inserted if possible.
+ */
+@Internal
+public final class RootArgumentTypeStrategy implements ArgumentTypeStrategy {
+
+	private final LogicalTypeRoot expectedRoot;
+
+	private final boolean expectedNullability;
+
+	public RootArgumentTypeStrategy(LogicalTypeRoot expectedRoot, boolean expectedNullability) {
+		this.expectedRoot = Preconditions.checkNotNull(expectedRoot);
+		this.expectedNullability = expectedNullability;
+	}
+
+	@Override
+	public Optional<DataType> inferArgumentType(CallContext callContext, int argumentPos, boolean throwOnFailure) {
+		final DataType actualDataType = callContext.getArgumentDataTypes().get(argumentPos);
+		final LogicalType actualType = actualDataType.getLogicalType();
+
+		if (!expectedNullability && actualType.isNullable()) {
+			if (throwOnFailure) {
+				throw callContext.newValidationError(
+					"Unsupported argument type. Expected nullable type of root '%s' but actual type was '%s'.",
+					expectedRoot,
+					actualType);
+			}
+			return Optional.empty();
+		}
+
+		return findDataType(
+			callContext,
+			throwOnFailure,
+			actualDataType,
+			expectedRoot,
+			expectedNullability);
+	}
+
+	@Override
+	public Argument getExpectedArgument(FunctionDefinition functionDefinition, int argumentPos) {
+		// "< ... >" to indicate that this is not a type
+		if (!expectedNullability) {
+			return Argument.of("<" + expectedRoot + " NOT NULL>");
+		}
+		return Argument.of("<" + expectedRoot + ">");
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+		RootArgumentTypeStrategy that = (RootArgumentTypeStrategy) o;
+		return expectedNullability == that.expectedNullability &&
+			expectedRoot == that.expectedRoot;
+	}
+
+	@Override
+	public int hashCode() {
+		return Objects.hash(expectedRoot, expectedNullability);
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * Finds a data type that is close to the given data type in terms of nullability and conversion
+	 * class but of the given logical root.
+	 *
+	 * <p>This method is shared with {@link FamilyArgumentTypeStrategy}.
+	 */
+	static Optional<DataType> findDataType(

Review comment:
       nit: Could we moved that to a helper class in the `strategies` package? It looks a bit counter-intuitive that an unrelated(`FamilyArgumentTypeStrategy` does not extend from this class) class uses this method.

##########
File path: flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java
##########
@@ -481,7 +484,70 @@
 				"Map strategy fails for an odd number of arguments",
 				InputTypeStrategies.SPECIFIC_FOR_MAP)
 				.calledWithArgumentTypes(DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.BIGINT())
-				.expectErrorMessage("Invalid number of arguments. 3 arguments passed.")
+				.expectErrorMessage("Invalid number of arguments. 3 arguments passed."),
+
+			TestSpec
+				.forStrategy(
+					"Logical type roots instead of concrete data types",
+					sequence(
+						logical(LogicalTypeRoot.VARCHAR),
+						logical(LogicalTypeRoot.DECIMAL),
+						logical(LogicalTypeRoot.DECIMAL),
+						logical(LogicalTypeRoot.BOOLEAN),
+						logical(LogicalTypeRoot.INTEGER, false)))
+				.calledWithArgumentTypes(
+					DataTypes.NULL(),
+					DataTypes.INT(),
+					DataTypes.DOUBLE(),
+					DataTypes.BOOLEAN().notNull(),
+					DataTypes.INT().notNull())
+				.expectSignature(
+					"f(<VARCHAR>, <DECIMAL>, <DECIMAL>, <BOOLEAN>, <INTEGER NOT NULL>)")
+				.expectArgumentTypes(
+					DataTypes.VARCHAR(1),
+					DataTypes.DECIMAL(10, 0),
+					DataTypes.DECIMAL(30, 15),
+					DataTypes.BOOLEAN(),

Review comment:
       How about we preserve the nullability? Right now there is no way to say that we accept both. Therefore e.g. we loose the nullability info in such case:
   
   ```
   .inputTypeStrategy(logical(LogicalTypeRoot.BOOLEAN))
   .outputTypeStrategy(TypeStrategies.argument(0))
   ```
   
   I would suggest either:
   * if the `expectedNullability` in `FamilyArgumentTypeStrategy`/`RootArgumentTypeStrategy` is false forward the nullability of the input argument
   * introduce three state value -> expect nullable, not null, both 

##########
File path: flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java
##########
@@ -481,7 +484,70 @@
 				"Map strategy fails for an odd number of arguments",
 				InputTypeStrategies.SPECIFIC_FOR_MAP)
 				.calledWithArgumentTypes(DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.BIGINT())
-				.expectErrorMessage("Invalid number of arguments. 3 arguments passed.")
+				.expectErrorMessage("Invalid number of arguments. 3 arguments passed."),
+
+			TestSpec
+				.forStrategy(
+					"Logical type roots instead of concrete data types",
+					sequence(
+						logical(LogicalTypeRoot.VARCHAR),
+						logical(LogicalTypeRoot.DECIMAL),
+						logical(LogicalTypeRoot.DECIMAL),
+						logical(LogicalTypeRoot.BOOLEAN),
+						logical(LogicalTypeRoot.INTEGER, false)))
+				.calledWithArgumentTypes(
+					DataTypes.NULL(),
+					DataTypes.INT(),
+					DataTypes.DOUBLE(),
+					DataTypes.BOOLEAN().notNull(),
+					DataTypes.INT().notNull())
+				.expectSignature(
+					"f(<VARCHAR>, <DECIMAL>, <DECIMAL>, <BOOLEAN>, <INTEGER NOT NULL>)")
+				.expectArgumentTypes(
+					DataTypes.VARCHAR(1),
+					DataTypes.DECIMAL(10, 0),
+					DataTypes.DECIMAL(30, 15),
+					DataTypes.BOOLEAN(),
+					DataTypes.INT().notNull()),
+
+			TestSpec
+				.forStrategy(
+					"Logical type roots with wrong implicit cast",
+					sequence(logical(LogicalTypeRoot.VARCHAR)))
+				.calledWithArgumentTypes(DataTypes.INT())
+				.expectSignature("f(<VARCHAR>)")
+				.expectErrorMessage(
+					"Unsupported argument type. Expected type root 'VARCHAR' but actual type was 'INT'."),
+
+			TestSpec
+				.forStrategy(
+					"Logical type roots with wrong nullability",
+					sequence(logical(LogicalTypeRoot.VARCHAR, false)))
+				.calledWithArgumentTypes(DataTypes.VARCHAR(5))
+				.expectSignature("f(<VARCHAR NOT NULL>)")
+				.expectErrorMessage(
+					"Unsupported argument type. Expected nullable type of root 'VARCHAR' but actual type was 'VARCHAR(5)'."),
+
+			TestSpec
+				.forStrategy(
+					"Logical type family instead of concrete data types",
+					sequence(
+						logical(LogicalTypeFamily.CHARACTER_STRING),
+						logical(LogicalTypeFamily.EXACT_NUMERIC),
+						logical(LogicalTypeFamily.APPROXIMATE_NUMERIC),

Review comment:
       Can we add a case with e.g. `BIGINT` used with `APPROXIMATE_NUMERIC` family? This should work due to the implicit casts right?

##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/RootArgumentTypeStrategy.java
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.inference.strategies;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.functions.FunctionDefinition;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.ArgumentTypeStrategy;
+import org.apache.flink.table.types.inference.CallContext;
+import org.apache.flink.table.types.inference.Signature.Argument;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeCasts;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.Optional;
+
+import static org.apache.flink.table.types.logical.LogicalTypeFamily.APPROXIMATE_NUMERIC;
+import static org.apache.flink.table.types.logical.LogicalTypeFamily.EXACT_NUMERIC;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.BINARY;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.CHAR;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE;
+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;
+
+/**
+ * Strategy for an argument that corresponds to a given {@link LogicalTypeRoot} and nullability.
+ *
+ * <p>Implicit casts will be inserted if possible.
+ */
+@Internal
+public final class RootArgumentTypeStrategy implements ArgumentTypeStrategy {
+
+	private final LogicalTypeRoot expectedRoot;
+
+	private final boolean expectedNullability;
+
+	public RootArgumentTypeStrategy(LogicalTypeRoot expectedRoot, boolean expectedNullability) {
+		this.expectedRoot = Preconditions.checkNotNull(expectedRoot);
+		this.expectedNullability = expectedNullability;
+	}
+
+	@Override
+	public Optional<DataType> inferArgumentType(CallContext callContext, int argumentPos, boolean throwOnFailure) {
+		final DataType actualDataType = callContext.getArgumentDataTypes().get(argumentPos);
+		final LogicalType actualType = actualDataType.getLogicalType();
+
+		if (!expectedNullability && actualType.isNullable()) {
+			if (throwOnFailure) {
+				throw callContext.newValidationError(
+					"Unsupported argument type. Expected nullable type of root '%s' but actual type was '%s'.",
+					expectedRoot,
+					actualType);
+			}
+			return Optional.empty();
+		}
+
+		return findDataType(
+			callContext,
+			throwOnFailure,
+			actualDataType,
+			expectedRoot,
+			expectedNullability);
+	}
+
+	@Override
+	public Argument getExpectedArgument(FunctionDefinition functionDefinition, int argumentPos) {
+		// "< ... >" to indicate that this is not a type
+		if (!expectedNullability) {
+			return Argument.of("<" + expectedRoot + " NOT NULL>");
+		}
+		return Argument.of("<" + expectedRoot + ">");
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+		RootArgumentTypeStrategy that = (RootArgumentTypeStrategy) o;
+		return expectedNullability == that.expectedNullability &&
+			expectedRoot == that.expectedRoot;
+	}
+
+	@Override
+	public int hashCode() {
+		return Objects.hash(expectedRoot, expectedNullability);
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * Finds a data type that is close to the given data type in terms of nullability and conversion
+	 * class but of the given logical root.
+	 *
+	 * <p>This method is shared with {@link FamilyArgumentTypeStrategy}.
+	 */
+	static Optional<DataType> findDataType(
+			CallContext callContext,
+			boolean throwOnFailure,
+			DataType actualDataType,
+			LogicalTypeRoot expectedRoot,
+			boolean expectedNullability) {
+		final LogicalType actualType = actualDataType.getLogicalType();
+		return Optional.ofNullable(findDataTypeOfRoot(actualDataType, expectedRoot))
+			// set nullability
+			.map(newDataType -> {
+				if (expectedNullability) {
+					return newDataType.nullable();
+				} else {
+					return newDataType.notNull();
+				}
+			})
+			// preserve bridging class if possible
+			.map(newDataType -> {
+				final Class<?> clazz = actualDataType.getConversionClass();
+				final LogicalType newType = newDataType.getLogicalType();
+				if (newType.supportsInputConversion(clazz) || newType.supportsOutputConversion(clazz)) {
+					return newDataType.bridgedTo(clazz);
+				}
+				return newDataType;
+			})
+			// check if type can be implicitly casted
+			.filter(newDataType -> {
+				if (supportsImplicitCast(actualType, newDataType.getLogicalType())) {
+					return true;
+				}
+				if (throwOnFailure) {
+					throw callContext.newValidationError(
+						"Unsupported argument type. Expected type root '%s' but actual type was '%s'.",
+						expectedRoot,
+						actualType);
+				}
+				return false;
+			});
+	}
+
+	/**
+	 * Returns a data type for the given data type and expected root.
+	 *
+	 * <p>This method is aligned with {@link LogicalTypeCasts#supportsImplicitCast(LogicalType, LogicalType)}.
+	 *
+	 * <p>The default output of this method represents the default data type for a NULL literal. Thus,
+	 * the output of this method needs to be checked again if an implicit cast is supported.
+	 */
+	private static @Nullable DataType findDataTypeOfRoot(
+			DataType actualDataType,
+			LogicalTypeRoot expectedRoot) {
+		final LogicalType actualType = actualDataType.getLogicalType();
+		if (hasRoot(actualType, expectedRoot)) {
+			return actualDataType;
+		}
+		switch (expectedRoot) {
+			case CHAR:
+				return DataTypes.CHAR(CharType.DEFAULT_LENGTH);
+			case VARCHAR:
+				if (hasRoot(actualType, CHAR)) {
+					return DataTypes.VARCHAR(getLength(actualType));
+				}
+				return DataTypes.VARCHAR(VarCharType.DEFAULT_LENGTH);
+			case BOOLEAN:
+				return DataTypes.BOOLEAN();
+			case BINARY:
+				return DataTypes.BINARY(BinaryType.DEFAULT_LENGTH);
+			case VARBINARY:
+				if (hasRoot(actualType, BINARY)) {
+					return DataTypes.VARBINARY(getLength(actualType));
+				}
+				return DataTypes.VARBINARY(VarBinaryType.DEFAULT_LENGTH);
+			case DECIMAL:
+				if (hasFamily(actualType, EXACT_NUMERIC)) {
+					return DataTypes.DECIMAL(getPrecision(actualType), getScale(actualType));
+				} else if (hasFamily(actualType, APPROXIMATE_NUMERIC)) {
+					final int precision = getPrecision(actualType);
+					// we don't know where the precision occurs (before or after the dot)
+					return DataTypes.DECIMAL(precision * 2, precision);
+				}
+				return DataTypes.DECIMAL(DecimalType.MIN_PRECISION, DecimalType.MIN_SCALE);

Review comment:
       Why not default precision and scale?

##########
File path: flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java
##########
@@ -481,7 +484,70 @@
 				"Map strategy fails for an odd number of arguments",
 				InputTypeStrategies.SPECIFIC_FOR_MAP)
 				.calledWithArgumentTypes(DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.BIGINT())
-				.expectErrorMessage("Invalid number of arguments. 3 arguments passed.")
+				.expectErrorMessage("Invalid number of arguments. 3 arguments passed."),
+
+			TestSpec
+				.forStrategy(
+					"Logical type roots instead of concrete data types",
+					sequence(
+						logical(LogicalTypeRoot.VARCHAR),
+						logical(LogicalTypeRoot.DECIMAL),
+						logical(LogicalTypeRoot.DECIMAL),
+						logical(LogicalTypeRoot.BOOLEAN),
+						logical(LogicalTypeRoot.INTEGER, false)))
+				.calledWithArgumentTypes(
+					DataTypes.NULL(),
+					DataTypes.INT(),
+					DataTypes.DOUBLE(),
+					DataTypes.BOOLEAN().notNull(),
+					DataTypes.INT().notNull())
+				.expectSignature(
+					"f(<VARCHAR>, <DECIMAL>, <DECIMAL>, <BOOLEAN>, <INTEGER NOT NULL>)")
+				.expectArgumentTypes(
+					DataTypes.VARCHAR(1),
+					DataTypes.DECIMAL(10, 0),
+					DataTypes.DECIMAL(30, 15),
+					DataTypes.BOOLEAN(),
+					DataTypes.INT().notNull()),
+
+			TestSpec
+				.forStrategy(
+					"Logical type roots with wrong implicit cast",
+					sequence(logical(LogicalTypeRoot.VARCHAR)))
+				.calledWithArgumentTypes(DataTypes.INT())
+				.expectSignature("f(<VARCHAR>)")
+				.expectErrorMessage(
+					"Unsupported argument type. Expected type root 'VARCHAR' but actual type was 'INT'."),
+
+			TestSpec
+				.forStrategy(
+					"Logical type roots with wrong nullability",
+					sequence(logical(LogicalTypeRoot.VARCHAR, false)))
+				.calledWithArgumentTypes(DataTypes.VARCHAR(5))
+				.expectSignature("f(<VARCHAR NOT NULL>)")
+				.expectErrorMessage(
+					"Unsupported argument type. Expected nullable type of root 'VARCHAR' but actual type was 'VARCHAR(5)'."),
+
+			TestSpec
+				.forStrategy(

Review comment:
       Can we add tests for invalid cases? E.g. Using `FLOAT` with `EXACT_NUMERIC`.

##########
File path: flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/table/validation/CalcValidationTest.scala
##########
@@ -102,14 +102,6 @@ class CalcValidationTest extends TableTestBase {
       case _: ValidationException => //ignore
     }
 
-    try {
-      util.addTable[(Int, Long, String)]("Table2")
-      .select('_1 as '*, '_2 as 'b, '_1 as 'c)

Review comment:
       Why did you remove this test? Is the field reference a problem here? Can we just change it to strings?
   
   ```
   .select('_1 as "*", '_2 as "b", '_1 as "c")
   ```

##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/DoubleType.java
##########
@@ -33,6 +33,8 @@
 @PublicEvolving
 public final class DoubleType extends LogicalType {
 
+	public static final int PRECISION = 15;

Review comment:
       Shouldn't it be 16? https://en.wikipedia.org/wiki/Double-precision_floating-point_format
   
   > With the 52 bits of the fraction (F) significand appearing in the memory format, the total precision is therefore 53 bits (approximately 16 decimal digits, 53 log10(2) ≈ 15.955). 

##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/FamilyArgumentTypeStrategy.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.inference.strategies;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.functions.FunctionDefinition;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.ArgumentTypeStrategy;
+import org.apache.flink.table.types.inference.CallContext;
+import org.apache.flink.table.types.inference.Signature;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.util.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+import static org.apache.flink.table.types.inference.strategies.RootArgumentTypeStrategy.findDataType;
+
+/**
+ * Strategy for an argument that corresponds to a given {@link LogicalTypeFamily} and nullability.
+ *
+ * <p>Implicit casts will be inserted if possible.
+ */
+@Internal
+public final class FamilyArgumentTypeStrategy implements ArgumentTypeStrategy {
+
+	private final LogicalTypeFamily expectedFamily;
+
+	private final boolean expectedNullability;
+
+	private static final Map<LogicalTypeFamily, LogicalTypeRoot> familyToRoot = new HashMap<>();
+	static {
+		// commonly used type roots for families
+		familyToRoot.put(LogicalTypeFamily.NUMERIC, LogicalTypeRoot.INTEGER);
+		familyToRoot.put(LogicalTypeFamily.EXACT_NUMERIC, LogicalTypeRoot.INTEGER);

Review comment:
       Why did you decide for `INTEGER` for `NUMERIC`, `EXACT_NUMERIC` and `BINARY` for `BINARY_STRING`? Shouldn't we use the type with the highest precision? That's what Calcite does. Calcite uses:
   * `DECIMAL(MAX_PRECISION, MAX_SCALE)` for `NUMERIC`, `EXACT_NUMERIC`
   * `VARBINARY` for `BINARY_STRING`
   
   See `org.apache.calcite.sql.type.SqlTypeFamily#getDefaultConcreteType`

##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/StrategyUtils.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.inference.strategies;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.CallContext;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeCasts;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.Optional;
+
+import static org.apache.flink.table.types.logical.LogicalTypeFamily.APPROXIMATE_NUMERIC;
+import static org.apache.flink.table.types.logical.LogicalTypeFamily.EXACT_NUMERIC;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.BINARY;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.CHAR;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE;
+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 shared logic in classes of this package.
+ */
+final class StrategyUtils {
+
+	/**
+	 * Finds a data type that is close to the given data type in terms of nullability and conversion
+	 * class but of the given logical root.
+	 */
+	static Optional<DataType> findDataType(
+			CallContext callContext,
+			boolean throwOnFailure,
+			DataType actualDataType,
+			LogicalTypeRoot expectedRoot,
+			@Nullable Boolean expectedNullability) {
+		final LogicalType actualType = actualDataType.getLogicalType();
+		return Optional.ofNullable(findDataTypeOfRoot(actualDataType, expectedRoot))
+			// set nullability
+			.map(newDataType -> {
+				if (Objects.equals(expectedNullability, Boolean.TRUE)) {
+					return newDataType.nullable();
+				} else if (Objects.equals(expectedNullability, Boolean.FALSE)) {
+					return newDataType.notNull();
+				}
+				return newDataType;
+			})
+			// preserve bridging class if possible
+			.map(newDataType -> {
+				final Class<?> clazz = actualDataType.getConversionClass();
+				final LogicalType newType = newDataType.getLogicalType();
+				if (newType.supportsInputConversion(clazz)) {

Review comment:
       Shouldn't it be `OutputConversionClass` ? Arguments of a UDF are "outputs` of the table ecosystem.

##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/StrategyUtils.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.inference.strategies;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.CallContext;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeCasts;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.Optional;
+
+import static org.apache.flink.table.types.logical.LogicalTypeFamily.APPROXIMATE_NUMERIC;
+import static org.apache.flink.table.types.logical.LogicalTypeFamily.EXACT_NUMERIC;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.BINARY;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.CHAR;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE;
+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 shared logic in classes of this package.
+ */
+final class StrategyUtils {
+
+	/**
+	 * Finds a data type that is close to the given data type in terms of nullability and conversion
+	 * class but of the given logical root.
+	 */
+	static Optional<DataType> findDataType(
+			CallContext callContext,
+			boolean throwOnFailure,
+			DataType actualDataType,
+			LogicalTypeRoot expectedRoot,
+			@Nullable Boolean expectedNullability) {
+		final LogicalType actualType = actualDataType.getLogicalType();
+		return Optional.ofNullable(findDataTypeOfRoot(actualDataType, expectedRoot))
+			// set nullability
+			.map(newDataType -> {
+				if (Objects.equals(expectedNullability, Boolean.TRUE)) {
+					return newDataType.nullable();
+				} else if (Objects.equals(expectedNullability, Boolean.FALSE)) {
+					return newDataType.notNull();
+				}
+				return newDataType;

Review comment:
       Use the nullability of the `actualDataType` here? Otherwise `expectedNullability = null` is equivalent to `expectedNullability = true`

##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/StrategyUtils.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.inference.strategies;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.CallContext;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeCasts;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.Optional;
+
+import static org.apache.flink.table.types.logical.LogicalTypeFamily.APPROXIMATE_NUMERIC;
+import static org.apache.flink.table.types.logical.LogicalTypeFamily.EXACT_NUMERIC;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.BINARY;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.CHAR;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE;
+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 shared logic in classes of this package.
+ */
+final class StrategyUtils {
+
+	/**
+	 * Finds a data type that is close to the given data type in terms of nullability and conversion
+	 * class but of the given logical root.
+	 */
+	static Optional<DataType> findDataType(
+			CallContext callContext,
+			boolean throwOnFailure,
+			DataType actualDataType,
+			LogicalTypeRoot expectedRoot,
+			@Nullable Boolean expectedNullability) {
+		final LogicalType actualType = actualDataType.getLogicalType();
+		return Optional.ofNullable(findDataTypeOfRoot(actualDataType, expectedRoot))
+			// set nullability
+			.map(newDataType -> {
+				if (Objects.equals(expectedNullability, Boolean.TRUE)) {
+					return newDataType.nullable();
+				} else if (Objects.equals(expectedNullability, Boolean.FALSE)) {
+					return newDataType.notNull();
+				}
+				return newDataType;

Review comment:
       You can see the problem with this test case:
   ```
   TestSpec
   	.forStrategy(
   		"...",
   		logical(LogicalTypeFamily.APPROXIMATE_NUMERIC))
   	.calledWithArgumentTypes(
   		DataTypes.BIGINT().notNull())
   	.expectArgumentTypes(
   		DataTypes.DOUBLE() // should be NOT NULL, it is nullable for now
           ),
   ```
   




----------------------------------------------------------------
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:
users@infra.apache.org