You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ja...@apache.org on 2019/08/07 14:37:15 UTC

[flink] 03/05: [FLINK-13495][table-planner-blink] Introduce isAssignable to use soft check in TableSourceUtil

This is an automated email from the ASF dual-hosted git repository.

jark pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 9dddae827e3f117b3527254701119c10909a3ac5
Author: JingsongLi <lz...@aliyun.com>
AuthorDate: Fri Aug 2 18:06:58 2019 +0200

    [FLINK-13495][table-planner-blink] Introduce isAssignable to use soft check in TableSourceUtil
---
 .../table/planner/sources/TableSourceUtil.scala    |   3 +-
 .../table/runtime/types/PlannerTypeUtils.java      |  44 +++
 .../runtime/types/LogicalTypeAssignableTest.java   | 301 +++++++++++++++++++++
 3 files changed, 347 insertions(+), 1 deletion(-)

diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sources/TableSourceUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sources/TableSourceUtil.scala
index 2f15be2..bbbc06d 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sources/TableSourceUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sources/TableSourceUtil.scala
@@ -27,6 +27,7 @@ import org.apache.flink.table.functions.BuiltInFunctionDefinitions
 import org.apache.flink.table.planner.calcite.FlinkTypeFactory
 import org.apache.flink.table.planner.expressions.RexNodeConverter
 import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter
+import org.apache.flink.table.runtime.types.PlannerTypeUtils.isAssignable
 import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
 import org.apache.flink.table.sources.{DefinedFieldMapping, DefinedProctimeAttribute, DefinedRowtimeAttributes, RowtimeAttributeDescriptor, TableSource}
 import org.apache.flink.table.types.logical.{LogicalType, TimestampKind, TimestampType, TinyIntType}
