From 601389b36a02c1986f4785a1303990098baba9b6 Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Wed, 13 May 2026 20:49:41 +0200 Subject: [PATCH 1/4] [FLINK-36602][table] Upgrade Calcite version to 1.38.0 --- flink-python/pyflink/table/tests/test_udf.py | 2 +- flink-table/flink-sql-parser/pom.xml | 8 +- .../src/main/codegen/data/Parser.tdd | 1 - .../src/main/codegen/templates/Parser.jj | 117 +- .../java/org/apache/calcite/sql/SqlJoin.java | 293 --- .../parser/validate/FlinkSqlConformance.java | 5 - .../sql/parser/FlinkSqlParserImplTest.java | 3 + .../flink-table-calcite-bridge/pom.xml | 6 +- .../functions/BuiltInFunctionDefinitions.java | 2 +- .../org/apache/calcite/plan/RelOptUtil.java | 107 +- .../calcite/rel/metadata/RelMdPredicates.java | 1051 -------- .../calcite/rel/rules/SubQueryRemoveRule.java | 10 +- .../rel/type/RelDataTypeFactoryImpl.java | 19 +- .../org/apache/calcite/rex/RexBuilder.java | 437 +++- .../org/apache/calcite/rex/RexChecker.java | 34 +- .../org/apache/calcite/rex/RexShuttle.java | 35 +- .../java/org/apache/calcite/rex/RexUtil.java | 67 +- .../apache/calcite/runtime/SqlFunctions.java | 647 ++++- .../calcite/sql/SqlIntervalQualifier.java | 51 +- .../java/org/apache/calcite/sql/SqlUtil.java | 76 +- .../apache/calcite/sql/type/BasicSqlType.java | 322 +++ .../calcite/sql/type/SqlTypeFactoryImpl.java | 18 +- .../apache/calcite/sql/type/SqlTypeUtil.java | 96 +- .../sql/validate/SqlValidatorImpl.java | 502 +++- .../apache/calcite/sql2rel/AggConverter.java | 33 +- .../calcite/sql2rel/RelDecorrelator.java | 73 +- .../calcite/sql2rel/SqlToRelConverter.java | 615 +++-- .../sql2rel/StandardConvertletTable.java | 2183 ----------------- .../planner/calcite/FlinkConvertletTable.java | 22 + .../planner/calcite/FlinkTypeFactory.java | 7 +- .../planner/calcite/FlinkTypeSystem.java | 10 + .../calcite/RelTimeIndicatorConverter.java | 1 + .../exec/serde/RexNodeJsonDeserializer.java | 6 +- .../nodes/exec/utils/CommonPythonUtil.java | 4 +- ...markIntoTableSourceScanAcrossCalcRule.java | 2 +- .../plan/schema/TimeIndicatorRelDataType.java | 76 + .../src/main/resources/META-INF/NOTICE | 4 +- .../planner/codegen/ExpressionReducer.scala | 14 +- .../BatchPhysicalOverAggregateRule.scala | 1 + .../schema/TimeIndicatorRelDataType.scala | 55 - .../planner/plan/utils/FlinkRelOptUtil.scala | 4 +- .../planner/plan/utils/FlinkRexUtil.scala | 2 +- .../plan/utils/OverAggregateUtil.scala | 1 + .../functions/JsonFunctionsITCase.java | 30 +- .../SqlNodeToCallOperationTest.java | 2 +- .../plan/batch/sql/RowLevelUpdateTest.java | 7 +- .../exec/serde/RexNodeJsonSerdeTest.java | 2 +- .../table/planner/plan/batch/sql/CalcTest.xml | 10 +- .../plan/batch/sql/DeadlockBreakupTest.xml | 6 +- .../batch/sql/ForwardHashExchangeTest.xml | 8 +- .../plan/batch/sql/RemoveCollationTest.xml | 4 +- .../plan/batch/sql/RemoveShuffleTest.xml | 16 +- .../plan/batch/sql/SubplanReuseTest.xml | 8 +- .../planner/plan/batch/sql/TableSinkTest.xml | 10 +- .../planner/plan/batch/sql/UnionTest.xml | 2 +- .../planner/plan/batch/sql/ValuesTest.xml | 3 +- .../plan/batch/sql/agg/OverAggregateTest.xml | 54 +- .../AggregateReduceFunctionsRuleTest.xml | 4 +- .../logical/ConvertToNotInOrInRuleTest.xml | 22 +- .../logical/FlinkPruneEmptyRulesTest.xml | 27 +- .../PushFilterIntoTableSourceScanRuleTest.xml | 2 +- ...veUnreachableCoalesceArgumentsRuleTest.xml | 4 +- .../logical/WindowGroupReorderRuleTest.xml | 32 +- .../logical/subquery/SubQuerySemiJoinTest.xml | 61 + ...ushLocalAggIntoTableSourceScanRuleTest.xml | 4 +- .../planner/plan/stream/sql/CalcTest.xml | 8 +- .../planner/plan/stream/sql/DeltaJoinTest.xml | 4 +- .../plan/stream/sql/MatchRecognizeTest.xml | 4 +- .../planner/plan/stream/sql/UnionTest.xml | 2 +- .../planner/plan/stream/sql/ValuesTest.xml | 3 +- .../plan/stream/sql/agg/AggregateTest.xml | 4 +- .../plan/stream/sql/agg/OverAggregateTest.xml | 46 +- .../planner/plan/stream/table/ValuesTest.xml | 82 +- .../FlinkRelMdColumnIntervalTest.scala | 6 +- .../metadata/FlinkRelMdHandlerTestBase.scala | 4 + .../metadata/FlinkRelMdSelectivityTest.scala | 1 + .../metadata/FlinkRelMdUpsertKeysTest.scala | 1 + .../logical/FlinkPruneEmptyRulesTest.scala | 2 +- .../subquery/SubQueryAntiJoinTest.scala | 2 +- .../subquery/SubQuerySemiJoinTest.scala | 40 +- flink-table/pom.xml | 2 +- 81 files changed, 3004 insertions(+), 4547 deletions(-) delete mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlJoin.java delete mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/type/BasicSqlType.java delete mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/schema/TimeIndicatorRelDataType.java delete mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/schema/TimeIndicatorRelDataType.scala diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index 182c46bbd9cf8..588add0ffba99 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -218,7 +218,7 @@ def udf_with_constant_params(p, null_param, tinyint_param, smallint_param, int_p "cast (1 as SMALLINT)," "cast (1 as INT)," "cast (1 as BIGINT)," - "cast (1.05 as DECIMAL)," + "cast (1.05 as DECIMAL(3,2))," "cast (1.23 as FLOAT)," "cast (1.98932 as DOUBLE)," "true," diff --git a/flink-table/flink-sql-parser/pom.xml b/flink-table/flink-sql-parser/pom.xml index 366d6d8ea1f2f..0264583bfecc7 100644 --- a/flink-table/flink-sql-parser/pom.xml +++ b/flink-table/flink-sql-parser/pom.xml @@ -68,13 +68,13 @@ under the License. ${calcite.version} + + + + + + + + + + + @@ -1491,6 +1520,38 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject(d=[$0]) +- LogicalFilter(condition=[true]) +- LogicalTableScan(table=[[default_catalog, default_database, r]]) +]]> + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml index 7852555acf870..229bac52ea0e8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml @@ -529,14 +529,14 @@ FROM inventory]]> (COUNT($3) OVER (PARTITION BY $1), 0), $SUM0($3) OVER (PARTITION BY $1), null:BIGINT)], name=[$1]) +LogicalProject(id=[$0], amount=[$2], EXPR$2=[CASE(>(COUNT($3) OVER (PARTITION BY $1), 0), SUM($3) OVER (PARTITION BY $1), null:BIGINT)], name=[$1]) +- LogicalTableScan(table=[[default_catalog, default_database, inventory]]) ]]> (w0$o0, 0), w0$o1, null:BIGINT) AS EXPR$2, name]) -+- OverAggregate(partitionBy=[name], window#0=[COUNT(price) AS w0$o0, $SUM0(price) AS w0$o1 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[id, name, amount, price, w0$o0, w0$o1]) ++- OverAggregate(partitionBy=[name], window#0=[COUNT(price) AS w0$o0, SUM(price) AS w0$o1 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[id, name, amount, price, w0$o0, w0$o1]) +- Sort(orderBy=[name ASC]) +- Exchange(distribution=[hash[name]]) +- TableSourceScan(table=[[default_catalog, default_database, inventory, project=[id, name, amount, price], metadata=[]]], fields=[id, name, amount, price]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/CalcTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/CalcTest.xml index 11b409c98fe18..b7b97659a739c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/CalcTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/CalcTest.xml @@ -181,13 +181,13 @@ Calc(select=[a, b, c], where=[((a < 10) AND (b > 20))]) @@ -198,13 +198,13 @@ Calc(select=[ARRAY(0.12, 0.5, 0.99) AS EXPR$0]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest.xml index 7c04bf6c2aaa6..0995440d9eeb7 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest.xml @@ -377,7 +377,7 @@ Sink(table=[default_catalog.default_database.snk], fields=[a0, a1, a2, a3, b0, b @@ -385,7 +385,7 @@ LogicalSink(table=[default_catalog.default_database.snk], fields=[a0, a1, a2, a3 (COUNT($1) OVER (PARTITION BY $0 ORDER BY $3 NULLS FIRST), 0), $SUM0($1) OVER (PARTITION BY $0 ORDER BY $3 NULLS FIRST), null:INTEGER)]), rowType=[RecordType(VARCHAR(2147483647) symbol, INTEGER price, INTEGER tax, TIMESTAMP_LTZ(3) *ROWTIME* matchRowtime, INTEGER price_sum)] +LogicalProject(symbol=[$0], price=[$1], tax=[$2], matchRowtime=[$3], price_sum=[CASE(>(COUNT($1) OVER (PARTITION BY $0 ORDER BY $3 NULLS FIRST), 0), SUM($1) OVER (PARTITION BY $0 ORDER BY $3 NULLS FIRST), null:INTEGER)]), rowType=[RecordType(VARCHAR(2147483647) symbol, INTEGER price, INTEGER tax, TIMESTAMP_LTZ(3) *ROWTIME* matchRowtime, INTEGER price_sum)] +- LogicalProject(symbol=[$0], price=[$1], tax=[$2], matchRowtime=[$3]), rowType=[RecordType(VARCHAR(2147483647) symbol, INTEGER price, INTEGER tax, TIMESTAMP_LTZ(3) *ROWTIME* matchRowtime)] +- LogicalMatch(partition=[[0]], order=[[1 ASC-nulls-first]], outputFields=[[symbol, price, tax, matchRowtime]], allRows=[false], after=[FLAG(SKIP TO NEXT ROW)], pattern=[_UTF-16LE'A'], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[>(PREV(A.$2, 0), 0)]], inputFields=[[symbol, ts_ltz, price, tax]]), rowType=[RecordType(VARCHAR(2147483647) symbol, INTEGER price, INTEGER tax, TIMESTAMP_LTZ(3) *ROWTIME* matchRowtime)] +- LogicalWatermarkAssigner(rowtime=[ts_ltz], watermark=[-($1, 1000:INTERVAL SECOND)]), rowType=[RecordType(VARCHAR(2147483647) symbol, TIMESTAMP_LTZ(3) *ROWTIME* ts_ltz, INTEGER price, INTEGER tax)] @@ -199,7 +199,7 @@ LogicalProject(symbol=[$0], price=[$1], tax=[$2], matchRowtime=[$3], price_sum=[ (w0$o0, 0), w0$o1, null:INTEGER) AS price_sum]), rowType=[RecordType(VARCHAR(2147483647) symbol, INTEGER price, INTEGER tax, TIMESTAMP_LTZ(3) *ROWTIME* matchRowtime, INTEGER price_sum)] -+- OverAggregate(partitionBy=[symbol], orderBy=[matchRowtime ASC], window=[ RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[symbol, price, tax, matchRowtime, COUNT(price) AS w0$o0, $SUM0(price) AS w0$o1]), rowType=[RecordType(VARCHAR(2147483647) symbol, INTEGER price, INTEGER tax, TIMESTAMP_LTZ(3) *ROWTIME* matchRowtime, BIGINT w0$o0, INTEGER w0$o1)] ++- OverAggregate(partitionBy=[symbol], orderBy=[matchRowtime ASC], window=[ RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[symbol, price, tax, matchRowtime, COUNT(price) AS w0$o0, SUM(price) AS w0$o1]), rowType=[RecordType(VARCHAR(2147483647) symbol, INTEGER price, INTEGER tax, TIMESTAMP_LTZ(3) *ROWTIME* matchRowtime, BIGINT w0$o0, INTEGER w0$o1)] +- Exchange(distribution=[hash[symbol]]), rowType=[RecordType(VARCHAR(2147483647) symbol, INTEGER price, INTEGER tax, TIMESTAMP_LTZ(3) *ROWTIME* matchRowtime)] +- Match(partitionBy=[symbol], orderBy=[ts_ltz ASC], measures=[FINAL(A.price) AS price, FINAL(A.tax) AS tax, FINAL(MATCH_ROWTIME(*.ts_ltz)) AS matchRowtime], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO NEXT ROW], pattern=[_UTF-16LE'A'], define=[{A=>(PREV(A.$2, 0), 0)}]), rowType=[RecordType(VARCHAR(2147483647) symbol, INTEGER price, INTEGER tax, TIMESTAMP_LTZ(3) *ROWTIME* matchRowtime)] +- Exchange(distribution=[hash[symbol]]), rowType=[RecordType(VARCHAR(2147483647) symbol, TIMESTAMP_LTZ(3) *ROWTIME* ts_ltz, INTEGER price, INTEGER tax)] diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnionTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnionTest.xml index 2969cdc7486ee..0dbdafc413b55 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnionTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnionTest.xml @@ -70,7 +70,7 @@ LogicalProject(a=[$0], b=[$1]), rowType=[RecordType(INTEGER a, DECIMAL(20, 1) b) +- LogicalUnion(all=[true]), rowType=[RecordType(INTEGER a, DECIMAL(20, 1) b)] :- LogicalProject(a=[$0], b=[$1]), rowType=[RecordType(INTEGER a, BIGINT b)] : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)] - +- LogicalProject(a=[$0], EXPR$1=[0:DECIMAL(2, 1)]), rowType=[RecordType(INTEGER a, DECIMAL(2, 1) EXPR$1)] + +- LogicalProject(a=[$0], EXPR$1=[0.0:DECIMAL(2, 1)]), rowType=[RecordType(INTEGER a, DECIMAL(2, 1) EXPR$1)] +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)] ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/ValuesTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/ValuesTest.xml index 098ecd36f3471..42260c1b33fc6 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/ValuesTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/ValuesTest.xml @@ -100,7 +100,8 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.xml index 92fccedf388e4..c2a34a6c49bac 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.xml @@ -68,7 +68,7 @@ FROM T GROUP BY a @@ -76,7 +76,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[SUM($2)], EXPR$3=[SUM($3 diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/OverAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/OverAggregateTest.xml index dc0185da0ec8d..fdd3264866764 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/OverAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/OverAggregateTest.xml @@ -111,14 +111,14 @@ FROM MyTable (COUNT($2) OVER (PARTITION BY $0 ORDER BY $3 NULLS FIRST RANGE 7200000:INTERVAL HOUR PRECEDING), 0), $SUM0($2) OVER (PARTITION BY $0 ORDER BY $3 NULLS FIRST RANGE 7200000:INTERVAL HOUR PRECEDING), null:BIGINT), COUNT($2) OVER (PARTITION BY $0 ORDER BY $3 NULLS FIRST RANGE 7200000:INTERVAL HOUR PRECEDING))]) +LogicalProject(a=[$0], avgA=[/(CASE(>(COUNT($2) OVER (PARTITION BY $0 ORDER BY $3 NULLS FIRST RANGE 7200000:INTERVAL HOUR PRECEDING), 0), SUM($2) OVER (PARTITION BY $0 ORDER BY $3 NULLS FIRST RANGE 7200000:INTERVAL HOUR PRECEDING), null:BIGINT), COUNT($2) OVER (PARTITION BY $0 ORDER BY $3 NULLS FIRST RANGE 7200000:INTERVAL HOUR PRECEDING))]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0), w0$o1, null:BIGINT) / w0$o0) AS avgA]) -+- OverAggregate(partitionBy=[a], orderBy=[proctime ASC], window=[ RANGE BETWEEN 7200000 PRECEDING AND CURRENT ROW], select=[a, c, proctime, COUNT(c) AS w0$o0, $SUM0(c) AS w0$o1]) ++- OverAggregate(partitionBy=[a], orderBy=[proctime ASC], window=[ RANGE BETWEEN 7200000 PRECEDING AND CURRENT ROW], select=[a, c, proctime, COUNT(c) AS w0$o0, SUM(c) AS w0$o1]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c, proctime]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime]) @@ -138,14 +138,14 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $2 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING), null:INTEGER)]) +LogicalProject(c=[$2], cnt1=[COUNT($0) OVER (PARTITION BY $2 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING)], sum1=[CASE(>(COUNT($0) OVER (PARTITION BY $2 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING), 0), SUM($0) OVER (PARTITION BY $2 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING), null:INTEGER)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0), w0$o1, null:INTEGER) AS sum1]) -+- OverAggregate(partitionBy=[c], orderBy=[proctime ASC], window=[ ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, c, proctime, COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1]) ++- OverAggregate(partitionBy=[c], orderBy=[proctime ASC], window=[ ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, c, proctime, COUNT(a) AS w0$o0, SUM(a) AS w0$o1]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c, proctime]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime]) @@ -158,14 +158,14 @@ Calc(select=[c, w0$o0 AS cnt1, CASE((w0$o0 > 0), w0$o1, null:INTEGER) AS sum1]) (COUNT($2) OVER (PARTITION BY $0 ORDER BY PROCTIME() NULLS FIRST ROWS 4 PRECEDING), 0), $SUM0($2) OVER (PARTITION BY $0 ORDER BY PROCTIME() NULLS FIRST ROWS 4 PRECEDING), null:BIGINT)], EXPR$2=[MIN($2) OVER (PARTITION BY $0 ORDER BY PROCTIME() NULLS FIRST ROWS 4 PRECEDING)]) +LogicalProject(a=[$0], EXPR$1=[CASE(>(COUNT($2) OVER (PARTITION BY $0 ORDER BY PROCTIME() NULLS FIRST ROWS 4 PRECEDING), 0), SUM($2) OVER (PARTITION BY $0 ORDER BY PROCTIME() NULLS FIRST ROWS 4 PRECEDING), null:BIGINT)], EXPR$2=[MIN($2) OVER (PARTITION BY $0 ORDER BY PROCTIME() NULLS FIRST ROWS 4 PRECEDING)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0), w0$o1, null:BIGINT) AS EXPR$1, w0$o2 AS EXPR$2]) -+- OverAggregate(partitionBy=[a], orderBy=[$2 ASC], window=[ ROWS BETWEEN 4 PRECEDING AND CURRENT ROW], select=[a, c, $2, COUNT(c) AS w0$o0, $SUM0(c) AS w0$o1, MIN(c) AS w0$o2]) ++- OverAggregate(partitionBy=[a], orderBy=[$2 ASC], window=[ ROWS BETWEEN 4 PRECEDING AND CURRENT ROW], select=[a, c, $2, COUNT(c) AS w0$o0, SUM(c) AS w0$o1, MIN(c) AS w0$o2]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c, PROCTIME() AS $2]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime]) @@ -183,14 +183,14 @@ FROM MyTable (COUNT($0) OVER (ORDER BY $3 NULLS FIRST), 0), $SUM0($0) OVER (ORDER BY $3 NULLS FIRST), null:INTEGER)]) +LogicalProject(c=[$2], cnt1=[COUNT($0) OVER (ORDER BY $3 NULLS FIRST)], cnt2=[CASE(>(COUNT($0) OVER (ORDER BY $3 NULLS FIRST), 0), SUM($0) OVER (ORDER BY $3 NULLS FIRST), null:INTEGER)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0), w0$o1, null:INTEGER) AS cnt2]) -+- OverAggregate(orderBy=[proctime ASC], window=[ RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, c, proctime, COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1]) ++- OverAggregate(orderBy=[proctime ASC], window=[ RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, c, proctime, COUNT(a) AS w0$o0, SUM(a) AS w0$o1]) +- Exchange(distribution=[single]) +- Calc(select=[a, c, proctime]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime]) @@ -231,14 +231,14 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $2 ORDER BY $3 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $3 NULLS FIRST), null:INTEGER)]) +LogicalProject(c=[$2], cnt1=[COUNT($0) OVER (PARTITION BY $2 ORDER BY $3 NULLS FIRST)], cnt2=[CASE(>(COUNT($0) OVER (PARTITION BY $2 ORDER BY $3 NULLS FIRST), 0), SUM($0) OVER (PARTITION BY $2 ORDER BY $3 NULLS FIRST), null:INTEGER)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0), w0$o1, null:INTEGER) AS cnt2]) -+- OverAggregate(partitionBy=[c], orderBy=[proctime ASC], window=[ RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, c, proctime, COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1]) ++- OverAggregate(partitionBy=[c], orderBy=[proctime ASC], window=[ RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, c, proctime, COUNT(a) AS w0$o0, SUM(a) AS w0$o1]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c, proctime]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime]) @@ -282,14 +282,14 @@ FROM MyTable (COUNT(DISTINCT $0) OVER (PARTITION BY $2 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING), 0), $SUM0(DISTINCT $0) OVER (PARTITION BY $2 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING), null:INTEGER)]) +LogicalProject(c=[$2], cnt1=[COUNT(DISTINCT $0) OVER (PARTITION BY $2 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING)], sum1=[CASE(>(COUNT(DISTINCT $0) OVER (PARTITION BY $2 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING), 0), SUM(DISTINCT $0) OVER (PARTITION BY $2 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING), null:INTEGER)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0), w0$o1, null:INTEGER) AS sum1]) -+- OverAggregate(partitionBy=[c], orderBy=[proctime ASC], window=[ ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, c, proctime, COUNT(DISTINCT a) AS w0$o0, $SUM0(DISTINCT a) AS w0$o1]) ++- OverAggregate(partitionBy=[c], orderBy=[proctime ASC], window=[ ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, c, proctime, COUNT(DISTINCT a) AS w0$o0, SUM(DISTINCT a) AS w0$o1]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c, proctime]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime]) @@ -313,14 +313,14 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $1 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING), null:INTEGER)], cnt2=[COUNT(DISTINCT $0) OVER (PARTITION BY $1 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING)], sum2=[CASE(>(COUNT(DISTINCT $2) OVER (PARTITION BY $1 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING), 0), $SUM0(DISTINCT $2) OVER (PARTITION BY $1 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING), null:BIGINT)]) +LogicalProject(b=[$1], cnt1=[COUNT($0) OVER (PARTITION BY $1 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING)], sum1=[CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING), 0), SUM($0) OVER (PARTITION BY $1 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING), null:INTEGER)], cnt2=[COUNT(DISTINCT $0) OVER (PARTITION BY $1 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING)], sum2=[CASE(>(COUNT(DISTINCT $2) OVER (PARTITION BY $1 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING), 0), SUM(DISTINCT $2) OVER (PARTITION BY $1 ORDER BY $3 NULLS FIRST ROWS 2 PRECEDING), null:BIGINT)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0), w0$o1, null:INTEGER) AS sum1, w0$o2 AS cnt2, CASE((w0$o3 > 0), w0$o4, null:BIGINT) AS sum2]) -+- OverAggregate(partitionBy=[b], orderBy=[proctime ASC], window=[ ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, b, c, proctime, COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1, COUNT(DISTINCT a) AS w0$o2, COUNT(DISTINCT c) AS w0$o3, $SUM0(DISTINCT c) AS w0$o4]) ++- OverAggregate(partitionBy=[b], orderBy=[proctime ASC], window=[ ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, b, c, proctime, COUNT(a) AS w0$o0, SUM(a) AS w0$o1, COUNT(DISTINCT a) AS w0$o2, COUNT(DISTINCT c) AS w0$o3, SUM(DISTINCT c) AS w0$o4]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[a, b, c, proctime]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime]) @@ -437,14 +437,14 @@ FROM MyTable (COUNT($0) OVER (ORDER BY $4 NULLS FIRST), 0), $SUM0($0) OVER (ORDER BY $4 NULLS FIRST), null:INTEGER)]) +LogicalProject(c=[$2], cnt1=[COUNT($0) OVER (ORDER BY $4 NULLS FIRST)], cnt2=[CASE(>(COUNT($0) OVER (ORDER BY $4 NULLS FIRST), 0), SUM($0) OVER (ORDER BY $4 NULLS FIRST), null:INTEGER)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0), w0$o1, null:INTEGER) AS cnt2]) -+- OverAggregate(orderBy=[rowtime ASC], window=[ RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, c, rowtime, COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1]) ++- OverAggregate(orderBy=[rowtime ASC], window=[ RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, c, rowtime, COUNT(a) AS w0$o0, SUM(a) AS w0$o1]) +- Exchange(distribution=[single]) +- Calc(select=[a, c, rowtime]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime]) @@ -462,14 +462,14 @@ FROM MyTable (COUNT($0) OVER (ORDER BY $4 NULLS FIRST ROWS UNBOUNDED PRECEDING), 0), $SUM0($0) OVER (ORDER BY $4 NULLS FIRST ROWS UNBOUNDED PRECEDING), null:INTEGER)]) +LogicalProject(c=[$2], cnt1=[COUNT($0) OVER (ORDER BY $4 NULLS FIRST ROWS UNBOUNDED PRECEDING)], cnt2=[CASE(>(COUNT($0) OVER (ORDER BY $4 NULLS FIRST ROWS UNBOUNDED PRECEDING), 0), SUM($0) OVER (ORDER BY $4 NULLS FIRST ROWS UNBOUNDED PRECEDING), null:INTEGER)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0), w0$o1, null:INTEGER) AS cnt2]) -+- OverAggregate(orderBy=[rowtime ASC], window=[ ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, c, rowtime, COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1]) ++- OverAggregate(orderBy=[rowtime ASC], window=[ ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, c, rowtime, COUNT(a) AS w0$o0, SUM(a) AS w0$o1]) +- Exchange(distribution=[single]) +- Calc(select=[a, c, rowtime]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime]) @@ -487,14 +487,14 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $2 ORDER BY $4 NULLS FIRST), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $4 NULLS FIRST), null:INTEGER)]) +LogicalProject(c=[$2], cnt1=[COUNT($0) OVER (PARTITION BY $2 ORDER BY $4 NULLS FIRST)], cnt2=[CASE(>(COUNT($0) OVER (PARTITION BY $2 ORDER BY $4 NULLS FIRST), 0), SUM($0) OVER (PARTITION BY $2 ORDER BY $4 NULLS FIRST), null:INTEGER)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0), w0$o1, null:INTEGER) AS cnt2]) -+- OverAggregate(partitionBy=[c], orderBy=[rowtime ASC], window=[ RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, c, rowtime, COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1]) ++- OverAggregate(partitionBy=[c], orderBy=[rowtime ASC], window=[ RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, c, rowtime, COUNT(a) AS w0$o0, SUM(a) AS w0$o1]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c, rowtime]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime]) @@ -512,14 +512,14 @@ FROM MyTable (COUNT($0) OVER (PARTITION BY $2 ORDER BY $4 NULLS FIRST ROWS UNBOUNDED PRECEDING), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $4 NULLS FIRST ROWS UNBOUNDED PRECEDING), null:INTEGER)]) +LogicalProject(c=[$2], cnt1=[COUNT($0) OVER (PARTITION BY $2 ORDER BY $4 NULLS FIRST ROWS UNBOUNDED PRECEDING)], cnt2=[CASE(>(COUNT($0) OVER (PARTITION BY $2 ORDER BY $4 NULLS FIRST ROWS UNBOUNDED PRECEDING), 0), SUM($0) OVER (PARTITION BY $2 ORDER BY $4 NULLS FIRST ROWS UNBOUNDED PRECEDING), null:INTEGER)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> 0), w0$o1, null:INTEGER) AS cnt2]) -+- OverAggregate(partitionBy=[c], orderBy=[rowtime ASC], window=[ ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, c, rowtime, COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1]) ++- OverAggregate(partitionBy=[c], orderBy=[rowtime ASC], window=[ ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, c, rowtime, COUNT(a) AS w0$o0, SUM(a) AS w0$o1]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c, rowtime]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime]) @@ -728,7 +728,7 @@ LogicalProject(c=[$2], cnt1=[COUNT($0) OVER (PARTITION BY $1 ORDER BY $3 NULLS F @@ -45,11 +45,11 @@ LogicalUnion(all=[true]) @@ -58,30 +58,30 @@ Union(all=[true], union=[f0]) @@ -90,7 +90,7 @@ Union(all=[true], union=[f0, f1, f2]) @@ -146,30 +146,30 @@ Union(all=[true], union=[f0, f1, f2]) @@ -178,30 +178,30 @@ Union(all=[true], union=[f0, f1, f2]) @@ -244,18 +244,18 @@ Union(all=[true], union=[a, b]) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnIntervalTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnIntervalTest.scala index 232213f48046e..aab16f95d4c9f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnIntervalTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnIntervalTest.scala @@ -139,7 +139,7 @@ class FlinkRelMdColumnIntervalTest extends FlinkRelMdHandlerTestBase { // id <= 20 val expr1 = relBuilder.call(LESS_THAN_OR_EQUAL, relBuilder.field(0), relBuilder.literal(20)) // id > 10.0 (note: the types of id and literal are different) - val expr2 = relBuilder.call(GREATER_THAN, relBuilder.field(0), relBuilder.literal(10.0)) + val expr2 = relBuilder.call(GREATER_THAN, relBuilder.field(0), relBuilder.literal(bd(10.0))) // DIV(id, 2) > 3 val expr3 = relBuilder.call( GREATER_THAN, @@ -165,13 +165,13 @@ class FlinkRelMdColumnIntervalTest extends FlinkRelMdHandlerTestBase { // id <= 20 AND id > 10 AND DIV(id, 2) > 3 val filter2 = relBuilder.push(ts).filter(expr1, expr2, expr3).build assertEquals( - ValueInterval(bd(10.0), bd(20), includeLower = false), + ValueInterval(bd(10.0), bd(20.0), includeLower = false), mq.getColumnInterval(filter2, 0)) // id <= 20 AND id > 10 AND score < 4.1 val filter3 = relBuilder.push(ts).filter(expr1, expr2, expr4).build assertEquals( - ValueInterval(bd(10.0), bd(20), includeLower = false), + ValueInterval(bd(10.0), bd(20.0), includeLower = false), mq.getColumnInterval(filter3, 0)) // score > 6.0 OR score <= 4.0 diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala index eaca9fa23c901..d8815d3aa6bf7 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala @@ -2435,6 +2435,7 @@ class FlinkRelMdHandlerTestBase { true, RexWindowBounds.create(SqlWindow.createUnboundedPreceding(new SqlParserPos(0, 0)), null), RexWindowBounds.create(SqlWindow.createCurrentRow(new SqlParserPos(0, 0)), null), + null, RelCollations.of( new RelFieldCollation( 1, @@ -2568,6 +2569,7 @@ class FlinkRelMdHandlerTestBase { true, RexWindowBounds.create(SqlWindow.createUnboundedPreceding(new SqlParserPos(0, 0)), null), RexWindowBounds.create(SqlWindow.createCurrentRow(new SqlParserPos(0, 0)), null), + null, RelCollations.of( new RelFieldCollation( 1, @@ -2589,6 +2591,7 @@ class FlinkRelMdHandlerTestBase { false, RexWindowBounds.create(SqlWindow.createUnboundedPreceding(new SqlParserPos(4, 15)), null), RexWindowBounds.create(SqlWindow.createCurrentRow(new SqlParserPos(0, 0)), null), + null, RelCollations.of( new RelFieldCollation( 2, @@ -2634,6 +2637,7 @@ class FlinkRelMdHandlerTestBase { false, RexWindowBounds.create(SqlWindow.createUnboundedPreceding(new SqlParserPos(7, 19)), null), RexWindowBounds.create(SqlWindow.createUnboundedFollowing(new SqlParserPos(0, 0)), null), + null, RelCollations.EMPTY, ImmutableList.of( new Window.RexWinAggCall( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSelectivityTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSelectivityTest.scala index eea6b0b433605..1fc2f71faeb13 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSelectivityTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSelectivityTest.scala @@ -520,6 +520,7 @@ class FlinkRelMdSelectivityTest extends FlinkRelMdHandlerTestBase { true, RexWindowBounds.create(SqlWindow.createUnboundedPreceding(new SqlParserPos(0, 0)), null), RexWindowBounds.create(SqlWindow.createCurrentRow(new SqlParserPos(0, 0)), null), + null, RelCollations.of( new RelFieldCollation( 1, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeysTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeysTest.scala index 996ef4f2de8bf..4a77662bd2b70 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeysTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeysTest.scala @@ -571,6 +571,7 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { true, RexWindowBounds.create(SqlWindow.createUnboundedPreceding(new SqlParserPos(0, 0)), null), RexWindowBounds.create(SqlWindow.createCurrentRow(new SqlParserPos(0, 0)), null), + null, RelCollations.of( new RelFieldCollation( 2, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.scala index 74e9ad050fea3..11258852f29a2 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.scala @@ -47,7 +47,7 @@ class FlinkPruneEmptyRulesTest extends TableTestBase { CoreRules.FILTER_REDUCE_EXPRESSIONS, CoreRules.PROJECT_REDUCE_EXPRESSIONS, CoreRules.FILTER_SET_OP_TRANSPOSE, - CoreRules.FILTER_PROJECT_TRANSPOSE, + FlinkFilterProjectTransposeRule.INSTANCE, CoreRules.PROJECT_MERGE, CoreRules.PROJECT_FILTER_VALUES_MERGE, FlinkPruneEmptyRules.UNION_INSTANCE, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.scala index 7ecef1016c0d1..065f5b97c7750 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.scala @@ -737,7 +737,7 @@ class SubQueryAntiJoinTest extends SubQueryTestBase { // TODO some bugs in SubQueryRemoveRule // the result RelNode (LogicalJoin(condition=[=($1, $11)], joinType=[left])) // after SubQueryRemoveRule is unexpected - assertThatExceptionOfType(classOf[AssertionError]) + assertThatExceptionOfType(classOf[NullPointerException]) .isThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[anti]")) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.scala index 5b0802ccf59f8..809ac66c577c2 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.scala @@ -964,25 +964,9 @@ class SubQuerySemiJoinTest extends SubQueryTestBase { @Test def testInWithCorrelatedOnHaving(): Unit = { - // TODO There are some bugs when converting SqlNode to RelNode: val sqlQuery = "SELECT SUM(a) AS s FROM x GROUP BY b " + "HAVING MAX(a) IN (SELECT d FROM y WHERE y.d = x.b)" - - // the logical plan is: - // - // LogicalProject(s=[$1]) - // LogicalFilter(condition=[IN($2, { - // LogicalProject(d=[$1]) - // LogicalFilter(condition=[=($1, $cor0.b)]) - // LogicalTableScan(table=[[builtin, default, r]]) - // })]) - // LogicalAggregate(group=[{0}], s=[SUM($1)], agg#1=[MAX($1)]) - // LogicalProject(b=[$1], a=[$0]) - // LogicalTableScan(table=[[builtin, default, l]]) - // - // LogicalFilter lost variablesSet information. - - util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]") + util.verifyRelPlan(sqlQuery) } @Test @@ -1622,25 +1606,9 @@ class SubQuerySemiJoinTest extends SubQueryTestBase { @Test def testExistsWithCorrelatedOnHaving(): Unit = { - // TODO There are some bugs when converting SqlNode to RelNode: - val sqlQuery1 = + val sqlQuery = "SELECT SUM(a) AS s FROM x GROUP BY b HAVING EXISTS (SELECT * FROM y WHERE y.d = x.b)" - - // the logical plan is: - // - // LogicalProject(s=[$1]) - // LogicalFilter(condition=[IN($2, { - // LogicalProject(d=[$1]) - // LogicalFilter(condition=[=($1, $cor0.b)]) - // LogicalTableScan(table=[[builtin, default, r]]) - // })]) - // LogicalAggregate(group=[{0}], s=[SUM($1)], agg#1=[MAX($1)]) - // LogicalProject(b=[$1], a=[$0]) - // LogicalTableScan(table=[[builtin, default, l]]) - // - // LogicalFilter lost variablesSet information. - - util.verifyRelPlanNotExpected(sqlQuery1, "joinType=[semi]") + util.verifyRelPlan(sqlQuery) } @Test @@ -1711,7 +1679,7 @@ class SubQuerySemiJoinTest extends SubQueryTestBase { // TODO some bugs in SubQueryRemoveRule // the result RelNode (LogicalJoin(condition=[=($1, $8)], joinType=[left])) // after SubQueryRemoveRule is unexpected - assertThatExceptionOfType(classOf[AssertionError]) + assertThatExceptionOfType(classOf[NullPointerException]) .isThrownBy(() => util.verifyRelPlanNotExpected(sqlQuery, "joinType=[semi]")) } diff --git a/flink-table/pom.xml b/flink-table/pom.xml index 476e911568487..eacd06e2cc1ce 100644 --- a/flink-table/pom.xml +++ b/flink-table/pom.xml @@ -79,7 +79,7 @@ under the License. - 1.37.0 + 1.38.0 3.1.11 33.4.0-jre 2.5.2 From f94c6a9f75c38da22ef896c5d4aec07921efeb58 Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Thu, 28 May 2026 13:50:12 +0200 Subject: [PATCH 2/4] [FLINK-36602][tests] Replace deprecated `JavaConversions` in `FlinkRelMdHandlerTest` and its children --- .../FlinkRelMdColumnIntervalTest.scala | 36 +- .../FlinkRelMdColumnNullCountTest.scala | 11 +- .../FlinkRelMdColumnOriginNullCountTest.scala | 11 +- .../FlinkRelMdColumnUniquenessTest.scala | 4 +- .../FlinkRelMdDistinctRowCountTest.scala | 5 +- .../metadata/FlinkRelMdDistributionTest.scala | 18 +- ...FlinkRelMdFilteredColumnIntervalTest.scala | 5 +- .../metadata/FlinkRelMdHandlerTestBase.scala | 345 ++++++++++-------- .../FlinkRelMdModifiedMonotonicityTest.scala | 11 +- .../metadata/FlinkRelMdRowCollationTest.scala | 24 +- .../metadata/FlinkRelMdRowCountTest.scala | 4 +- .../metadata/FlinkRelMdSelectivityTest.scala | 29 +- .../plan/metadata/FlinkRelMdSizeTest.scala | 106 +++--- .../metadata/FlinkRelMdUniqueGroupsTest.scala | 24 +- .../metadata/FlinkRelMdUniqueKeysTest.scala | 173 +++++---- .../metadata/FlinkRelMdUpsertKeysTest.scala | 183 +++++----- 16 files changed, 533 insertions(+), 456 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnIntervalTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnIntervalTest.scala index aab16f95d4c9f..1902cf6e4be06 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnIntervalTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnIntervalTest.scala @@ -27,7 +27,7 @@ import org.apache.flink.table.types.logical.IntType import org.apache.calcite.rel.RelDistributions import org.apache.calcite.rel.core.JoinRelType import org.apache.calcite.rel.logical.LogicalExchange -import org.apache.calcite.rex.{RexCall, RexUtil} +import org.apache.calcite.rex.{RexCall, RexNode, RexUtil} import org.apache.calcite.sql.fun.SqlStdOperatorTable._ import org.apache.calcite.util.{DateString, TimestampString, TimeString} import org.junit.jupiter.api.Assertions._ @@ -35,8 +35,6 @@ import org.junit.jupiter.api.Test import java.sql.{Date, Time, Timestamp} -import scala.collection.JavaConversions._ - class FlinkRelMdColumnIntervalTest extends FlinkRelMdHandlerTestBase { @Test @@ -238,7 +236,11 @@ class FlinkRelMdColumnIntervalTest extends FlinkRelMdHandlerTestBase { val expr7 = relBuilder.call(GREATER_THAN, relBuilder.field(2), relBuilder.literal(1.9d)) // calc => projects + filter(id <= 20) - val calc1 = createLogicalCalc(studentLogicalScan, outputRowType, projects, List(expr1)) + val calc1 = createLogicalCalc( + studentLogicalScan, + outputRowType, + projects, + java.util.List.of[RexNode](expr1)) assertEquals(ValueInterval(bd(0), bd(20)), mq.getColumnInterval(calc1, 0)) assertNull(mq.getColumnInterval(calc1, 1)) assertEqualsAsDouble(ValueInterval(bd(2.9), bd(5.0)), mq.getColumnInterval(calc1, 2)) @@ -254,7 +256,11 @@ class FlinkRelMdColumnIntervalTest extends FlinkRelMdHandlerTestBase { // calc => project + filter(id <= 20 AND id > 10 AND DIV(id, 2) > 3) val calc2 = - createLogicalCalc(studentLogicalScan, outputRowType, projects, List(expr1, expr2, expr3)) + createLogicalCalc( + studentLogicalScan, + outputRowType, + projects, + java.util.List.of(expr1, expr2, expr3)) assertEquals( ValueInterval(bd(10), bd(20), includeLower = false), mq.getColumnInterval(calc2, 0)) @@ -262,7 +268,11 @@ class FlinkRelMdColumnIntervalTest extends FlinkRelMdHandlerTestBase { // calc => project + filter(id <= 20 AND id > 10 AND score < 4.1) val calc3 = - createLogicalCalc(studentLogicalScan, outputRowType, projects, List(expr1, expr2, expr4)) + createLogicalCalc( + studentLogicalScan, + outputRowType, + projects, + java.util.List.of(expr1, expr2, expr4)) assertEquals( ValueInterval(bd(10), bd(20), includeLower = false), mq.getColumnInterval(calc3, 0)) @@ -272,7 +282,7 @@ class FlinkRelMdColumnIntervalTest extends FlinkRelMdHandlerTestBase { studentLogicalScan, outputRowType, projects, - List(relBuilder.call(OR, expr5, expr6))) + java.util.List.of[RexNode](relBuilder.call(OR, expr5, expr6))) assertEqualsAsDouble(ValueInterval(bd(2.9), bd(5.0)), mq.getColumnInterval(calc4, 2)) // calc => project + filter(score > 6.0 OR score <= 4.0 OR id < 20) @@ -280,7 +290,7 @@ class FlinkRelMdColumnIntervalTest extends FlinkRelMdHandlerTestBase { studentLogicalScan, outputRowType, projects, - List(relBuilder.call(OR, expr5, expr6, expr1))) + java.util.List.of[RexNode](relBuilder.call(OR, expr5, expr6, expr1))) assertEqualsAsDouble(ValueInterval(bd(2.9), bd(5.0)), mq.getColumnInterval(calc5, 2)) // calc => project + filter((id <= 20 AND score < 4.1) OR NOT(DIV(id, 2) > 3 OR score > 1.9)) @@ -288,7 +298,7 @@ class FlinkRelMdColumnIntervalTest extends FlinkRelMdHandlerTestBase { studentLogicalScan, outputRowType, projects, - List( + java.util.List.of[RexNode]( relBuilder.call( OR, relBuilder.call(AND, expr1, expr4), @@ -301,7 +311,7 @@ class FlinkRelMdColumnIntervalTest extends FlinkRelMdHandlerTestBase { studentLogicalScan, outputRowType, projects, - List( + java.util.List.of[RexNode]( relBuilder.call( OR, relBuilder.call(AND, expr1, expr4), @@ -331,7 +341,11 @@ class FlinkRelMdColumnIntervalTest extends FlinkRelMdHandlerTestBase { Array("f0", "f1", "f2", "f3"), Array(new IntType(), new IntType(), new IntType(), new IntType())) val calc8 = - createLogicalCalc(studentLogicalScan, rowType, List(expr8, expr9, expr10, expr11), List()) + createLogicalCalc( + studentLogicalScan, + rowType, + java.util.List.of(expr8, expr9, expr10, expr11), + java.util.List.of()) assertEquals(ValueInterval(bd(0), bd(1)), mq.getColumnInterval(calc8, 0)) assertEquals(ValueInterval(bd(10), bd(12)), mq.getColumnInterval(calc8, 1)) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCountTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCountTest.scala index 9d7a4073c8f08..4d4af154745d1 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCountTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCountTest.scala @@ -18,12 +18,11 @@ package org.apache.flink.table.planner.plan.metadata import org.apache.calcite.rel.core.JoinRelType +import org.apache.calcite.rex.RexNode import org.apache.calcite.sql.fun.SqlStdOperatorTable._ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test -import scala.collection.JavaConversions._ - class FlinkRelMdColumnNullCountTest extends FlinkRelMdHandlerTestBase { @Test @@ -126,7 +125,7 @@ class FlinkRelMdColumnNullCountTest extends FlinkRelMdHandlerTestBase { studentLogicalScan, studentLogicalScan.getRowType, relBuilder.fields(), - List(expr1)) + java.util.List.of[RexNode](expr1)) assertEquals(0.0, mq.getColumnNullCount(calc2, 0)) assertEquals(0.0, mq.getColumnNullCount(calc2, 1)) assertNull(mq.getColumnNullCount(calc2, 2)) @@ -141,7 +140,7 @@ class FlinkRelMdColumnNullCountTest extends FlinkRelMdHandlerTestBase { studentLogicalScan, studentLogicalScan.getRowType, relBuilder.fields(), - List(expr2)) + java.util.List.of[RexNode](expr2)) assertEquals(0.0, mq.getColumnNullCount(calc3, 0)) assertEquals(0.0, mq.getColumnNullCount(calc3, 1)) assertNull(mq.getColumnNullCount(calc3, 2)) @@ -156,7 +155,7 @@ class FlinkRelMdColumnNullCountTest extends FlinkRelMdHandlerTestBase { studentLogicalScan, logicalProject.getRowType, logicalProject.getProjects, - List(expr1)) + java.util.List.of[RexNode](expr1)) assertEquals(0.0, mq.getColumnNullCount(calc4, 0)) assertEquals(0.0, mq.getColumnNullCount(calc4, 1)) assertNull(mq.getColumnNullCount(calc4, 2)) @@ -175,7 +174,7 @@ class FlinkRelMdColumnNullCountTest extends FlinkRelMdHandlerTestBase { studentLogicalScan, logicalProject.getRowType, logicalProject.getProjects, - List(expr2)) + java.util.List.of[RexNode](expr2)) assertEquals(0.0, mq.getColumnNullCount(calc5, 0)) assertEquals(0.0, mq.getColumnNullCount(calc5, 1)) assertNull(mq.getColumnNullCount(calc5, 2)) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnOriginNullCountTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnOriginNullCountTest.scala index ef23c4858dda1..4a137ce040fc7 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnOriginNullCountTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnOriginNullCountTest.scala @@ -18,12 +18,11 @@ package org.apache.flink.table.planner.plan.metadata import org.apache.calcite.rel.core.JoinRelType +import org.apache.calcite.rex.RexNode import org.apache.calcite.sql.fun.SqlStdOperatorTable.{EQUALS, LESS_THAN_OR_EQUAL} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test -import scala.collection.JavaConversions._ - class FlinkRelMdColumnOriginNullCountTest extends FlinkRelMdHandlerTestBase { @Test @@ -69,7 +68,7 @@ class FlinkRelMdColumnOriginNullCountTest extends FlinkRelMdHandlerTestBase { val ts = relBuilder.scan("MyTable3").build() relBuilder.push(ts) - val projects = List( + val projects = java.util.List.of( relBuilder.call(EQUALS, relBuilder.field(0), relBuilder.literal(1)), relBuilder.field(0), relBuilder.field(1), @@ -95,14 +94,14 @@ class FlinkRelMdColumnOriginNullCountTest extends FlinkRelMdHandlerTestBase { studentLogicalScan, studentLogicalScan.getRowType, relBuilder.fields(), - List(expr)) + java.util.List.of[RexNode](expr)) (0 until calc1.getRowType.getFieldCount).foreach { idx => assertNull(mq.getColumnOriginNullCount(calc1, idx)) } val ts = relBuilder.scan("MyTable3").build() relBuilder.push(ts) - val projects = List( + val projects = java.util.List.of( relBuilder.call(EQUALS, relBuilder.field(0), relBuilder.literal(1)), relBuilder.field(0), relBuilder.field(1), @@ -110,7 +109,7 @@ class FlinkRelMdColumnOriginNullCountTest extends FlinkRelMdHandlerTestBase { relBuilder.literal(null) ) val outputRowType = relBuilder.project(projects).build().getRowType - val calc2 = createLogicalCalc(ts, outputRowType, projects, List()) + val calc2 = createLogicalCalc(ts, outputRowType, projects, java.util.List.of()) assertEquals(null, mq.getColumnOriginNullCount(calc2, 0)) assertEquals(1.0, mq.getColumnOriginNullCount(calc2, 1)) assertEquals(0.0, mq.getColumnOriginNullCount(calc2, 2)) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnUniquenessTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnUniquenessTest.scala index 57920d5d6ccc9..49d07882c63f4 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnUniquenessTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnUniquenessTest.scala @@ -29,8 +29,6 @@ import org.apache.calcite.util.ImmutableBitSet import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test -import scala.collection.JavaConversions._ - class FlinkRelMdColumnUniquenessTest extends FlinkRelMdHandlerTestBase { @Test @@ -93,7 +91,7 @@ class FlinkRelMdColumnUniquenessTest extends FlinkRelMdHandlerTestBase { // project: id, cast(id as long not null), name, cast(name as varchar not null) relBuilder.push(studentLogicalScan) - val exprs = List( + val exprs = java.util.List.of( relBuilder.field(0), relBuilder.cast(relBuilder.field(0), BIGINT), relBuilder.field(1), diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCountTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCountTest.scala index 568307603f898..4beb3b1802ad6 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCountTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCountTest.scala @@ -20,13 +20,12 @@ package org.apache.flink.table.planner.plan.metadata import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalRank import org.apache.flink.table.planner.plan.utils.FlinkRelMdUtil +import org.apache.calcite.rex.RexNode import org.apache.calcite.sql.fun.SqlStdOperatorTable._ import org.apache.calcite.util.ImmutableBitSet import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test -import scala.collection.JavaConversions._ - class FlinkRelMdDistinctRowCountTest extends FlinkRelMdHandlerTestBase { @Test @@ -187,7 +186,7 @@ class FlinkRelMdDistinctRowCountTest extends FlinkRelMdHandlerTestBase { studentLogicalScan, logicalProject.getRowType, logicalProject.getProjects, - List(expr1)) + java.util.List.of[RexNode](expr1)) assertEquals(1.0, mq.getDistinctRowCount(calc, ImmutableBitSet.of(), null)) assertEquals(25.0, mq.getDistinctRowCount(calc, ImmutableBitSet.of(0), null)) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistributionTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistributionTest.scala index 0b741f2fb22e7..b8325f725f625 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistributionTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistributionTest.scala @@ -24,12 +24,11 @@ import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalD import org.apache.flink.table.types.logical.{BigIntType, DoubleType} import com.google.common.collect.ImmutableList +import org.apache.calcite.rex.RexNode import org.apache.calcite.sql.fun.SqlStdOperatorTable._ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test -import scala.collection.JavaConversions._ - class FlinkRelMdDistributionTest extends FlinkRelMdHandlerTestBase { @Test @@ -70,7 +69,11 @@ class FlinkRelMdDistributionTest extends FlinkRelMdHandlerTestBase { relBuilder.push(scan1) val expr4 = relBuilder.call(LESS_THAN, relBuilder.field(4), relBuilder.literal(170.0)) val calc = - createLogicalCalc(scan1, logicalProject.getRowType, logicalProject.getProjects, List(expr4)) + createLogicalCalc( + scan1, + logicalProject.getRowType, + logicalProject.getProjects, + java.util.List.of[RexNode](expr4)) assertEquals( FlinkRelDistribution.hash(Array(6), requireStrict = false), mq.flinkDistribution(calc)) @@ -80,7 +83,7 @@ class FlinkRelMdDistributionTest extends FlinkRelMdHandlerTestBase { createDataStreamScan(ImmutableList.of("student"), flinkLogicalTraits.replace(distribution01)) relBuilder.push(scan2) // projects: $0==1, $0, $1, true, 2.1, 2 - val projects1 = List( + val projects1 = java.util.List.of( relBuilder.call(EQUALS, relBuilder.field(0), relBuilder.literal(1)), relBuilder.field(0), relBuilder.field(1), @@ -92,13 +95,14 @@ class FlinkRelMdDistributionTest extends FlinkRelMdHandlerTestBase { relBuilder.push(scan2) val expr1 = relBuilder.call(LESS_THAN_OR_EQUAL, relBuilder.field(0), relBuilder.literal(2)) // calc => projects + filter: $0 <= 2 - val calc1 = createLogicalCalc(scan2, outputRowType, projects1, List(expr1)) + val calc1 = + createLogicalCalc(scan2, outputRowType, projects1, java.util.List.of[RexNode](expr1)) assertEquals( FlinkRelDistribution.hash(Array(1, 2), requireStrict = false), mq.flinkDistribution(calc1)) // projects: $0==1, $0, 2.1, true, 2.1, 2 - val projects2 = List( + val projects2 = java.util.List.of( relBuilder.call(EQUALS, relBuilder.field(0), relBuilder.literal(1)), relBuilder.field(0), makeLiteral(2.1, new DoubleType(), isNullable = false, allowCast = true), @@ -106,7 +110,7 @@ class FlinkRelMdDistributionTest extends FlinkRelMdHandlerTestBase { makeLiteral(2.1, new DoubleType(), isNullable = false, allowCast = true), makeLiteral(2L, new BigIntType(), isNullable = false, allowCast = true) ) - val calc2 = createLogicalCalc(scan2, outputRowType, projects2, List()) + val calc2 = createLogicalCalc(scan2, outputRowType, projects2, java.util.List.of()) assertEquals(FlinkRelDistribution.ANY, mq.flinkDistribution(calc2)) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdFilteredColumnIntervalTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdFilteredColumnIntervalTest.scala index 6ea6a129fa1ec..96a24e332fa39 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdFilteredColumnIntervalTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdFilteredColumnIntervalTest.scala @@ -26,7 +26,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable.{DIVIDE, EQUALS, GREATER_T import org.junit.jupiter.api.{BeforeEach, Test} import org.junit.jupiter.api.Assertions.{assertEquals, assertNull} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters.seqAsJavaListConverter class FlinkRelMdFilteredColumnIntervalTest extends FlinkRelMdHandlerTestBase { private var ts: RelNode = _ @@ -202,7 +202,8 @@ class FlinkRelMdFilteredColumnIntervalTest extends FlinkRelMdHandlerTestBase { new BooleanType() ) ) - val calc = createLogicalCalc(ts, outputRowType, projects, List(expr1)) + val calc = + createLogicalCalc(ts, outputRowType, projects.asJava, java.util.List.of[RexNode](expr1)) assertEquals(ValueInterval(bd(-5), bd(2)), mq.getFilteredColumnInterval(calc, 0, -1)) assertEquals(ValueInterval(bd(0d), bd(6.1d)), mq.getFilteredColumnInterval(calc, 1, -1)) assertEquals(ValueInterval(bd(-5), bd(2)), mq.getFilteredColumnInterval(calc, 0, 2)) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala index d8815d3aa6bf7..2662d3c3139ad 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala @@ -77,7 +77,7 @@ import java.time.Duration import java.util import java.util.Collections -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters.{collectionAsScalaIterableConverter, seqAsJavaListConverter} class FlinkRelMdHandlerTestBase { @@ -177,11 +177,20 @@ class FlinkRelMdHandlerTestBase { createTableSourceTable(ImmutableList.of("TableSourceTable1"), streamPhysicalTraits) protected lazy val flinkLogicalIntermediateTableScan: FlinkLogicalIntermediateTableScan = - createIntermediateScan(streamExchangeById, flinkLogicalTraits, Set(ImmutableBitSet.of(0))) + createIntermediateScan( + streamExchangeById, + flinkLogicalTraits, + java.util.Set.of[ImmutableBitSet](ImmutableBitSet.of(0))) protected lazy val batchPhysicalIntermediateTableScan: BatchPhysicalIntermediateTableScan = - createIntermediateScan(batchExchangeById, batchPhysicalTraits, Set(ImmutableBitSet.of(0))) + createIntermediateScan( + batchExchangeById, + batchPhysicalTraits, + java.util.Set.of[ImmutableBitSet](ImmutableBitSet.of(0))) protected lazy val streamPhysicalIntermediateTableScan: StreamPhysicalIntermediateTableScan = - createIntermediateScan(streamExchangeById, streamPhysicalTraits, Set(ImmutableBitSet.of(0))) + createIntermediateScan( + streamExchangeById, + streamPhysicalTraits, + java.util.Set.of[ImmutableBitSet](ImmutableBitSet.of(0))) protected lazy val tablePartiallyProjectedKeyLogicalScan: LogicalTableScan = createTableSourceTable( @@ -248,7 +257,7 @@ class FlinkRelMdHandlerTestBase { List(null, "false", "2017-09-01", "10:00:01", null, "3.12", null, null), List("3", "true", null, "10:00:02", "2017-10-01 01:00:00", "3.0", null, "xyz"), List("2", "true", "2017-10-02", "09:59:59", "2017-07-01 01:00:00", "-1", null, "F") - ).map(createLiteralList(valuesType, _)) + ).map(createLiteralList(valuesType, _)).asJava relBuilder.values(tupleList, valuesType) relBuilder.build().asInstanceOf[LogicalValues] } @@ -257,7 +266,7 @@ class FlinkRelMdHandlerTestBase { // case sex = 'M' then 1 else 2, true, 2.1, 2, cast(score as double not null) as s from student protected lazy val logicalProject: LogicalProject = { relBuilder.push(studentLogicalScan) - val projects = List( + val projects = java.util.List.of( // id relBuilder.field(0), // name @@ -304,7 +313,7 @@ class FlinkRelMdHandlerTestBase { studentLogicalScan, logicalProject.getRowType, logicalProject.getProjects, - List(expr)) + java.util.List.of[RexNode](expr)) (filter, calc) } @@ -383,7 +392,12 @@ class FlinkRelMdHandlerTestBase { } protected lazy val intermediateTable = - new IntermediateRelTable(Seq(""), streamExchangeById, null, false, Set(ImmutableBitSet.of(0))) + new IntermediateRelTable( + java.util.List.of[String](""), + streamExchangeById, + null, + false, + java.util.Set.of[ImmutableBitSet](ImmutableBitSet.of(0))) protected lazy val intermediateScan = new FlinkLogicalIntermediateTableScan( cluster, @@ -403,7 +417,7 @@ class FlinkRelMdHandlerTestBase { protected def createSorts(sortKeys: () => Seq[RexNode]): (RelNode, RelNode, RelNode, RelNode) = { val logicalSort = relBuilder .scan("student") - .sort(sortKeys()) + .sort(sortKeys().asJava) .build .asInstanceOf[LogicalSort] val collation = logicalSort.getCollation @@ -513,7 +527,7 @@ class FlinkRelMdHandlerTestBase { sortKeys: () => Seq[RexNode]): (RelNode, RelNode, RelNode, RelNode, RelNode, RelNode) = { val logicalSortLimit = relBuilder .scan("student") - .sort(sortKeys()) + .sort(sortKeys().asJava) .limit(10, 20) .build .asInstanceOf[LogicalSort] @@ -873,10 +887,10 @@ class FlinkRelMdHandlerTestBase { ) val builder = typeFactory.builder() - firstRow.getRowType.getFieldList.dropRight(2).foreach(builder.add) + firstRow.getRowType.getFieldList.asScala.dropRight(2).foreach(builder.add) val projectProgram = RexProgram.create( firstRow.getRowType, - Array(0, 1, 2).map(i => RexInputRef.of(i, firstRow.getRowType)).toList, + Array(0, 1, 2).map(i => RexInputRef.of(i, firstRow.getRowType)).toList.asJava, null, builder.build(), rexBuilder @@ -1016,8 +1030,8 @@ class FlinkRelMdHandlerTestBase { false, false, false, - Seq().toList, - Seq(Integer.valueOf(3)).toList, + java.util.List.of[RexNode](), + java.util.List.of[Integer](Integer.valueOf(3)), -1, null, RelCollations.of(), @@ -1034,7 +1048,7 @@ class FlinkRelMdHandlerTestBase { studentLogicalScan, ImmutableBitSet.of(0), null, - Seq(tableAggCall)) + Seq(tableAggCall).asJava) val flinkLogicalTableAgg = new FlinkLogicalTableAggregate( cluster, @@ -1042,7 +1056,7 @@ class FlinkRelMdHandlerTestBase { studentLogicalScan, ImmutableBitSet.of(0), null, - Seq(tableAggCall) + Seq(tableAggCall).asJava ) val builder = typeFactory.builder() @@ -1075,7 +1089,7 @@ class FlinkRelMdHandlerTestBase { relBuilder.scan("TemporalTable1") val ts = relBuilder.peek() val project = relBuilder - .project(relBuilder.fields(Seq[Integer](2, 0, 1, 4).toList)) + .project(relBuilder.fields(Seq[Integer](2, 0, 1, 4).toList.asJava)) .build() .asInstanceOf[Project] val program = @@ -1122,9 +1136,9 @@ class FlinkRelMdHandlerTestBase { streamExchange, flinkLogicalWindowAgg.getRowType, Array(1), - flinkLogicalWindowAgg.getAggCallList, + flinkLogicalWindowAgg.getAggCallList.asScala.toSeq, tumblingGroupWindow, - namedPropertiesOfWindowAgg, + namedPropertiesOfWindowAgg.asScala.toSeq, emitStrategy ) @@ -1170,11 +1184,11 @@ class FlinkRelMdHandlerTestBase { logicalAgg.getAggCallList ) - val aggCalls = logicalAgg.getAggCallList + val aggCalls = logicalAgg.getAggCallList.asScala val aggFunctionFactory = new AggFunctionFactory( FlinkTypeFactory.toLogicalRowType(studentBatchScan.getRowType), Array.empty[Int], - Array.fill(aggCalls.size())(false), + Array.fill(aggCalls.size)(false), false) val aggCallToAggFunction = aggCalls.zipWithIndex.map { case (call, index) => (call, aggFunctionFactory.createAggFunction(call, index)) @@ -1210,7 +1224,7 @@ class FlinkRelMdHandlerTestBase { Array(3), auxGrouping = Array(), true, - aggCallToAggFunction) + aggCallToAggFunction.toSeq) val batchExchange1 = new BatchPhysicalExchange( cluster, @@ -1226,7 +1240,7 @@ class FlinkRelMdHandlerTestBase { batchLocalAgg.getInput.getRowType, Array(0), auxGrouping = Array(), - aggCallToAggFunction, + aggCallToAggFunction.toSeq, isMerge = true) val batchExchange2 = new BatchPhysicalExchange( @@ -1243,17 +1257,17 @@ class FlinkRelMdHandlerTestBase { batchExchange2.getRowType, Array(3), auxGrouping = Array(), - aggCallToAggFunction, + aggCallToAggFunction.toSeq, isMerge = false) val aggCallNeedRetractions = - AggregateUtil.deriveAggCallNeedRetractions(1, aggCalls, needRetraction = false, null) + AggregateUtil.deriveAggCallNeedRetractions(1, aggCalls.toSeq, needRetraction = false, null) val streamLocalAgg = new StreamPhysicalLocalGroupAggregate( cluster, streamPhysicalTraits, studentStreamScan, Array(3), - aggCalls, + aggCalls.toSeq, aggCallNeedRetractions, false, PartialFinalType.NONE) @@ -1269,7 +1283,7 @@ class FlinkRelMdHandlerTestBase { streamExchange1, rowTypeOfGlobalAgg, Array(0), - aggCalls, + aggCalls.toSeq, aggCallNeedRetractions, streamLocalAgg.getInput.getRowType, AggregateUtil.needRetraction(streamLocalAgg), @@ -1286,7 +1300,7 @@ class FlinkRelMdHandlerTestBase { streamExchange2, rowTypeOfGlobalAgg, Array(3), - aggCalls) + aggCalls.toSeq) ( logicalAgg, @@ -1328,7 +1342,7 @@ class FlinkRelMdHandlerTestBase { streamGlobalAggWithoutLocalWithFilter) = { relBuilder.push(studentLogicalScan) - val projects = List( + val projects = java.util.List.of( relBuilder.field(0), relBuilder.field(1), relBuilder.field(2), @@ -1371,7 +1385,7 @@ class FlinkRelMdHandlerTestBase { false, false, false, - List(Integer.valueOf(argIndex)), + java.util.List.of[Integer](Integer.valueOf(argIndex)), filterArg, null, RelCollations.EMPTY, @@ -1401,10 +1415,10 @@ class FlinkRelMdHandlerTestBase { val logicalAggWithFilter = LogicalAggregate.create( calcOnStudentScan, - List(), + java.util.List.of[RelHint](), ImmutableBitSet.of(3), - List(ImmutableBitSet.of(3)), - aggCallList) + java.util.List.of[ImmutableBitSet](ImmutableBitSet.of(3)), + aggCallList.asJava) val flinkLogicalAggWithFilter = new FlinkLogicalAggregate( cluster, @@ -1414,11 +1428,11 @@ class FlinkRelMdHandlerTestBase { logicalAggWithFilter.getGroupSets, logicalAggWithFilter.getAggCallList) - val aggCalls = logicalAggWithFilter.getAggCallList + val aggCalls = logicalAggWithFilter.getAggCallList.asScala val aggFunctionFactory = new AggFunctionFactory( FlinkTypeFactory.toLogicalRowType(calcOnStudentScan.getRowType), Array.empty[Int], - Array.fill(aggCalls.size())(false), + Array.fill(aggCalls.size)(false), false) val aggCallToAggFunction = aggCalls.zipWithIndex.map { case (call, index) => (call, aggFunctionFactory.createAggFunction(call, index)) @@ -1476,7 +1490,7 @@ class FlinkRelMdHandlerTestBase { Array(3), auxGrouping = Array(), true, - aggCallToAggFunction) + aggCallToAggFunction.toSeq) val batchExchange1 = new BatchPhysicalExchange( cluster, @@ -1492,7 +1506,7 @@ class FlinkRelMdHandlerTestBase { batchLocalAggWithFilter.getInput.getRowType, Array(0), auxGrouping = Array(), - aggCallToAggFunction, + aggCallToAggFunction.toSeq, isMerge = true) val batchExchange2 = new BatchPhysicalExchange( @@ -1509,17 +1523,17 @@ class FlinkRelMdHandlerTestBase { batchExchange2.getRowType, Array(3), auxGrouping = Array(), - aggCallToAggFunction, + aggCallToAggFunction.toSeq, isMerge = false) val aggCallNeedRetractions = - AggregateUtil.deriveAggCallNeedRetractions(1, aggCalls, needRetraction = false, null) + AggregateUtil.deriveAggCallNeedRetractions(1, aggCalls.toSeq, needRetraction = false, null) val streamLocalAggWithFilter = new StreamPhysicalLocalGroupAggregate( cluster, streamPhysicalTraits, calcOnStudentScan, Array(3), - aggCalls, + aggCalls.toSeq, aggCallNeedRetractions, false, PartialFinalType.NONE) @@ -1535,7 +1549,7 @@ class FlinkRelMdHandlerTestBase { streamExchange1, rowTypeOfGlobalAgg, Array(0), - aggCalls, + aggCalls.toSeq, aggCallNeedRetractions, streamLocalAggWithFilter.getInput.getRowType, AggregateUtil.needRetraction(streamLocalAggWithFilter), @@ -1552,7 +1566,7 @@ class FlinkRelMdHandlerTestBase { streamExchange2, rowTypeOfGlobalAgg, Array(3), - aggCalls) + aggCalls.toSeq) ( logicalAggWithFilter, @@ -1599,13 +1613,13 @@ class FlinkRelMdHandlerTestBase { logicalAggWithAuxGroup.getAggCallList ) - val aggCalls = logicalAggWithAuxGroup.getAggCallList.filter { + val aggCalls = logicalAggWithAuxGroup.getAggCallList.asScala.filter { call => call.getAggregation != FlinkSqlOperatorTable.AUXILIARY_GROUP } val aggFunctionFactory = new AggFunctionFactory( FlinkTypeFactory.toLogicalRowType(studentBatchScan.getRowType), Array.empty[Int], - Array.fill(aggCalls.size())(false), + Array.fill(aggCalls.size)(false), false) val aggCallToAggFunction = aggCalls.zipWithIndex.map { case (call, index) => (call, aggFunctionFactory.createAggFunction(call, index)) @@ -1629,7 +1643,7 @@ class FlinkRelMdHandlerTestBase { Array(0), auxGrouping = Array(1, 4), true, - aggCallToAggFunction) + aggCallToAggFunction.toSeq) val hash0 = FlinkRelDistribution.hash(Array(0), requireStrict = true) val batchExchange = new BatchPhysicalExchange( @@ -1655,7 +1669,7 @@ class FlinkRelMdHandlerTestBase { batchLocalAggWithAuxGroup.getInput.getRowType, Array(0), auxGrouping = Array(1, 2), - aggCallToAggFunction, + aggCallToAggFunction.toSeq, isMerge = true) val batchExchange2 = new BatchPhysicalExchange( @@ -1672,7 +1686,7 @@ class FlinkRelMdHandlerTestBase { batchExchange2.getRowType, Array(0), auxGrouping = Array(1, 4), - aggCallToAggFunction, + aggCallToAggFunction.toSeq, isMerge = false) ( @@ -1700,8 +1714,8 @@ class FlinkRelMdHandlerTestBase { intervalOfMillis(900000) ) - protected lazy val namedPropertiesOfWindowAgg: Seq[NamedWindowProperty] = - Seq( + protected lazy val namedPropertiesOfWindowAgg: java.util.List[NamedWindowProperty] = + java.util.List.of( new NamedWindowProperty("w$start", new WindowStart(windowRef)), new NamedWindowProperty("w$end", new WindowStart(windowRef)), new NamedWindowProperty("w$rowtime", new RowtimeAttribute(windowRef)), @@ -1725,7 +1739,7 @@ class FlinkRelMdHandlerTestBase { relBuilder.scan("TemporalTable1") val ts = relBuilder.peek() val project = relBuilder - .project(relBuilder.fields(Seq[Integer](0, 1, 4, 2).toList)) + .project(relBuilder.fields(Seq[Integer](0, 1, 4, 2).toList.asJava)) .build() .asInstanceOf[Project] val program = @@ -1736,7 +1750,7 @@ class FlinkRelMdHandlerTestBase { false, false, false, - List[Integer](3), + java.util.List.of[Integer](3), -1, null, RelCollations.EMPTY, @@ -1780,17 +1794,17 @@ class FlinkRelMdHandlerTestBase { AggregateUtil.transformToBatchAggregateFunctions( typeFactory, FlinkTypeFactory.toLogicalRowType(batchExchange1.getRowType), - flinkLogicalWindowAgg.getAggCallList) - val aggCallToAggFunction = flinkLogicalWindowAgg.getAggCallList.zip(aggregates) + flinkLogicalWindowAgg.getAggCallList.asScala.toSeq) + val aggCallToAggFunction = flinkLogicalWindowAgg.getAggCallList.asScala.zip(aggregates) val localWindowAggTypes = (Array(0, 1).map(batchCalc.getRowType.getFieldList.get(_).getType) ++ // grouping Array(longType) ++ // assignTs - aggCallOfWindowAgg.map(_.getType)).toList // agg calls + aggCallOfWindowAgg.asScala.map(_.getType)).toList.asJava // agg calls val localWindowAggNames = (Array(0, 1).map(batchCalc.getRowType.getFieldNames.get(_)) ++ // grouping Array("assignedWindow$") ++ // assignTs - Array("count$0")).toList // agg calls + Array("count$0")).toList.asJava // agg calls val localWindowAggRowType = typeFactory.createStructType(localWindowAggTypes, localWindowAggNames) val batchLocalWindowAgg = new BatchPhysicalLocalHashWindowAggregate( @@ -1801,11 +1815,11 @@ class FlinkRelMdHandlerTestBase { batchCalc.getRowType, Array(0, 1), Array.empty, - aggCallToAggFunction, + aggCallToAggFunction.toSeq, tumblingGroupWindow, inputTimeFieldIndex = 2, inputTimeIsDate = false, - namedPropertiesOfWindowAgg, + namedPropertiesOfWindowAgg.asScala.toSeq, enableAssignPane = false) val batchExchange2 = new BatchPhysicalExchange( cluster, @@ -1820,11 +1834,11 @@ class FlinkRelMdHandlerTestBase { batchCalc.getRowType, Array(0, 1), Array.empty, - aggCallToAggFunction, + aggCallToAggFunction.toSeq, tumblingGroupWindow, inputTimeFieldIndex = 2, inputTimeIsDate = false, - namedPropertiesOfWindowAgg, + namedPropertiesOfWindowAgg.asScala.toSeq, enableAssignPane = false, isMerge = true ) @@ -1837,11 +1851,11 @@ class FlinkRelMdHandlerTestBase { batchExchange1.getRowType, Array(0, 1), Array.empty, - aggCallToAggFunction, + aggCallToAggFunction.toSeq, tumblingGroupWindow, inputTimeFieldIndex = 2, inputTimeIsDate = false, - namedPropertiesOfWindowAgg, + namedPropertiesOfWindowAgg.asScala.toSeq, enableAssignPane = false, isMerge = false ) @@ -1863,9 +1877,9 @@ class FlinkRelMdHandlerTestBase { streamExchange, flinkLogicalWindowAgg.getRowType, Array(0, 1), - flinkLogicalWindowAgg.getAggCallList, + flinkLogicalWindowAgg.getAggCallList.asScala.toSeq, tumblingGroupWindow, - namedPropertiesOfWindowAgg, + namedPropertiesOfWindowAgg.asScala.toSeq, emitStrategy ) @@ -1895,7 +1909,7 @@ class FlinkRelMdHandlerTestBase { relBuilder.scan("TemporalTable1") val ts = relBuilder.peek() val project = relBuilder - .project(relBuilder.fields(Seq[Integer](0, 1, 4).toList)) + .project(relBuilder.fields(Seq[Integer](0, 1, 4).toList.asJava)) .build() .asInstanceOf[Project] val program = @@ -1906,7 +1920,7 @@ class FlinkRelMdHandlerTestBase { false, false, false, - List[Integer](0), + java.util.List.of[Integer](0), -1, null, RelCollations.EMPTY, @@ -1950,17 +1964,17 @@ class FlinkRelMdHandlerTestBase { AggregateUtil.transformToBatchAggregateFunctions( typeFactory, FlinkTypeFactory.toLogicalRowType(batchExchange1.getRowType), - flinkLogicalWindowAgg.getAggCallList) - val aggCallToAggFunction = flinkLogicalWindowAgg.getAggCallList.zip(aggregates) + flinkLogicalWindowAgg.getAggCallList.asScala.toSeq) + val aggCallToAggFunction = flinkLogicalWindowAgg.getAggCallList.asScala.zip(aggregates) val localWindowAggTypes = (Array(batchCalc.getRowType.getFieldList.get(1).getType) ++ // grouping Array(longType) ++ // assignTs - aggCallOfWindowAgg.map(_.getType)).toList // agg calls + aggCallOfWindowAgg.asScala.map(_.getType)).toList.asJava // agg calls val localWindowAggNames = (Array(batchCalc.getRowType.getFieldNames.get(1)) ++ // grouping Array("assignedWindow$") ++ // assignTs - Array("count$0")).toList // agg calls + Array("count$0")).toList.asJava // agg calls val localWindowAggRowType = typeFactory.createStructType(localWindowAggTypes, localWindowAggNames) val batchLocalWindowAgg = new BatchPhysicalLocalHashWindowAggregate( @@ -1971,11 +1985,11 @@ class FlinkRelMdHandlerTestBase { batchCalc.getRowType, Array(1), Array.empty, - aggCallToAggFunction, + aggCallToAggFunction.toSeq, tumblingGroupWindow, inputTimeFieldIndex = 2, inputTimeIsDate = false, - namedPropertiesOfWindowAgg, + namedPropertiesOfWindowAgg.asScala.toSeq, enableAssignPane = false) val batchExchange2 = new BatchPhysicalExchange( cluster, @@ -1990,11 +2004,11 @@ class FlinkRelMdHandlerTestBase { batchCalc.getRowType, Array(0), Array.empty, - aggCallToAggFunction, + aggCallToAggFunction.toSeq, tumblingGroupWindow, inputTimeFieldIndex = 2, inputTimeIsDate = false, - namedPropertiesOfWindowAgg, + namedPropertiesOfWindowAgg.asScala.toSeq, enableAssignPane = false, isMerge = true ) @@ -2007,11 +2021,11 @@ class FlinkRelMdHandlerTestBase { batchExchange1.getRowType, Array(1), Array.empty, - aggCallToAggFunction, + aggCallToAggFunction.toSeq, tumblingGroupWindow, inputTimeFieldIndex = 2, inputTimeIsDate = false, - namedPropertiesOfWindowAgg, + namedPropertiesOfWindowAgg.asScala.toSeq, enableAssignPane = false, isMerge = false ) @@ -2033,9 +2047,9 @@ class FlinkRelMdHandlerTestBase { streamExchange, flinkLogicalWindowAgg.getRowType, Array(1), - flinkLogicalWindowAgg.getAggCallList, + flinkLogicalWindowAgg.getAggCallList.asScala.toSeq, tumblingGroupWindow, - namedPropertiesOfWindowAgg, + namedPropertiesOfWindowAgg.asScala.toSeq, emitStrategy ) @@ -2064,7 +2078,7 @@ class FlinkRelMdHandlerTestBase { relBuilder.scan("TemporalTable2") val ts = relBuilder.peek() val project = relBuilder - .project(relBuilder.fields(Seq[Integer](0, 2, 4, 1).toList)) + .project(relBuilder.fields(Seq[Integer](0, 2, 4, 1).toList.asJava)) .build() .asInstanceOf[Project] val program = @@ -2075,7 +2089,7 @@ class FlinkRelMdHandlerTestBase { false, false, false, - List[Integer](1), + java.util.List.of[Integer](1), -1, null, RelCollations.EMPTY, @@ -2088,7 +2102,7 @@ class FlinkRelMdHandlerTestBase { false, false, false, - List[Integer](3), + java.util.List.of[Integer](3), -1, null, RelCollations.EMPTY, @@ -2129,24 +2143,24 @@ class FlinkRelMdHandlerTestBase { val hash0 = FlinkRelDistribution.hash(Array(0), requireStrict = true) val batchExchange1 = new BatchPhysicalExchange(cluster, batchPhysicalTraits.replace(hash0), batchCalc, hash0) - val aggCallsWithoutAuxGroup = flinkLogicalWindowAggWithAuxGroup.getAggCallList.drop(1) + val aggCallsWithoutAuxGroup = flinkLogicalWindowAggWithAuxGroup.getAggCallList.asScala.drop(1) val (_, _, aggregates) = AggregateUtil.transformToBatchAggregateFunctions( typeFactory, FlinkTypeFactory.toLogicalRowType(batchExchange1.getRowType), - aggCallsWithoutAuxGroup) + aggCallsWithoutAuxGroup.toSeq) val aggCallToAggFunction = aggCallsWithoutAuxGroup.zip(aggregates) val localWindowAggTypes = (Array(batchCalc.getRowType.getFieldList.get(0).getType) ++ // grouping Array(longType) ++ // assignTs Array(batchCalc.getRowType.getFieldList.get(1).getType) ++ // auxGrouping - aggCallsWithoutAuxGroup.map(_.getType)).toList // agg calls + aggCallsWithoutAuxGroup.map(_.getType)).toList.asJava // agg calls val localWindowAggNames = (Array(batchCalc.getRowType.getFieldNames.get(0)) ++ // grouping Array("assignedWindow$") ++ // assignTs Array(batchCalc.getRowType.getFieldNames.get(1)) ++ // auxGrouping - Array("count$0")).toList // agg calls + Array("count$0")).toList.asJava // agg calls val localWindowAggRowType = typeFactory.createStructType(localWindowAggTypes, localWindowAggNames) val batchLocalWindowAggWithAuxGroup = new BatchPhysicalLocalHashWindowAggregate( @@ -2157,11 +2171,11 @@ class FlinkRelMdHandlerTestBase { batchCalc.getRowType, Array(0), Array(1), - aggCallToAggFunction, + aggCallToAggFunction.toSeq, tumblingGroupWindow, inputTimeFieldIndex = 2, inputTimeIsDate = false, - namedPropertiesOfWindowAgg, + namedPropertiesOfWindowAgg.asScala.toSeq, enableAssignPane = false) val batchExchange2 = new BatchPhysicalExchange( cluster, @@ -2176,11 +2190,11 @@ class FlinkRelMdHandlerTestBase { batchCalc.getRowType, Array(0), Array(2), // local output grouping keys: grouping + assignTs + auxGrouping - aggCallToAggFunction, + aggCallToAggFunction.toSeq, tumblingGroupWindow, inputTimeFieldIndex = 2, inputTimeIsDate = false, - namedPropertiesOfWindowAgg, + namedPropertiesOfWindowAgg.asScala.toSeq, enableAssignPane = false, isMerge = true ) @@ -2193,11 +2207,11 @@ class FlinkRelMdHandlerTestBase { batchExchange1.getRowType, Array(0), Array(1), - aggCallToAggFunction, + aggCallToAggFunction.toSeq, tumblingGroupWindow, inputTimeFieldIndex = 2, inputTimeIsDate = false, - namedPropertiesOfWindowAgg, + namedPropertiesOfWindowAgg.asScala.toSeq, enableAssignPane = false, isMerge = false ) @@ -2250,7 +2264,10 @@ class FlinkRelMdHandlerTestBase { val rowTypeOfCalc = createRowType("id", "name", "score", "age", "class") val rexProgram = RexProgram.create( studentFlinkLogicalScan.getRowType, - Array(0, 1, 2, 3, 6).map(i => RexInputRef.of(i, studentFlinkLogicalScan.getRowType)).toList, + Array(0, 1, 2, 3, 6) + .map(i => RexInputRef.of(i, studentFlinkLogicalScan.getRowType)) + .toList + .asJava, null, rowTypeOfCalc, rexBuilder @@ -2292,20 +2309,23 @@ class FlinkRelMdHandlerTestBase { "cnt") val projectProgram = RexProgram.create( flinkLogicalOverAgg.getRowType, - (0 until flinkLogicalOverAgg.getRowType.getFieldCount).flatMap { - i => - if (i < 8 || i >= 10) { - Array[RexNode](RexInputRef.of(i, flinkLogicalOverAgg.getRowType)) - } else if (i == 8) { - Array[RexNode]( - rexBuilder.makeCall( - SqlStdOperatorTable.DIVIDE, - RexInputRef.of(8, flinkLogicalOverAgg.getRowType), - RexInputRef.of(9, flinkLogicalOverAgg.getRowType))) - } else { - Array.empty[RexNode] - } - }.toList, + (0 until flinkLogicalOverAgg.getRowType.getFieldCount) + .flatMap { + i => + if (i < 8 || i >= 10) { + Array[RexNode](RexInputRef.of(i, flinkLogicalOverAgg.getRowType)) + } else if (i == 8) { + Array[RexNode]( + rexBuilder.makeCall( + SqlStdOperatorTable.DIVIDE, + RexInputRef.of(8, flinkLogicalOverAgg.getRowType), + RexInputRef.of(9, flinkLogicalOverAgg.getRowType))) + } else { + Array.empty[RexNode] + } + } + .toList + .asJava, null, rowTypeOfWindowAggOutput, rexBuilder @@ -2344,7 +2364,7 @@ class FlinkRelMdHandlerTestBase { sort1, outputRowType1, sort1.getRowType, - Seq(overAggGroups(0)), + Seq(overAggGroups.get(0)), flinkLogicalOverAgg ) @@ -2376,7 +2396,7 @@ class FlinkRelMdHandlerTestBase { sort2, outputRowType2, sort2.getRowType, - Seq(overAggGroups(1)), + Seq(overAggGroups.get(1)), flinkLogicalOverAgg ) @@ -2406,7 +2426,7 @@ class FlinkRelMdHandlerTestBase { exchange2, outputRowType3, exchange2.getRowType, - Seq(overAggGroups(2)), + Seq(overAggGroups.get(2)), flinkLogicalOverAgg ) @@ -2483,7 +2503,10 @@ class FlinkRelMdHandlerTestBase { val rowTypeOfCalc = createRowType("id", "name", "score", "age", "class") val rexProgram = RexProgram.create( studentFlinkLogicalScan.getRowType, - Array(0, 1, 2, 3, 6).map(i => RexInputRef.of(i, studentFlinkLogicalScan.getRowType)).toList, + Array(0, 1, 2, 3, 6) + .map(i => RexInputRef.of(i, studentFlinkLogicalScan.getRowType)) + .toList + .asJava, null, rowTypeOfCalc, rexBuilder @@ -2527,20 +2550,23 @@ class FlinkRelMdHandlerTestBase { createRowType("id", "name", "score", "age", "class", "rk", "drk", "avg_score") val projectProgram = RexProgram.create( flinkLogicalOverAgg.getRowType, - (0 until flinkLogicalOverAgg.getRowType.getFieldCount).flatMap { - i => - if (i < 7) { - Array[RexNode](RexInputRef.of(i, flinkLogicalOverAgg.getRowType)) - } else if (i == 7) { - Array[RexNode]( - rexBuilder.makeCall( - SqlStdOperatorTable.DIVIDE, - RexInputRef.of(7, flinkLogicalOverAgg.getRowType), - RexInputRef.of(8, flinkLogicalOverAgg.getRowType))) - } else { - Array.empty[RexNode] - } - }.toList, + (0 until flinkLogicalOverAgg.getRowType.getFieldCount) + .flatMap { + i => + if (i < 7) { + Array[RexNode](RexInputRef.of(i, flinkLogicalOverAgg.getRowType)) + } else if (i == 7) { + Array[RexNode]( + rexBuilder.makeCall( + SqlStdOperatorTable.DIVIDE, + RexInputRef.of(7, flinkLogicalOverAgg.getRowType), + RexInputRef.of(8, flinkLogicalOverAgg.getRowType))) + } else { + Array.empty[RexNode] + } + } + .toList + .asJava, null, rowTypeOfWindowAggOutput, rexBuilder @@ -3284,7 +3310,7 @@ class FlinkRelMdHandlerTestBase { relBuilder.push(studentLogicalScan) if (windowFunctionCall) { - val projects = List( + val projects = java.util.List.of( relBuilder.field(0), relBuilder.field(1), relBuilder.call(FlinkSqlOperatorTable.PROCTIME)) @@ -3342,7 +3368,10 @@ class FlinkRelMdHandlerTestBase { FunctionIdentifier.of("STRING_SPLIT"), new JavaUserDefinedTableFunctions.StringSplit())), rexBuilder.makeFieldAccess(correlVar, 1), - rexBuilder.makeCall(stringType, CAST, List(rexBuilder.makeFieldAccess(correlVar, 0))) + rexBuilder.makeCall( + stringType, + CAST, + java.util.List.of[RexNode](rexBuilder.makeFieldAccess(correlVar, 0))) ) new FlinkLogicalTableFunctionScan( cluster, @@ -3421,7 +3450,7 @@ class FlinkRelMdHandlerTestBase { )) } } - val program = RexProgram.create(tvf.getRowType, projects, null, outputType, rexBuilder) + val program = RexProgram.create(tvf.getRowType, projects.asJava, null, outputType, rexBuilder) new StreamPhysicalCalc( cluster, streamPhysicalTraits, @@ -3451,7 +3480,12 @@ class FlinkRelMdHandlerTestBase { protected def createUnionOnWindowTVF( tvf1: CommonPhysicalWindowTableFunction, tvf2: CommonPhysicalWindowTableFunction): Union = { - new StreamPhysicalUnion(cluster, streamPhysicalTraits, List(tvf1, tvf2), true, tvf1.getRowType) + new StreamPhysicalUnion( + cluster, + streamPhysicalTraits, + java.util.List.of(tvf1, tvf2), + true, + tvf1.getRowType) } // hash by field a @@ -3487,8 +3521,8 @@ class FlinkRelMdHandlerTestBase { relBuilder.push(windowTableFunctionScan) val groupKey = if (groupByWindow) - List(relBuilder.field(0), relBuilder.field(3), relBuilder.field(4)) - else List(relBuilder.field(0)) + java.util.List.of(relBuilder.field(0), relBuilder.field(3), relBuilder.field(4)) + else java.util.List.of[RexInputRef](relBuilder.field(0)) val logicalAgg = relBuilder .aggregate( @@ -3535,7 +3569,7 @@ class FlinkRelMdHandlerTestBase { traitSet, streamTumbleWindowTVFRel, Array(), - logicalAgg.getAggCallList, + logicalAgg.getAggCallList.asScala.toSeq, new WindowAttachedWindowingStrategy(tumbleWindowSpec, timeAttributeType, 5, 6), namedWindowProperties) @@ -3656,7 +3690,13 @@ class FlinkRelMdHandlerTestBase { upsertKeys: util.Set[ImmutableBitSet], statistic: FlinkStatistic = FlinkStatistic.UNKNOWN): T = { val intermediateTable = - new IntermediateRelTable(Seq(""), relNode, null, false, upsertKeys, statistic) + new IntermediateRelTable( + java.util.List.of[String](""), + relNode, + null, + false, + upsertKeys, + statistic) val conventionTrait = traitSet.getTrait(ConventionTraitDef.INSTANCE) val scan = conventionTrait match { @@ -3722,26 +3762,29 @@ class FlinkRelMdHandlerTestBase { literalValues: Seq[String]): util.List[RexLiteral] = { require(literalValues.length == rowType.getFieldCount) val rexBuilder = relBuilder.getRexBuilder - literalValues.zipWithIndex.map { - case (v, index) => - val fieldType = rowType.getFieldList.get(index).getType - if (v == null) { - rexBuilder.makeNullLiteral(fieldType) - } else { - fieldType.getSqlTypeName match { - case BIGINT => rexBuilder.makeLiteral(v.toLong, fieldType, true) - case INTEGER => rexBuilder.makeLiteral(v.toInt, fieldType, true) - case BOOLEAN => rexBuilder.makeLiteral(v.toBoolean) - case DATE => rexBuilder.makeDateLiteral(new DateString(v)) - case TIME => rexBuilder.makeTimeLiteral(new TimeString(v), 0) - case TIMESTAMP => rexBuilder.makeTimestampLiteral(new TimestampString(v), 0) - case DOUBLE => rexBuilder.makeApproxLiteral(BigDecimal.valueOf(v.toDouble)) - case FLOAT => rexBuilder.makeApproxLiteral(BigDecimal.valueOf(v.toFloat)) - case VARCHAR => rexBuilder.makeLiteral(v) - case _ => throw new TableException(s"${fieldType.getSqlTypeName} is not supported!") - } - }.asInstanceOf[RexLiteral] - }.toList + literalValues.zipWithIndex + .map { + case (v, index) => + val fieldType = rowType.getFieldList.get(index).getType + if (v == null) { + rexBuilder.makeNullLiteral(fieldType) + } else { + fieldType.getSqlTypeName match { + case BIGINT => rexBuilder.makeLiteral(v.toLong, fieldType, true) + case INTEGER => rexBuilder.makeLiteral(v.toInt, fieldType, true) + case BOOLEAN => rexBuilder.makeLiteral(v.toBoolean) + case DATE => rexBuilder.makeDateLiteral(new DateString(v)) + case TIME => rexBuilder.makeTimeLiteral(new TimeString(v), 0) + case TIMESTAMP => rexBuilder.makeTimestampLiteral(new TimestampString(v), 0) + case DOUBLE => rexBuilder.makeApproxLiteral(BigDecimal.valueOf(v.toDouble)) + case FLOAT => rexBuilder.makeApproxLiteral(BigDecimal.valueOf(v.toFloat)) + case VARCHAR => rexBuilder.makeLiteral(v) + case _ => throw new TableException(s"${fieldType.getSqlTypeName} is not supported!") + } + }.asInstanceOf[RexLiteral] + } + .toList + .asJava } protected def createLogicalCalc( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdModifiedMonotonicityTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdModifiedMonotonicityTest.scala index a086c081426ef..586c0daa92c78 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdModifiedMonotonicityTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdModifiedMonotonicityTest.scala @@ -26,7 +26,7 @@ import org.apache.calcite.rel.RelCollations import org.apache.calcite.rel.core.JoinRelType import org.apache.calcite.rel.hint.RelHint import org.apache.calcite.rel.logical.LogicalCalc -import org.apache.calcite.rex.{RexNode, RexProgram} +import org.apache.calcite.rex.{RexInputRef, RexNode, RexProgram} import org.apache.calcite.sql.fun.SqlStdOperatorTable._ import org.apache.calcite.sql.validate.SqlMonotonicity import org.apache.calcite.sql.validate.SqlMonotonicity._ @@ -36,7 +36,7 @@ import org.junit.jupiter.api.Test import java.util -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters.seqAsJavaListConverter import scala.language.postfixOps class FlinkRelMdModifiedMonotonicityTest extends FlinkRelMdHandlerTestBase { @@ -60,8 +60,9 @@ class FlinkRelMdModifiedMonotonicityTest extends FlinkRelMdHandlerTestBase { relBuilder.push(inputAgg) // project `age` field and corresponding output type - val projection = List(relBuilder.field("age")) - val ageFieldType = inputAgg.getRowType.getFieldList.filter(x => x.getName.equals("age")) + val projection = java.util.List.of[RexInputRef](relBuilder.field("age")) + val ageFieldType = + inputAgg.getRowType.getFieldList.stream().filter(x => x.getName.equals("age")).toList val outputType = new RelRecordType(ageFieldType) // select age from (select id, age, count() from student by id, age) where ... @@ -250,7 +251,7 @@ class FlinkRelMdModifiedMonotonicityTest extends FlinkRelMdHandlerTestBase { projectWithMaxAgg, ImmutableBitSet.of(0), null, - Seq(tableAggCall) + Seq(tableAggCall).toList.asJava ) assertEquals(null, mq.getRelModifiedMonotonicity(tableAggregate)) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCollationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCollationTest.scala index 5ee117dc1a42b..c92937cca6f1b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCollationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCollationTest.scala @@ -31,7 +31,7 @@ import org.apache.calcite.util.ImmutableBitSet import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.Test -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters.seqAsJavaListConverter class FlinkRelMdRowCollationTest extends FlinkRelMdHandlerTestBase { @@ -47,7 +47,7 @@ class FlinkRelMdRowCollationTest extends FlinkRelMdHandlerTestBase { List("1", "9.0", "true", "2"), List("2", "6.0", "false", "3"), List("3", "3.0", "true", "4") - ).map(createLiteralList(valuesType, _)) + ).map(createLiteralList(valuesType, _)).asJava relBuilder.clear() relBuilder.values(tupleList, valuesType) relBuilder.build().asInstanceOf[LogicalValues] @@ -61,11 +61,20 @@ class FlinkRelMdRowCollationTest extends FlinkRelMdHandlerTestBase { // Test intermediate table scan. val flinkLogicalIntermediateTableScan: FlinkLogicalIntermediateTableScan = - createIntermediateScan(flinkLogicalSort, flinkLogicalTraits, Set(ImmutableBitSet.of(0))) + createIntermediateScan( + flinkLogicalSort, + flinkLogicalTraits, + java.util.Set.of[ImmutableBitSet](ImmutableBitSet.of(0))) val batchPhysicalIntermediateTableScan: BatchPhysicalIntermediateTableScan = - createIntermediateScan(batchSort, batchPhysicalTraits, Set(ImmutableBitSet.of(0))) + createIntermediateScan( + batchSort, + batchPhysicalTraits, + java.util.Set.of[ImmutableBitSet](ImmutableBitSet.of(0))) val streamPhysicalIntermediateTableScan: StreamPhysicalIntermediateTableScan = - createIntermediateScan(streamSort, streamPhysicalTraits, Set(ImmutableBitSet.of(0))) + createIntermediateScan( + streamSort, + streamPhysicalTraits, + java.util.Set.of[ImmutableBitSet](ImmutableBitSet.of(0))) Array( flinkLogicalIntermediateTableScan, batchPhysicalIntermediateTableScan, @@ -104,7 +113,7 @@ class FlinkRelMdRowCollationTest extends FlinkRelMdHandlerTestBase { val project: LogicalProject = { relBuilder.push(collationValues) - val projects = List( + val projects = java.util.List.of( // a + b relBuilder.call(PLUS, relBuilder.field(0), relBuilder.literal(1)), // c @@ -125,7 +134,8 @@ class FlinkRelMdRowCollationTest extends FlinkRelMdHandlerTestBase { .add("a", SqlTypeName.BIGINT) .add("ts", SqlTypeName.VARCHAR) .build() - val tupleList = List(List("3", "2015-07-24 10:00:00")).map(createLiteralList(valuesType, _)) + val tupleList = + List(List("3", "2015-07-24 10:00:00")).map(createLiteralList(valuesType, _)).asJava relBuilder.values(tupleList, valuesType) val project2 = relBuilder .project( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCountTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCountTest.scala index 3792805bfff6a..ad44ac902db6f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCountTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCountTest.scala @@ -28,8 +28,6 @@ import org.apache.calcite.util.ImmutableBitSet import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test -import scala.collection.JavaConversions._ - class FlinkRelMdRowCountTest extends FlinkRelMdHandlerTestBase { @Test @@ -174,7 +172,7 @@ class FlinkRelMdRowCountTest extends FlinkRelMdHandlerTestBase { false, false, false, - List[Integer](3), + java.util.List.of[Integer](3), -1, null, RelCollations.EMPTY, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSelectivityTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSelectivityTest.scala index 1fc2f71faeb13..e8894c4f05f23 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSelectivityTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSelectivityTest.scala @@ -40,7 +40,7 @@ import org.junit.jupiter.api.Test import java.util import java.util.Collections -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters.seqAsJavaListConverter class FlinkRelMdSelectivityTest extends FlinkRelMdHandlerTestBase { @@ -72,9 +72,9 @@ class FlinkRelMdSelectivityTest extends FlinkRelMdHandlerTestBase { val expr1 = relBuilder.call(LESS_THAN_OR_EQUAL, relBuilder.field(0), relBuilder.literal(2)) val expr2 = relBuilder.call(GREATER_THAN, relBuilder.field(0), relBuilder.literal(-1)) val expr3 = relBuilder.call(LESS_THAN, relBuilder.field(1), relBuilder.literal(1.1d)) - relBuilder.filter(List(expr1, expr2, expr3)) + relBuilder.filter(java.util.List.of(expr1, expr2, expr3)) // top projects: $0==1, $0, $1, true, 2.1, 2 - val projects = List( + val projects = java.util.List.of( relBuilder.call(EQUALS, relBuilder.field(0), relBuilder.literal(1)), relBuilder.field(0), relBuilder.field(1), @@ -101,7 +101,7 @@ class FlinkRelMdSelectivityTest extends FlinkRelMdHandlerTestBase { val expr1 = relBuilder.call(LESS_THAN_OR_EQUAL, relBuilder.field(0), relBuilder.literal(2)) val expr2 = relBuilder.call(GREATER_THAN, relBuilder.field(0), relBuilder.literal(-1)) val expr3 = relBuilder.call(LESS_THAN, relBuilder.field(1), relBuilder.literal(1.1d)) - val filter = relBuilder.filter(List(expr1, expr2, expr3)).build() + val filter = relBuilder.filter(java.util.List.of(expr1, expr2, expr3)).build() relBuilder.push(filter) val pred1 = relBuilder.call(LESS_THAN_OR_EQUAL, relBuilder.field(0), relBuilder.literal(1)) assertEquals((1.0 + 1.0) / (2.0 + 1.0), mq.getSelectivity(filter, pred1)) @@ -115,7 +115,7 @@ class FlinkRelMdSelectivityTest extends FlinkRelMdHandlerTestBase { relBuilder.push(ts) // projects: $0==1, $0, $1, true, 2.1, 2 - val projects = List( + val projects = java.util.List.of( relBuilder.call(EQUALS, relBuilder.field(0), relBuilder.literal(1)), relBuilder.field(0), relBuilder.field(1), @@ -132,7 +132,8 @@ class FlinkRelMdSelectivityTest extends FlinkRelMdHandlerTestBase { val expr2 = relBuilder.call(GREATER_THAN, relBuilder.field(0), relBuilder.literal(-1)) val expr3 = relBuilder.call(LESS_THAN, relBuilder.field(1), relBuilder.literal(1.1d)) val rexBuilder = relBuilder.getRexBuilder - val predicate = RexUtil.composeConjunction(rexBuilder, List(expr1, expr2, expr3), true) + val predicate = + RexUtil.composeConjunction(rexBuilder, java.util.List.of(expr1, expr2, expr3), true) val program = RexProgram.create(ts.getRowType, projects, predicate, outputRowType, rexBuilder) val calc = new BatchPhysicalCalc(cluster, batchPhysicalTraits, ts, program, outputRowType) @@ -331,14 +332,14 @@ class FlinkRelMdSelectivityTest extends FlinkRelMdHandlerTestBase { val aggWithAuxGroupAndExpand = relBuilder .push(expand) .aggregate( - relBuilder.groupKey(relBuilder.fields(Seq[Integer](0, 4).toList)), + relBuilder.groupKey(relBuilder.fields(Seq[Integer](0, 4).toList.asJava)), Lists.newArrayList( AggregateCall.create( FlinkSqlOperatorTable.AUXILIARY_GROUP, false, false, false, - List[Integer](1), + java.util.List.of[Integer](1), -1, null, RelCollations.EMPTY, @@ -351,7 +352,7 @@ class FlinkRelMdSelectivityTest extends FlinkRelMdHandlerTestBase { false, false, false, - List[Integer](2), + java.util.List.of[Integer](2), -1, null, RelCollations.EMPTY, @@ -364,7 +365,7 @@ class FlinkRelMdSelectivityTest extends FlinkRelMdHandlerTestBase { false, false, false, - List[Integer](3), + java.util.List.of[Integer](3), -1, null, RelCollations.EMPTY, @@ -544,9 +545,9 @@ class FlinkRelMdSelectivityTest extends FlinkRelMdHandlerTestBase { ) )) val scan: FlinkLogicalDataStreamTableScan = - createDataStreamScan(List("MyTable4"), flinkLogicalTraits) + createDataStreamScan(java.util.List.of[String]("MyTable4"), flinkLogicalTraits) val builder = typeFactory.builder - scan.getRowType.getFieldList.foreach(f => builder.add(f.getName, f.getType)) + scan.getRowType.getFieldList.stream().forEach(f => builder.add(f.getName, f.getType)) builder.add(rankAggCall.getName, rankAggCall.getType) builder.add(maxAggCall.getName, maxAggCall.getType) val overWindow = new FlinkLogicalOverAggregate( @@ -607,7 +608,7 @@ class FlinkRelMdSelectivityTest extends FlinkRelMdHandlerTestBase { relBuilder.push(ts).push(right) val joinCondition = RexUtil.composeConjunction( rexBuilder, - List( + java.util.List.of( relBuilder.call(EQUALS, relBuilder.field(2, 0, 0), relBuilder.field(2, 1, 0)), relBuilder.call(GREATER_THAN, relBuilder.field(2, 0, 0), relBuilder.literal(-1)), relBuilder.call(GREATER_THAN, relBuilder.field(2, 1, 1), relBuilder.literal(0.1d)) @@ -619,7 +620,7 @@ class FlinkRelMdSelectivityTest extends FlinkRelMdHandlerTestBase { right, Collections.emptyList(), joinCondition, - Set.empty[CorrelationId], + java.util.Set.of[CorrelationId], JoinRelType.INNER) relBuilder.push(join) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSizeTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSizeTest.scala index 706ca89348948..738d5c22463cf 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSizeTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSizeTest.scala @@ -20,8 +20,6 @@ package org.apache.flink.table.planner.plan.metadata import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Test -import scala.collection.JavaConversions._ - class FlinkRelMdSizeTest extends FlinkRelMdHandlerTestBase { @Test @@ -45,47 +43,47 @@ class FlinkRelMdSizeTest extends FlinkRelMdHandlerTestBase { .foreach { scan => assertEquals( - Seq(8.0, 7.2, 8.0, 4.0, 8.0, 1.0, 4.0), - mq.getAverageColumnSizes(scan).toList) + java.util.List.of(8.0, 7.2, 8.0, 4.0, 8.0, 1.0, 4.0), + mq.getAverageColumnSizes(scan)) } Array(empLogicalScan, empBatchScan, empStreamScan).foreach { scan => assertEquals( - Seq(4.0, 12.0, 12.0, 4.0, 12.0, 8.0, 8.0, 4.0), - mq.getAverageColumnSizes(scan).toList) + java.util.List.of(4.0, 12.0, 12.0, 4.0, 12.0, 8.0, 8.0, 4.0), + mq.getAverageColumnSizes(scan)) } } @Test def testAverageColumnSizeOnValues(): Unit = { assertEquals( - Seq(6.25, 1.0, 9.25, 12.0, 9.25, 8.0, 1.0, 3.75), - mq.getAverageColumnSizes(logicalValues).toList) + java.util.List.of(6.25, 1.0, 9.25, 12.0, 9.25, 8.0, 1.0, 3.75), + mq.getAverageColumnSizes(logicalValues)) assertEquals( - Seq(8.0, 1.0, 12.0, 12.0, 12.0, 8.0, 4.0, 12.0), - mq.getAverageColumnSizes(emptyValues).toList) + java.util.List.of(8.0, 1.0, 12.0, 12.0, 12.0, 8.0, 4.0, 12.0), + mq.getAverageColumnSizes(emptyValues)) } @Test def testAverageColumnSizeOnProject(): Unit = { assertEquals( - Seq(8.0, 7.2, 8.0, 4.0, 8.0, 8.0, 8.0, 4.0, 1.0, 8.0, 8.0, 8.0), - mq.getAverageColumnSizes(logicalProject).toList) + java.util.List.of(8.0, 7.2, 8.0, 4.0, 8.0, 8.0, 8.0, 4.0, 1.0, 8.0, 8.0, 8.0), + mq.getAverageColumnSizes(logicalProject)) } @Test def testAverageColumnSizeOnFilter(): Unit = { assertEquals( - Seq(8.0, 7.2, 8.0, 4.0, 8.0, 1.0, 4.0), - mq.getAverageColumnSizes(logicalFilter).toList) + java.util.List.of(8.0, 7.2, 8.0, 4.0, 8.0, 1.0, 4.0), + mq.getAverageColumnSizes(logicalFilter)) } @Test def testAverageColumnSizeOnCalc(): Unit = { assertEquals( - Seq(8.0, 7.2, 8.0, 4.0, 8.0, 8.0, 8.0, 4.0, 1.0, 8.0, 8.0, 8.0), - mq.getAverageColumnSizes(logicalCalc).toList) + java.util.List.of(8.0, 7.2, 8.0, 4.0, 8.0, 8.0, 8.0, 4.0, 1.0, 8.0, 8.0, 8.0), + mq.getAverageColumnSizes(logicalCalc)) } @Test @@ -93,8 +91,8 @@ class FlinkRelMdSizeTest extends FlinkRelMdHandlerTestBase { Array(logicalExpand, flinkLogicalExpand, batchExpand, streamExpand).foreach { expand => assertEquals( - Seq(8.0, 7.2, 8.0, 4.0, 8.0, 1.0, 4.0, 8.0), - mq.getAverageColumnSizes(expand).toList) + java.util.List.of(8.0, 7.2, 8.0, 4.0, 8.0, 1.0, 4.0, 8.0), + mq.getAverageColumnSizes(expand)) } } @@ -103,8 +101,8 @@ class FlinkRelMdSizeTest extends FlinkRelMdHandlerTestBase { Array(batchExchange, streamExchange).foreach { exchange => assertEquals( - Seq(8.0, 7.2, 8.0, 4.0, 8.0, 1.0, 4.0), - mq.getAverageColumnSizes(exchange).toList) + java.util.List.of(8.0, 7.2, 8.0, 4.0, 8.0, 1.0, 4.0), + mq.getAverageColumnSizes(exchange)) } } @@ -113,12 +111,12 @@ class FlinkRelMdSizeTest extends FlinkRelMdHandlerTestBase { Array(logicalRank, flinkLogicalRank, batchGlobalRank, streamRank).foreach { rank => assertEquals( - Seq(8.0, 7.2, 8.0, 4.0, 8.0, 1.0, 4.0, 8.0), - mq.getAverageColumnSizes(rank).toList) + java.util.List.of(8.0, 7.2, 8.0, 4.0, 8.0, 1.0, 4.0, 8.0), + mq.getAverageColumnSizes(rank)) } assertEquals( - Seq(8.0, 7.2, 8.0, 4.0, 8.0, 1.0, 4.0), - mq.getAverageColumnSizes(batchLocalRank).toList) + java.util.List.of(8.0, 7.2, 8.0, 4.0, 8.0, 1.0, 4.0), + mq.getAverageColumnSizes(batchLocalRank)) } @Test @@ -140,7 +138,9 @@ class FlinkRelMdSizeTest extends FlinkRelMdHandlerTestBase { streamLimit ).foreach { sort => - assertEquals(Seq(8.0, 7.2, 8.0, 4.0, 8.0, 1.0, 4.0), mq.getAverageColumnSizes(sort).toList) + assertEquals( + java.util.List.of(8.0, 7.2, 8.0, 4.0, 8.0, 1.0, 4.0), + mq.getAverageColumnSizes(sort)) } } @@ -153,7 +153,8 @@ class FlinkRelMdSizeTest extends FlinkRelMdHandlerTestBase { batchGlobalAggWithoutLocal, streamGlobalAggWithLocal, streamGlobalAggWithoutLocal).foreach { - agg => assertEquals(Seq(4.0, 8.0, 8.0, 8.0, 8.0, 8.0), mq.getAverageColumnSizes(agg).toList) + agg => + assertEquals(java.util.List.of(4.0, 8.0, 8.0, 8.0, 8.0, 8.0), mq.getAverageColumnSizes(agg)) } Array( @@ -161,7 +162,8 @@ class FlinkRelMdSizeTest extends FlinkRelMdHandlerTestBase { flinkLogicalAggWithAuxGroup, batchGlobalAggWithLocalWithAuxGroup, batchGlobalAggWithoutLocalWithAuxGroup).foreach { - agg => assertEquals(Seq(8.0, 7.2, 8.0, 8.0, 8.0, 8.0), mq.getAverageColumnSizes(agg).toList) + agg => + assertEquals(java.util.List.of(8.0, 7.2, 8.0, 8.0, 8.0, 8.0), mq.getAverageColumnSizes(agg)) } } @@ -172,9 +174,14 @@ class FlinkRelMdSizeTest extends FlinkRelMdHandlerTestBase { flinkLogicalWindowAgg, batchGlobalWindowAggWithoutLocalAgg, batchGlobalWindowAggWithLocalAgg).foreach { - agg => assertEquals(Seq(4d, 32d, 8d, 12d, 12d, 12d, 12d), mq.getAverageColumnSizes(agg).toSeq) + agg => + assertEquals( + java.util.List.of(4d, 32d, 8d, 12d, 12d, 12d, 12d), + mq.getAverageColumnSizes(agg)) } - assertEquals(Seq(4.0, 32.0, 8.0, 8.0), mq.getAverageColumnSizes(batchLocalWindowAgg).toSeq) + assertEquals( + java.util.List.of(4.0, 32.0, 8.0, 8.0), + mq.getAverageColumnSizes(batchLocalWindowAgg)) Array( logicalWindowAggWithAuxGroup, @@ -182,11 +189,14 @@ class FlinkRelMdSizeTest extends FlinkRelMdHandlerTestBase { batchGlobalWindowAggWithoutLocalAggWithAuxGroup, batchGlobalWindowAggWithLocalAggWithAuxGroup ).foreach { - agg => assertEquals(Seq(8d, 4d, 8d, 12d, 12d, 12d, 12d), mq.getAverageColumnSizes(agg).toSeq) + agg => + assertEquals( + java.util.List.of(8d, 4d, 8d, 12d, 12d, 12d, 12d), + mq.getAverageColumnSizes(agg)) } assertEquals( - Seq(8d, 8d, 4d, 8d), - mq.getAverageColumnSizes(batchLocalWindowAggWithAuxGroup).toSeq) + java.util.List.of(8d, 8d, 4d, 8d), + mq.getAverageColumnSizes(batchLocalWindowAggWithAuxGroup)) } @Test @@ -194,19 +204,19 @@ class FlinkRelMdSizeTest extends FlinkRelMdHandlerTestBase { Array(flinkLogicalOverAgg, batchOverAgg).foreach { agg => assertEquals( - Seq(8.0, 7.2, 8.0, 4.0, 4.0, 8.0, 8.0, 8.0, 8.0, 8.0, 8.0), - mq.getAverageColumnSizes(agg).toList) + java.util.List.of(8.0, 7.2, 8.0, 4.0, 4.0, 8.0, 8.0, 8.0, 8.0, 8.0, 8.0), + mq.getAverageColumnSizes(agg)) } assertEquals( - Seq(8.0, 12.0, 8.0, 4.0, 4.0, 8.0, 8.0, 8.0), - mq.getAverageColumnSizes(streamOverAgg).toList) + java.util.List.of(8.0, 12.0, 8.0, 4.0, 4.0, 8.0, 8.0, 8.0), + mq.getAverageColumnSizes(streamOverAgg)) } @Test def testAverageColumnSizeOnJoin(): Unit = { assertEquals( - Seq(4.0, 8.0, 12.0, 88.8, 4.0, 8.0, 8.0, 4.0, 8.0), - mq.getAverageColumnSizes(logicalInnerJoinOnUniqueKeys).toList) + java.util.List.of(4.0, 8.0, 12.0, 88.8, 4.0, 8.0, 8.0, 4.0, 8.0), + mq.getAverageColumnSizes(logicalInnerJoinOnUniqueKeys)) Array( logicalInnerJoinOnDisjointKeys, logicalLeftJoinNotOnUniqueKeys, @@ -214,39 +224,45 @@ class FlinkRelMdSizeTest extends FlinkRelMdHandlerTestBase { logicalFullJoinWithoutEquiCond).foreach { join => assertEquals( - Seq(4.0, 8.0, 12.0, 88.8, 4.0, 4.0, 8.0, 12.0, 10.52, 4.0), - mq.getAverageColumnSizes(join).toList) + java.util.List.of(4.0, 8.0, 12.0, 88.8, 4.0, 4.0, 8.0, 12.0, 10.52, 4.0), + mq.getAverageColumnSizes(join)) } Array(logicalSemiJoinOnUniqueKeys, logicalAntiJoinNotOnUniqueKeys).foreach { - join => assertEquals(Seq(4.0, 8.0, 12.0, 88.8, 4.0), mq.getAverageColumnSizes(join).toList) + join => + assertEquals(java.util.List.of(4.0, 8.0, 12.0, 88.8, 4.0), mq.getAverageColumnSizes(join)) } } @Test def testAverageColumnSizeOnUnion(): Unit = { Array(logicalUnion, logicalUnionAll).foreach { - union => assertEquals(Seq(4.0, 8.0, 12.0, 49.66, 4.0), mq.getAverageColumnSizes(union).toList) + union => + assertEquals(java.util.List.of(4.0, 8.0, 12.0, 49.66, 4.0), mq.getAverageColumnSizes(union)) } } @Test def testAverageColumnSizeOnIntersect(): Unit = { Array(logicalIntersect, logicalIntersectAll).foreach { - union => assertEquals(Seq(4.0, 8.0, 12.0, 88.8, 4.0), mq.getAverageColumnSizes(union).toList) + union => + assertEquals(java.util.List.of(4.0, 8.0, 12.0, 88.8, 4.0), mq.getAverageColumnSizes(union)) } } @Test def testAverageColumnSizeOnMinus(): Unit = { Array(logicalMinus, logicalMinusAll).foreach { - union => assertEquals(Seq(4.0, 8.0, 12.0, 88.8, 4.0), mq.getAverageColumnSizes(union).toList) + union => + assertEquals(java.util.List.of(4.0, 8.0, 12.0, 88.8, 4.0), mq.getAverageColumnSizes(union)) } } @Test def testAverageColumnSizeOnDefault(): Unit = { - assertEquals(Seq(8.0, 12.0, 8.0, 4.0, 8.0, 12.0, 4.0), mq.getAverageColumnSizes(testRel).toList) + assertEquals( + java.util.List.of(8.0, 12.0, 8.0, 4.0, 8.0, 12.0, 4.0), + mq.getAverageColumnSizes(testRel)) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueGroupsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueGroupsTest.scala index 8ae002b351b6a..7f63798a2b8b1 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueGroupsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueGroupsTest.scala @@ -21,14 +21,13 @@ import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalExpand import org.apache.flink.table.planner.plan.utils.ExpandUtil import com.google.common.collect.{ImmutableList, ImmutableSet} +import org.apache.calcite.rex.RexNode import org.apache.calcite.sql.fun.SqlStdOperatorTable.{GREATER_THAN, LESS_THAN_OR_EQUAL, MULTIPLY, PLUS} import org.apache.calcite.util.ImmutableBitSet import org.assertj.core.api.Assertions.assertThatThrownBy import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Test -import scala.collection.JavaConversions._ - class FlinkRelMdUniqueGroupsTest extends FlinkRelMdHandlerTestBase { @Test @@ -86,7 +85,7 @@ class FlinkRelMdUniqueGroupsTest extends FlinkRelMdHandlerTestBase { // a <= 2 and b > 10 val expr1 = relBuilder.call(LESS_THAN_OR_EQUAL, relBuilder.field(0), relBuilder.literal(2)) val expr2 = relBuilder.call(GREATER_THAN, relBuilder.field(1), relBuilder.literal(10d)) - val filter1 = relBuilder.filter(List(expr1, expr2)).build() + val filter1 = relBuilder.filter(java.util.List.of(expr1, expr2)).build() assertEquals(ImmutableBitSet.of(0), mq.getUniqueGroups(filter1, ImmutableBitSet.of(0))) assertEquals(ImmutableBitSet.of(0, 1), mq.getUniqueGroups(filter1, ImmutableBitSet.of(0, 1))) @@ -95,7 +94,7 @@ class FlinkRelMdUniqueGroupsTest extends FlinkRelMdHandlerTestBase { // a <= 2 and b > 10 val expr3 = relBuilder.call(LESS_THAN_OR_EQUAL, relBuilder.field(0), relBuilder.literal(2)) val expr4 = relBuilder.call(GREATER_THAN, relBuilder.field(1), relBuilder.literal(10d)) - val filter2 = relBuilder.filter(List(expr3, expr4)).build() + val filter2 = relBuilder.filter(java.util.List.of(expr3, expr4)).build() assertEquals(ImmutableBitSet.of(0), mq.getUniqueGroups(filter2, ImmutableBitSet.of(0))) assertEquals(ImmutableBitSet.of(0), mq.getUniqueGroups(filter2, ImmutableBitSet.of(0, 1))) assertEquals(ImmutableBitSet.of(0), mq.getUniqueGroups(filter2, ImmutableBitSet.of(0, 1, 2))) @@ -107,7 +106,7 @@ class FlinkRelMdUniqueGroupsTest extends FlinkRelMdHandlerTestBase { relBuilder.scan("MyTable4") // a, b, c val proj1 = relBuilder - .project(List(relBuilder.field(0), relBuilder.field(1), relBuilder.field(2))) + .project(java.util.List.of(relBuilder.field(0), relBuilder.field(1), relBuilder.field(2))) .build() assertEquals(ImmutableBitSet.of(0), mq.getUniqueGroups(proj1, ImmutableBitSet.of(0))) assertEquals(ImmutableBitSet.of(0), mq.getUniqueGroups(proj1, ImmutableBitSet.of(0, 1))) @@ -119,7 +118,7 @@ class FlinkRelMdUniqueGroupsTest extends FlinkRelMdHandlerTestBase { // a + b, b * 2, 1 val proj2 = relBuilder .project( - List( + java.util.List.of( relBuilder.call(PLUS, relBuilder.field(0), relBuilder.field(1)), relBuilder.call(MULTIPLY, relBuilder.field(1), relBuilder.literal(2)), relBuilder.literal(1) @@ -135,7 +134,7 @@ class FlinkRelMdUniqueGroupsTest extends FlinkRelMdHandlerTestBase { // a, b * 2, c, 1, 2 val proj3 = relBuilder .project( - List( + java.util.List.of( relBuilder.field(0), relBuilder.call(MULTIPLY, relBuilder.field(1), relBuilder.literal(2)), relBuilder.field(2), @@ -156,8 +155,8 @@ class FlinkRelMdUniqueGroupsTest extends FlinkRelMdHandlerTestBase { relBuilder.scan("MyTable4") // a, a as a1, $2 val proj4 = relBuilder - .project( - List(relBuilder.field(0), relBuilder.alias(relBuilder.field(0), "a1"), relBuilder.field(2))) + .project(java.util.List + .of(relBuilder.field(0), relBuilder.alias(relBuilder.field(0), "a1"), relBuilder.field(2))) .build() assertEquals(ImmutableBitSet.of(0), mq.getUniqueGroups(proj4, ImmutableBitSet.of(0))) assertEquals(ImmutableBitSet.of(0), mq.getUniqueGroups(proj4, ImmutableBitSet.of(0, 1))) @@ -167,7 +166,8 @@ class FlinkRelMdUniqueGroupsTest extends FlinkRelMdHandlerTestBase { relBuilder.clear() relBuilder.scan("MyTable4") // true, 1 - val proj5 = relBuilder.project(List(relBuilder.literal(true), relBuilder.literal(1))).build() + val proj5 = + relBuilder.project(java.util.List.of(relBuilder.literal(true), relBuilder.literal(1))).build() assertEquals(ImmutableBitSet.of(0), mq.getUniqueGroups(proj5, ImmutableBitSet.of(0))) assertEquals(ImmutableBitSet.of(1), mq.getUniqueGroups(proj5, ImmutableBitSet.of(1))) assertEquals(ImmutableBitSet.of(0), mq.getUniqueGroups(proj5, ImmutableBitSet.of(0, 1))) @@ -179,7 +179,7 @@ class FlinkRelMdUniqueGroupsTest extends FlinkRelMdHandlerTestBase { // project: a, b * 2, c, 1, 2 // filter: a > 1 relBuilder.push(ts) - val projects = List( + val projects = java.util.List.of( relBuilder.field(0), relBuilder.call(MULTIPLY, relBuilder.field(1), relBuilder.literal(2)), relBuilder.field(2), @@ -188,7 +188,7 @@ class FlinkRelMdUniqueGroupsTest extends FlinkRelMdHandlerTestBase { ) val condition = relBuilder.call(LESS_THAN_OR_EQUAL, relBuilder.field(0), relBuilder.literal(2)) val outputRowType = relBuilder.push(ts).project(projects).build().getRowType - val calc = createLogicalCalc(ts, outputRowType, projects, List(condition)) + val calc = createLogicalCalc(ts, outputRowType, projects, java.util.List.of[RexNode](condition)) assertEquals(ImmutableBitSet.of(0), mq.getUniqueGroups(calc, ImmutableBitSet.of(0))) assertEquals(ImmutableBitSet.of(1), mq.getUniqueGroups(calc, ImmutableBitSet.of(1))) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeysTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeysTest.scala index 41fd47cc62a61..38f0c1fe0d45c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeysTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeysTest.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.utils.ExpandUtil import com.google.common.collect.{ImmutableList, ImmutableSet} import org.apache.calcite.prepare.CalciteCatalogReader import org.apache.calcite.rel.hint.RelHint +import org.apache.calcite.rex.RexNode import org.apache.calcite.sql.`type`.SqlTypeName.VARCHAR import org.apache.calcite.sql.fun.SqlStdOperatorTable.{EQUALS, LESS_THAN} import org.apache.calcite.util.ImmutableBitSet @@ -33,14 +34,14 @@ import org.junit.jupiter.api.Test import java.util.Collections -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters.setAsJavaSetConverter class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { @Test def testGetUniqueKeysOnTableScan(): Unit = { Array(studentLogicalScan, studentBatchScan, studentStreamScan).foreach { - scan => assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(scan).toSet) + scan => assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(scan)) } Array(empLogicalScan, empBatchScan, empStreamScan).foreach { @@ -49,21 +50,21 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { val table = relBuilder.getRelOptSchema .asInstanceOf[CalciteCatalogReader] - .getTable(Seq("projected_table_source_table")) + .getTable(java.util.List.of[String]("projected_table_source_table")) .asInstanceOf[TableSourceTable] val tableSourceScan = new StreamPhysicalTableSourceScan( cluster, streamPhysicalTraits, Collections.emptyList[RelHint](), table) - assertEquals(uniqueKeys(Array(0, 2)), mq.getUniqueKeys(tableSourceScan).toSet) + assertEquals(uniqueKeys(Array(0, 2)), mq.getUniqueKeys(tableSourceScan)) } @Test def testGetUniqueKeysOnProjectedTableScanWithPartialCompositePrimaryKey(): Unit = { val table = relBuilder.getRelOptSchema .asInstanceOf[CalciteCatalogReader] - .getTable(Seq("projected_table_source_table_with_partial_pk")) + .getTable(java.util.List.of[String]("projected_table_source_table_with_partial_pk")) .asInstanceOf[TableSourceTable] val tableSourceScan = new StreamPhysicalTableSourceScan( cluster, @@ -81,11 +82,11 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { @Test def testGetUniqueKeysOnProject(): Unit = { - assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(logicalProject).toSet) + assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(logicalProject)) relBuilder.push(studentLogicalScan) // id=1, id, cast(id AS bigint not null), cast(id AS int), $1 - val exprs = List( + val exprs = java.util.List.of( relBuilder.call(EQUALS, relBuilder.field(0), relBuilder.literal(1)), relBuilder.field(0), rexBuilder.makeCast(longType, relBuilder.field(0)), @@ -94,8 +95,8 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { ) val project1 = relBuilder.project(exprs).build() // INT -> BIGINT is an injective cast, so position 2 is also a unique key - assertEquals(uniqueKeys(Array(1), Array(2)), mq.getUniqueKeys(project1).toSet) - assertEquals(uniqueKeys(Array(1), Array(2)), mq.getUniqueKeys(project1, true).toSet) + assertEquals(uniqueKeys(Array(1), Array(2)), mq.getUniqueKeys(project1)) + assertEquals(uniqueKeys(Array(1), Array(2)), mq.getUniqueKeys(project1, true)) } @Test @@ -108,14 +109,14 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { // Project: CAST(id AS STRING), name // id (position 0 in source) is the unique key - val exprs = List( + val exprs = java.util.List.of( rexBuilder.makeCast(stringType, relBuilder.field(0)), // CAST(id AS STRING) relBuilder.field(1) // name ) val project = relBuilder.project(exprs).build() // The casted id at position 0 should still be recognized as unique - assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(project).toSet) + assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(project)) } @Test @@ -127,7 +128,7 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { val stringType = typeFactory.createSqlType(VARCHAR, 100) // Project: CAST(id AS STRING), id, name - val exprs = List( + val exprs = java.util.List.of( rexBuilder.makeCast(stringType, relBuilder.field(0)), // CAST(id AS STRING) - injective relBuilder.field(0), // id (raw reference) relBuilder.field(1) // name @@ -135,7 +136,7 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { val project = relBuilder.project(exprs).build() // Both position 0 (STRING cast of id) and position 1 (raw id) are unique keys - assertEquals(uniqueKeys(Array(0), Array(1)), mq.getUniqueKeys(project).toSet) + assertEquals(uniqueKeys(Array(0), Array(1)), mq.getUniqueKeys(project)) } @Test @@ -148,7 +149,7 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { // Project: id, CAST(name AS STRING) // id is the unique key; name is NOT a key (even after casting) - val exprs = List( + val exprs = java.util.List.of( relBuilder.field(0), // id - the unique key rexBuilder.makeCast(stringType, relBuilder.field(1)) // CAST(name AS STRING) - not a key ) @@ -156,7 +157,7 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { // Only position 0 (id) is a unique key // Position 1 (cast of name) is NOT a key because name wasn't a key to begin with - assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(project).toSet) + assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(project)) } @Test @@ -168,39 +169,39 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { val stringType = typeFactory.createSqlType(VARCHAR, 100) // First, project id as STRING to simulate a STRING key column - val stringKeyExprs = List( + val stringKeyExprs = java.util.List.of( rexBuilder.makeCast(stringType, relBuilder.field(0)), // CAST(id AS STRING) relBuilder.field(1) // name ) val stringKeyProject = relBuilder.project(stringKeyExprs).build() // At this point, position 0 is a STRING that's still a unique key - assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(stringKeyProject).toSet) + assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(stringKeyProject)) // Now cast the STRING back to INT - this is a non-injective (narrowing) cast relBuilder.push(stringKeyProject) - val narrowedExprs = List( + val narrowedExprs = java.util.List.of( rexBuilder.makeCast(intType, relBuilder.field(0)), // CAST(string_id AS INT) - NOT injective relBuilder.field(1) // name ) val narrowedProject = relBuilder.project(narrowedExprs).build() // The key is LOST because STRING->INT is not injective - assertEquals(uniqueKeys(), mq.getUniqueKeys(narrowedProject).toSet) + assertEquals(uniqueKeys(), mq.getUniqueKeys(narrowedProject)) } @Test def testGetUniqueKeysOnFilter(): Unit = { - assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(logicalFilter).toSet) + assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(logicalFilter)) } @Test def testGetUniqueKeysOnWatermark(): Unit = { - assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(logicalWatermarkAssigner).toSet) + assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(logicalWatermarkAssigner)) } @Test def testGetUniqueKeysOnMiniBatchAssigner(): Unit = { - assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(streamMiniBatchAssigner).toSet) + assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(streamMiniBatchAssigner)) } @Test @@ -212,11 +213,11 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { studentLogicalScan, logicalProject.getRowType, logicalProject.getProjects, - List(expr)) - assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(logicalCalc).toSet) + java.util.List.of[RexNode](expr)) + assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(logicalCalc)) // id=1, id, cast(id AS bigint not null), cast(id AS int), $1 - val exprs = List( + val exprs = java.util.List.of( relBuilder.call(EQUALS, relBuilder.field(0), relBuilder.literal(1)), relBuilder.field(0), rexBuilder.makeCast(longType, relBuilder.field(0)), @@ -224,16 +225,17 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { relBuilder.field(1) ) val rowType = relBuilder.project(exprs).build().getRowType - val calc2 = createLogicalCalc(studentLogicalScan, rowType, exprs, List(expr)) + val calc2 = + createLogicalCalc(studentLogicalScan, rowType, exprs, java.util.List.of[RexNode](expr)) // INT -> BIGINT is an injective cast, so position 2 is also a unique key - assertEquals(uniqueKeys(Array(1), Array(2)), mq.getUniqueKeys(calc2).toSet) - assertEquals(uniqueKeys(Array(1), Array(2)), mq.getUniqueKeys(calc2, true).toSet) + assertEquals(uniqueKeys(Array(1), Array(2)), mq.getUniqueKeys(calc2)) + assertEquals(uniqueKeys(Array(1), Array(2)), mq.getUniqueKeys(calc2, true)) } @Test def testGetUniqueKeysOnExpand(): Unit = { Array(logicalExpand, flinkLogicalExpand, batchExpand, streamExpand).foreach { - expand => assertEquals(uniqueKeys(Array(0, 7)), mq.getUniqueKeys(expand).toSet) + expand => assertEquals(uniqueKeys(Array(0, 7)), mq.getUniqueKeys(expand)) } val expandProjects = ExpandUtil.createExpandProjects( @@ -259,20 +261,18 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { @Test def testGetUniqueKeysOnExchange(): Unit = { Array(batchExchange, streamExchange).foreach { - exchange => assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(exchange).toSet) + exchange => assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(exchange)) } } @Test def testGetUniqueKeysOnRank(): Unit = { Array(logicalRank, flinkLogicalRank, batchLocalRank, batchGlobalRank, streamRank).foreach { - rank => assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(rank).toSet) + rank => assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(rank)) } Array(logicalRowNumber, flinkLogicalRowNumber, streamRowNumber) - .foreach { - rank => assertEquals(uniqueKeys(Array(0), Array(7)), mq.getUniqueKeys(rank).toSet) - } + .foreach(rank => assertEquals(uniqueKeys(Array(0), Array(7)), mq.getUniqueKeys(rank))) } @Test @@ -292,25 +292,25 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { flinkLogicalLimit, batchLimit, streamLimit - ).foreach(sort => assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(sort).toSet)) + ).foreach(sort => assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(sort))) } @Test def testGetUniqueKeysOnStreamExecDeduplicate(): Unit = { - assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(streamProcTimeDeduplicateFirstRow).toSet) - assertEquals(uniqueKeys(Array(1, 2)), mq.getUniqueKeys(streamProcTimeDeduplicateLastRow).toSet) - assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(streamRowTimeDeduplicateFirstRow).toSet) - assertEquals(uniqueKeys(Array(1, 2)), mq.getUniqueKeys(streamRowTimeDeduplicateLastRow).toSet) + assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(streamProcTimeDeduplicateFirstRow)) + assertEquals(uniqueKeys(Array(1, 2)), mq.getUniqueKeys(streamProcTimeDeduplicateLastRow)) + assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(streamRowTimeDeduplicateFirstRow)) + assertEquals(uniqueKeys(Array(1, 2)), mq.getUniqueKeys(streamRowTimeDeduplicateLastRow)) } @Test def testGetUniqueKeysOnStreamExecChangelogNormalize(): Unit = { - assertEquals(uniqueKeys(Array(1, 0)), mq.getUniqueKeys(streamChangelogNormalize).toSet) + assertEquals(uniqueKeys(Array(1, 0)), mq.getUniqueKeys(streamChangelogNormalize)) } @Test def testGetUniqueKeysOnStreamExecDropUpdateBefore(): Unit = { - assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(streamDropUpdateBefore).toSet) + assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(streamDropUpdateBefore)) } @Test @@ -322,7 +322,7 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { batchGlobalAggWithoutLocal, streamGlobalAggWithLocal, streamGlobalAggWithoutLocal).foreach { - agg => assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(agg).toSet) + agg => assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(agg)) } assertNull(mq.getUniqueKeys(batchLocalAgg)) assertNull(mq.getUniqueKeys(streamLocalAgg)) @@ -332,7 +332,7 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { flinkLogicalAggWithAuxGroup, batchGlobalAggWithLocalWithAuxGroup, batchGlobalAggWithoutLocalWithAuxGroup).foreach { - agg => assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(agg).toSet) + agg => assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(agg)) } assertNull(mq.getUniqueKeys(batchLocalAggWithAuxGroup)) } @@ -376,93 +376,88 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { @Test def testGetUniqueKeysOnOverAgg(): Unit = { Array(flinkLogicalOverAgg, batchOverAgg).foreach { - agg => assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(agg).toSet) + agg => assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(agg)) } - assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(streamOverAgg).toSet) + assertEquals(uniqueKeys(Array(0)), mq.getUniqueKeys(streamOverAgg)) } @Test def testGetUniqueKeysOnJoin(): Unit = { assertEquals( uniqueKeys(Array(1), Array(5), Array(1, 5), Array(5, 6), Array(1, 5, 6)), - mq.getUniqueKeys(logicalInnerJoinOnUniqueKeys).toSet) - assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalInnerJoinNotOnUniqueKeys).toSet) - assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalInnerJoinOnRHSUniqueKeys).toSet) - assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalInnerJoinWithoutEquiCond).toSet) - assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalInnerJoinWithEquiAndNonEquiCond).toSet) + mq.getUniqueKeys(logicalInnerJoinOnUniqueKeys)) + assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalInnerJoinNotOnUniqueKeys)) + assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalInnerJoinOnRHSUniqueKeys)) + assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalInnerJoinWithoutEquiCond)) + assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalInnerJoinWithEquiAndNonEquiCond)) assertEquals( uniqueKeys(Array(1), Array(1, 5), Array(1, 5, 6)), - mq.getUniqueKeys(logicalLeftJoinOnUniqueKeys).toSet) - assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalLeftJoinNotOnUniqueKeys).toSet) - assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalLeftJoinOnRHSUniqueKeys).toSet) - assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalLeftJoinWithoutEquiCond).toSet) - assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalLeftJoinWithEquiAndNonEquiCond).toSet) + mq.getUniqueKeys(logicalLeftJoinOnUniqueKeys)) + assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalLeftJoinNotOnUniqueKeys)) + assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalLeftJoinOnRHSUniqueKeys)) + assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalLeftJoinWithoutEquiCond)) + assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalLeftJoinWithEquiAndNonEquiCond)) assertEquals( uniqueKeys(Array(5), Array(1, 5), Array(5, 6), Array(1, 5, 6)), - mq.getUniqueKeys(logicalRightJoinOnUniqueKeys).toSet) - assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalRightJoinNotOnUniqueKeys).toSet) - assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalRightJoinOnLHSUniqueKeys).toSet) - assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalRightJoinWithoutEquiCond).toSet) - assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalRightJoinWithEquiAndNonEquiCond).toSet) + mq.getUniqueKeys(logicalRightJoinOnUniqueKeys)) + assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalRightJoinNotOnUniqueKeys)) + assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalRightJoinOnLHSUniqueKeys)) + assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalRightJoinWithoutEquiCond)) + assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalRightJoinWithEquiAndNonEquiCond)) assertEquals( uniqueKeys(Array(1, 5), Array(1, 5, 6)), - mq.getUniqueKeys(logicalFullJoinOnUniqueKeys).toSet) - assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalFullJoinNotOnUniqueKeys).toSet) - assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalFullJoinOnRHSUniqueKeys).toSet) - assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalFullJoinWithoutEquiCond).toSet) - assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalFullJoinWithEquiAndNonEquiCond).toSet) - - assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(logicalSemiJoinOnUniqueKeys).toSet) - assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(logicalSemiJoinNotOnUniqueKeys).toSet) + mq.getUniqueKeys(logicalFullJoinOnUniqueKeys)) + assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalFullJoinNotOnUniqueKeys)) + assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalFullJoinOnRHSUniqueKeys)) + assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalFullJoinWithoutEquiCond)) + assertEquals(uniqueKeys(), mq.getUniqueKeys(logicalFullJoinWithEquiAndNonEquiCond)) + + assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(logicalSemiJoinOnUniqueKeys)) + assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(logicalSemiJoinNotOnUniqueKeys)) assertNull(mq.getUniqueKeys(logicalSemiJoinOnRHSUniqueKeys)) - assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(logicalSemiJoinWithoutEquiCond).toSet) - assertEquals( - uniqueKeys(Array(1)), - mq.getUniqueKeys(logicalSemiJoinWithEquiAndNonEquiCond).toSet) + assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(logicalSemiJoinWithoutEquiCond)) + assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(logicalSemiJoinWithEquiAndNonEquiCond)) - assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(logicalAntiJoinOnUniqueKeys).toSet) - assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(logicalAntiJoinNotOnUniqueKeys).toSet) + assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(logicalAntiJoinOnUniqueKeys)) + assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(logicalAntiJoinNotOnUniqueKeys)) assertNull(mq.getUniqueKeys(logicalAntiJoinOnRHSUniqueKeys)) - assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(logicalAntiJoinWithoutEquiCond).toSet) - assertEquals( - uniqueKeys(Array(1)), - mq.getUniqueKeys(logicalAntiJoinWithEquiAndNonEquiCond).toSet) + assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(logicalAntiJoinWithoutEquiCond)) + assertEquals(uniqueKeys(Array(1)), mq.getUniqueKeys(logicalAntiJoinWithEquiAndNonEquiCond)) } @Test def testGetUniqueKeysOnLookupJoin(): Unit = { Array(batchLookupJoin, streamLookupJoin).foreach { - join => assertEquals(uniqueKeys(), mq.getUniqueKeys(join).toSet) + join => assertEquals(uniqueKeys(), mq.getUniqueKeys(join)) } } @Test def testGetUniqueKeysOnLookupJoinWithPk(): Unit = { Array(batchLookupJoinWithPk, streamLookupJoinWithPk).foreach { - join => - assertEquals(uniqueKeys(Array(7), Array(0, 7), Array(0)), mq.getUniqueKeys(join).toSet) + join => assertEquals(uniqueKeys(Array(7), Array(0, 7), Array(0)), mq.getUniqueKeys(join)) } } @Test def testGetUniqueKeysOnLookupJoinNotContainsPk(): Unit = { Array(batchLookupJoinNotContainsPk, streamLookupJoinNotContainsPk).foreach { - join => assertEquals(uniqueKeys(), mq.getUniqueKeys(join).toSet) + join => assertEquals(uniqueKeys(), mq.getUniqueKeys(join)) } } @Test def testGetUniqueKeysOnSetOp(): Unit = { Array(logicalUnionAll, logicalIntersectAll, logicalMinusAll).foreach { - setOp => assertEquals(uniqueKeys(), mq.getUniqueKeys(setOp).toSet) + setOp => assertEquals(uniqueKeys(), mq.getUniqueKeys(setOp)) } Array(logicalUnion, logicalIntersect, logicalMinus).foreach { - setOp => assertEquals(uniqueKeys(Array(0, 1, 2, 3, 4)), mq.getUniqueKeys(setOp).toSet) + setOp => assertEquals(uniqueKeys(Array(0, 1, 2, 3, 4)), mq.getUniqueKeys(setOp)) } } @@ -475,19 +470,19 @@ class FlinkRelMdUniqueKeysTest extends FlinkRelMdHandlerTestBase { def testGetUniqueKeysOnTableScanTable(): Unit = { assertEquals( uniqueKeys(Array(0, 1), Array(0, 1, 5)), - mq.getUniqueKeys(logicalLeftJoinOnContainedUniqueKeys).toSet + mq.getUniqueKeys(logicalLeftJoinOnContainedUniqueKeys) ) assertEquals( uniqueKeys(Array(0, 1, 5)), - mq.getUniqueKeys(logicalLeftJoinOnDisjointUniqueKeys).toSet + mq.getUniqueKeys(logicalLeftJoinOnDisjointUniqueKeys) ) assertEquals( uniqueKeys(), - mq.getUniqueKeys(logicalLeftJoinWithNoneKeyTableUniqueKeys).toSet + mq.getUniqueKeys(logicalLeftJoinWithNoneKeyTableUniqueKeys) ) } - private def uniqueKeys(keys: Array[Int]*): Set[ImmutableBitSet] = { - keys.map(k => ImmutableBitSet.of(k: _*)).toSet + private def uniqueKeys(keys: Array[Int]*): java.util.Set[ImmutableBitSet] = { + keys.map(k => ImmutableBitSet.of(k: _*)).toSet.asJava } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeysTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeysTest.scala index 4a77662bd2b70..4ae6f389e5182 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeysTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUpsertKeysTest.scala @@ -42,14 +42,14 @@ import org.junit.jupiter.api.Test import java.util.Collections -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters.setAsJavaSetConverter class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { @Test def testGetUpsertKeysOnTableScan(): Unit = { Array(studentLogicalScan, studentBatchScan, studentStreamScan).foreach { - scan => assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(scan).toSet) + scan => assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(scan)) } Array(empLogicalScan, empBatchScan, empStreamScan).foreach { @@ -58,21 +58,21 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { val table = relBuilder.getRelOptSchema .asInstanceOf[CalciteCatalogReader] - .getTable(Seq("projected_table_source_table")) + .getTable(java.util.List.of[String]("projected_table_source_table")) .asInstanceOf[TableSourceTable] val tableSourceScan = new StreamPhysicalTableSourceScan( cluster, streamPhysicalTraits, Collections.emptyList[RelHint](), table) - assertEquals(toBitSet(Array(0, 2)), mq.getUpsertKeys(tableSourceScan).toSet) + assertEquals(toBitSet(Array(0, 2)), mq.getUpsertKeys(tableSourceScan)) } @Test def testGetUpsertKeysOnProjectedTableScanWithPartialCompositePrimaryKey(): Unit = { val table = relBuilder.getRelOptSchema .asInstanceOf[CalciteCatalogReader] - .getTable(Seq("projected_table_source_table_with_partial_pk")) + .getTable(java.util.List.of[String]("projected_table_source_table_with_partial_pk")) .asInstanceOf[TableSourceTable] val tableSourceScan = new StreamPhysicalTableSourceScan( cluster, @@ -90,11 +90,11 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { @Test def testGetUpsertKeysOnProject(): Unit = { - assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(logicalProject).toSet) + assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(logicalProject)) relBuilder.push(studentLogicalScan) // id=1, id, cast(id AS bigint not null), cast(id AS int), $1 - val exprs = List( + val exprs = java.util.List.of( relBuilder.call(EQUALS, relBuilder.field(0), relBuilder.literal(1)), relBuilder.field(0), // INT -> BIGINT is an injective cast, so position 2 is now also an upsert key @@ -104,7 +104,7 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { ) val project1 = relBuilder.project(exprs).build() - assertEquals(toBitSet(Array(1), Array(2)), mq.getUpsertKeys(project1).toSet) + assertEquals(toBitSet(Array(1), Array(2)), mq.getUpsertKeys(project1)) } @Test @@ -116,14 +116,14 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { val stringType = typeFactory.createSqlType(VARCHAR, 100) // Project: CAST(id AS STRING), name - val exprs = List( + val exprs = java.util.List.of( rexBuilder.makeCast(stringType, relBuilder.field(0)), // CAST(id AS STRING) relBuilder.field(1) // name ) val project = relBuilder.project(exprs).build() // The casted id at position 0 should still be recognized as upsert key - assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(project).toSet) + assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(project)) } @Test @@ -135,7 +135,7 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { val stringType = typeFactory.createSqlType(VARCHAR, 100) // Project: CAST(id AS STRING), id, name - val exprs = List( + val exprs = java.util.List.of( rexBuilder.makeCast(stringType, relBuilder.field(0)), // CAST(id AS STRING) - injective relBuilder.field(0), // id (raw reference) relBuilder.field(1) // name @@ -143,7 +143,7 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { val project = relBuilder.project(exprs).build() // Both position 0 (STRING cast of id) and position 1 (raw id) are upsert keys - assertEquals(toBitSet(Array(0), Array(1)), mq.getUpsertKeys(project).toSet) + assertEquals(toBitSet(Array(0), Array(1)), mq.getUpsertKeys(project)) } @Test @@ -155,38 +155,38 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { val stringType = typeFactory.createSqlType(VARCHAR, 100) // First, project id as STRING to simulate a STRING key column - val stringKeyExprs = List( + val stringKeyExprs = java.util.List.of( rexBuilder.makeCast(stringType, relBuilder.field(0)), // CAST(id AS STRING) relBuilder.field(1) // name ) val stringKeyProject = relBuilder.project(stringKeyExprs).build() - assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(stringKeyProject).toSet) + assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(stringKeyProject)) // Now cast the STRING back to INT - this is a non-injective cast relBuilder.push(stringKeyProject) - val narrowedExprs = List( + val narrowedExprs = java.util.List.of( rexBuilder.makeCast(intType, relBuilder.field(0)), // CAST(string_id AS INT) - NOT injective relBuilder.field(1) // name ) val narrowedProject = relBuilder.project(narrowedExprs).build() // The key is LOST because STRING->INT is not injective - assertEquals(toBitSet(), mq.getUpsertKeys(narrowedProject).toSet) + assertEquals(toBitSet(), mq.getUpsertKeys(narrowedProject)) } @Test def testGetUpsertKeysOnFilter(): Unit = { - assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(logicalFilter).toSet) + assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(logicalFilter)) } @Test def testGetUpsertKeysOnWatermark(): Unit = { - assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(logicalWatermarkAssigner).toSet) + assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(logicalWatermarkAssigner)) } @Test def testGetUpsertKeysOnMiniBatchAssigner(): Unit = { - assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(streamMiniBatchAssigner).toSet) + assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(streamMiniBatchAssigner)) } @Test @@ -198,11 +198,11 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { studentLogicalScan, logicalProject.getRowType, logicalProject.getProjects, - List(expr)) - assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(logicalCalc).toSet) + java.util.List.of[RexNode](expr)) + assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(logicalCalc)) // id=1, id, cast(id AS bigint not null), cast(id AS int), $1 - val exprs = List( + val exprs = java.util.List.of( relBuilder.call(EQUALS, relBuilder.field(0), relBuilder.literal(1)), relBuilder.field(0), rexBuilder.makeCast(longType, relBuilder.field(0)), @@ -210,15 +210,16 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { relBuilder.field(1) ) val rowType = relBuilder.project(exprs).build().getRowType - val calc2 = createLogicalCalc(studentLogicalScan, rowType, exprs, List(expr)) + val calc2 = + createLogicalCalc(studentLogicalScan, rowType, exprs, java.util.List.of[RexNode](expr)) // INT -> BIGINT is an injective cast, so position 2 is now also an upsert key - assertEquals(toBitSet(Array(1), Array(2)), mq.getUpsertKeys(calc2).toSet) + assertEquals(toBitSet(Array(1), Array(2)), mq.getUpsertKeys(calc2)) } @Test def testGetUpsertKeysOnExpand(): Unit = { Array(logicalExpand, flinkLogicalExpand, batchExpand, streamExpand).foreach { - expand => assertEquals(toBitSet(Array(0, 7)), mq.getUpsertKeys(expand).toSet) + expand => assertEquals(toBitSet(Array(0, 7)), mq.getUpsertKeys(expand)) } val expandProjects = ExpandUtil.createExpandProjects( @@ -244,18 +245,18 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { @Test def testGetUpsertKeysOnExchange(): Unit = { Array(batchExchange, streamExchange).foreach { - exchange => assertEquals(toBitSet(), mq.getUpsertKeys(exchange).toSet) + exchange => assertEquals(toBitSet(), mq.getUpsertKeys(exchange)) } Array(batchExchangeById, streamExchangeById).foreach { - exchange => assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(exchange).toSet) + exchange => assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(exchange)) } } @Test def testGetUpsertKeysOnRank(): Unit = { Array(logicalRank, flinkLogicalRank, batchLocalRank, batchGlobalRank, streamRank).foreach { - rank => assertEquals(toBitSet(), mq.getUpsertKeys(rank).toSet) + rank => assertEquals(toBitSet(), mq.getUpsertKeys(rank)) } Array( @@ -263,22 +264,20 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { flinkLogicalRankById, batchLocalRankById, batchGlobalRankById, - streamRankById).foreach { - rank => assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(rank).toSet) - } + streamRankById).foreach(rank => assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(rank))) Array(logicalRowNumber, flinkLogicalRowNumber, streamRowNumber) - .foreach(rank => assertEquals(toBitSet(Array(0), Array(7)), mq.getUpsertKeys(rank).toSet)) + .foreach(rank => assertEquals(toBitSet(Array(0), Array(7)), mq.getUpsertKeys(rank))) } @Test def testGetUpsertKeysOnSort(): Unit = { def testWithoutKey(rel: RelNode): Unit = { - assertEquals(toBitSet(), mq.getUpsertKeys(rel).toSet) + assertEquals(toBitSet(), mq.getUpsertKeys(rel)) } def testWithKey(rel: RelNode): Unit = { - assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(rel).toSet) + assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(rel)) } testWithoutKey(logicalSort) @@ -311,20 +310,20 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { @Test def testGetUpsertKeysOnStreamExecDeduplicate(): Unit = { - assertEquals(toBitSet(Array(1)), mq.getUpsertKeys(streamProcTimeDeduplicateFirstRow).toSet) - assertEquals(toBitSet(Array(1, 2)), mq.getUpsertKeys(streamProcTimeDeduplicateLastRow).toSet) - assertEquals(toBitSet(Array(1)), mq.getUpsertKeys(streamRowTimeDeduplicateFirstRow).toSet) - assertEquals(toBitSet(Array(1, 2)), mq.getUpsertKeys(streamRowTimeDeduplicateLastRow).toSet) + assertEquals(toBitSet(Array(1)), mq.getUpsertKeys(streamProcTimeDeduplicateFirstRow)) + assertEquals(toBitSet(Array(1, 2)), mq.getUpsertKeys(streamProcTimeDeduplicateLastRow)) + assertEquals(toBitSet(Array(1)), mq.getUpsertKeys(streamRowTimeDeduplicateFirstRow)) + assertEquals(toBitSet(Array(1, 2)), mq.getUpsertKeys(streamRowTimeDeduplicateLastRow)) } @Test def testGetUpsertKeysOnStreamExecChangelogNormalize(): Unit = { - assertEquals(toBitSet(Array(1, 0)), mq.getUpsertKeys(streamChangelogNormalize).toSet) + assertEquals(toBitSet(Array(1, 0)), mq.getUpsertKeys(streamChangelogNormalize)) } @Test def testGetUpsertKeysOnStreamExecDropUpdateBefore(): Unit = { - assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(streamDropUpdateBefore).toSet) + assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(streamDropUpdateBefore)) } @Test @@ -336,7 +335,7 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { batchGlobalAggWithoutLocal, streamGlobalAggWithLocal, streamGlobalAggWithoutLocal).foreach { - agg => assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(agg).toSet) + agg => assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(agg)) } assertNull(mq.getUpsertKeys(batchLocalAgg)) assertNull(mq.getUpsertKeys(streamLocalAgg)) @@ -346,7 +345,7 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { flinkLogicalAggWithAuxGroup, batchGlobalAggWithLocalWithAuxGroup, batchGlobalAggWithoutLocalWithAuxGroup).foreach { - agg => assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(agg).toSet) + agg => assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(agg)) } assertNull(mq.getUpsertKeys(batchLocalAggWithAuxGroup)) } @@ -390,88 +389,88 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { @Test def testGetUpsertKeysOnOverAgg(): Unit = { Array(flinkLogicalOverAgg, batchOverAgg, streamOverAgg).foreach { - agg => assertEquals(toBitSet(), mq.getUpsertKeys(agg).toSet) + agg => assertEquals(toBitSet(), mq.getUpsertKeys(agg)) } - assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(streamOverAggById).toSet) + assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(streamOverAggById)) } @Test def testGetUpsertKeysOnJoin(): Unit = { assertEquals( toBitSet(Array(1), Array(5), Array(1, 5), Array(5, 6), Array(1, 5, 6)), - mq.getUpsertKeys(logicalInnerJoinOnUniqueKeys).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalInnerJoinNotOnUniqueKeys).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalInnerJoinOnRHSUniqueKeys).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalInnerJoinWithoutEquiCond).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalInnerJoinWithEquiAndNonEquiCond).toSet) + mq.getUpsertKeys(logicalInnerJoinOnUniqueKeys)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalInnerJoinNotOnUniqueKeys)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalInnerJoinOnRHSUniqueKeys)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalInnerJoinWithoutEquiCond)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalInnerJoinWithEquiAndNonEquiCond)) assertEquals( toBitSet(Array(1), Array(1, 5), Array(1, 5, 6)), - mq.getUpsertKeys(logicalLeftJoinOnUniqueKeys).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalLeftJoinNotOnUniqueKeys).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalLeftJoinOnRHSUniqueKeys).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalLeftJoinWithoutEquiCond).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalLeftJoinWithEquiAndNonEquiCond).toSet) + mq.getUpsertKeys(logicalLeftJoinOnUniqueKeys)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalLeftJoinNotOnUniqueKeys)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalLeftJoinOnRHSUniqueKeys)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalLeftJoinWithoutEquiCond)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalLeftJoinWithEquiAndNonEquiCond)) assertEquals( toBitSet(Array(5), Array(1, 5), Array(5, 6), Array(1, 5, 6)), - mq.getUpsertKeys(logicalRightJoinOnUniqueKeys).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalRightJoinNotOnUniqueKeys).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalRightJoinOnLHSUniqueKeys).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalRightJoinWithoutEquiCond).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalRightJoinWithEquiAndNonEquiCond).toSet) + mq.getUpsertKeys(logicalRightJoinOnUniqueKeys)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalRightJoinNotOnUniqueKeys)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalRightJoinOnLHSUniqueKeys)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalRightJoinWithoutEquiCond)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalRightJoinWithEquiAndNonEquiCond)) assertEquals( toBitSet(Array(1, 5), Array(1, 5, 6)), - mq.getUpsertKeys(logicalFullJoinOnUniqueKeys).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalFullJoinNotOnUniqueKeys).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalFullJoinOnRHSUniqueKeys).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalFullJoinWithoutEquiCond).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalFullJoinWithEquiAndNonEquiCond).toSet) - - assertEquals(toBitSet(Array(1)), mq.getUpsertKeys(logicalSemiJoinOnUniqueKeys).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalSemiJoinNotOnUniqueKeys).toSet) + mq.getUpsertKeys(logicalFullJoinOnUniqueKeys)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalFullJoinNotOnUniqueKeys)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalFullJoinOnRHSUniqueKeys)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalFullJoinWithoutEquiCond)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalFullJoinWithEquiAndNonEquiCond)) + + assertEquals(toBitSet(Array(1)), mq.getUpsertKeys(logicalSemiJoinOnUniqueKeys)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalSemiJoinNotOnUniqueKeys)) assertNull(mq.getUpsertKeys(logicalSemiJoinOnRHSUniqueKeys)) - assertEquals(toBitSet(Array(1)), mq.getUpsertKeys(logicalSemiJoinWithoutEquiCond).toSet) - assertEquals(toBitSet(Array(1)), mq.getUpsertKeys(logicalSemiJoinWithEquiAndNonEquiCond).toSet) + assertEquals(toBitSet(Array(1)), mq.getUpsertKeys(logicalSemiJoinWithoutEquiCond)) + assertEquals(toBitSet(Array(1)), mq.getUpsertKeys(logicalSemiJoinWithEquiAndNonEquiCond)) - assertEquals(toBitSet(Array(1)), mq.getUpsertKeys(logicalAntiJoinOnUniqueKeys).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalAntiJoinNotOnUniqueKeys).toSet) + assertEquals(toBitSet(Array(1)), mq.getUpsertKeys(logicalAntiJoinOnUniqueKeys)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalAntiJoinNotOnUniqueKeys)) assertNull(mq.getUpsertKeys(logicalAntiJoinOnRHSUniqueKeys)) - assertEquals(toBitSet(Array(1)), mq.getUpsertKeys(logicalAntiJoinWithoutEquiCond).toSet) - assertEquals(toBitSet(), mq.getUpsertKeys(logicalAntiJoinWithEquiAndNonEquiCond).toSet) + assertEquals(toBitSet(Array(1)), mq.getUpsertKeys(logicalAntiJoinWithoutEquiCond)) + assertEquals(toBitSet(), mq.getUpsertKeys(logicalAntiJoinWithEquiAndNonEquiCond)) } @Test def testGetUpsertKeysOnLookupJoin(): Unit = { Array(batchLookupJoin, streamLookupJoin).foreach { - join => assertEquals(toBitSet(), mq.getUpsertKeys(join).toSet) + join => assertEquals(toBitSet(), mq.getUpsertKeys(join)) } } @Test def testGetUpsertKeysOnLookupJoinWithPk(): Unit = { Array(batchLookupJoinWithPk, streamLookupJoinWithPk).foreach { - join => assertEquals(toBitSet(Array(7), Array(0, 7), Array(0)), mq.getUpsertKeys(join).toSet) + join => assertEquals(toBitSet(Array(7), Array(0, 7), Array(0)), mq.getUpsertKeys(join)) } } @Test def testGetUpsertKeysOnLookupJoinNotContainsPk(): Unit = { Array(batchLookupJoinNotContainsPk, streamLookupJoinNotContainsPk).foreach { - join => assertEquals(toBitSet(), mq.getUpsertKeys(join).toSet) + join => assertEquals(toBitSet(), mq.getUpsertKeys(join)) } } @Test def testGetUpsertKeysOnSetOp(): Unit = { Array(logicalUnionAll, logicalIntersectAll, logicalMinusAll).foreach { - setOp => assertEquals(toBitSet(), mq.getUpsertKeys(setOp).toSet) + setOp => assertEquals(toBitSet(), mq.getUpsertKeys(setOp)) } Array(logicalUnion, logicalIntersect, logicalMinus).foreach { - setOp => assertEquals(toBitSet(Array(0, 1, 2, 3, 4)), mq.getUpsertKeys(setOp).toSet) + setOp => assertEquals(toBitSet(Array(0, 1, 2, 3, 4)), mq.getUpsertKeys(setOp)) } } @@ -482,7 +481,7 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { @Test def testGetUpsertKeysOnIntermediateScan(): Unit = { - assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(intermediateScan).toSet) + assertEquals(toBitSet(Array(0)), mq.getUpsertKeys(intermediateScan)) } @Test @@ -490,7 +489,7 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { // Immutable columns: {0, 1, 2} (PK 'a' + immutable 'c', 'd') assertEquals( toBitSet(Array(0), Array(0, 1, 2)), - mq.getUpsertKeys(tableWithImmutableColsLogicalScan).toSet) + mq.getUpsertKeys(tableWithImmutableColsLogicalScan)) } @Test @@ -502,7 +501,7 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { streamPhysicalTraits.replace(hash1), tableWithImmutableColsStreamScan, hash1) - assertEquals(toBitSet(Array(0), Array(0, 1, 2)), mq.getUpsertKeys(exchange1).toSet) + assertEquals(toBitSet(Array(0), Array(0, 1, 2)), mq.getUpsertKeys(exchange1)) // Hash exchange on column 3 (rowtime, NOT immutable) val hash3 = FlinkRelDistribution.hash(Array(3), requireStrict = true) @@ -511,7 +510,7 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { streamPhysicalTraits.replace(hash3), tableWithImmutableColsStreamScan, hash3) - assertEquals(toBitSet(), mq.getUpsertKeys(exchange3).toSet) + assertEquals(toBitSet(), mq.getUpsertKeys(exchange3)) } @Test @@ -519,12 +518,12 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { // Sort on column 1 (c, immutable) relBuilder.push(tableWithImmutableColsLogicalScan) val sort1 = relBuilder.sort(relBuilder.field(1)).build() - assertEquals(toBitSet(Array(0), Array(0, 1, 2)), mq.getUpsertKeys(sort1).toSet) + assertEquals(toBitSet(Array(0), Array(0, 1, 2)), mq.getUpsertKeys(sort1)) // Sort on column 3 (rowtime, NOT immutable) relBuilder.push(tableWithImmutableColsLogicalScan) val sort3 = relBuilder.sort(relBuilder.field(3)).build() - assertEquals(toBitSet(), mq.getUpsertKeys(sort3).toSet) + assertEquals(toBitSet(), mq.getUpsertKeys(sort3)) } @Test @@ -553,11 +552,11 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { // Rank partitioned by column 1 (c, immutable) val rank1 = buildRank(1) - assertEquals(toBitSet(Array(0), Array(0, 1, 2)), mq.getUpsertKeys(rank1).toSet) + assertEquals(toBitSet(Array(0), Array(0, 1, 2)), mq.getUpsertKeys(rank1)) // Rank partitioned by column 3 (rowtime, NOT immutable) val rank3 = buildRank(3) - assertEquals(toBitSet(), mq.getUpsertKeys(rank3).toSet) + assertEquals(toBitSet(), mq.getUpsertKeys(rank3)) } @Test @@ -621,11 +620,11 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { // Over agg partitioned by column 1 (c, immutable) val over1 = buildOverAgg(1) - assertEquals(toBitSet(Array(0), Array(0, 1, 2)), mq.getUpsertKeys(over1).toSet) + assertEquals(toBitSet(Array(0), Array(0, 1, 2)), mq.getUpsertKeys(over1)) // Over agg partitioned by column 3 (rowtime, NOT immutable) val over3 = buildOverAgg(3) - assertEquals(toBitSet(), mq.getUpsertKeys(over3).toSet) + assertEquals(toBitSet(), mq.getUpsertKeys(over3)) } @Test @@ -639,7 +638,7 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { JoinRelType.SEMI, relBuilder.call(EQUALS, relBuilder.field(2, 0, 1), relBuilder.field(2, 1, 1))) .build() - assertEquals(toBitSet(Array(0), Array(0, 1, 2)), mq.getUpsertKeys(join1).toSet) + assertEquals(toBitSet(Array(0), Array(0, 1, 2)), mq.getUpsertKeys(join1)) } @Test @@ -659,10 +658,10 @@ class FlinkRelMdUpsertKeysTest extends FlinkRelMdHandlerTestBase { .build() assertEquals( toBitSet(Array(0, 4), Array(0, 4, 5, 6), Array(0, 1, 2, 4), Array(0, 1, 2, 4, 5, 6)), - mq.getUpsertKeys(join).toSet) + mq.getUpsertKeys(join)) } - private def toBitSet(keys: Array[Int]*): Set[ImmutableBitSet] = { - keys.map(k => ImmutableBitSet.of(k: _*)).toSet + private def toBitSet(keys: Array[Int]*): java.util.Set[ImmutableBitSet] = { + keys.map(k => ImmutableBitSet.of(k: _*)).toSet.asJava } } From 4417d430a57383bc6492a9cb6ac745378662c925 Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Thu, 28 May 2026 15:54:22 +0200 Subject: [PATCH 3/4] [FLINK-39817][table] Upgrade Calcite to 1.39.0 --- flink-table/flink-sql-parser/pom.xml | 8 +- .../src/main/codegen/data/Parser.tdd | 4 + .../src/main/codegen/templates/Parser.jj | 102 +- .../calcite/sql/type/SqlTypeFamily.java | 292 -- .../apache/calcite/sql/type/SqlTypeName.java | 1078 ---- .../parser/validate/FlinkSqlConformance.java | 15 + .../flink-table-calcite-bridge/pom.xml | 8 +- .../calcite/jdbc/SimpleCalciteSchema.java | 56 +- .../org/apache/calcite/plan/RelOptUtil.java | 4596 ----------------- .../calcite/rel/rules/SubQueryRemoveRule.java | 115 +- .../rel/type/RelDataTypeFactoryImpl.java | 14 +- .../org/apache/calcite/rex/RexBuilder.java | 2206 -------- .../org/apache/calcite/rex/RexChecker.java | 203 - .../apache/calcite/rex/RexFieldAccess.java | 141 - .../org/apache/calcite/rex/RexProgram.java | 984 ---- .../org/apache/calcite/rex/RexShuttle.java | 284 - .../java/org/apache/calcite/rex/RexUtil.java | 18 +- .../apache/calcite/runtime/SqlFunctions.java | 384 +- .../java/org/apache/calcite/sql/SqlUtil.java | 6 +- .../apache/calcite/sql/type/BasicSqlType.java | 2 +- .../sql/type/SqlTypeAssignmentRule.java | 4 +- .../calcite/sql/type/SqlTypeFactoryImpl.java | 64 +- .../apache/calcite/sql/type/SqlTypeUtil.java | 86 +- .../sql/validate/SqlValidatorImpl.java | 408 +- .../apache/calcite/sql2rel/AggConverter.java | 2 +- .../calcite/sql2rel/RelDecorrelator.java | 486 +- .../calcite/sql2rel/SqlToRelConverter.java | 492 +- .../calcite/FlinkCalciteSqlValidator.java | 11 + .../src/main/resources/META-INF/NOTICE | 6 +- .../planner/plan/utils/TemporalJoinUtil.scala | 3 - .../planner/functions/CastFunctionITCase.java | 3 +- .../table/planner/plan/batch/sql/CalcTest.xml | 2 +- .../plan/batch/sql/DeadlockBreakupTest.xml | 2 +- .../batch/sql/DynamicFunctionPlanTest.xml | 9 +- .../plan/batch/sql/SetOperatorsTest.xml | 4 +- .../ReplaceIntersectWithSemiJoinRuleTest.xml | 2 +- .../ReplaceMinusWithAntiJoinRuleTest.xml | 2 +- .../logical/WindowGroupReorderRuleTest.xml | 2 +- .../logical/subquery/SubQueryAntiJoinTest.xml | 180 + .../logical/subquery/SubQuerySemiJoinTest.xml | 109 + ...PushCalcPastChangelogNormalizeRuleTest.xml | 18 +- .../planner/plan/stream/sql/CalcTest.xml | 2 +- .../plan/stream/sql/SetOperatorsTest.xml | 4 +- .../planner/plan/stream/table/ValuesTest.xml | 2 +- .../expressions/SqlExpressionTest.scala | 3 +- .../subquery/SubQueryAntiJoinTest.scala | 16 +- .../subquery/SubQuerySemiJoinTest.scala | 12 +- flink-table/pom.xml | 2 +- 48 files changed, 1879 insertions(+), 10573 deletions(-) delete mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java delete mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java delete mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/plan/RelOptUtil.java delete mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rex/RexBuilder.java delete mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rex/RexChecker.java delete mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rex/RexFieldAccess.java delete mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rex/RexProgram.java delete mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rex/RexShuttle.java diff --git a/flink-table/flink-sql-parser/pom.xml b/flink-table/flink-sql-parser/pom.xml index 0264583bfecc7..8be0c191ffb04 100644 --- a/flink-table/flink-sql-parser/pom.xml +++ b/flink-table/flink-sql-parser/pom.xml @@ -68,11 +68,11 @@ under the License. ${calcite.version} - 3.1.11 + 3.1.12 diff --git a/flink-table/flink-sql-parser/pom.xml b/flink-table/flink-sql-parser/pom.xml index 8be0c191ffb04..2c4c37d47bb67 100644 --- a/flink-table/flink-sql-parser/pom.xml +++ b/flink-table/flink-sql-parser/pom.xml @@ -68,10 +68,10 @@ under the License. ${calcite.version}