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/02 02:09:05 UTC
[flink] 01/03: [FLINK-13290][table-api] Add method to check
LogicalType compatible
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 73bb670ec6888ce55a9185a222b7b4a7cdb62d05
Author: Jark Wu <im...@gmail.com>
AuthorDate: Wed Jul 31 16:44:51 2019 +0800
[FLINK-13290][table-api] Add method to check LogicalType compatible
Add a areTypesCompatible() method to LogicalTypeChecks. This will compare two LogicalTypes without field names and other logical attributes (e.g. description, isFinal).
---
.../types/logical/utils/LogicalTypeChecks.java | 61 +++++
.../types/LogicalTypeCompatibleCheckTest.java | 300 +++++++++++++++++++++
2 files changed, 361 insertions(+)
diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeChecks.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeChecks.java
index 3e2962e..45e2318 100644
--- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeChecks.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeChecks.java
@@ -39,6 +39,10 @@ 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 java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
/**
* Utilities for checking {@link LogicalType} and avoiding a lot of type casting and repetitive work.
*/
@@ -145,6 +149,20 @@ public final class LogicalTypeChecks {
return logicalType.accept(SINGLE_FIELD_INTERVAL_EXTRACTOR);
}
+ /**
+ * Returns true if the two given types are compatible. Types are compatible is for atomic types
+ * (VARCHAR, INT, BOOLEAN, etc..), they must be fully equal (i.e. {@link LogicalType#equals(Object)}),
+ * for complex types (ARRAY, ROW, MAP, etc..), they must be in the same type but ignore field
+ * names and other logical attributes, and all the children types ({@link LogicalType#getChildren()})
+ * must be compatible too.
+ */
+ public static boolean areTypesCompatible(LogicalType thisType, LogicalType thatType) {
+ checkNotNull(thisType);
+ checkNotNull(thatType);
+ TypeCompatibleVisitor visitor = new TypeCompatibleVisitor(thisType);
+ return thatType.accept(visitor);
+ }
+
private LogicalTypeChecks() {
// no instantiation
}
@@ -332,4 +350,47 @@ public final class LogicalTypeChecks {
}
}
}
+
+ private static class TypeCompatibleVisitor extends LogicalTypeDefaultVisitor<Boolean> {
+
+ private final LogicalType thisType;
+
+ private TypeCompatibleVisitor(LogicalType thisType) {
+ checkNotNull(thisType);
+ this.thisType = thisType;
+ }
+
+ @Override
+ protected Boolean defaultMethod(LogicalType thatType) {
+ checkNotNull(thatType);
+ if (thisType == thatType) {
+ return true;
+ }
+ if (thisType.getClass() != thatType.getClass() ||
+ thisType.isNullable() != thatType.isNullable() ||
+ thisType.getTypeRoot() != thatType.getTypeRoot()) {
+ return false;
+ }
+
+ List<LogicalType> thisChildren = thisType.getChildren();
+ List<LogicalType> thatChildren = thatType.getChildren();
+ if (thisChildren.size() != thatChildren.size()) {
+ return false;
+ }
+ if (thisChildren.isEmpty()) {
+ // if it is an atomic type, delegate to equals method.
+ return thisType.equals(thatType);
+ } else {
+ // if it is composite type, only need to check children types
+ for (int i = 0; i < thisChildren.size(); i++) {
+ LogicalType thisChild = thisChildren.get(i);
+ LogicalType thatChild = thatChildren.get(i);
+ if (!areTypesCompatible(thisChild, thatChild)) {
+ return false;
+ }
+ }
+ return true;
+ }
+ }
+ }
}
diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCompatibleCheckTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCompatibleCheckTest.java
new file mode 100644
index 0000000..ba38c08
--- /dev/null
+++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCompatibleCheckTest.java
@@ -0,0 +1,300 @@
+/*
+ * 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;
+
+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.apache.flink.table.types.logical.utils.LogicalTypeChecks;
+
+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 LogicalTypeChecks#areTypesCompatible(LogicalType, LogicalType)}.
+ */
+@RunWith(Parameterized.class)
+public class LogicalTypeCompatibleCheckTest {
+
+ @Parameters(name = "{index}: [{0} COMPATIBLE {1} => {2}")
+ public static List<Object[]> testData() {
+ return Arrays.asList(
+ new Object[][]{
+ {new CharType(), new CharType(5), false},
+
+ {new VarCharType(), new VarCharType(33), false},
+
+ {new BooleanType(), new BooleanType(false), false},
+
+ {new BinaryType(), new BinaryType(22), false},
+
+ {new VarBinaryType(), new VarBinaryType(44), false},
+
+ {new DecimalType(), new DecimalType(10, 2), false},
+
+ {new TinyIntType(), new TinyIntType(false), false},
+
+ {new SmallIntType(), new SmallIntType(false), false},
+
+ {new IntType(), new IntType(false), false},
+
+ {new BigIntType(), new BigIntType(false), false},
+
+ {new FloatType(), new FloatType(false), false},
+
+ {new DoubleType(), new DoubleType(false), false},
+
+ {new DateType(), new DateType(false), false},
+
+ {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()),
+ false
+ },
+
+ {
+ 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(LogicalTypesTest.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(
+ LogicalTypeChecks.areTypesCompatible(sourceType, targetType),
+ equalTo(equals));
+ assertTrue(LogicalTypeChecks.areTypesCompatible(sourceType, sourceType.copy()));
+ assertTrue(LogicalTypeChecks.areTypesCompatible(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;
+ }
+}