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

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

commit c60c0369d36713d591f6f0b27921f713fcf9e670
Author: Timo Walther <[email protected]>
AuthorDate: Thu Jun 17 11:32:48 2021 +0200

    [hotfix][table-common] Move specific input type strategies out of 
InputTypeStrategies
---
 .../functions/BuiltInFunctionDefinitions.java      | 14 ++--
 .../table/types/inference/InputTypeStrategies.java | 45 -------------
 .../strategies/CastInputTypeStrategy.java          |  2 +-
 .../CurrentWatermarkInputTypeStrategy.java         |  2 +-
 .../inference/strategies/MapInputTypeStrategy.java |  2 +-
 .../strategies/SpecificInputTypeStrategies.java    | 78 ++++++++++++++++++++++
 .../strategies/SpecificTypeStrategies.java         |  2 +-
 .../inference/ComparableInputTypeStrategyTest.java | 46 +++++++------
 .../types/inference/InputTypeStrategiesTest.java   | 18 ++---
 .../CurrentWatermarkInputTypeStrategyTest.java     | 11 ++-
 10 files changed, 128 insertions(+), 92 deletions(-)

diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
index ccafce4..38f632d 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
@@ -25,6 +25,7 @@ import 
org.apache.flink.table.types.inference.ArgumentTypeStrategy;
 import org.apache.flink.table.types.inference.ConstantArgumentCount;
 import org.apache.flink.table.types.inference.InputTypeStrategies;
 import org.apache.flink.table.types.inference.TypeStrategies;
+import 
org.apache.flink.table.types.inference.strategies.SpecificInputTypeStrategies;
 import 
org.apache.flink.table.types.inference.strategies.SpecificTypeStrategies;
 import org.apache.flink.table.types.logical.LogicalTypeFamily;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
@@ -47,9 +48,6 @@ import static 
org.apache.flink.table.types.inference.InputTypeStrategies.COMMON_
 import static 
org.apache.flink.table.types.inference.InputTypeStrategies.LITERAL;
 import static 
org.apache.flink.table.types.inference.InputTypeStrategies.NO_ARGS;
 import static 
org.apache.flink.table.types.inference.InputTypeStrategies.OUTPUT_IF_NULL;
-import static 
org.apache.flink.table.types.inference.InputTypeStrategies.SPECIFIC_FOR_CAST;
-import static 
org.apache.flink.table.types.inference.InputTypeStrategies.TWO_EQUALS_COMPARABLE;
-import static 
org.apache.flink.table.types.inference.InputTypeStrategies.TWO_FULLY_COMPARABLE;
 import static org.apache.flink.table.types.inference.InputTypeStrategies.and;
 import static 
org.apache.flink.table.types.inference.InputTypeStrategies.commonType;
 import static 
org.apache.flink.table.types.inference.InputTypeStrategies.comparable;
@@ -66,6 +64,8 @@ import static 
org.apache.flink.table.types.inference.TypeStrategies.first;
 import static 
org.apache.flink.table.types.inference.TypeStrategies.matchFamily;
 import static org.apache.flink.table.types.inference.TypeStrategies.nullable;
 import static 
org.apache.flink.table.types.inference.TypeStrategies.varyingString;
+import static 
org.apache.flink.table.types.inference.strategies.SpecificInputTypeStrategies.TWO_EQUALS_COMPARABLE;
+import static 
org.apache.flink.table.types.inference.strategies.SpecificInputTypeStrategies.TWO_FULLY_COMPARABLE;
 
 /** Dictionary of function definitions for all built-in functions. */
 @PublicEvolving