@@ -116,7 +117,7 @@ object TableSourceUtil {
 
         val (physicalName, idx, tpe) = resolveInputField(name, tableSource)
         // validate that mapped fields are are same type
-        if (fromTypeInfoToLogicalType(tpe) != t) {
+        if (!isAssignable(fromTypeInfoToLogicalType(tpe), t)) {
           throw new ValidationException(s"Type $t of table field '$name' does not " +
             s"match with type $tpe of the field '$physicalName' of the TableSource return type.")
         }
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/PlannerTypeUtils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/PlannerTypeUtils.java
index bdb1134..77d346b 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/PlannerTypeUtils.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/PlannerTypeUtils.java
@@ -110,4 +110,48 @@ public class PlannerTypeUtils {
 				return t1.copy(true).equals(t2.copy(true));
 		}
 	}
+
+	/**
+	 * Now in the conversion to the TypeInformation from DataType, type may loose some information
+	 * about nullable and precision. So we add this method to do a soft check.
+	 *
+	 * <p>The difference of {@link #isInteroperable} is ignore decimal precision.
+	 *
+	 * <p>Now not ignore timestamp precision, because we only support one precision for timestamp type now.
+	 */
+	public static boolean isAssignable(LogicalType t1, LogicalType t2) {
+		// Soft check for CharType, it is converted to String TypeInformation and loose char information.
+		if (t1.getTypeRoot().getFamilies().contains(CHARACTER_STRING) &&
+				t2.getTypeRoot().getFamilies().contains(CHARACTER_STRING)) {
+			return true;
+		}
+		if (t1.getTypeRoot().getFamilies().contains(BINARY_STRING) &&
+				t2.getTypeRoot().getFamilies().contains(BINARY_STRING)) {
+			return true;
+		}
+		if (t1.getTypeRoot() != t2.getTypeRoot()) {
+			return false;
+		}
+
+		switch (t1.getTypeRoot()) {
+			case DECIMAL:
+				return true;
+			default:
+				if (t1.getChildren().isEmpty()) {
+					return t1.copy(true).equals(t2.copy(true));
+				} else {
+					List<LogicalType> children1 = t1.getChildren();
+					List<LogicalType> children2 = t2.getChildren();
+					if (children1.size() != children2.size()) {
+						return false;
+					}
+					for (int i = 0; i < children1.size(); i++) {
+						if (!isAssignable(children1.get(i), children2.get(i))) {
+							return false;
+						}
+					}
+					return true;
+				}
+		}
+	}
 }
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/types/LogicalTypeAssignableTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/types/LogicalTypeAssignableTest.java
new file mode 100644
index 0000000..af67f51
--- /dev/null
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/types/LogicalTypeAssignableTest.java
@@ -0,0 +1,301 @@
+/*
+ * 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.runtime.types;
+
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DayTimeIntervalType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DistinctType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.MultisetType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.RowType.RowField;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.StructuredType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampKind;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.table.types.logical.TypeInformationAnyType;
+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.ZonedTimestampType;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for {@link PlannerTypeUtils#isAssignable(LogicalType, LogicalType)}.
+ */
+@RunWith(Parameterized.class)
+public class LogicalTypeAssignableTest {
+
+	@Parameters(name = "{index}: [{0} COMPATIBLE {1} => {2}")
+	public static List<Object[]> testData() {
+		return Arrays.asList(
+			new Object[][]{
+				{new CharType(), new CharType(5), true},
+
+				{new CharType(), new VarCharType(5), true},
+
+				{new VarCharType(), new VarCharType(33), true},
+
+				{new BooleanType(), new BooleanType(false), true},
+
+				{new BinaryType(), new BinaryType(22), true},
+
+				{new VarBinaryType(), new VarBinaryType(44), true},
+
+				{new DecimalType(), new DecimalType(10, 2), true},
+
+				{new TinyIntType(), new TinyIntType(false), true},
+
+				{new SmallIntType(), new SmallIntType(false), true},
+
+				{new IntType(), new IntType(false), true},
+
+				{new BigIntType(), new BigIntType(false), true},
+
+				{new FloatType(), new FloatType(false), true},
+
+				{new DoubleType(), new DoubleType(false), true},
+
+				{new DateType(), new DateType(false), true},
+
+				{new TimeType(), new TimeType(9), false},
+
+				{new TimestampType(9), new TimestampType(3), false},
+
+				{new ZonedTimestampType(9), new ZonedTimestampType(3), false},
+
+				{new ZonedTimestampType(false, TimestampKind.PROCTIME, 9), new ZonedTimestampType(3), false},
+
+				{
+					new YearMonthIntervalType(YearMonthIntervalType.YearMonthResolution.YEAR_TO_MONTH, 2),
+					new YearMonthIntervalType(YearMonthIntervalType.YearMonthResolution.MONTH),
+					false
+				},
+
+				{
+					new DayTimeIntervalType(DayTimeIntervalType.DayTimeResolution.DAY_TO_SECOND, 2, 6),
+					new DayTimeIntervalType(DayTimeIntervalType.DayTimeResolution.DAY_TO_SECOND, 2, 7),
+					false
+				},
+
+				{
+					new ArrayType(new TimestampType()),
+					new ArrayType(new SmallIntType()),
+					false,
+				},
+
+				{
+					new MultisetType(new TimestampType()),
+					new MultisetType(new SmallIntType()),
+					false
+				},
+
+				{
+					new MapType(new VarCharType(20), new TimestampType()),
+					new MapType(new VarCharType(99), new TimestampType()),
+					true
+				},
+
+				{
+					new RowType(
+						Arrays.asList(
+							new RowType.RowField("a", new VarCharType()),
+							new RowType.RowField("b", new VarCharType()),
+							new RowType.RowField("c", new VarCharType()),
+							new RowType.RowField("d", new TimestampType()))),
+					new RowType(
+						Arrays.asList(
+							new RowType.RowField("_a", new VarCharType()),
+							new RowType.RowField("_b", new VarCharType()),
+							new RowType.RowField("_c", new VarCharType()),
+							new RowType.RowField("_d", new TimestampType()))),
+					// field name doesn't matter
+					true
+				},
+
+				{
+					new RowType(
+						Arrays.asList(
+							new RowField("f1", new IntType()),
+							new RowField("f2", new VarCharType())
+						)
+					),
+					new RowType(
+						Arrays.asList(
+							new RowField("f1", new IntType()),
+							new RowField("f2", new BooleanType())
+						)
+					),
+					false
+				},
+
+				{
+					new ArrayType(
+						new RowType(
+							Arrays.asList(
+								new RowField("f1", new IntType()),
+								new RowField("f2", new IntType())
+							)
+						)
+					),
+					new ArrayType(
+						new RowType(
+							Arrays.asList(
+								new RowField("f3", new IntType()),
+								new RowField("f4", new IntType())
+							)
+						)
+					),
+					true
+				},
+
+				{
+					new MapType(
+						new IntType(),
+						new RowType(
+							Arrays.asList(
+								new RowField("f1", new IntType()),
+								new RowField("f2", new IntType())
+							)
+						)
+					),
+					new MapType(
+						new IntType(),
+						new RowType(
+							Arrays.asList(
+								new RowField("f3", new IntType()),
+								new RowField("f4", new IntType())
+							)
+						)
+					),
+					true
+				},
+
+				{
+					new MultisetType(
+						new RowType(
+							Arrays.asList(
+								new RowField("f1", new IntType()),
+								new RowField("f2", new IntType())
+							)
+						)
+					),
+					new MultisetType(
+						new RowType(
+							Arrays.asList(
+								new RowField("f1", new IntType()),
+								new RowField("f2", new IntType())
+							)
+						)
+					),
+					true
+				},
+
+				{
+					new TypeInformationAnyType<>(Types.GENERIC(PlannerTypeUtils.class)),
+					new TypeInformationAnyType<>(Types.GENERIC(Object.class)),
+					false
+				},
+
+				{
+					createUserType(new IntType(), new VarCharType()),
+					createUserType(new IntType(), new VarCharType()),
+					true
+				},
+
+				{
+					createDistinctType(new DecimalType(10, 2)),
+					createDistinctType(new DecimalType(10, 2)),
+					true
+				}
+			}
+		);
+	}
+
+	@Parameter
+	public LogicalType sourceType;
+
+	@Parameter(1)
+	public LogicalType targetType;
+
+	@Parameter(2)
+	public boolean equals;
+
+	@Test
+	public void testAreTypesCompatible() {
+		assertThat(
+			PlannerTypeUtils.isAssignable(sourceType, targetType),
+			equalTo(equals));
+		assertTrue(PlannerTypeUtils.isAssignable(sourceType, sourceType.copy()));
+		assertTrue(PlannerTypeUtils.isAssignable(targetType, targetType.copy()));
+	}
+
+	private static DistinctType createDistinctType(LogicalType sourceType) {
+		return new DistinctType.Builder(
+			ObjectIdentifier.of("cat", "db", UUID.randomUUID().toString()),
+			sourceType)
+			.setDescription("Money type desc.")
+			.build();
+	}
+
+	private static StructuredType createUserType(LogicalType... children) {
+		return new StructuredType.Builder(
+			ObjectIdentifier.of("cat", "db", "User"),
+			Arrays.stream(children).map(lt ->
+				new StructuredType.StructuredAttribute(UUID.randomUUID().toString(), lt))
+				.collect(Collectors.toList()))
+			.setDescription("User type desc.")
+			.setFinal(true)
+			.setInstantiable(true)
+			.setImplementationClass(User.class)
+			.build();
+	}
+
+	private static final class User {
+		public int setting;
+	}
+}