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;
+ }
+}