@@ -1214,7 +1214,7 @@ public final class BuiltInFunctionDefinitions {
             BuiltInFunctionDefinition.newBuilder()
                     .name("array")
                     .kind(SCALAR)
-                    .inputTypeStrategy(InputTypeStrategies.SPECIFIC_FOR_ARRAY)
+                    .inputTypeStrategy(SpecificInputTypeStrategies.ARRAY)
                     .outputTypeStrategy(SpecificTypeStrategies.ARRAY)
                     .build();
 
@@ -1229,7 +1229,7 @@ public final class BuiltInFunctionDefinitions {
             BuiltInFunctionDefinition.newBuilder()
                     .name("map")
                     .kind(SCALAR)
-                    .inputTypeStrategy(InputTypeStrategies.SPECIFIC_FOR_MAP)
+                    .inputTypeStrategy(SpecificInputTypeStrategies.MAP)
                     .outputTypeStrategy(SpecificTypeStrategies.MAP)
                     .build();
 
@@ -1394,7 +1394,7 @@ public final class BuiltInFunctionDefinitions {
             BuiltInFunctionDefinition.newBuilder()
                     .name("CURRENT_WATERMARK")
                     .kind(SCALAR)
-                    
.inputTypeStrategy(InputTypeStrategies.SPECIFIC_FOR_CURRENT_WATERMARK)
+                    
.inputTypeStrategy(SpecificInputTypeStrategies.CURRENT_WATERMARK)
                     
.outputTypeStrategy(SpecificTypeStrategies.CURRENT_WATERMARK)
                     .notDeterministic()
                     .runtimeProvided()
@@ -1472,7 +1472,7 @@ public final class BuiltInFunctionDefinitions {
             BuiltInFunctionDefinition.newBuilder()
                     .name("cast")
                     .kind(SCALAR)
-                    .inputTypeStrategy(SPECIFIC_FOR_CAST)
+                    .inputTypeStrategy(SpecificInputTypeStrategies.CAST)
                     .outputTypeStrategy(
                             nullable(ConstantArgumentCount.to(0), 
TypeStrategies.argument(1)))
                     .build();
diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/InputTypeStrategies.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/InputTypeStrategies.java
index 7838bef..b4dc725 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/InputTypeStrategies.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/InputTypeStrategies.java
@@ -19,21 +19,17 @@
 package org.apache.flink.table.types.inference;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
 import org.apache.flink.table.types.DataType;
 import 
org.apache.flink.table.types.inference.strategies.AndArgumentTypeStrategy;
 import 
org.apache.flink.table.types.inference.strategies.AnyArgumentTypeStrategy;
-import org.apache.flink.table.types.inference.strategies.CastInputTypeStrategy;
 import 
org.apache.flink.table.types.inference.strategies.CommonArgumentTypeStrategy;
 import 
org.apache.flink.table.types.inference.strategies.CommonInputTypeStrategy;
 import 
org.apache.flink.table.types.inference.strategies.ComparableTypeStrategy;
 import 
org.apache.flink.table.types.inference.strategies.CompositeArgumentTypeStrategy;
 import 
org.apache.flink.table.types.inference.strategies.ConstraintArgumentTypeStrategy;
-import 
org.apache.flink.table.types.inference.strategies.CurrentWatermarkInputTypeStrategy;
 import 
org.apache.flink.table.types.inference.strategies.ExplicitArgumentTypeStrategy;
 import 
org.apache.flink.table.types.inference.strategies.FamilyArgumentTypeStrategy;
 import 
org.apache.flink.table.types.inference.strategies.LiteralArgumentTypeStrategy;
-import org.apache.flink.table.types.inference.strategies.MapInputTypeStrategy;
 import 
org.apache.flink.table.types.inference.strategies.OrArgumentTypeStrategy;
 import org.apache.flink.table.types.inference.strategies.OrInputTypeStrategy;
 import 
org.apache.flink.table.types.inference.strategies.OutputArgumentTypeStrategy;
@@ -303,47 +299,6 @@ public final class InputTypeStrategies {
     }
 
     // 
--------------------------------------------------------------------------------------------
-    // Specific input type strategies
-    // 
--------------------------------------------------------------------------------------------
-
-    /** Strategy specific for {@link BuiltInFunctionDefinitions#CAST}. */
-    public static final InputTypeStrategy SPECIFIC_FOR_CAST = new 
CastInputTypeStrategy();
-
-    /**
-     * Strategy specific for {@link BuiltInFunctionDefinitions#ARRAY}.
-     *
-     * <p>It expects at least one argument. All the arguments must have a 
common super type.
-     */
-    public static final InputTypeStrategy SPECIFIC_FOR_ARRAY =
-            new CommonInputTypeStrategy(ConstantArgumentCount.from(1));
-
-    /**
-     * Strategy specific for {@link BuiltInFunctionDefinitions#MAP}.
-     *
-     * <p>It expects at least two arguments. There must be even number of 
arguments. All the keys
-     * and values must have a common super type respectively.
-     */
-    public static final InputTypeStrategy SPECIFIC_FOR_MAP = new 
MapInputTypeStrategy();
-
-    /**
-     * Strategy that checks all types are fully comparable with each other. 
Requires exactly two
-     * arguments.
-     */
-    public static final InputTypeStrategy TWO_FULLY_COMPARABLE =
-            comparable(ConstantArgumentCount.of(2), 
StructuredComparision.FULL);
-
-    /**
-     * Strategy that checks all types are equals comparable with each other. 
Requires exactly two
-     * arguments.
-     */
-    public static final InputTypeStrategy TWO_EQUALS_COMPARABLE =
-            comparable(ConstantArgumentCount.of(2), 
StructuredComparision.EQUALS);
-
-    /** Strategy specific for {@link 
BuiltInFunctionDefinitions#CURRENT_WATERMARK}. */
-    public static final InputTypeStrategy SPECIFIC_FOR_CURRENT_WATERMARK =
-            new CurrentWatermarkInputTypeStrategy();
-
-    // 
--------------------------------------------------------------------------------------------
 
     private InputTypeStrategies() {
         // no instantiation
diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/CastInputTypeStrategy.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/CastInputTypeStrategy.java
index 15b2fc0..92bdbe5 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/CastInputTypeStrategy.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/CastInputTypeStrategy.java
@@ -43,7 +43,7 @@ import static 
org.apache.flink.table.types.logical.utils.LogicalTypeCasts.suppor
  * second one. The second one must be a type literal.
  */
 @Internal
-public final class CastInputTypeStrategy implements InputTypeStrategy {
+class CastInputTypeStrategy implements InputTypeStrategy {
 
     @Override
     public ArgumentCount getArgumentCount() {
diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/CurrentWatermarkInputTypeStrategy.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/CurrentWatermarkInputTypeStrategy.java
index f20a7c0..8933dd0 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/CurrentWatermarkInputTypeStrategy.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/CurrentWatermarkInputTypeStrategy.java
@@ -42,7 +42,7 @@ import java.util.Optional;
  * <p>It expects a single argument representing a rowtime attribute.
  */
 @Internal
-public class CurrentWatermarkInputTypeStrategy implements InputTypeStrategy {
+class CurrentWatermarkInputTypeStrategy implements InputTypeStrategy {
 
     @Override
     public ArgumentCount getArgumentCount() {
diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/MapInputTypeStrategy.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/MapInputTypeStrategy.java
index d0f3487..6a9d84c 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/MapInputTypeStrategy.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/MapInputTypeStrategy.java
@@ -44,7 +44,7 @@ import java.util.stream.IntStream;
  * values must have a common super type respectively.
  */
 @Internal
-public final class MapInputTypeStrategy implements InputTypeStrategy {
+class MapInputTypeStrategy implements InputTypeStrategy {
 
     private static final ArgumentCount AT_LEAST_TWO_EVEN =
             new ArgumentCount() {
diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificInputTypeStrategies.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificInputTypeStrategies.java
new file mode 100644
index 0000000..110056d
--- /dev/null
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificInputTypeStrategies.java
@@ -0,0 +1,78 @@
+/*
+ * 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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.types.inference.ConstantArgumentCount;
+import org.apache.flink.table.types.inference.InputTypeStrategies;
+import org.apache.flink.table.types.inference.InputTypeStrategy;
+import org.apache.flink.table.types.logical.StructuredType;
+
+import static 
org.apache.flink.table.types.inference.InputTypeStrategies.comparable;
+
+/**
+ * Entry point for specific input type strategies not covered in {@link 
InputTypeStrategies}.
+ *
+ * <p>This primarily serves the purpose of reducing visibility of individual 
type strategy
+ * implementations to avoid polluting the API classpath.
+ */
+@Internal
+public final class SpecificInputTypeStrategies {
+
+    /** See {@link CastInputTypeStrategy}. */
+    public static final InputTypeStrategy CAST = new CastInputTypeStrategy();
+
+    /** See {@link MapInputTypeStrategy}. */
+    public static final InputTypeStrategy MAP = new MapInputTypeStrategy();
+
+    /** See {@link CurrentWatermarkTypeStrategy}. */
+    public static final InputTypeStrategy CURRENT_WATERMARK =
+            new CurrentWatermarkInputTypeStrategy();
+
+    // 
--------------------------------------------------------------------------------------------
+    // Strategies composed of other strategies
+    // 
--------------------------------------------------------------------------------------------
+
+    /**
+     * Strategy specific for {@link BuiltInFunctionDefinitions#ARRAY}.
+     *
+     * <p>It expects at least one argument. All the arguments must have a 
common super type.
+     */
+    public static final InputTypeStrategy ARRAY =
+            new CommonInputTypeStrategy(ConstantArgumentCount.from(1));
+
+    /**
+     * Strategy that checks all types are fully comparable with each other. 
Requires exactly two
+     * arguments.
+     */
+    public static final InputTypeStrategy TWO_FULLY_COMPARABLE =
+            comparable(ConstantArgumentCount.of(2), 
StructuredType.StructuredComparision.FULL);
+
+    /**
+     * Strategy that checks all types are equals comparable with each other. 
Requires exactly two
+     * arguments.
+     */
+    public static final InputTypeStrategy TWO_EQUALS_COMPARABLE =
+            comparable(ConstantArgumentCount.of(2), 
StructuredType.StructuredComparision.EQUALS);
+
+    private SpecificInputTypeStrategies() {
+        // no instantiation
+    }
+}
diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificTypeStrategies.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificTypeStrategies.java
index 5d7431c..69b1e5f 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificTypeStrategies.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificTypeStrategies.java
@@ -29,7 +29,7 @@ import org.apache.flink.table.types.inference.TypeStrategy;
  * implementations to avoid polluting the API classpath.
  */
 @Internal
-public class SpecificTypeStrategies {
+public final class SpecificTypeStrategies {
 
     /** See {@link RowTypeStrategy}. */
     public static final TypeStrategy ROW = new RowTypeStrategy();
diff --git 
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/ComparableInputTypeStrategyTest.java
 
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/ComparableInputTypeStrategyTest.java
index 1e1bf06..1e0150b 100644
--- 
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/ComparableInputTypeStrategyTest.java
+++ 
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/ComparableInputTypeStrategyTest.java
@@ -28,6 +28,7 @@ import org.apache.flink.table.types.AtomicDataType;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.FieldsDataType;
 import 
org.apache.flink.table.types.inference.strategies.ComparableTypeStrategy;
+import 
org.apache.flink.table.types.inference.strategies.SpecificInputTypeStrategies;
 import org.apache.flink.table.types.logical.DistinctType;
 import org.apache.flink.table.types.logical.StructuredType;
 import 
org.apache.flink.table.types.logical.StructuredType.StructuredComparision;
@@ -89,16 +90,17 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                 DataTypes.DATE()),
                 TestSpec.forStrategy(
                                 "VARCHAR and CHAR types are comparable",
-                                InputTypeStrategies.TWO_FULLY_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_FULLY_COMPARABLE)
                         .calledWithArgumentTypes(DataTypes.VARCHAR(10), 
DataTypes.CHAR(13))
                         .expectArgumentTypes(DataTypes.VARCHAR(10), 
DataTypes.CHAR(13)),
                 TestSpec.forStrategy(
                                 "VARBINARY and BINARY types are comparable",
-                                InputTypeStrategies.TWO_FULLY_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_FULLY_COMPARABLE)
                         .calledWithArgumentTypes(DataTypes.VARBINARY(10), 
DataTypes.BINARY(13))
                         .expectArgumentTypes(DataTypes.VARBINARY(10), 
DataTypes.BINARY(13)),
                 TestSpec.forStrategy(
-                                "Comparable array types", 
InputTypeStrategies.TWO_FULLY_COMPARABLE)
+                                "Comparable array types",
+                                
SpecificInputTypeStrategies.TWO_FULLY_COMPARABLE)
                         .calledWithArgumentTypes(
                                 DataTypes.ARRAY(DataTypes.TINYINT()),
                                 DataTypes.ARRAY(DataTypes.DECIMAL(10, 2)))
@@ -106,7 +108,8 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                 DataTypes.ARRAY(DataTypes.TINYINT()),
                                 DataTypes.ARRAY(DataTypes.DECIMAL(10, 2))),
                 TestSpec.forStrategy(
-                                "Comparable map types", 
InputTypeStrategies.TWO_FULLY_COMPARABLE)
+                                "Comparable map types",
+                                
SpecificInputTypeStrategies.TWO_FULLY_COMPARABLE)
                         .calledWithArgumentTypes(
                                 DataTypes.MAP(DataTypes.TINYINT(), 
DataTypes.TIMESTAMP()),
                                 DataTypes.MAP(
@@ -119,7 +122,7 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                         DataTypes.TIMESTAMP_WITH_TIME_ZONE())),
                 TestSpec.forStrategy(
                                 "Fully comparable structured types",
-                                InputTypeStrategies.TWO_FULLY_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_FULLY_COMPARABLE)
                         .calledWithArgumentTypes(
                                 structuredType(
                                                 "type",
@@ -144,7 +147,7 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                         .nullable()),
                 TestSpec.forStrategy(
                                 "Equals comparable structured types",
-                                InputTypeStrategies.TWO_EQUALS_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_EQUALS_COMPARABLE)
                         .calledWithArgumentTypes(
                                 structuredType(
                                         "type",
@@ -165,7 +168,7 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                         StructuredComparision.EQUALS)),
                 TestSpec.forStrategy(
                                 "Comparable arrays of structured types",
-                                InputTypeStrategies.TWO_EQUALS_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_EQUALS_COMPARABLE)
                         .calledWithArgumentTypes(
                                 DataTypes.ARRAY(
                                         structuredType(
@@ -194,7 +197,7 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                                 .nullable())),
                 TestSpec.forStrategy(
                                 "Distinct types are comparable if the source 
type is comparable",
-                                InputTypeStrategies.TWO_EQUALS_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_EQUALS_COMPARABLE)
                         .calledWithArgumentTypes(
                                 distinctType("type", 
DataTypes.INT()).notNull(),
                                 distinctType("type", 
DataTypes.INT()).nullable())
@@ -203,7 +206,7 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                 distinctType("type", 
DataTypes.INT()).nullable()),
                 TestSpec.forStrategy(
                                 "Comparable multisets of distinct types",
-                                InputTypeStrategies.TWO_EQUALS_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_EQUALS_COMPARABLE)
                         .calledWithArgumentTypes(
                                 DataTypes.MULTISET(distinctType("type", 
DataTypes.INT()).notNull()),
                                 DataTypes.MULTISET(
@@ -214,12 +217,12 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                         distinctType("type", 
DataTypes.INT()).nullable())),
                 TestSpec.forStrategy(
                                 "Everything is comparable with null type",
-                                InputTypeStrategies.TWO_EQUALS_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_EQUALS_COMPARABLE)
                         .calledWithArgumentTypes(DataTypes.INT(), 
DataTypes.NULL())
                         .expectArgumentTypes(DataTypes.INT(), 
DataTypes.NULL()),
                 TestSpec.forStrategy(
                                 "RAW types are comparable if the originating 
class implements Comparable",
-                                InputTypeStrategies.TWO_EQUALS_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_EQUALS_COMPARABLE)
                         .calledWithArgumentTypes(
                                 rawType(ComparableClass.class).notNull(),
                                 rawType(ComparableClass.class).nullable())
@@ -228,7 +231,7 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                 rawType(ComparableClass.class).nullable()),
                 TestSpec.forStrategy(
                                 "Comparable map of raw types",
-                                InputTypeStrategies.TWO_EQUALS_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_EQUALS_COMPARABLE)
                         .calledWithArgumentTypes(
                                 DataTypes.MAP(
                                         
rawType(ComparableClass.class).notNull(),
@@ -245,7 +248,7 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                         rawType(ComparableClass.class))),
                 TestSpec.forStrategy(
                                 "RAW types are not comparable if the 
originating class does not implement Comparable",
-                                InputTypeStrategies.TWO_EQUALS_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_EQUALS_COMPARABLE)
                         .calledWithArgumentTypes(
                                 rawType(NotComparableClass.class),
                                 rawType(NotComparableClass.class))
@@ -257,7 +260,7 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                         NotComparableClass.class.getName())),
                 TestSpec.forStrategy(
                                 "RAW types are not comparable if the types are 
different",
-                                InputTypeStrategies.TWO_EQUALS_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_EQUALS_COMPARABLE)
                         .calledWithArgumentTypes(
                                 rawType(NotComparableClass.class),
                                 DataTypes.RAW(
@@ -275,7 +278,7 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                         NotComparableClass.class.getName())),
                 TestSpec.forStrategy(
                                 "Not fully comparable structured types",
-                                InputTypeStrategies.TWO_FULLY_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_FULLY_COMPARABLE)
                         .calledWithArgumentTypes(
                                 structuredType(
                                         "type",
@@ -290,7 +293,7 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                         + " with each other. Can not compare 
`cat`.`db`.`type` with `cat`.`db`.`type`"),
                 TestSpec.forStrategy(
                                 "Two different structured types are not 
comparable",
-                                InputTypeStrategies.TWO_EQUALS_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_EQUALS_COMPARABLE)
                         .calledWithArgumentTypes(
                                 structuredType(
                                         "type1",
@@ -305,7 +308,7 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                         + " Can not compare `cat`.`db`.`type1` 
with `cat`.`db`.`type2`"),
                 TestSpec.forStrategy(
                                 "Two different different distinct types are 
not comparable even if point to the same type",
-                                InputTypeStrategies.TWO_EQUALS_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_EQUALS_COMPARABLE)
                         .calledWithArgumentTypes(
                                 distinctType("type1", DataTypes.INT()),
                                 distinctType("type2", DataTypes.INT()))
@@ -314,7 +317,7 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                         + " Can not compare `cat`.`db`.`type1` 
with `cat`.`db`.`type2`"),
                 TestSpec.forStrategy(
                                 "Not comparable array types",
-                                InputTypeStrategies.TWO_FULLY_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_FULLY_COMPARABLE)
                         .calledWithArgumentTypes(
                                 DataTypes.ARRAY(DataTypes.TINYINT()),
                                 DataTypes.ARRAY(DataTypes.VARCHAR(2)))
@@ -323,7 +326,7 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                         + " with each other. Can not compare 
ARRAY<TINYINT> with ARRAY<VARCHAR(2)>"),
                 TestSpec.forStrategy(
                                 "Not comparable key types in map types",
-                                InputTypeStrategies.TWO_FULLY_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_FULLY_COMPARABLE)
                         .calledWithArgumentTypes(
                                 DataTypes.MAP(DataTypes.TINYINT(), 
DataTypes.TIMESTAMP()),
                                 DataTypes.MAP(
@@ -334,7 +337,7 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                         + " MAP<VARCHAR(3), TIMESTAMP(6) WITH 
TIME ZONE>"),
                 TestSpec.forStrategy(
                                 "Not comparable value types in map types",
-                                InputTypeStrategies.TWO_FULLY_COMPARABLE)
+                                
SpecificInputTypeStrategies.TWO_FULLY_COMPARABLE)
                         .calledWithArgumentTypes(
                                 DataTypes.MAP(DataTypes.TINYINT(), 
DataTypes.TIMESTAMP()),
                                 DataTypes.MAP(DataTypes.DECIMAL(10, 3), 
DataTypes.INT()))
@@ -342,7 +345,8 @@ public class ComparableInputTypeStrategyTest extends 
InputTypeStrategiesTestBase
                                 "All types in a comparison should support both 
'EQUALS' and 'ORDER' comparison"
                                         + " with each other. Can not compare 
MAP<TINYINT, TIMESTAMP(6)> with MAP<DECIMAL(10, 3), INT>"),
                 TestSpec.forStrategy(
-                                "Not comparable types", 
InputTypeStrategies.TWO_FULLY_COMPARABLE)
+                                "Not comparable types",
+                                
SpecificInputTypeStrategies.TWO_FULLY_COMPARABLE)
                         .calledWithArgumentTypes(DataTypes.TIMESTAMP(), 
DataTypes.BIGINT())
                         .expectErrorMessage(
                                 "All types in a comparison should support both 
'EQUALS' and 'ORDER' comparison"
diff --git 
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java
 
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java
index 4458c19..59b8d19 100644
--- 
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java
+++ 
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.table.types.inference;
 
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.types.DataType;
+import 
org.apache.flink.table.types.inference.strategies.SpecificInputTypeStrategies;
 import org.apache.flink.table.types.logical.LogicalTypeFamily;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 
@@ -442,7 +443,7 @@ public class InputTypeStrategiesTest extends 
InputTypeStrategiesTestBase {
                                 "Invalid number of arguments. At least 2 
arguments expected but 1 passed."),
                 TestSpec.forStrategy(
                                 "Array strategy infers a common type",
-                                InputTypeStrategies.SPECIFIC_FOR_ARRAY)
+                                SpecificInputTypeStrategies.ARRAY)
                         .expectSignature("f(<COMMON>, <COMMON>...)")
                         .calledWithArgumentTypes(
                                 DataTypes.INT().notNull(),
@@ -456,18 +457,17 @@ public class InputTypeStrategiesTest extends 
InputTypeStrategiesTestBase {
                                 DataTypes.DOUBLE()),
                 TestSpec.forStrategy(
                                 "Array strategy fails for no arguments",
-                                InputTypeStrategies.SPECIFIC_FOR_ARRAY)
+                                SpecificInputTypeStrategies.ARRAY)
                         .calledWithArgumentTypes()
                         .expectErrorMessage(
                                 "Invalid number of arguments. At least 1 
arguments expected but 0 passed."),
                 TestSpec.forStrategy(
                                 "Array strategy fails for null arguments",
-                                InputTypeStrategies.SPECIFIC_FOR_ARRAY)
+                                SpecificInputTypeStrategies.ARRAY)
                         .calledWithArgumentTypes(DataTypes.NULL())
                         .expectErrorMessage("Could not find a common type for 
arguments: [NULL]"),
                 TestSpec.forStrategy(
-                                "Map strategy infers common types",
-                                InputTypeStrategies.SPECIFIC_FOR_MAP)
+                                "Map strategy infers common types", 
SpecificInputTypeStrategies.MAP)
                         .calledWithArgumentTypes(
                                 DataTypes.INT().notNull(),
                                 DataTypes.DOUBLE(),
@@ -480,24 +480,24 @@ public class InputTypeStrategiesTest extends 
InputTypeStrategiesTestBase {
                                 DataTypes.DOUBLE()),
                 TestSpec.forStrategy(
                                 "Map strategy fails for no arguments",
-                                InputTypeStrategies.SPECIFIC_FOR_MAP)
+                                SpecificInputTypeStrategies.MAP)
                         .calledWithArgumentTypes()
                         .expectErrorMessage(
                                 "Invalid number of arguments. At least 2 
arguments expected but 0 passed."),
                 TestSpec.forStrategy(
                                 "Map strategy fails for an odd number of 
arguments",
-                                InputTypeStrategies.SPECIFIC_FOR_MAP)
+                                SpecificInputTypeStrategies.MAP)
                         .calledWithArgumentTypes(
                                 DataTypes.BIGINT(), DataTypes.BIGINT(), 
DataTypes.BIGINT())
                         .expectErrorMessage("Invalid number of arguments. 3 
arguments passed."),
-                TestSpec.forStrategy("Cast strategy", 
InputTypeStrategies.SPECIFIC_FOR_CAST)
+                TestSpec.forStrategy("Cast strategy", 
SpecificInputTypeStrategies.CAST)
                         .calledWithArgumentTypes(DataTypes.INT(), 
DataTypes.BIGINT())
                         .calledWithLiteralAt(1, DataTypes.BIGINT())
                         .expectSignature("f(<ANY>, <TYPE LITERAL>)")
                         .expectArgumentTypes(DataTypes.INT(), 
DataTypes.BIGINT()),
                 TestSpec.forStrategy(
                                 "Cast strategy for invalid target type",
-                                InputTypeStrategies.SPECIFIC_FOR_CAST)
+                                SpecificInputTypeStrategies.CAST)
                         .calledWithArgumentTypes(DataTypes.BOOLEAN(), 
DataTypes.DATE())
                         .calledWithLiteralAt(1, DataTypes.DATE())
                         .expectErrorMessage("Unsupported cast from 'BOOLEAN' 
to 'DATE'."),
diff --git 
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/strategies/CurrentWatermarkInputTypeStrategyTest.java
 
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/strategies/CurrentWatermarkInputTypeStrategyTest.java
index 742d679..7734998 100644
--- 
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/strategies/CurrentWatermarkInputTypeStrategyTest.java
+++ 
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/strategies/CurrentWatermarkInputTypeStrategyTest.java
@@ -20,7 +20,6 @@ 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.InputTypeStrategies;
 import org.apache.flink.table.types.inference.InputTypeStrategiesTestBase;
 import org.apache.flink.table.types.logical.LocalZonedTimestampType;
 import org.apache.flink.table.types.logical.TimestampKind;
@@ -41,29 +40,29 @@ public class CurrentWatermarkInputTypeStrategyTest extends 
InputTypeStrategiesTe
         return asList(
                 TestSpec.forStrategy(
                                 "TIMESTAMP(3) *ROWTIME* works",
-                                
InputTypeStrategies.SPECIFIC_FOR_CURRENT_WATERMARK)
+                                SpecificInputTypeStrategies.CURRENT_WATERMARK)
                         
.calledWithArgumentTypes(createRowtimeType(TimestampKind.ROWTIME, 3))
                         
.expectArgumentTypes(createRowtimeType(TimestampKind.ROWTIME, 3)),
                 TestSpec.forStrategy(
                                 "TIMESTAMP_LTZ(3) *ROWTIME* works",
-                                
InputTypeStrategies.SPECIFIC_FOR_CURRENT_WATERMARK)
+                                SpecificInputTypeStrategies.CURRENT_WATERMARK)
                         
.calledWithArgumentTypes(createRowtimeLtzType(TimestampKind.ROWTIME, 3))
                         
.expectArgumentTypes(createRowtimeLtzType(TimestampKind.ROWTIME, 3)),
                 TestSpec.forStrategy(
                                 "TIMESTAMP(3) doesn't work",
-                                
InputTypeStrategies.SPECIFIC_FOR_CURRENT_WATERMARK)
+                                SpecificInputTypeStrategies.CURRENT_WATERMARK)
                         
.calledWithArgumentTypes(createRowtimeType(TimestampKind.REGULAR, 3))
                         .expectErrorMessage(
                                 "The argument of CURRENT_WATERMARK() must be a 
rowtime attribute, but was 'TIMESTAMP(3) NOT NULL'."),
                 TestSpec.forStrategy(
                                 "TIMESTAMP_LTZ(3) doesn't work",
-                                
InputTypeStrategies.SPECIFIC_FOR_CURRENT_WATERMARK)
+                                SpecificInputTypeStrategies.CURRENT_WATERMARK)
                         
.calledWithArgumentTypes(createRowtimeLtzType(TimestampKind.REGULAR, 3))
                         .expectErrorMessage(
                                 "The argument of CURRENT_WATERMARK() must be a 
rowtime attribute, but was 'TIMESTAMP_LTZ(3) NOT NULL'."),
                 TestSpec.forStrategy(
                                 "BIGINT doesn't work",
-                                
InputTypeStrategies.SPECIFIC_FOR_CURRENT_WATERMARK)
+                                SpecificInputTypeStrategies.CURRENT_WATERMARK)
                         .calledWithArgumentTypes(DataTypes.BIGINT())
                         .expectErrorMessage(
                                 "CURRENT_WATERMARK() must be called with a 
single rowtime attribute argument, but 'BIGINT' cannot be a time attribute."));

Reply via email to